Skip to content

Commit

Permalink
[DBZ-PGYB][yugabyte/yuyabyte-db#24204] Changes to support LSN types w…
Browse files Browse the repository at this point in the history
…ith replication slot (#162)

YugabyteDB logical replication now supports creating replication slots
with two types of LSN:
1. `SEQUENCE` - This is a monotonic increasing number that will
determine the record in global order within the context of a slot.
However, this LSN can’t be compared across two LSN’s of different slots.
2. `HYBRID_TIME` - This will mean that the LSN will be denoted by the
`HybridTime` of the transaction commit record. All the records of the
transaction that is streamed will have the same LSN as that of the
commit record. The user has to ensure that the changes of a transaction
are applied in totality and the acknowledgement is sent only if the
commit record of a transaction is processed. With this mode, the LSN
value can be compared across the different slots.

To ensure that the connector also supports streaming for both LSN types,
this PR introduces the following changes:
1. Adds a configuration property `slot.lsn.type` which accepts two
parameters i.e. `SEQUENCE` or `HYBRID_TIME` with the default being
`SEQUENCE`
    a. **Note that this property only accepts parameters in uppercase.**
2. Depending on the LSN type provided, the connector processes events
accordingly.

This closes yugabyte/yuyabyte-db#24204
  • Loading branch information
vaibhav-yb authored Feb 5, 2025
1 parent 060f600 commit 985efdd
Show file tree
Hide file tree
Showing 6 changed files with 224 additions and 20 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -380,6 +380,62 @@ public static SecureConnectionMode parse(String value, String defaultValue) {
}
}

public enum LsnType implements EnumeratedValue {
SEQUENCE("SEQUENCE") {
@Override
public String getLsnTypeName() {
return getValue();
}

@Override
public boolean isSequence() {
return true;
}

@Override
public boolean isHybridTime() {
return false;
}
},
HYBRID_TIME("HYBRID_TIME") {
@Override
public String getLsnTypeName() {
return getValue();
}

@Override
public boolean isSequence() {
return false;
}

@Override
public boolean isHybridTime() {
return true;
}
};

private final String lsnTypeName;

LsnType(String lsnTypeName) {
this.lsnTypeName = lsnTypeName;
}

public static LsnType parse(String s) {
return valueOf(s.trim().toUpperCase());
}

@Override
public String getValue() {
return lsnTypeName;
}

public abstract boolean isSequence();

public abstract boolean isHybridTime();

public abstract String getLsnTypeName();
}

public enum LogicalDecoder implements EnumeratedValue {
PGOUTPUT("pgoutput") {
@Override
Expand Down Expand Up @@ -580,6 +636,14 @@ public static SchemaRefreshMode parse(String value) {
+ "'. " +
"Defaults to '" + LogicalDecoder.YBOUTPUT.getValue() + "'.");

public static final Field SLOT_LSN_TYPE = Field.create("slot.lsn.type")
.withDisplayName("Slot LSN type")
.withType(Type.STRING)
.withWidth(Width.MEDIUM)
.withImportance(Importance.MEDIUM)
.withEnum(LsnType.class, LsnType.SEQUENCE)
.withDescription("LSN type being used with the replication slot");

public static final Field SLOT_NAME = Field.create("slot.name")
.withDisplayName("Slot")
.withType(Type.STRING)
Expand Down Expand Up @@ -1083,6 +1147,10 @@ protected String slotName() {
return getConfig().getString(SLOT_NAME);
}

public LsnType slotLsnType() {
return LsnType.parse(getConfig().getString(SLOT_LSN_TYPE));
}

protected boolean dropSlotOnStop() {
if (getConfig().hasKey(DROP_SLOT_ON_STOP.name())) {
return getConfig().getBoolean(DROP_SLOT_ON_STOP);
Expand Down Expand Up @@ -1218,6 +1286,7 @@ protected SourceInfoStructMaker<? extends AbstractSourceInfo> getSourceInfoStruc
DATABASE_NAME,
PLUGIN_NAME,
SLOT_NAME,
SLOT_LSN_TYPE,
PUBLICATION_NAME,
PUBLICATION_AUTOCREATE_MODE,
REPLICA_IDENTITY_AUTOSET_VALUES,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -9,6 +9,7 @@
import java.util.Map;
import java.util.Objects;
import java.util.OptionalLong;
import java.util.concurrent.ConcurrentLinkedQueue;
import java.util.concurrent.atomic.AtomicReference;

import org.apache.kafka.connect.errors.ConnectException;
Expand Down Expand Up @@ -80,12 +81,16 @@ public class PostgresStreamingChangeEventSource implements StreamingChangeEventS
*/
private long numberOfEventsSinceLastEventSentOrWalGrowingWarning = 0;
private Lsn lastCompletelyProcessedLsn;
private Lsn lastSentFeedback = Lsn.valueOf(2L);
private PostgresOffsetContext effectiveOffset;

protected ConcurrentLinkedQueue<Lsn> commitTimes;

/**
* For DEBUGGING
*/
private OptionalLong lastTxnidForWhichCommitSeen = OptionalLong.empty();
private long recordCount = 0;

public PostgresStreamingChangeEventSource(PostgresConnectorConfig connectorConfig, Snapshotter snapshotter,
PostgresConnection connection, PostgresEventDispatcher<TableId> dispatcher, ErrorHandler errorHandler, Clock clock,
Expand All @@ -101,7 +106,7 @@ public PostgresStreamingChangeEventSource(PostgresConnectorConfig connectorConfi
this.snapshotter = snapshotter;
this.replicationConnection = (PostgresReplicationConnection) replicationConnection;
this.connectionProbeTimer = ElapsedTimeStrategy.constant(Clock.system(), connectorConfig.statusUpdateInterval());

this.commitTimes = new ConcurrentLinkedQueue<>();
}

@Override
Expand All @@ -121,6 +126,20 @@ private void initSchema() {
}
}

public Lsn getLsn(PostgresOffsetContext offsetContext, PostgresConnectorConfig.LsnType lsnType) {
if (lsnType.isSequence()) {
return this.effectiveOffset.lastCompletelyProcessedLsn() != null ? this.effectiveOffset.lastCompletelyProcessedLsn()
: this.effectiveOffset.lsn();
} else {
// We are in the block for HYBRID_TIME lsn type and last commit can be null for cases
// where we have just started/restarted the connector, in that case, we simply sent the
// initial value of lastSentFeedback and let the server handle the time we
// should get the changes from.
return this.effectiveOffset.lastCommitLsn() == null ?
lastSentFeedback : this.effectiveOffset.lastCommitLsn();
}
}

@Override
public void execute(ChangeEventSourceContext context, PostgresPartition partition, PostgresOffsetContext offsetContext)
throws InterruptedException {
Expand Down Expand Up @@ -148,17 +167,24 @@ public void execute(ChangeEventSourceContext context, PostgresPartition partitio
}

if (hasStartLsnStoredInContext) {
// start streaming from the last recorded position in the offset
final Lsn lsn = this.effectiveOffset.lastCompletelyProcessedLsn() != null ? this.effectiveOffset.lastCompletelyProcessedLsn()
: this.effectiveOffset.lsn();
final Lsn lsn = getLsn(this.effectiveOffset, connectorConfig.slotLsnType());
final Operation lastProcessedMessageType = this.effectiveOffset.lastProcessedMessageType();
LOGGER.info("Retrieved latest position from stored offset '{}'", lsn);
walPosition = new WalPositionLocator(this.effectiveOffset.lastCommitLsn(), lsn, lastProcessedMessageType);

if (this.effectiveOffset.lastCommitLsn() == null) {
LOGGER.info("Last commit stored in offset is null");
}

LOGGER.info("Retrieved last committed LSN from stored offset '{}'", lsn);

walPosition = new WalPositionLocator(this.effectiveOffset.lastCommitLsn(), lsn,
lastProcessedMessageType, connectorConfig.slotLsnType().isHybridTime() /* isLsnTypeHybridTime */);

replicationStream.compareAndSet(null, replicationConnection.startStreaming(lsn, walPosition));
lastSentFeedback = lsn;
}
else {
LOGGER.info("No previous LSN found in Kafka, streaming from the latest xlogpos or flushed LSN...");
walPosition = new WalPositionLocator();
walPosition = new WalPositionLocator(this.connectorConfig.slotLsnType().isHybridTime());
replicationStream.compareAndSet(null, replicationConnection.startStreaming(walPosition));
}
// for large dbs, the refresh of schema can take too much time
Expand Down Expand Up @@ -188,7 +214,13 @@ public void execute(ChangeEventSourceContext context, PostgresPartition partitio
} catch (Exception e) {
LOGGER.info("Commit failed while preparing for reconnect", e);
}
walPosition.enableFiltering();

// Do not filter anything when lsn type is hybrid time. This is to avoid the WalPositionLocator complaining
// about the LSN not being present in the lsnSeen set.
if (connectorConfig.slotLsnType().isSequence()) {
walPosition.enableFiltering();
}

stream.stopKeepAlive();
replicationConnection.reconnect();

Expand All @@ -198,7 +230,11 @@ public void execute(ChangeEventSourceContext context, PostgresPartition partitio
replicationConnection.getConnectedNodeIp());
}

replicationStream.set(replicationConnection.startStreaming(walPosition.getLastEventStoredLsn(), walPosition));
// For the HybridTime mode, we always want to resume from the position of last commit so that we
// send complete transactions and do not resume from the last event stored LSN.
Lsn lastStoredLsn = connectorConfig.slotLsnType().isHybridTime() ? walPosition.getLastCommitStoredLsn() : walPosition.getLastEventStoredLsn();
replicationStream.set(replicationConnection.startStreaming(lastStoredLsn, walPosition));

stream = this.replicationStream.get();
stream.startKeepAlive(Threads.newSingleThreadExecutor(YugabyteDBConnector.class, connectorConfig.getLogicalName(), KEEP_ALIVE_THREAD_NAME));
}
Expand Down Expand Up @@ -292,6 +328,8 @@ private void processReplicationMessages(PostgresPartition partition, PostgresOff
LOGGER.debug("Processing BEGIN with end LSN {} and txnid {}", lsn, message.getTransactionId());
} else {
LOGGER.debug("Processing COMMIT with end LSN {} and txnid {}", lsn, message.getTransactionId());
LOGGER.debug("Record count in the txn {} is {} with commit time {}", message.getTransactionId(), recordCount, lsn.asLong() - 1);
recordCount = 0;
}

OptionalLong currentTxnid = message.getTransactionId();
Expand All @@ -308,7 +346,7 @@ private void processReplicationMessages(PostgresPartition partition, PostgresOff
// Don't skip on BEGIN message as it would flush LSN for the whole transaction
// too early
if (message.getOperation() == Operation.COMMIT) {
commitMessage(partition, offsetContext, lsn);
commitMessage(partition, offsetContext, lsn, message);
}
return;
}
Expand All @@ -321,7 +359,7 @@ private void processReplicationMessages(PostgresPartition partition, PostgresOff
dispatcher.dispatchTransactionStartedEvent(partition, toString(message.getTransactionId()), offsetContext, message.getCommitTime());
}
else if (message.getOperation() == Operation.COMMIT) {
commitMessage(partition, offsetContext, lsn);
commitMessage(partition, offsetContext, lsn, message);
dispatcher.dispatchTransactionCommittedEvent(partition, offsetContext, message.getCommitTime());
}
maybeWarnAboutGrowingWalBacklog(true);
Expand All @@ -333,7 +371,7 @@ else if (message.getOperation() == Operation.MESSAGE) {

// non-transactional message that will not be followed by a COMMIT message
if (message.isLastEventForLsn()) {
commitMessage(partition, offsetContext, lsn);
commitMessage(partition, offsetContext, lsn, message);
}

dispatcher.dispatchLogicalDecodingMessage(
Expand All @@ -346,6 +384,9 @@ else if (message.getOperation() == Operation.MESSAGE) {
}
// DML event
else {
LOGGER.trace("Processing DML event with lsn {} and lastCompletelyProcessedLsn {}", lsn, lastCompletelyProcessedLsn);
++recordCount;

TableId tableId = null;
if (message.getOperation() != Operation.NOOP) {
tableId = PostgresSchema.parse(message.getTable());
Expand Down Expand Up @@ -384,7 +425,17 @@ private void searchWalPosition(ChangeEventSourceContext context, PostgresPartiti
while (context.isRunning() && resumeLsn.get() == null) {

boolean receivedMessage = stream.readPending(message -> {
final Lsn lsn = stream.lastReceivedLsn();
final Lsn lsn;
if (connectorConfig.slotLsnType().isHybridTime()) {
// Last commit can be null for cases where
// we have just started/restarted the connector, in that case, we simply sent the
// initial value of lastSentFeedback and let the server handle the time we
// should get the changes from.

lsn = walPosition.getLastCommitStoredLsn() != null ? walPosition.getLastCommitStoredLsn() : lastSentFeedback;
} else {
lsn = stream.lastReceivedLsn();
}
resumeLsn.set(walPosition.resumeFromLsn(lsn, message).orElse(null));
});

Expand Down Expand Up @@ -412,9 +463,17 @@ private void probeConnectionIfNeeded() throws SQLException {
}
}

private void commitMessage(PostgresPartition partition, PostgresOffsetContext offsetContext, final Lsn lsn) throws SQLException, InterruptedException {
private void commitMessage(PostgresPartition partition, PostgresOffsetContext offsetContext, final Lsn lsn, ReplicationMessage message) throws SQLException, InterruptedException {
lastCompletelyProcessedLsn = lsn;
offsetContext.updateCommitPosition(lsn, lastCompletelyProcessedLsn);

if (this.connectorConfig.slotLsnType().isHybridTime()) {
if (message.getOperation() == Operation.COMMIT) {
LOGGER.debug("Adding '{}' as lsn to the commit times queue", Lsn.valueOf(lsn.asLong() - 1));
commitTimes.add(Lsn.valueOf(lsn.asLong() - 1));
}
}

maybeWarnAboutGrowingWalBacklog(false);
dispatcher.dispatchHeartbeatEvent(partition, offsetContext);
}
Expand Down Expand Up @@ -470,11 +529,23 @@ public void commitOffset(Map<String, ?> partition, Map<String, ?> offset) {
return;
}

Lsn finalLsn;
if (this.connectorConfig.slotLsnType().isHybridTime()) {
finalLsn = getLsnToBeFlushed(lsn);
} else {
finalLsn = lsn;
}

if (LOGGER.isDebugEnabled()) {
LOGGER.debug("Flushing LSN to server: {}", lsn);
LOGGER.debug("Flushing LSN to server: {}", finalLsn);
}
// tell the server the point up to which we've processed data, so it can be free to recycle WAL segments
replicationStream.flushLsn(lsn);
replicationStream.flushLsn(finalLsn);

if (this.connectorConfig.slotLsnType().isHybridTime()) {
lastSentFeedback = finalLsn;
cleanCommitTimeQueue(finalLsn);
}
}
else {
LOGGER.debug("Streaming has already stopped, ignoring commit callback...");
Expand All @@ -485,6 +556,45 @@ public void commitOffset(Map<String, ?> partition, Map<String, ?> offset) {
}
}

/**
* Returns the LSN that should be flushed to the service. The {@code commitTimes} list will have
* a list of all the commit times for which we have received a commit record. All we want now
* is that whenever we get a commit callback, we should be flushing a time just smaller than
* the one we have gotten the callback on.
* @param lsn the {@link Lsn} received in callback
* @return the {@link Lsn} to be flushed
*/
protected Lsn getLsnToBeFlushed(Lsn lsn) {
if (commitTimes == null || commitTimes.isEmpty()) {
// This means that the queue has not been initialised and the task is still starting.
return lastSentFeedback;
}

Lsn result = lastSentFeedback;

if (LOGGER.isDebugEnabled()) {
LOGGER.debug("Queue at this time: {}", commitTimes);
}

for (Lsn commitLsn : commitTimes) {
if (commitLsn.compareTo(lsn) < 0) {
LOGGER.debug("Assigning result as {}", commitLsn);
result = commitLsn;
} else {
// This will be the loop exit when we encounter any bigger element.
break;
}
}

return result;
}

protected void cleanCommitTimeQueue(Lsn lsn) {
if (commitTimes != null) {
commitTimes.removeIf(ele -> ele.compareTo(lsn) < 1);
}
}

@Override
public PostgresOffsetContext getOffsetContext() {
return effectiveOffset;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -144,7 +144,7 @@ public boolean isValid() {

@Override
public String toString() {
return "LSN{" + asString() + '}';
return "LSN{" + asLong() + '}';
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -115,6 +115,13 @@ public PostgresConnection(JdbcConfiguration config, PostgresValueConverterBuilde
final PostgresValueConverter valueConverter = valueConverterBuilder.build(this.typeRegistry);
this.defaultValueConverter = new PostgresDefaultValueConverter(valueConverter, this.getTimestampUtils(), typeRegistry);
}

try {
LOGGER.debug("Setting GUC to disable catalog version check");
execute("SET yb_disable_catalog_version_check = true;");
} catch (Exception e) {
LOGGER.error("Error while setting GUC yb_disable_catalog_version_check", e);
}
}

public PostgresConnection(JdbcConfiguration config, PostgresValueConverterBuilder valueConverterBuilder, String connectionUsage) {
Expand Down
Loading

0 comments on commit 985efdd

Please sign in to comment.