DBZ-521 Misc. improvements and fixes;

* Actually assing last processed LSN to commit()
* passing LSN to commit() instead of entire SourceRecord
* Avoiding exception in case of wal2json batch with exactly one element
* Using "Lsn" instead of "LSN" in method names
* JavaDoc clarifications
This commit is contained in:
Gunnar Morling 2018-01-18 11:07:00 +01:00
parent 0b5348339a
commit 6f6ce33958
8 changed files with 99 additions and 91 deletions

View File

@ -49,10 +49,7 @@ public class PostgresConnectorTask extends SourceTask {
private RecordsProducer producer;
private Metronome metronome;
private Duration pollInterval;
private volatile SourceRecord lastRecordForRecovery = null;
public PostgresConnectorTask() {
}
private volatile long lastProcessedLsn;
@Override
public void start(Map<String, String> props) {
@ -160,7 +157,7 @@ private void createSnapshotProducer(PostgresTaskContext taskContext, SourceInfo
@Override
public void commit() throws InterruptedException {
if (running.get()) {
producer.commit(lastRecordForRecovery);
producer.commit(lastProcessedLsn);
}
}
@ -194,7 +191,8 @@ public List<SourceRecord> poll() throws InterruptedException {
for (int i = records.size() - 1; i >= 0; i--) {
SourceRecord r = records.get(i);
if (((Map<String, Boolean>)r.sourceOffset()).getOrDefault(SourceInfo.LAST_EVENT_FOR_LSN, Boolean.TRUE)) {
lastRecordForRecovery = r;
Map<String, ?> offset = r.sourceOffset();
lastProcessedLsn = (Long)offset.get(SourceInfo.LSN_KEY);
break;
}
}

View File

@ -7,6 +7,7 @@
package io.debezium.connector.postgresql;
import java.util.function.Consumer;
import org.apache.kafka.connect.source.SourceRecord;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -43,9 +44,9 @@ protected RecordsProducer(PostgresTaskContext taskContext, SourceInfo sourceInfo
protected abstract void start(Consumer<SourceRecord> recordsConsumer);
/**
* Notification that offsets have been committed to Kafka and LSN recorded up to the record
* Notification that offsets have been committed to Kafka up to the given LSN.
*/
protected abstract void commit(SourceRecord lastRecordForRecovery);
protected abstract void commit(long lsn);
/**
* Requests that this producer be stopped. This is normally a request coming from a {@link PostgresConnectorTask} instance

View File

@ -108,8 +108,8 @@ private void startStreaming(Consumer<SourceRecord> consumer) {
}
@Override
protected void commit(final SourceRecord lastRecordForRecovery) {
streamProducer.ifPresent(x -> x.commit(lastRecordForRecovery));
protected void commit(long lsn) {
streamProducer.ifPresent(x -> x.commit(lsn));
}
@Override

View File

@ -116,7 +116,7 @@ private void streamChanges(Consumer<SourceRecord> consumer) {
while (!Thread.currentThread().isInterrupted()) {
try {
// this will block until a message is available
stream.read(x -> process(x, stream.lastReceivedLSN(), consumer));
stream.read(x -> process(x, stream.lastReceivedLsn(), consumer));
} catch (SQLException e) {
Throwable cause = e.getCause();
if (cause != null && (cause instanceof IOException)) {
@ -136,20 +136,14 @@ private void streamChanges(Consumer<SourceRecord> consumer) {
}
@Override
protected synchronized void commit(final SourceRecord lastRecordForRecovery) {
protected synchronized void commit(long lsn) {
LoggingContext.PreviousContext previousContext = taskContext.configureLoggingContext(CONTEXT_NAME);
try {
ReplicationStream replicationStream = this.replicationStream.get();
if (replicationStream != null) {
// tell the server the point up to which we've processed data, so it can be free to recycle WAL segments
logger.debug("flushing offsets to server...");
if (lastRecordForRecovery != null) {
Map<String, ?> offset = lastRecordForRecovery.sourceOffset();
Long lsn = (Long)offset.get(SourceInfo.LSN_KEY);
if (lsn != null) {
replicationStream.flushLSN();
}
}
replicationStream.flushLsn(lsn);
} else {
logger.debug("streaming has already stopped, ignoring commit callback...");
}

View File

@ -207,25 +207,29 @@ public void close() throws SQLException {
}
@Override
public void flushLSN() throws SQLException {
public void flushLastReceivedLsn() throws SQLException {
if (lastReceivedLSN == null) {
// nothing to flush yet, since we haven't read anything...
return;
}
stream.setFlushedLSN(lastReceivedLSN);
stream.setAppliedLSN(lastReceivedLSN);
doFlushLsn(lastReceivedLSN);
}
@Override
public void flushLsn(long lsn) throws SQLException {
doFlushLsn(LogSequenceNumber.valueOf(lsn));
}
private void doFlushLsn(LogSequenceNumber lsn) throws SQLException {
stream.setFlushedLSN(lsn);
stream.setAppliedLSN(lsn);
stream.forceUpdateStatus();
}
@Override
public void flushLSN(Long lsn) throws SQLException {
stream.setFlushedLSN(LogSequenceNumber.valueOf(lsn));
stream.setAppliedLSN(LogSequenceNumber.valueOf(lsn));
stream.forceUpdateStatus();
}
@Override
public Long lastReceivedLSN() {
public Long lastReceivedLsn() {
return lastReceivedLSN != null ? lastReceivedLSN.asLong() : null;
}

View File

@ -23,7 +23,7 @@ public interface ReplicationMessageProcessor {
/**
* Blocks and waits for a replication message to be sent over a replication connection. Once a message has been received,
* the value of the {@link #lastReceivedLSN() last received LSN} will also be updated accordingly.
* the value of the {@link #lastReceivedLsn() last received LSN} will also be updated accordingly.
*
* @param processor - a callback to which the arrived message is passed
* @throws SQLException if anything unexpected fails
@ -33,7 +33,7 @@ public interface ReplicationMessageProcessor {
/**
* Attempts to read a replication message from a replication connection, returning that message if it's available or returning
* {@code null} if nothing is available. Once a message has been received, the value of the {@link #lastReceivedLSN() last received LSN}
* {@code null} if nothing is available. Once a message has been received, the value of the {@link #lastReceivedLsn() last received LSN}
* will also be updated accordingly.
*
* @param processor - a callback to which the arrived message is passed
@ -46,6 +46,10 @@ public interface ReplicationMessageProcessor {
* Sends a message to the server informing it about that latest position in the WAL that this stream has read via
* {@link ReplicationConnection#startStreaming()} or {@link ReplicationConnection#startStreaming(Long)}.
* <p>
* Due to the internal buffering the messages sent to Kafka (and thus committed offsets) will usually lag behind the
* latest received LSN, which is why {@link #flushLsn(long)} should be called typically in order to prevent committing
* an LSN which hasn't been committed to Kafka yet.
* <p>
* This essentially tells the server that this stream has successfully processed messages up to the current read cursor
* and so the server is free to discard older segments with earlier LSNs. It also affects the catch-up behavior once a slot
* is restarted and the server attempt to bring it up-to-date.
@ -53,9 +57,15 @@ public interface ReplicationMessageProcessor {
*
* @throws SQLException if anything goes wrong
*/
void flushLSN() throws SQLException;
void flushLastReceivedLsn() throws SQLException;
void flushLSN(Long lsn) throws SQLException;
/**
* Sends a message to the server informing it about the latest processed LSN.
*
* @throws SQLException if anything goes wrong
*/
void flushLsn(long lsn) throws SQLException;
/**
* Returns the value for the latest server received LSN during a read operation. The value is always updated once messages
@ -64,7 +74,7 @@ public interface ReplicationMessageProcessor {
*
* @return a {@link Long} value, possibly null if this is called before anything has been read
*/
Long lastReceivedLSN();
Long lastReceivedLsn();
/**
* //TODO author=Horia Chiorean date=13/10/2016 description=Don't use this for now, because of the bug from the PG server

View File

@ -9,6 +9,7 @@
import java.nio.ByteBuffer;
import java.sql.SQLException;
import java.util.Arrays;
import java.util.Iterator;
import org.apache.kafka.connect.errors.ConnectException;
import org.postgresql.replication.fluent.logical.ChainedLogicalStreamBuilder;
@ -18,6 +19,7 @@
import io.debezium.connector.postgresql.connection.MessageDecoder;
import io.debezium.connector.postgresql.connection.ReplicationStream.ReplicationMessageProcessor;
import io.debezium.document.Array;
import io.debezium.document.Array.Entry;
import io.debezium.document.Document;
import io.debezium.document.DocumentReader;
import io.debezium.document.Value;
@ -51,13 +53,11 @@ public void processMessage(ByteBuffer buffer, ReplicationMessageProcessor proces
final String timestamp = message.getString("timestamp");
final long commitTime = dateTime.systemTimestamp(timestamp);
final Array changes = message.getArray("change");
if (changes.size() > 0) {
for (int i = 0; i < changes.size() - 2; i++) {
final Value v = changes.get(i);
processor.process(new Wal2JsonReplicationMessage(txId, commitTime, v.asDocument(), containsMetadata, false));
}
final Value v = changes.get(changes.size() - 1);
processor.process(new Wal2JsonReplicationMessage(txId, commitTime, v.asDocument(), containsMetadata, true));
Iterator<Entry> it = changes.iterator();
while (it.hasNext()) {
Value value = it.next().getValue();
processor.process(new Wal2JsonReplicationMessage(txId, commitTime, value.asDocument(), containsMetadata, !it.hasNext()));
}
} catch (final IOException e) {
throw new ConnectException(e);

View File

@ -20,6 +20,7 @@
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicLong;
import java.util.function.Consumer;
import org.junit.Before;
import org.junit.Test;
@ -49,13 +50,13 @@ public void shouldCreateAndDropReplicationSlots() throws Exception {
// create a replication connection which should be dropped once it's closed
try (ReplicationConnection connection = TestHelper.createForReplication("test1", true)) {
ReplicationStream stream = connection.startStreaming();
assertNull(stream.lastReceivedLSN());
assertNull(stream.lastReceivedLsn());
stream.close();
}
// create a replication connection which should be dropped once it's closed
try (ReplicationConnection connection = TestHelper.createForReplication("test2", true)) {
ReplicationStream stream = connection.startStreaming();
assertNull(stream.lastReceivedLSN());
assertNull(stream.lastReceivedLsn());
stream.close();
}
}
@ -133,7 +134,7 @@ public void shouldReceiveMissedChangesWhileDown() throws Exception {
public void shouldResumeFromLastReceivedLSN() throws Exception {
String slotName = "test";
AtomicLong lastReceivedLSN = new AtomicLong(0);
startInsertStop(slotName, stream -> lastReceivedLSN.compareAndSet(0, stream.lastReceivedLSN()));
startInsertStop(slotName, stream -> lastReceivedLSN.compareAndSet(0, stream.lastReceivedLsn()));
assertTrue(lastReceivedLSN.get() > 0);
// resume replication from the last received LSN and don't expect anything else
@ -206,7 +207,7 @@ public void shouldGeneratesEventsForMultipleSchemas() throws Exception {
private void flushLSN(ReplicationStream stream) {
try {
stream.flushLSN();
stream.flushLastReceivedLsn();
} catch (SQLException e) {
throw new RuntimeException(e);
}