DBZ-777 Fix pgoutput decoder
This commit is contained in:
parent
de2345f87b
commit
5307a5822d
@ -18,6 +18,7 @@
|
|||||||
import io.debezium.connector.postgresql.connection.PostgresConnection;
|
import io.debezium.connector.postgresql.connection.PostgresConnection;
|
||||||
import io.debezium.connector.postgresql.connection.ReplicationMessage;
|
import io.debezium.connector.postgresql.connection.ReplicationMessage;
|
||||||
import io.debezium.data.Envelope.Operation;
|
import io.debezium.data.Envelope.Operation;
|
||||||
|
import io.debezium.function.Predicates;
|
||||||
import io.debezium.pipeline.spi.ChangeRecordEmitter;
|
import io.debezium.pipeline.spi.ChangeRecordEmitter;
|
||||||
import io.debezium.pipeline.spi.OffsetContext;
|
import io.debezium.pipeline.spi.OffsetContext;
|
||||||
import io.debezium.relational.Column;
|
import io.debezium.relational.Column;
|
||||||
@ -111,7 +112,7 @@ public DataCollectionSchema synchronizeTableSchema(DataCollectionSchema tableSch
|
|||||||
final boolean metadataInMessage = message.hasTypeMetadata();
|
final boolean metadataInMessage = message.hasTypeMetadata();
|
||||||
final TableId tableId = (TableId) tableSchema.id();
|
final TableId tableId = (TableId) tableSchema.id();
|
||||||
final Table table = schema.tableFor(tableId);
|
final Table table = schema.tableFor(tableId);
|
||||||
if (getOperation() == Operation.DELETE || message.shouldSchemaBeSynchronized()) {
|
if (getOperation() == Operation.DELETE || !message.shouldSchemaBeSynchronized()) {
|
||||||
return tableSchema;
|
return tableSchema;
|
||||||
}
|
}
|
||||||
final List<ReplicationMessage.Column> columns = message.getNewTupleList();
|
final List<ReplicationMessage.Column> columns = message.getNewTupleList();
|
||||||
@ -137,10 +138,12 @@ private Object[] columnValues(List<ReplicationMessage.Column> columns, TableId t
|
|||||||
|
|
||||||
// based on the schema columns, create the values on the same position as the columns
|
// based on the schema columns, create the values on the same position as the columns
|
||||||
List<Column> schemaColumns = table.columns();
|
List<Column> schemaColumns = table.columns();
|
||||||
|
// based on the replication message without toasted columns for now
|
||||||
|
List<ReplicationMessage.Column> columnsWithoutToasted = columns.stream().filter(Predicates.not(ReplicationMessage.Column::isToastedColumn)).collect(Collectors.toList());
|
||||||
// JSON does not deliver a list of all columns for REPLICA IDENTITY DEFAULT
|
// JSON does not deliver a list of all columns for REPLICA IDENTITY DEFAULT
|
||||||
Object[] values = new Object[columns.size() < schemaColumns.size() ? schemaColumns.size() : columns.size()];
|
Object[] values = new Object[columnsWithoutToasted.size() < schemaColumns.size() ? schemaColumns.size() : columnsWithoutToasted.size()];
|
||||||
|
|
||||||
for (ReplicationMessage.Column column: columns) {
|
for (ReplicationMessage.Column column: columnsWithoutToasted) {
|
||||||
//DBZ-298 Quoted column names will be sent like that in messages, but stored unquoted in the column names
|
//DBZ-298 Quoted column names will be sent like that in messages, but stored unquoted in the column names
|
||||||
final String columnName = Strings.unquoteIdentifierPart(column.getName());
|
final String columnName = Strings.unquoteIdentifierPart(column.getName());
|
||||||
final Column tableColumn = table.columnWithName(columnName);
|
final Column tableColumn = table.columnWithName(columnName);
|
||||||
|
@ -96,6 +96,7 @@ public void execute(ChangeEventSourceContext context) throws InterruptedExceptio
|
|||||||
if (message == null) {
|
if (message == null) {
|
||||||
LOGGER.trace("Received empty message");
|
LOGGER.trace("Received empty message");
|
||||||
lastCompletelyProcessedLsn = lsn;
|
lastCompletelyProcessedLsn = lsn;
|
||||||
|
dispatcher.dispatchHeartbeatEvent(offsetContext);
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
if (message.isLastEventForLsn()) {
|
if (message.isLastEventForLsn()) {
|
||||||
|
@ -336,12 +336,11 @@ private void decodeInsert(ByteBuffer buffer, TypeRegistry typeRegistry, Replicat
|
|||||||
|
|
||||||
Optional<Table> resolvedTable = resolveRelation(relationId);
|
Optional<Table> resolvedTable = resolveRelation(relationId);
|
||||||
if (!resolvedTable.isPresent()) {
|
if (!resolvedTable.isPresent()) {
|
||||||
return;
|
processor.process(null);
|
||||||
}
|
}
|
||||||
|
else {
|
||||||
Table table = resolvedTable.get();
|
Table table = resolvedTable.get();
|
||||||
List<Column> columns = resolveColumnsFromStreamTupleData(buffer, typeRegistry, table);
|
List<Column> columns = resolveColumnsFromStreamTupleData(buffer, typeRegistry, table);
|
||||||
|
|
||||||
processor.process(new PgOutputReplicationMessage(
|
processor.process(new PgOutputReplicationMessage(
|
||||||
Operation.INSERT,
|
Operation.INSERT,
|
||||||
table.id().toDoubleQuotedString(),
|
table.id().toDoubleQuotedString(),
|
||||||
@ -350,6 +349,7 @@ private void decodeInsert(ByteBuffer buffer, TypeRegistry typeRegistry, Replicat
|
|||||||
null,
|
null,
|
||||||
columns));
|
columns));
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Callback handler for the 'U' update replication stream message.
|
* Callback handler for the 'U' update replication stream message.
|
||||||
@ -365,9 +365,9 @@ private void decodeUpdate(ByteBuffer buffer, TypeRegistry typeRegistry, Replicat
|
|||||||
|
|
||||||
Optional<Table> resolvedTable = resolveRelation(relationId);
|
Optional<Table> resolvedTable = resolveRelation(relationId);
|
||||||
if (!resolvedTable.isPresent()) {
|
if (!resolvedTable.isPresent()) {
|
||||||
return;
|
processor.process(null);
|
||||||
}
|
}
|
||||||
|
else {
|
||||||
Table table = resolvedTable.get();
|
Table table = resolvedTable.get();
|
||||||
|
|
||||||
// When reading the tuple-type, we could get 3 different values, 'O', 'K', or 'N'.
|
// When reading the tuple-type, we could get 3 different values, 'O', 'K', or 'N'.
|
||||||
@ -386,7 +386,6 @@ private void decodeUpdate(ByteBuffer buffer, TypeRegistry typeRegistry, Replicat
|
|||||||
}
|
}
|
||||||
|
|
||||||
List<Column> columns = resolveColumnsFromStreamTupleData(buffer, typeRegistry, table);
|
List<Column> columns = resolveColumnsFromStreamTupleData(buffer, typeRegistry, table);
|
||||||
|
|
||||||
processor.process(new PgOutputReplicationMessage(
|
processor.process(new PgOutputReplicationMessage(
|
||||||
Operation.UPDATE,
|
Operation.UPDATE,
|
||||||
table.id().toDoubleQuotedString(),
|
table.id().toDoubleQuotedString(),
|
||||||
@ -395,6 +394,7 @@ private void decodeUpdate(ByteBuffer buffer, TypeRegistry typeRegistry, Replicat
|
|||||||
oldColumns,
|
oldColumns,
|
||||||
columns));
|
columns));
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Callback handler for the 'D' delete replication stream message.
|
* Callback handler for the 'D' delete replication stream message.
|
||||||
@ -412,9 +412,9 @@ private void decodeDelete(ByteBuffer buffer, TypeRegistry typeRegistry, Replicat
|
|||||||
|
|
||||||
Optional<Table> resolvedTable = resolveRelation(relationId);
|
Optional<Table> resolvedTable = resolveRelation(relationId);
|
||||||
if (!resolvedTable.isPresent()) {
|
if (!resolvedTable.isPresent()) {
|
||||||
return;
|
processor.process(null);
|
||||||
}
|
}
|
||||||
|
else {
|
||||||
Table table = resolvedTable.get();
|
Table table = resolvedTable.get();
|
||||||
List<Column> columns = resolveColumnsFromStreamTupleData(buffer, typeRegistry, table);
|
List<Column> columns = resolveColumnsFromStreamTupleData(buffer, typeRegistry, table);
|
||||||
processor.process(new PgOutputReplicationMessage(
|
processor.process(new PgOutputReplicationMessage(
|
||||||
@ -425,6 +425,7 @@ private void decodeDelete(ByteBuffer buffer, TypeRegistry typeRegistry, Replicat
|
|||||||
columns,
|
columns,
|
||||||
null));
|
null));
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Resolves a given replication message relation identifier to a {@link Table}.
|
* Resolves a given replication message relation identifier to a {@link Table}.
|
||||||
|
Loading…
Reference in New Issue
Block a user