DBZ-777 PostgreSQL connector rewrite to the standard framework
This commit is contained in:
parent
89ae4b2151
commit
f4246df6e4
@ -11,6 +11,8 @@
|
|||||||
|
|
||||||
import io.debezium.config.CommonConnectorConfig;
|
import io.debezium.config.CommonConnectorConfig;
|
||||||
import io.debezium.connector.LegacyV1AbstractSourceInfoStructMaker;
|
import io.debezium.connector.LegacyV1AbstractSourceInfoStructMaker;
|
||||||
|
import io.debezium.connector.SnapshotRecord;
|
||||||
|
import io.debezium.time.Conversions;
|
||||||
|
|
||||||
public class LegacyV1PostgresSourceInfoStructMaker extends LegacyV1AbstractSourceInfoStructMaker<SourceInfo> {
|
public class LegacyV1PostgresSourceInfoStructMaker extends LegacyV1AbstractSourceInfoStructMaker<SourceInfo> {
|
||||||
|
|
||||||
@ -51,8 +53,22 @@ public Struct struct(SourceInfo sourceInfo) {
|
|||||||
result.put(SourceInfo.DATABASE_NAME_KEY, sourceInfo.database());
|
result.put(SourceInfo.DATABASE_NAME_KEY, sourceInfo.database());
|
||||||
result.put(SourceInfo.SCHEMA_NAME_KEY, sourceInfo.schemaName());
|
result.put(SourceInfo.SCHEMA_NAME_KEY, sourceInfo.schemaName());
|
||||||
result.put(SourceInfo.TABLE_NAME_KEY, sourceInfo.tableName());
|
result.put(SourceInfo.TABLE_NAME_KEY, sourceInfo.tableName());
|
||||||
// use the offset information without the snapshot part (see below)
|
if (sourceInfo.timestamp() != null) {
|
||||||
sourceInfo.offset().forEach(result::put);
|
result.put(SourceInfo.TIMESTAMP_USEC_KEY, Conversions.toEpochMicros(sourceInfo.timestamp()));
|
||||||
|
}
|
||||||
|
if (sourceInfo.txId() != null) {
|
||||||
|
result.put(SourceInfo.TXID_KEY, sourceInfo.txId());
|
||||||
|
}
|
||||||
|
if (sourceInfo.lsn() != null) {
|
||||||
|
result.put(SourceInfo.LSN_KEY, sourceInfo.lsn());
|
||||||
|
}
|
||||||
|
if (sourceInfo.xmin() != null) {
|
||||||
|
result.put(SourceInfo.XMIN_KEY, sourceInfo.xmin());
|
||||||
|
}
|
||||||
|
if (sourceInfo.isSnapshot()) {
|
||||||
|
result.put(SourceInfo.SNAPSHOT_KEY, true);
|
||||||
|
result.put(SourceInfo.LAST_SNAPSHOT_RECORD_KEY, sourceInfo.snapshot() == SnapshotRecord.LAST);
|
||||||
|
}
|
||||||
return result;
|
return result;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -0,0 +1,80 @@
|
|||||||
|
/*
|
||||||
|
* Copyright Debezium Authors.
|
||||||
|
*
|
||||||
|
* Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*/
|
||||||
|
package io.debezium.connector.postgresql;
|
||||||
|
|
||||||
|
import io.debezium.connector.postgresql.connection.PostgresConnection;
|
||||||
|
import io.debezium.connector.postgresql.connection.ReplicationConnection;
|
||||||
|
import io.debezium.connector.postgresql.spi.SlotCreationResult;
|
||||||
|
import io.debezium.connector.postgresql.spi.Snapshotter;
|
||||||
|
import io.debezium.pipeline.ErrorHandler;
|
||||||
|
import io.debezium.pipeline.EventDispatcher;
|
||||||
|
import io.debezium.pipeline.source.spi.ChangeEventSourceFactory;
|
||||||
|
import io.debezium.pipeline.source.spi.SnapshotChangeEventSource;
|
||||||
|
import io.debezium.pipeline.source.spi.SnapshotProgressListener;
|
||||||
|
import io.debezium.pipeline.source.spi.StreamingChangeEventSource;
|
||||||
|
import io.debezium.pipeline.spi.OffsetContext;
|
||||||
|
import io.debezium.relational.TableId;
|
||||||
|
import io.debezium.util.Clock;
|
||||||
|
|
||||||
|
public class PostgresChangeEventSourceFactory implements ChangeEventSourceFactory {
|
||||||
|
|
||||||
|
private final PostgresConnectorConfig configuration;
|
||||||
|
private final PostgresConnection jdbcConnection;
|
||||||
|
private final ErrorHandler errorHandler;
|
||||||
|
private final EventDispatcher<TableId> dispatcher;
|
||||||
|
private final Clock clock;
|
||||||
|
private final PostgresSchema schema;
|
||||||
|
private final PostgresTaskContext taskContext;
|
||||||
|
private final Snapshotter snapshotter;
|
||||||
|
private final ReplicationConnection replicationConnection;
|
||||||
|
private final SlotCreationResult slotCreatedInfo;
|
||||||
|
|
||||||
|
public PostgresChangeEventSourceFactory(PostgresConnectorConfig configuration, Snapshotter snapshotter, PostgresConnection jdbcConnection,
|
||||||
|
ErrorHandler errorHandler, EventDispatcher<TableId> dispatcher, Clock clock, PostgresSchema schema, PostgresTaskContext taskContext,
|
||||||
|
ReplicationConnection replicationConnection, SlotCreationResult slotCreatedInfo) {
|
||||||
|
this.configuration = configuration;
|
||||||
|
this.jdbcConnection = jdbcConnection;
|
||||||
|
this.errorHandler = errorHandler;
|
||||||
|
this.dispatcher = dispatcher;
|
||||||
|
this.clock = clock;
|
||||||
|
this.schema = schema;
|
||||||
|
this.taskContext = taskContext;
|
||||||
|
this.snapshotter = snapshotter;
|
||||||
|
this.replicationConnection = replicationConnection;
|
||||||
|
this.slotCreatedInfo = slotCreatedInfo;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public SnapshotChangeEventSource getSnapshotChangeEventSource(OffsetContext offsetContext, SnapshotProgressListener snapshotProgressListener) {
|
||||||
|
return new PostgresSnapshotChangeEventSource(
|
||||||
|
configuration,
|
||||||
|
snapshotter,
|
||||||
|
(PostgresOffsetContext) offsetContext,
|
||||||
|
jdbcConnection,
|
||||||
|
schema,
|
||||||
|
dispatcher,
|
||||||
|
clock,
|
||||||
|
snapshotProgressListener,
|
||||||
|
slotCreatedInfo
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public StreamingChangeEventSource getStreamingChangeEventSource(OffsetContext offsetContext) {
|
||||||
|
return new PostgresStreamingChangeEventSource(
|
||||||
|
configuration,
|
||||||
|
snapshotter,
|
||||||
|
(PostgresOffsetContext) offsetContext,
|
||||||
|
jdbcConnection,
|
||||||
|
dispatcher,
|
||||||
|
errorHandler,
|
||||||
|
clock,
|
||||||
|
schema,
|
||||||
|
taskContext,
|
||||||
|
replicationConnection
|
||||||
|
);
|
||||||
|
}
|
||||||
|
}
|
@ -0,0 +1,318 @@
|
|||||||
|
/*
|
||||||
|
* Copyright Debezium Authors.
|
||||||
|
*
|
||||||
|
* Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*/
|
||||||
|
package io.debezium.connector.postgresql;
|
||||||
|
|
||||||
|
import java.sql.SQLException;
|
||||||
|
import java.util.Iterator;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Objects;
|
||||||
|
import java.util.Optional;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
|
import org.apache.kafka.connect.errors.ConnectException;
|
||||||
|
import org.postgresql.jdbc.PgConnection;
|
||||||
|
import org.slf4j.Logger;
|
||||||
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
|
import io.debezium.connector.postgresql.connection.PostgresConnection;
|
||||||
|
import io.debezium.connector.postgresql.connection.ReplicationMessage;
|
||||||
|
import io.debezium.data.Envelope.Operation;
|
||||||
|
import io.debezium.pipeline.spi.ChangeRecordEmitter;
|
||||||
|
import io.debezium.pipeline.spi.OffsetContext;
|
||||||
|
import io.debezium.relational.Column;
|
||||||
|
import io.debezium.relational.ColumnEditor;
|
||||||
|
import io.debezium.relational.RelationalChangeRecordEmitter;
|
||||||
|
import io.debezium.relational.Table;
|
||||||
|
import io.debezium.relational.TableEditor;
|
||||||
|
import io.debezium.relational.TableId;
|
||||||
|
import io.debezium.relational.TableSchema;
|
||||||
|
import io.debezium.schema.DataCollectionSchema;
|
||||||
|
import io.debezium.util.Clock;
|
||||||
|
import io.debezium.util.Strings;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Emits change data based on a logical decoding event coming as protobuf or JSON message.
|
||||||
|
*
|
||||||
|
* @author Horia Chiorean (hchiorea@redhat.com), Jiri Pechanec
|
||||||
|
*/
|
||||||
|
public class PostgresChangeRecordEmitter extends RelationalChangeRecordEmitter {
|
||||||
|
|
||||||
|
private static final Logger LOGGER = LoggerFactory.getLogger(PostgresChangeRecordEmitter.class);
|
||||||
|
|
||||||
|
private final ReplicationMessage message;
|
||||||
|
private final PostgresSchema schema;
|
||||||
|
private final PostgresConnectorConfig connectorConfig;
|
||||||
|
private final PostgresConnection connection;
|
||||||
|
|
||||||
|
public PostgresChangeRecordEmitter(OffsetContext offset, Clock clock, PostgresConnectorConfig connectorConfig, PostgresSchema schema, PostgresConnection connection, ReplicationMessage message) {
|
||||||
|
super(offset, clock);
|
||||||
|
|
||||||
|
this.schema = schema;
|
||||||
|
this.message = message;
|
||||||
|
this.connectorConfig = connectorConfig;
|
||||||
|
this.connection = connection;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Operation getOperation() {
|
||||||
|
switch (message.getOperation()) {
|
||||||
|
case INSERT:
|
||||||
|
return Operation.CREATE;
|
||||||
|
case UPDATE:
|
||||||
|
return Operation.UPDATE;
|
||||||
|
case DELETE:
|
||||||
|
return Operation.DELETE;
|
||||||
|
default:
|
||||||
|
throw new IllegalArgumentException("Received event of unexpected command type: " + message.getOperation());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Object[] getOldColumnValues() {
|
||||||
|
final TableId tableId = PostgresSchema.parse(message.getTable());
|
||||||
|
Objects.requireNonNull(tableId);
|
||||||
|
|
||||||
|
try {
|
||||||
|
switch (getOperation()) {
|
||||||
|
case CREATE:
|
||||||
|
return null;
|
||||||
|
case UPDATE:
|
||||||
|
return columnValues(message.getOldTupleList(), tableId, true, message.hasTypeMetadata());
|
||||||
|
default:
|
||||||
|
return columnValues(message.getOldTupleList(), tableId, true, message.hasTypeMetadata());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
catch (SQLException e) {
|
||||||
|
throw new ConnectException(e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Object[] getNewColumnValues() {
|
||||||
|
final TableId tableId = PostgresSchema.parse(message.getTable());
|
||||||
|
Objects.requireNonNull(tableId);
|
||||||
|
|
||||||
|
try {
|
||||||
|
switch (getOperation()) {
|
||||||
|
case CREATE:
|
||||||
|
return columnValues(message.getNewTupleList(), tableId, true, message.hasTypeMetadata());
|
||||||
|
case UPDATE:
|
||||||
|
return columnValues(message.getNewTupleList(), tableId, true, message.hasTypeMetadata());
|
||||||
|
default:
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
catch (SQLException e) {
|
||||||
|
throw new ConnectException(e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public DataCollectionSchema synchronizeTableSchema(DataCollectionSchema tableSchema) {
|
||||||
|
final boolean metadataInMessage = message.hasTypeMetadata();
|
||||||
|
final TableId tableId = (TableId) tableSchema.id();
|
||||||
|
final Table table = schema.tableFor(tableId);
|
||||||
|
final List<ReplicationMessage.Column> columns = getOperation() == Operation.DELETE ? message.getOldTupleList() : message.getNewTupleList();
|
||||||
|
// check if we need to refresh our local schema due to DB schema changes for this table
|
||||||
|
if (schemaChanged(columns, table, metadataInMessage)) {
|
||||||
|
// Refresh the schema so we get information about primary keys
|
||||||
|
refreshTableFromDatabase(tableId);
|
||||||
|
// Update the schema with metadata coming from decoder message
|
||||||
|
if (metadataInMessage) {
|
||||||
|
schema.refresh(tableFromFromMessage(columns, schema.tableFor(tableId)));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return schema.schemaFor(tableId);
|
||||||
|
}
|
||||||
|
|
||||||
|
private Object[] columnValues(List<ReplicationMessage.Column> columns, TableId tableId, boolean refreshSchemaIfChanged, boolean metadataInMessage)
|
||||||
|
throws SQLException {
|
||||||
|
if (columns == null || columns.isEmpty()) {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
final Table table = schema.tableFor(tableId);
|
||||||
|
Objects.requireNonNull(table);
|
||||||
|
|
||||||
|
// based on the schema columns, create the values on the same position as the columns
|
||||||
|
List<Column> schemaColumns = table.columns();
|
||||||
|
// 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()];
|
||||||
|
|
||||||
|
for (ReplicationMessage.Column column: columns) {
|
||||||
|
//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 Column tableColumn = table.columnWithName(columnName);
|
||||||
|
if (tableColumn == null) {
|
||||||
|
LOGGER.warn(
|
||||||
|
"Internal schema is out-of-sync with incoming decoder events; column {} will be omitted from the change event.",
|
||||||
|
column.getName());
|
||||||
|
continue;
|
||||||
|
}
|
||||||
|
int position = tableColumn.position() - 1;
|
||||||
|
if (position < 0 || position >= values.length) {
|
||||||
|
LOGGER.warn(
|
||||||
|
"Internal schema is out-of-sync with incoming decoder events; column {} will be omitted from the change event.",
|
||||||
|
column.getName());
|
||||||
|
continue;
|
||||||
|
}
|
||||||
|
values[position] = column.getValue(() -> (PgConnection) connection.connection(), connectorConfig.includeUnknownDatatypes());
|
||||||
|
}
|
||||||
|
|
||||||
|
return values;
|
||||||
|
}
|
||||||
|
|
||||||
|
private Optional<DataCollectionSchema> newTable(TableId tableId) {
|
||||||
|
refreshTableFromDatabase(tableId);
|
||||||
|
final TableSchema tableSchema = schema.schemaFor(tableId);
|
||||||
|
if (tableSchema == null) {
|
||||||
|
LOGGER.warn("cannot load schema for table '{}'", tableId);
|
||||||
|
return Optional.empty();
|
||||||
|
}
|
||||||
|
else {
|
||||||
|
LOGGER.debug("refreshed DB schema to include table '{}'", tableId);
|
||||||
|
return Optional.of(tableSchema);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private void refreshTableFromDatabase(TableId tableId) {
|
||||||
|
try {
|
||||||
|
schema.refresh(connection, tableId, connectorConfig.skipRefreshSchemaOnMissingToastableData());
|
||||||
|
}
|
||||||
|
catch (SQLException e) {
|
||||||
|
throw new ConnectException("Database error while refresing table schema");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
static Optional<DataCollectionSchema> updateSchema(TableId tableId, ChangeRecordEmitter changeRecordEmitter) {
|
||||||
|
LOGGER.debug("Schema for table '{}' is missing", tableId);
|
||||||
|
return ((PostgresChangeRecordEmitter) changeRecordEmitter).newTable(tableId);
|
||||||
|
}
|
||||||
|
|
||||||
|
private boolean schemaChanged(List<ReplicationMessage.Column> columns, Table table, boolean metadataInMessage) {
|
||||||
|
int tableColumnCount = table.columns().size();
|
||||||
|
int replicationColumnCount = columns.size();
|
||||||
|
|
||||||
|
boolean msgHasMissingColumns = tableColumnCount > replicationColumnCount;
|
||||||
|
|
||||||
|
if (msgHasMissingColumns && connectorConfig.skipRefreshSchemaOnMissingToastableData()) {
|
||||||
|
// if we are ignoring missing toastable data for the purpose of schema sync, we need to modify the
|
||||||
|
// hasMissingColumns boolean to account for this. If there are untoasted columns missing from the replication
|
||||||
|
// message, we'll still have missing columns and thus require a schema refresh. However, we can /possibly/
|
||||||
|
// avoid the refresh if there are only toastable columns missing from the message.
|
||||||
|
msgHasMissingColumns = hasMissingUntoastedColumns(table, columns);
|
||||||
|
}
|
||||||
|
|
||||||
|
boolean msgHasAdditionalColumns = tableColumnCount < replicationColumnCount;
|
||||||
|
|
||||||
|
if (msgHasMissingColumns || msgHasAdditionalColumns) {
|
||||||
|
// the table metadata has less or more columns than the event, which means the table structure has changed,
|
||||||
|
// so we need to trigger a refresh...
|
||||||
|
LOGGER.info("Different column count {} present in the server message as schema in memory contains {}; refreshing table schema",
|
||||||
|
replicationColumnCount,
|
||||||
|
tableColumnCount);
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
// go through the list of columns from the message to figure out if any of them are new or have changed their type based
|
||||||
|
// on what we have in the table metadata....
|
||||||
|
return columns.stream().filter(message -> {
|
||||||
|
String columnName = message.getName();
|
||||||
|
Column column = table.columnWithName(columnName);
|
||||||
|
if (column == null) {
|
||||||
|
LOGGER.info("found new column '{}' present in the server message which is not part of the table metadata; refreshing table schema", columnName);
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
else {
|
||||||
|
final int localType = column.nativeType();
|
||||||
|
final int incomingType = message.getType().getOid();
|
||||||
|
if (localType != incomingType) {
|
||||||
|
LOGGER.info("detected new type for column '{}', old type was {} ({}), new type is {} ({}); refreshing table schema", columnName, localType, column.typeName(),
|
||||||
|
incomingType, message.getType().getName());
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
if (metadataInMessage) {
|
||||||
|
final int localLength = column.length();
|
||||||
|
final int incomingLength = message.getTypeMetadata().getLength();
|
||||||
|
if (localLength != incomingLength) {
|
||||||
|
LOGGER.info("detected new length for column '{}', old length was {}, new length is {}; refreshing table schema", columnName, localLength,
|
||||||
|
incomingLength);
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
final int localScale = column.scale().get();
|
||||||
|
final int incomingScale = message.getTypeMetadata().getScale();
|
||||||
|
if (localScale != incomingScale) {
|
||||||
|
LOGGER.info("detected new scale for column '{}', old scale was {}, new scale is {}; refreshing table schema", columnName, localScale,
|
||||||
|
incomingScale);
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
final boolean localOptional = column.isOptional();
|
||||||
|
final boolean incomingOptional = message.isOptional();
|
||||||
|
if (localOptional != incomingOptional) {
|
||||||
|
LOGGER.info("detected new optional status for column '{}', old value was {}, new value is {}; refreshing table schema", columnName, localOptional, incomingOptional);
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return false;
|
||||||
|
}).findFirst().isPresent();
|
||||||
|
}
|
||||||
|
|
||||||
|
private boolean hasMissingUntoastedColumns(Table table, List<ReplicationMessage.Column> columns) {
|
||||||
|
List<String> msgColumnNames = columns.stream()
|
||||||
|
.map(ReplicationMessage.Column::getName)
|
||||||
|
.collect(Collectors.toList());
|
||||||
|
|
||||||
|
// Compute list of table columns not present in the replication message
|
||||||
|
List<String> missingColumnNames = table.columns()
|
||||||
|
.stream()
|
||||||
|
.filter(c -> !msgColumnNames.contains(c.name()))
|
||||||
|
.map(Column::name)
|
||||||
|
.collect(Collectors.toList());
|
||||||
|
|
||||||
|
List<String> toastableColumns = schema.getToastableColumnsForTableId(table.id());
|
||||||
|
|
||||||
|
if (LOGGER.isDebugEnabled()) {
|
||||||
|
LOGGER.debug("msg columns: '{}' --- missing columns: '{}' --- toastableColumns: '{}",
|
||||||
|
String.join(",", msgColumnNames),
|
||||||
|
String.join(",", missingColumnNames),
|
||||||
|
String.join(",", toastableColumns));
|
||||||
|
}
|
||||||
|
// Return `true` if we have some columns not in the replication message that are not toastable or that we do
|
||||||
|
// not recognize
|
||||||
|
return !toastableColumns.containsAll(missingColumnNames);
|
||||||
|
}
|
||||||
|
|
||||||
|
private Table tableFromFromMessage(List<ReplicationMessage.Column> columns, Table table) {
|
||||||
|
final TableEditor combinedTable = table.edit()
|
||||||
|
.setColumns(columns.stream()
|
||||||
|
.map(column -> {
|
||||||
|
final PostgresType type = column.getType();
|
||||||
|
final ColumnEditor columnEditor = Column.editor()
|
||||||
|
.name(column.getName())
|
||||||
|
.jdbcType(type.getJdbcId())
|
||||||
|
.type(type.getName())
|
||||||
|
.optional(column.isOptional())
|
||||||
|
.nativeType(type.getOid());
|
||||||
|
columnEditor.length(column.getTypeMetadata().getLength());
|
||||||
|
columnEditor.scale(column.getTypeMetadata().getScale());
|
||||||
|
return columnEditor.create();
|
||||||
|
})
|
||||||
|
.collect(Collectors.toList())
|
||||||
|
);
|
||||||
|
final List<String> pkCandidates = table.filterColumnNames(c -> table.isPrimaryKeyColumn(c.name()));
|
||||||
|
final Iterator<String> itPkCandidates = pkCandidates.iterator();
|
||||||
|
while (itPkCandidates.hasNext()) {
|
||||||
|
final String candidateName = itPkCandidates.next();
|
||||||
|
if (!combinedTable.hasUniqueValues() && combinedTable.columnWithName(candidateName) == null) {
|
||||||
|
LOGGER.error("Potentional inconsistency in key for message {}", columns);
|
||||||
|
itPkCandidates.remove();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
combinedTable.setPrimaryKeyNames(pkCandidates);
|
||||||
|
return combinedTable.create();
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
@ -7,6 +7,8 @@
|
|||||||
package io.debezium.connector.postgresql;
|
package io.debezium.connector.postgresql;
|
||||||
|
|
||||||
import java.time.Duration;
|
import java.time.Duration;
|
||||||
|
import java.util.Arrays;
|
||||||
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.concurrent.TimeUnit;
|
import java.util.concurrent.TimeUnit;
|
||||||
|
|
||||||
@ -39,6 +41,8 @@
|
|||||||
import io.debezium.jdbc.JdbcConfiguration;
|
import io.debezium.jdbc.JdbcConfiguration;
|
||||||
import io.debezium.jdbc.TemporalPrecisionMode;
|
import io.debezium.jdbc.TemporalPrecisionMode;
|
||||||
import io.debezium.relational.RelationalDatabaseConnectorConfig;
|
import io.debezium.relational.RelationalDatabaseConnectorConfig;
|
||||||
|
import io.debezium.relational.TableId;
|
||||||
|
import io.debezium.relational.Tables.TableFilter;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The configuration properties for the {@link PostgresConnector}
|
* The configuration properties for the {@link PostgresConnector}
|
||||||
@ -586,32 +590,6 @@ public static SchemaRefreshMode parse(String value) {
|
|||||||
.withImportance(Importance.MEDIUM)
|
.withImportance(Importance.MEDIUM)
|
||||||
.withDescription("A name of class to that creates SSL Sockets. Use org.postgresql.ssl.NonValidatingFactory to disable SSL validation in development environments");
|
.withDescription("A name of class to that creates SSL Sockets. Use org.postgresql.ssl.NonValidatingFactory to disable SSL validation in development environments");
|
||||||
|
|
||||||
/**
|
|
||||||
* A comma-separated list of regular expressions that match schema names to be monitored.
|
|
||||||
* May not be used with {@link #SCHEMA_BLACKLIST}.
|
|
||||||
*/
|
|
||||||
public static final Field SCHEMA_WHITELIST = Field.create("schema.whitelist")
|
|
||||||
.withDisplayName("Schemas")
|
|
||||||
.withType(Type.LIST)
|
|
||||||
.withWidth(Width.LONG)
|
|
||||||
.withImportance(Importance.HIGH)
|
|
||||||
.withDependents(TABLE_WHITELIST_NAME)
|
|
||||||
.withDescription("The schemas for which events should be captured");
|
|
||||||
|
|
||||||
/**
|
|
||||||
* A comma-separated list of regular expressions that match schema names to be excluded from monitoring.
|
|
||||||
* May not be used with {@link #SCHEMA_WHITELIST}.
|
|
||||||
*/
|
|
||||||
public static final Field SCHEMA_BLACKLIST = Field.create("schema.blacklist")
|
|
||||||
.withDisplayName("Exclude Schemas")
|
|
||||||
.withType(Type.STRING)
|
|
||||||
.withWidth(Width.LONG)
|
|
||||||
.withImportance(Importance.MEDIUM)
|
|
||||||
.withValidation(PostgresConnectorConfig::validateSchemaBlacklist)
|
|
||||||
.withInvisibleRecommender()
|
|
||||||
.withDescription("");
|
|
||||||
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* A comma-separated list of regular expressions that match the fully-qualified names of tables to be monitored.
|
* A comma-separated list of regular expressions that match the fully-qualified names of tables to be monitored.
|
||||||
* Fully-qualified names for tables are of the form {@code <schemaName>.<tableName>} or
|
* Fully-qualified names for tables are of the form {@code <schemaName>.<tableName>} or
|
||||||
@ -792,8 +770,8 @@ protected PostgresConnectorConfig(Configuration config) {
|
|||||||
super(
|
super(
|
||||||
config,
|
config,
|
||||||
config.getString(RelationalDatabaseConnectorConfig.SERVER_NAME),
|
config.getString(RelationalDatabaseConnectorConfig.SERVER_NAME),
|
||||||
null, // TODO whitelist handling implemented locally here for the time being
|
new SystemTablesPredicate(),
|
||||||
null,
|
x -> x.schema() + "." + x.table(),
|
||||||
DEFAULT_SNAPSHOT_FETCH_SIZE
|
DEFAULT_SNAPSHOT_FETCH_SIZE
|
||||||
);
|
);
|
||||||
|
|
||||||
@ -924,16 +902,6 @@ protected static ConfigDef configDef() {
|
|||||||
return config;
|
return config;
|
||||||
}
|
}
|
||||||
|
|
||||||
private static int validateSchemaBlacklist(Configuration config, Field field, Field.ValidationOutput problems) {
|
|
||||||
String whitelist = config.getString(SCHEMA_WHITELIST);
|
|
||||||
String blacklist = config.getString(SCHEMA_BLACKLIST);
|
|
||||||
if (whitelist != null && blacklist != null) {
|
|
||||||
problems.accept(SCHEMA_BLACKLIST, blacklist, "Schema whitelist is already specified");
|
|
||||||
return 1;
|
|
||||||
}
|
|
||||||
return 0;
|
|
||||||
}
|
|
||||||
|
|
||||||
private static int validateTableBlacklist(Configuration config, Field field, Field.ValidationOutput problems) {
|
private static int validateTableBlacklist(Configuration config, Field field, Field.ValidationOutput problems) {
|
||||||
String whitelist = config.getString(TABLE_WHITELIST);
|
String whitelist = config.getString(TABLE_WHITELIST);
|
||||||
String blacklist = config.getString(TABLE_BLACKLIST);
|
String blacklist = config.getString(TABLE_BLACKLIST);
|
||||||
@ -948,4 +916,12 @@ private static int validateTableBlacklist(Configuration config, Field field, Fie
|
|||||||
public String getContextName() {
|
public String getContextName() {
|
||||||
return Module.contextName();
|
return Module.contextName();
|
||||||
}
|
}
|
||||||
}
|
|
||||||
|
private static class SystemTablesPredicate implements TableFilter {
|
||||||
|
protected static final List<String> SYSTEM_SCHEMAS = Arrays.asList("pg_catalog", "information_schema");
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean isIncluded(TableId t) {
|
||||||
|
return !SYSTEM_SCHEMAS.contains(t.schema().toLowerCase());
|
||||||
|
}
|
||||||
|
}}
|
||||||
|
@ -10,14 +10,9 @@
|
|||||||
import java.sql.SQLException;
|
import java.sql.SQLException;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.concurrent.atomic.AtomicBoolean;
|
import java.util.concurrent.atomic.AtomicReference;
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
import io.debezium.connector.postgresql.snapshot.SnapshotterWrapper;
|
|
||||||
import io.debezium.connector.postgresql.spi.Snapshotter;
|
|
||||||
import io.debezium.connector.postgresql.spi.SlotState;
|
|
||||||
import io.debezium.relational.TableId;
|
|
||||||
import io.debezium.schema.TopicSelector;
|
|
||||||
import org.apache.kafka.connect.errors.ConnectException;
|
import org.apache.kafka.connect.errors.ConnectException;
|
||||||
import org.apache.kafka.connect.source.SourceRecord;
|
import org.apache.kafka.connect.source.SourceRecord;
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
@ -28,6 +23,17 @@
|
|||||||
import io.debezium.connector.base.ChangeEventQueue;
|
import io.debezium.connector.base.ChangeEventQueue;
|
||||||
import io.debezium.connector.common.BaseSourceTask;
|
import io.debezium.connector.common.BaseSourceTask;
|
||||||
import io.debezium.connector.postgresql.connection.PostgresConnection;
|
import io.debezium.connector.postgresql.connection.PostgresConnection;
|
||||||
|
import io.debezium.connector.postgresql.connection.ReplicationConnection;
|
||||||
|
import io.debezium.connector.postgresql.spi.SlotCreationResult;
|
||||||
|
import io.debezium.connector.postgresql.spi.SlotState;
|
||||||
|
import io.debezium.connector.postgresql.spi.Snapshotter;
|
||||||
|
import io.debezium.pipeline.ChangeEventSourceCoordinator;
|
||||||
|
import io.debezium.pipeline.DataChangeEvent;
|
||||||
|
import io.debezium.pipeline.ErrorHandler;
|
||||||
|
import io.debezium.pipeline.EventDispatcher;
|
||||||
|
import io.debezium.relational.TableId;
|
||||||
|
import io.debezium.schema.TopicSelector;
|
||||||
|
import io.debezium.util.Clock;
|
||||||
import io.debezium.util.LoggingContext;
|
import io.debezium.util.LoggingContext;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -37,142 +43,199 @@
|
|||||||
*/
|
*/
|
||||||
public class PostgresConnectorTask extends BaseSourceTask {
|
public class PostgresConnectorTask extends BaseSourceTask {
|
||||||
|
|
||||||
|
private static final Logger LOGGER = LoggerFactory.getLogger(PostgresConnectorTask.class);
|
||||||
private static final String CONTEXT_NAME = "postgres-connector-task";
|
private static final String CONTEXT_NAME = "postgres-connector-task";
|
||||||
private final Logger logger = LoggerFactory.getLogger(getClass());
|
|
||||||
private final AtomicBoolean running = new AtomicBoolean(false);
|
|
||||||
|
|
||||||
private PostgresTaskContext taskContext;
|
private static enum State {
|
||||||
private RecordsProducer producer;
|
RUNNING, STOPPED;
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
private final AtomicReference<State> state = new AtomicReference<State>(State.STOPPED);
|
||||||
* In case of wal2json, all records of one TX will be sent with the same LSN. This is the last LSN that was
|
|
||||||
* completely processed, i.e. we've seen all events originating from that TX.
|
|
||||||
*/
|
|
||||||
private volatile Long lastCompletelyProcessedLsn;
|
|
||||||
|
|
||||||
/**
|
private volatile PostgresTaskContext taskContext;
|
||||||
* A queue with change events filled by the snapshot and streaming producers, consumed
|
private volatile ChangeEventQueue<DataChangeEvent> queue;
|
||||||
* by Kafka Connect via this task.
|
private volatile PostgresConnection jdbcConnection;
|
||||||
*/
|
private volatile ChangeEventSourceCoordinator coordinator;
|
||||||
private ChangeEventQueue<ChangeEvent> changeEventQueue;
|
private volatile ErrorHandler errorHandler;
|
||||||
|
private volatile PostgresSchema schema;
|
||||||
|
private volatile Map<String, ?> lastOffset;
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void start(Configuration config) {
|
public void start(Configuration config) {
|
||||||
if (running.get()) {
|
if (!state.compareAndSet(State.STOPPED, State.RUNNING)) {
|
||||||
// already running
|
LOGGER.info("Connector has already been started");
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
PostgresConnectorConfig connectorConfig = new PostgresConnectorConfig(config);
|
final PostgresConnectorConfig connectorConfig = new PostgresConnectorConfig(config);
|
||||||
|
final TopicSelector<TableId> topicSelector = PostgresTopicSelector.create(connectorConfig);
|
||||||
TypeRegistry typeRegistry;
|
final Snapshotter snapshotter = connectorConfig.getSnapshotter();
|
||||||
Charset databaseCharset;
|
|
||||||
|
|
||||||
try (final PostgresConnection connection = new PostgresConnection(connectorConfig.jdbcConfig())) {
|
|
||||||
typeRegistry = connection.getTypeRegistry();
|
|
||||||
databaseCharset = connection.getDatabaseCharset();
|
|
||||||
}
|
|
||||||
|
|
||||||
Snapshotter snapshotter = connectorConfig.getSnapshotter();
|
|
||||||
if (snapshotter == null) {
|
if (snapshotter == null) {
|
||||||
logger.error("Unable to load snapshotter, if using custom snapshot mode, double check your settings");
|
LOGGER.error("Unable to load snapshotter, if using custom snapshot mode, double check your settings");
|
||||||
throw new ConnectException("Unable to load snapshotter, if using custom snapshot mode, double check your settings");
|
throw new ConnectException("Unable to load snapshotter, if using custom snapshot mode, double check your settings");
|
||||||
}
|
}
|
||||||
|
|
||||||
// create the task context and schema...
|
|
||||||
TopicSelector<TableId> topicSelector = PostgresTopicSelector.create(connectorConfig);
|
|
||||||
PostgresSchema schema = new PostgresSchema(connectorConfig, typeRegistry, databaseCharset, topicSelector);
|
|
||||||
this.taskContext = new PostgresTaskContext(connectorConfig, schema, topicSelector);
|
|
||||||
|
|
||||||
SourceInfo sourceInfo = new SourceInfo(connectorConfig);
|
jdbcConnection = new PostgresConnection(connectorConfig.jdbcConfig());
|
||||||
Map<String, Object> existingOffset = context.offsetStorageReader().offset(sourceInfo.partition());
|
final TypeRegistry typeRegistry = jdbcConnection.getTypeRegistry();
|
||||||
|
final Charset databaseCharset = jdbcConnection.getDatabaseCharset();
|
||||||
|
|
||||||
|
schema = new PostgresSchema(connectorConfig, typeRegistry, databaseCharset, topicSelector);
|
||||||
|
this.taskContext = new PostgresTaskContext(connectorConfig, schema, topicSelector);
|
||||||
|
final PostgresOffsetContext previousOffset = (PostgresOffsetContext) getPreviousOffset(new PostgresOffsetContext.Loader(connectorConfig));
|
||||||
|
final Clock clock = Clock.system();
|
||||||
|
|
||||||
|
final SourceInfo sourceInfo = new SourceInfo(connectorConfig);
|
||||||
LoggingContext.PreviousContext previousContext = taskContext.configureLoggingContext(CONTEXT_NAME);
|
LoggingContext.PreviousContext previousContext = taskContext.configureLoggingContext(CONTEXT_NAME);
|
||||||
try {
|
try {
|
||||||
//Print out the server information
|
//Print out the server information
|
||||||
SlotState slotInfo = null;
|
SlotState slotInfo = null;
|
||||||
try (PostgresConnection connection = taskContext.createConnection()) {
|
try (PostgresConnection connection = taskContext.createConnection()) {
|
||||||
if (logger.isInfoEnabled()) {
|
if (LOGGER.isInfoEnabled()) {
|
||||||
logger.info(connection.serverInfo().toString());
|
LOGGER.info(connection.serverInfo().toString());
|
||||||
}
|
}
|
||||||
slotInfo = connection.getReplicationSlotState(connectorConfig.slotName(), connectorConfig.plugin().getPostgresPluginName());
|
slotInfo = connection.getReplicationSlotState(connectorConfig.slotName(), connectorConfig.plugin().getPostgresPluginName());
|
||||||
}
|
}
|
||||||
catch (SQLException e) {
|
catch (SQLException e) {
|
||||||
logger.warn("unable to load info of replication slot, debezium will try to create the slot");
|
LOGGER.warn("unable to load info of replication slot, debezium will try to create the slot");
|
||||||
}
|
}
|
||||||
|
|
||||||
SnapshotterWrapper snapWrapper;
|
if (previousOffset == null) {
|
||||||
if (existingOffset == null) {
|
LOGGER.info("No previous offset found");
|
||||||
logger.info("No previous offset found");
|
|
||||||
// if we have no initial offset, indicate that to Snapshotter by passing null
|
// if we have no initial offset, indicate that to Snapshotter by passing null
|
||||||
snapWrapper = new SnapshotterWrapper(snapshotter, connectorConfig, null, slotInfo);
|
snapshotter.init(connectorConfig, null, slotInfo);
|
||||||
}
|
}
|
||||||
else {
|
else {
|
||||||
logger.info("Found previous offset {}", sourceInfo);
|
LOGGER.info("Found previous offset {}", sourceInfo);
|
||||||
sourceInfo.load(existingOffset);
|
snapshotter.init(connectorConfig, previousOffset.asOffsetState(), slotInfo);
|
||||||
snapWrapper = new SnapshotterWrapper(snapshotter, connectorConfig, sourceInfo.asOffsetState(), slotInfo);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
createRecordProducer(taskContext, sourceInfo, snapWrapper);
|
ReplicationConnection replicationConnection = null;
|
||||||
|
SlotCreationResult slotCreatedInfo = null;
|
||||||
|
if (snapshotter.shouldStream()) {
|
||||||
|
boolean shouldExport = snapshotter.exportSnapshot();
|
||||||
|
try {
|
||||||
|
replicationConnection = taskContext.createReplicationConnection(shouldExport);
|
||||||
|
// we need to create the slot before we start streaming if it doesn't exist
|
||||||
|
// otherwise we can't stream back changes happening while the snapshot is taking place
|
||||||
|
if (slotInfo == null) {
|
||||||
|
slotCreatedInfo = replicationConnection.createReplicationSlot().orElse(null);
|
||||||
|
}
|
||||||
|
else {
|
||||||
|
slotCreatedInfo = null;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
catch (SQLException ex) {
|
||||||
|
throw new ConnectException(ex);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
changeEventQueue = new ChangeEventQueue.Builder<ChangeEvent>()
|
queue = new ChangeEventQueue.Builder<DataChangeEvent>()
|
||||||
.pollInterval(connectorConfig.getPollInterval())
|
.pollInterval(connectorConfig.getPollInterval())
|
||||||
.maxBatchSize(connectorConfig.getMaxBatchSize())
|
.maxBatchSize(connectorConfig.getMaxBatchSize())
|
||||||
.maxQueueSize(connectorConfig.getMaxQueueSize())
|
.maxQueueSize(connectorConfig.getMaxQueueSize())
|
||||||
.loggingContextSupplier(() -> taskContext.configureLoggingContext(CONTEXT_NAME))
|
.loggingContextSupplier(() -> taskContext.configureLoggingContext(CONTEXT_NAME))
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
producer.start(changeEventQueue::enqueue, changeEventQueue::producerFailure);
|
errorHandler = new ErrorHandler(PostgresConnector.class, connectorConfig.getLogicalName(), queue, this::cleanupResources);
|
||||||
running.compareAndSet(false, true);
|
|
||||||
|
final EventDispatcher<TableId> dispatcher = new EventDispatcher<>(
|
||||||
|
connectorConfig,
|
||||||
|
topicSelector,
|
||||||
|
schema,
|
||||||
|
queue,
|
||||||
|
connectorConfig.getTableFilters().dataCollectionFilter(),
|
||||||
|
DataChangeEvent::new);
|
||||||
|
dispatcher.setInconsistentSchemaHandler(PostgresChangeRecordEmitter::updateSchema);
|
||||||
|
|
||||||
|
coordinator = new ChangeEventSourceCoordinator(
|
||||||
|
previousOffset,
|
||||||
|
errorHandler,
|
||||||
|
PostgresConnector.class,
|
||||||
|
connectorConfig.getLogicalName(),
|
||||||
|
new PostgresChangeEventSourceFactory(
|
||||||
|
connectorConfig,
|
||||||
|
snapshotter,
|
||||||
|
jdbcConnection,
|
||||||
|
errorHandler,
|
||||||
|
dispatcher,
|
||||||
|
clock,
|
||||||
|
schema,
|
||||||
|
taskContext,
|
||||||
|
replicationConnection,
|
||||||
|
slotCreatedInfo
|
||||||
|
),
|
||||||
|
dispatcher,
|
||||||
|
schema
|
||||||
|
);
|
||||||
|
|
||||||
|
coordinator.start(taskContext, this.queue, new PostgresEventMetadataProvider());
|
||||||
}
|
}
|
||||||
finally {
|
finally {
|
||||||
previousContext.restore();
|
previousContext.restore();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private void createRecordProducer(PostgresTaskContext taskContext, SourceInfo sourceInfo, SnapshotterWrapper snapshotter) {
|
|
||||||
Snapshotter snapInstance = snapshotter.getSnapshotter();
|
|
||||||
if (snapInstance.shouldSnapshot()) {
|
|
||||||
if (snapInstance.shouldStream()) {
|
|
||||||
logger.info("Taking a new snapshot of the DB and streaming logical changes once the snapshot is finished...");
|
|
||||||
producer = new RecordsSnapshotProducer(taskContext, sourceInfo, snapshotter);
|
|
||||||
}
|
|
||||||
else {
|
|
||||||
logger.info("Taking only a snapshot of the DB without streaming any changes afterwards...");
|
|
||||||
producer = new RecordsSnapshotProducer(taskContext, sourceInfo, snapshotter);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
else if (snapInstance.shouldStream()) {
|
|
||||||
logger.info("Not attempting to take a snapshot, immediately starting to stream logical changes...");
|
|
||||||
producer = new RecordsStreamProducer(taskContext, sourceInfo);
|
|
||||||
}
|
|
||||||
else {
|
|
||||||
throw new ConnectException("Snapshotter neither is snapshotting or streaming, invalid!");
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void commit() throws InterruptedException {
|
public void commit() throws InterruptedException {
|
||||||
if (running.get()) {
|
if (coordinator != null) {
|
||||||
if (lastCompletelyProcessedLsn != null) {
|
coordinator.commitOffset(lastOffset);
|
||||||
producer.commit(lastCompletelyProcessedLsn);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public List<SourceRecord> poll() throws InterruptedException {
|
public List<SourceRecord> poll() throws InterruptedException {
|
||||||
List<ChangeEvent> events = changeEventQueue.poll();
|
final List<DataChangeEvent> records = queue.poll();
|
||||||
|
|
||||||
if (events.size() > 0) {
|
final List<SourceRecord> sourceRecords = records.stream()
|
||||||
lastCompletelyProcessedLsn = events.get(events.size() - 1).getLastCompletelyProcessedLsn();
|
.map(DataChangeEvent::getRecord)
|
||||||
|
.collect(Collectors.toList());
|
||||||
|
|
||||||
|
if (!sourceRecords.isEmpty()) {
|
||||||
|
this.lastOffset = sourceRecords.get(sourceRecords.size() - 1).sourceOffset();
|
||||||
}
|
}
|
||||||
return events.stream().map(ChangeEvent::getRecord).collect(Collectors.toList());
|
|
||||||
|
return sourceRecords;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void stop() {
|
public void stop() {
|
||||||
if (running.compareAndSet(true, false)) {
|
cleanupResources();
|
||||||
producer.stop();
|
}
|
||||||
|
|
||||||
|
private void cleanupResources() {
|
||||||
|
if (!state.compareAndSet(State.RUNNING, State.STOPPED)) {
|
||||||
|
LOGGER.info("Connector has already been stopped");
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
|
try {
|
||||||
|
if (coordinator != null) {
|
||||||
|
coordinator.stop();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
catch (InterruptedException e) {
|
||||||
|
Thread.interrupted();
|
||||||
|
LOGGER.error("Interrupted while stopping coordinator", e);
|
||||||
|
throw new ConnectException("Interrupted while stopping coordinator, failing the task");
|
||||||
|
}
|
||||||
|
|
||||||
|
try {
|
||||||
|
if (errorHandler != null) {
|
||||||
|
errorHandler.stop();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
catch (InterruptedException e) {
|
||||||
|
Thread.interrupted();
|
||||||
|
LOGGER.error("Interrupted while stopping", e);
|
||||||
|
}
|
||||||
|
|
||||||
|
if (jdbcConnection != null) {
|
||||||
|
jdbcConnection.close();
|
||||||
|
}
|
||||||
|
|
||||||
|
if (schema != null) {
|
||||||
|
schema.close();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -185,4 +248,4 @@ public String version() {
|
|||||||
protected Iterable<Field> getAllConfigurationFields() {
|
protected Iterable<Field> getAllConfigurationFields() {
|
||||||
return PostgresConnectorConfig.ALL_FIELDS;
|
return PostgresConnectorConfig.ALL_FIELDS;
|
||||||
}
|
}
|
||||||
}
|
}
|
@ -0,0 +1,70 @@
|
|||||||
|
/*
|
||||||
|
* Copyright Debezium Authors.
|
||||||
|
*
|
||||||
|
* Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*/
|
||||||
|
package io.debezium.connector.postgresql;
|
||||||
|
|
||||||
|
import java.time.Instant;
|
||||||
|
import java.util.Map;
|
||||||
|
|
||||||
|
import org.apache.kafka.connect.data.Struct;
|
||||||
|
|
||||||
|
import io.debezium.data.Envelope;
|
||||||
|
import io.debezium.pipeline.source.spi.EventMetadataProvider;
|
||||||
|
import io.debezium.pipeline.spi.OffsetContext;
|
||||||
|
import io.debezium.schema.DataCollectionId;
|
||||||
|
import io.debezium.time.Conversions;
|
||||||
|
import io.debezium.util.Collect;
|
||||||
|
|
||||||
|
class PostgresEventMetadataProvider implements EventMetadataProvider {
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Instant getEventTimestamp(DataCollectionId source, OffsetContext offset, Object key, Struct value) {
|
||||||
|
if (value == null) {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
final Struct sourceInfo = value.getStruct(Envelope.FieldName.SOURCE);
|
||||||
|
if (source == null) {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
if (sourceInfo.schema().field(SourceInfo.TIMESTAMP_USEC_KEY) != null) {
|
||||||
|
final Long timestamp = sourceInfo.getInt64(SourceInfo.TIMESTAMP_USEC_KEY);
|
||||||
|
return timestamp == null ? null : Conversions.toInstantFromMicros(timestamp);
|
||||||
|
}
|
||||||
|
final Long timestamp = sourceInfo.getInt64(SourceInfo.TIMESTAMP_KEY);
|
||||||
|
return timestamp == null ? null : Conversions.toInstantFromMillis(timestamp);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Map<String, String> getEventSourcePosition(DataCollectionId source, OffsetContext offset, Object key, Struct value) {
|
||||||
|
if (value == null) {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
final Struct sourceInfo = value.getStruct(Envelope.FieldName.SOURCE);
|
||||||
|
if (source == null) {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
Long xmin = sourceInfo.getInt64(SourceInfo.XMIN_KEY);
|
||||||
|
|
||||||
|
Map<String, String> r = Collect.hashMapOf(
|
||||||
|
SourceInfo.LSN_KEY, Long.toString(sourceInfo.getInt64(SourceInfo.LSN_KEY))
|
||||||
|
);
|
||||||
|
if (xmin != null) {
|
||||||
|
r.put(SourceInfo.XMIN_KEY, Long.toString(xmin));
|
||||||
|
}
|
||||||
|
return r;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String getTransactionId(DataCollectionId source, OffsetContext offset, Object key, Struct value) {
|
||||||
|
if (value == null) {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
final Struct sourceInfo = value.getStruct(Envelope.FieldName.SOURCE);
|
||||||
|
if (source == null) {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
return Long.toString(sourceInfo.getInt64(SourceInfo.TXID_KEY));
|
||||||
|
}
|
||||||
|
}
|
@ -0,0 +1,217 @@
|
|||||||
|
/*
|
||||||
|
* Copyright Debezium Authors.
|
||||||
|
*
|
||||||
|
* Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*/
|
||||||
|
package io.debezium.connector.postgresql;
|
||||||
|
|
||||||
|
import java.sql.SQLException;
|
||||||
|
import java.time.Instant;
|
||||||
|
import java.util.Collections;
|
||||||
|
import java.util.HashMap;
|
||||||
|
import java.util.Map;
|
||||||
|
|
||||||
|
import org.apache.kafka.connect.data.Schema;
|
||||||
|
import org.apache.kafka.connect.data.Struct;
|
||||||
|
import org.apache.kafka.connect.errors.ConnectException;
|
||||||
|
import org.slf4j.Logger;
|
||||||
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
|
import io.debezium.connector.SnapshotRecord;
|
||||||
|
import io.debezium.connector.postgresql.connection.PostgresConnection;
|
||||||
|
import io.debezium.connector.postgresql.connection.ReplicationConnection;
|
||||||
|
import io.debezium.connector.postgresql.spi.OffsetState;
|
||||||
|
import io.debezium.pipeline.spi.OffsetContext;
|
||||||
|
import io.debezium.relational.TableId;
|
||||||
|
import io.debezium.time.Conversions;
|
||||||
|
import io.debezium.util.Clock;
|
||||||
|
|
||||||
|
public class PostgresOffsetContext implements OffsetContext {
|
||||||
|
private static final Logger LOGGER = LoggerFactory.getLogger(PostgresSnapshotChangeEventSource.class);
|
||||||
|
|
||||||
|
private static final String SERVER_PARTITION_KEY = "server";
|
||||||
|
public static final String LAST_COMPLETELY_PROCESSED_LSN_KEY = "lsn_proc";
|
||||||
|
|
||||||
|
private final Schema sourceInfoSchema;
|
||||||
|
private final SourceInfo sourceInfo;
|
||||||
|
private final Map<String, String> partition;
|
||||||
|
private boolean lastSnapshotRecord;
|
||||||
|
private Long lastCompletelyProcessedLsn;
|
||||||
|
|
||||||
|
private PostgresOffsetContext(PostgresConnectorConfig connectorConfig, Long lsn, Long lastCompletelyProcessedLsn, Long txId, Instant time, boolean snapshot, boolean lastSnapshotRecord) {
|
||||||
|
partition = Collections.singletonMap(SERVER_PARTITION_KEY, connectorConfig.getLogicalName());
|
||||||
|
sourceInfo = new SourceInfo(connectorConfig);
|
||||||
|
|
||||||
|
this.lastCompletelyProcessedLsn = lastCompletelyProcessedLsn;
|
||||||
|
sourceInfo.update(lsn, time, txId, null, sourceInfo.xmin());
|
||||||
|
sourceInfoSchema = sourceInfo.schema();
|
||||||
|
|
||||||
|
this.lastSnapshotRecord = lastSnapshotRecord;
|
||||||
|
if (this.lastSnapshotRecord) {
|
||||||
|
postSnapshotCompletion();
|
||||||
|
}
|
||||||
|
else {
|
||||||
|
sourceInfo.setSnapshot(snapshot ? SnapshotRecord.TRUE : SnapshotRecord.FALSE);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Map<String, ?> getPartition() {
|
||||||
|
return partition;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Map<String, ?> getOffset() {
|
||||||
|
Map<String, Object> result = new HashMap<>();
|
||||||
|
if (sourceInfo.timestamp() != null) {
|
||||||
|
result.put(SourceInfo.TIMESTAMP_USEC_KEY, Conversions.toEpochMicros(sourceInfo.timestamp()));
|
||||||
|
}
|
||||||
|
if (sourceInfo.txId() != null) {
|
||||||
|
result.put(SourceInfo.TXID_KEY, sourceInfo.txId());
|
||||||
|
}
|
||||||
|
if (sourceInfo.lsn() != null) {
|
||||||
|
result.put(SourceInfo.LSN_KEY, sourceInfo.lsn());
|
||||||
|
}
|
||||||
|
if (sourceInfo.xmin() != null) {
|
||||||
|
result.put(SourceInfo.XMIN_KEY, sourceInfo.xmin());
|
||||||
|
}
|
||||||
|
if (sourceInfo.isSnapshot()) {
|
||||||
|
result.put(SourceInfo.SNAPSHOT_KEY, true);
|
||||||
|
result.put(SourceInfo.LAST_SNAPSHOT_RECORD_KEY, lastSnapshotRecord);
|
||||||
|
}
|
||||||
|
if (lastCompletelyProcessedLsn != null) {
|
||||||
|
result.put(LAST_COMPLETELY_PROCESSED_LSN_KEY, lastCompletelyProcessedLsn);
|
||||||
|
}
|
||||||
|
return result;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Schema getSourceInfoSchema() {
|
||||||
|
return sourceInfoSchema;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Struct getSourceInfo() {
|
||||||
|
return sourceInfo.struct();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean isSnapshotRunning() {
|
||||||
|
return sourceInfo.isSnapshot();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void preSnapshotStart() {
|
||||||
|
sourceInfo.setSnapshot(SnapshotRecord.TRUE);
|
||||||
|
lastSnapshotRecord = false;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void preSnapshotCompletion() {
|
||||||
|
lastSnapshotRecord = true;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void postSnapshotCompletion() {
|
||||||
|
sourceInfo.setSnapshot(SnapshotRecord.FALSE);
|
||||||
|
}
|
||||||
|
|
||||||
|
public void updateSnapshotPosition(Instant timestamp, TableId tableId) {
|
||||||
|
sourceInfo.update(timestamp, tableId);
|
||||||
|
}
|
||||||
|
|
||||||
|
public void updateWalPosition(Long lsn, Long lastCompletelyProcessedLsn, Instant commitTime, Long txId, TableId tableId, Long xmin) {
|
||||||
|
this.lastCompletelyProcessedLsn = lastCompletelyProcessedLsn;
|
||||||
|
sourceInfo.update(lsn, commitTime, txId, tableId, xmin);
|
||||||
|
}
|
||||||
|
|
||||||
|
boolean hasLastKnownPosition() {
|
||||||
|
return sourceInfo.lsn() != null;
|
||||||
|
}
|
||||||
|
|
||||||
|
Long lsn() {
|
||||||
|
return sourceInfo.lsn();
|
||||||
|
}
|
||||||
|
|
||||||
|
Long xmin() {
|
||||||
|
return sourceInfo.xmin();
|
||||||
|
}
|
||||||
|
|
||||||
|
public static class Loader implements OffsetContext.Loader {
|
||||||
|
|
||||||
|
private final PostgresConnectorConfig connectorConfig;
|
||||||
|
|
||||||
|
public Loader(PostgresConnectorConfig connectorConfig) {
|
||||||
|
this.connectorConfig = connectorConfig;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Map<String, ?> getPartition() {
|
||||||
|
return Collections.singletonMap(SERVER_PARTITION_KEY, connectorConfig.getLogicalName());
|
||||||
|
}
|
||||||
|
|
||||||
|
private Long readOptionalLong(Map<String, ?> offset, String key) {
|
||||||
|
final Object obj = offset.get(key);
|
||||||
|
return (obj == null) ? null : ((Number) obj).longValue();
|
||||||
|
}
|
||||||
|
|
||||||
|
@SuppressWarnings("unchecked")
|
||||||
|
@Override
|
||||||
|
public OffsetContext load(Map<String, ?> offset) {
|
||||||
|
final Long lsn = readOptionalLong(offset, SourceInfo.LSN_KEY);
|
||||||
|
final Long lastCompletelyProcessedLsn = readOptionalLong(offset, LAST_COMPLETELY_PROCESSED_LSN_KEY);
|
||||||
|
final Long txId = readOptionalLong(offset, SourceInfo.TXID_KEY);
|
||||||
|
|
||||||
|
final Instant useconds = Conversions.toInstantFromMicros((Long) offset.get(SourceInfo.TIMESTAMP_USEC_KEY));
|
||||||
|
final boolean snapshot = (boolean) ((Map<String, Object>) offset).getOrDefault(SourceInfo.SNAPSHOT_KEY, Boolean.FALSE);
|
||||||
|
final boolean lastSnapshotRecord = (boolean) ((Map<String, Object>) offset).getOrDefault(SourceInfo.LAST_SNAPSHOT_RECORD_KEY, Boolean.FALSE);
|
||||||
|
return new PostgresOffsetContext(connectorConfig, lsn, lastCompletelyProcessedLsn, txId, useconds, snapshot, lastSnapshotRecord);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String toString() {
|
||||||
|
return "PostgresOffsetContext [sourceInfo=" + sourceInfo
|
||||||
|
+ ", partition=" + partition
|
||||||
|
+ ", lastSnapshotRecord=" + lastSnapshotRecord + "]";
|
||||||
|
}
|
||||||
|
|
||||||
|
public static PostgresOffsetContext initialContext(PostgresConnectorConfig connectorConfig, PostgresConnection jdbcConnection, Clock clock) {
|
||||||
|
try {
|
||||||
|
LOGGER.info("Creating initial offset context");
|
||||||
|
final long lsn = jdbcConnection.currentXLogLocation();
|
||||||
|
final long txId = jdbcConnection.currentTransactionId().longValue();
|
||||||
|
LOGGER.info("Read xlogStart at '{}' from transaction '{}'", ReplicationConnection.format(lsn), txId);
|
||||||
|
return new PostgresOffsetContext(
|
||||||
|
connectorConfig,
|
||||||
|
lsn,
|
||||||
|
null,
|
||||||
|
txId,
|
||||||
|
clock.currentTimeAsInstant(),
|
||||||
|
false,
|
||||||
|
false);
|
||||||
|
}
|
||||||
|
catch (SQLException e) {
|
||||||
|
throw new ConnectException("Database processing error", e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public OffsetState asOffsetState() {
|
||||||
|
return new OffsetState(
|
||||||
|
sourceInfo.lsn(),
|
||||||
|
sourceInfo.txId(),
|
||||||
|
sourceInfo.xmin(),
|
||||||
|
sourceInfo.timestamp(),
|
||||||
|
sourceInfo.isSnapshot());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void markLastSnapshotRecord() {
|
||||||
|
sourceInfo.setSnapshot(SnapshotRecord.LAST);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void event(TableId tableId, Instant instant) {
|
||||||
|
sourceInfo.update(instant, tableId);
|
||||||
|
}
|
||||||
|
}
|
@ -0,0 +1,251 @@
|
|||||||
|
/*
|
||||||
|
* Copyright Debezium Authors.
|
||||||
|
*
|
||||||
|
* Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*/
|
||||||
|
package io.debezium.connector.postgresql;
|
||||||
|
|
||||||
|
import java.sql.ResultSet;
|
||||||
|
import java.sql.ResultSetMetaData;
|
||||||
|
import java.sql.SQLException;
|
||||||
|
import java.time.Duration;
|
||||||
|
import java.util.Optional;
|
||||||
|
import java.util.Set;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
|
import org.postgresql.util.PGmoney;
|
||||||
|
import org.slf4j.Logger;
|
||||||
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
|
import io.debezium.connector.postgresql.connection.PostgresConnection;
|
||||||
|
import io.debezium.connector.postgresql.connection.ReplicationConnection;
|
||||||
|
import io.debezium.connector.postgresql.spi.SlotCreationResult;
|
||||||
|
import io.debezium.connector.postgresql.spi.Snapshotter;
|
||||||
|
import io.debezium.data.SpecialValueDecimal;
|
||||||
|
import io.debezium.pipeline.EventDispatcher;
|
||||||
|
import io.debezium.pipeline.source.spi.SnapshotProgressListener;
|
||||||
|
import io.debezium.pipeline.spi.OffsetContext;
|
||||||
|
import io.debezium.relational.Column;
|
||||||
|
import io.debezium.relational.RelationalSnapshotChangeEventSource;
|
||||||
|
import io.debezium.relational.Table;
|
||||||
|
import io.debezium.relational.TableId;
|
||||||
|
import io.debezium.schema.SchemaChangeEvent;
|
||||||
|
import io.debezium.schema.SchemaChangeEvent.SchemaChangeEventType;
|
||||||
|
import io.debezium.util.Clock;
|
||||||
|
|
||||||
|
public class PostgresSnapshotChangeEventSource extends RelationalSnapshotChangeEventSource {
|
||||||
|
|
||||||
|
private static final Logger LOGGER = LoggerFactory.getLogger(PostgresSnapshotChangeEventSource.class);
|
||||||
|
|
||||||
|
private final PostgresConnectorConfig connectorConfig;
|
||||||
|
private final PostgresConnection jdbcConnection;
|
||||||
|
private final PostgresSchema schema;
|
||||||
|
private final Snapshotter snapshotter;
|
||||||
|
private final SlotCreationResult slotCreatedInfo;
|
||||||
|
|
||||||
|
public PostgresSnapshotChangeEventSource(PostgresConnectorConfig connectorConfig, Snapshotter snapshotter, PostgresOffsetContext previousOffset, PostgresConnection jdbcConnection, PostgresSchema schema, EventDispatcher<TableId> dispatcher, Clock clock, SnapshotProgressListener snapshotProgressListener, SlotCreationResult slotCreatedInfo) {
|
||||||
|
super(connectorConfig, previousOffset, jdbcConnection, dispatcher, clock, snapshotProgressListener);
|
||||||
|
this.connectorConfig = connectorConfig;
|
||||||
|
this.jdbcConnection = jdbcConnection;
|
||||||
|
this.schema = schema;
|
||||||
|
this.snapshotter = snapshotter;
|
||||||
|
this.slotCreatedInfo = slotCreatedInfo;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected SnapshottingTask getSnapshottingTask(OffsetContext previousOffset) {
|
||||||
|
boolean snapshotSchema = true;
|
||||||
|
boolean snapshotData = true;
|
||||||
|
|
||||||
|
snapshotData = snapshotter.shouldSnapshot();
|
||||||
|
if (snapshotData) {
|
||||||
|
LOGGER.info("According to the connector configuration data will be snapshotted");
|
||||||
|
}
|
||||||
|
else {
|
||||||
|
LOGGER.info("According to the connector configuration no snapshot will be executed");
|
||||||
|
snapshotSchema = false;
|
||||||
|
}
|
||||||
|
|
||||||
|
return new SnapshottingTask(snapshotSchema, snapshotData);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected SnapshotContext prepare(ChangeEventSourceContext context) throws Exception {
|
||||||
|
return new PostgresSnapshotContext(connectorConfig.databaseName());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void connectionCreated(SnapshotContext snapshotContext) throws Exception {
|
||||||
|
LOGGER.info("Setting isolation level");
|
||||||
|
String transactionStatement = snapshotter.snapshotTransactionIsolationLevelStatement(slotCreatedInfo);
|
||||||
|
LOGGER.info("Opening transaction with statement {}", transactionStatement);
|
||||||
|
jdbcConnection.executeWithoutCommitting(transactionStatement);
|
||||||
|
schema.refresh(jdbcConnection, false);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Set<TableId> getAllTableIds(SnapshotContext ctx) throws Exception {
|
||||||
|
return jdbcConnection.readTableNames(ctx.catalogName, null, null, new String[] {"TABLE"});
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void lockTablesForSchemaSnapshot(ChangeEventSourceContext sourceContext, SnapshotContext snapshotContext) throws SQLException, InterruptedException {
|
||||||
|
final Duration lockTimeout = connectorConfig.snapshotLockTimeout();
|
||||||
|
final Optional<String> lockStatement = snapshotter.snapshotTableLockingStatement(lockTimeout, schema.tableIds());
|
||||||
|
|
||||||
|
if (lockStatement.isPresent()) {
|
||||||
|
LOGGER.info("Waiting a maximum of '{}' seconds for each table lock", lockTimeout.getSeconds());
|
||||||
|
jdbcConnection.executeWithoutCommitting(lockStatement.get());
|
||||||
|
//now that we have the locks, refresh the schema
|
||||||
|
schema.refresh(jdbcConnection, false);
|
||||||
|
}
|
||||||
|
else {
|
||||||
|
// if we are not in an exported snapshot, this may result in some inconsistencies.
|
||||||
|
// Let the user know
|
||||||
|
if (!snapshotter.exportSnapshot()) {
|
||||||
|
LOGGER.warn("Step 2: skipping locking each table, this may result in inconsistent schema!");
|
||||||
|
}
|
||||||
|
else {
|
||||||
|
LOGGER.info("Step 2: skipping locking each table in an exported snapshot");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void releaseSchemaSnapshotLocks(SnapshotContext snapshotContext) throws SQLException {
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void determineSnapshotOffset(SnapshotContext ctx) throws Exception {
|
||||||
|
PostgresOffsetContext offset = (PostgresOffsetContext) ctx.offset;
|
||||||
|
final long xlogStart = getTransactionStartLsn();
|
||||||
|
final long txId = jdbcConnection.currentTransactionId().longValue();
|
||||||
|
LOGGER.info("Read xlogStart at '{}' from transaction '{}'", ReplicationConnection.format(xlogStart), txId);
|
||||||
|
if (offset == null) {
|
||||||
|
offset = PostgresOffsetContext.initialContext(connectorConfig, jdbcConnection, getClock());
|
||||||
|
ctx.offset = offset;
|
||||||
|
}
|
||||||
|
|
||||||
|
// use the old xmin, as we don't want to update it if in xmin recovery
|
||||||
|
offset.updateWalPosition(xlogStart, null, clock.currentTime(), txId, null, offset.xmin());
|
||||||
|
}
|
||||||
|
|
||||||
|
private long getTransactionStartLsn() throws SQLException {
|
||||||
|
if (snapshotter.exportSnapshot() && slotCreatedInfo != null) {
|
||||||
|
// When performing an exported snapshot based on a newly created replication slot, the txLogStart position
|
||||||
|
// should be based on the replication slot snapshot transaction point. This is crucial so that if any
|
||||||
|
// SQL operations occur mid-snapshot that they'll be properly captured when streaming begins; otherwise
|
||||||
|
// they'll be lost.
|
||||||
|
return slotCreatedInfo.startLsn();
|
||||||
|
}
|
||||||
|
return jdbcConnection.currentXLogLocation();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void readTableStructure(ChangeEventSourceContext sourceContext, SnapshotContext snapshotContext) throws SQLException, InterruptedException {
|
||||||
|
Set<String> schemas = snapshotContext.capturedTables.stream()
|
||||||
|
.map(TableId::schema)
|
||||||
|
.collect(Collectors.toSet());
|
||||||
|
|
||||||
|
// reading info only for the schemas we're interested in as per the set of captured tables;
|
||||||
|
// while the passed table name filter alone would skip all non-included tables, reading the schema
|
||||||
|
// would take much longer that way
|
||||||
|
for (String schema : schemas) {
|
||||||
|
if (!sourceContext.isRunning()) {
|
||||||
|
throw new InterruptedException("Interrupted while reading structure of schema " + schema);
|
||||||
|
}
|
||||||
|
|
||||||
|
LOGGER.info("Reading structure of schema '{}'", snapshotContext.catalogName);
|
||||||
|
jdbcConnection.readSchema(
|
||||||
|
snapshotContext.tables,
|
||||||
|
snapshotContext.catalogName,
|
||||||
|
schema,
|
||||||
|
connectorConfig.getTableFilters().dataCollectionFilter(),
|
||||||
|
null,
|
||||||
|
false
|
||||||
|
);
|
||||||
|
}
|
||||||
|
schema.refresh(jdbcConnection, false);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected SchemaChangeEvent getCreateTableEvent(SnapshotContext snapshotContext, Table table) throws SQLException {
|
||||||
|
return new SchemaChangeEvent(
|
||||||
|
snapshotContext.offset.getPartition(),
|
||||||
|
snapshotContext.offset.getOffset(),
|
||||||
|
snapshotContext.catalogName,
|
||||||
|
table.id().schema(),
|
||||||
|
null,
|
||||||
|
table,
|
||||||
|
SchemaChangeEventType.CREATE,
|
||||||
|
true);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void complete(SnapshotContext snapshotContext) {
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Optional<String> getSnapshotSelect(SnapshotContext snapshotContext, TableId tableId) {
|
||||||
|
return snapshotter.buildSnapshotQuery(tableId);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Object getColumnValue(ResultSet rs, int columnIndex, Column column) throws SQLException {
|
||||||
|
try {
|
||||||
|
final ResultSetMetaData metaData = rs.getMetaData();
|
||||||
|
final String columnTypeName = metaData.getColumnTypeName(columnIndex);
|
||||||
|
final PostgresType type = schema.getTypeRegistry().get(columnTypeName);
|
||||||
|
|
||||||
|
LOGGER.trace("Type of incoming data is: {}", type.getOid());
|
||||||
|
LOGGER.trace("ColumnTypeName is: {}", columnTypeName);
|
||||||
|
LOGGER.trace("Type is: {}", type);
|
||||||
|
|
||||||
|
if (type.isArrayType()) {
|
||||||
|
return rs.getArray(columnIndex);
|
||||||
|
}
|
||||||
|
|
||||||
|
switch (type.getOid()) {
|
||||||
|
case PgOid.MONEY:
|
||||||
|
//TODO author=Horia Chiorean date=14/11/2016 description=workaround for https://github.com/pgjdbc/pgjdbc/issues/100
|
||||||
|
return new PGmoney(rs.getString(columnIndex)).val;
|
||||||
|
case PgOid.BIT:
|
||||||
|
return rs.getString(columnIndex);
|
||||||
|
case PgOid.NUMERIC:
|
||||||
|
final String s = rs.getString(columnIndex);
|
||||||
|
if (s == null) {
|
||||||
|
return s;
|
||||||
|
}
|
||||||
|
|
||||||
|
Optional<SpecialValueDecimal> value = PostgresValueConverter.toSpecialValue(s);
|
||||||
|
return value.isPresent() ? value.get() : new SpecialValueDecimal(rs.getBigDecimal(columnIndex));
|
||||||
|
case PgOid.TIME:
|
||||||
|
// To handle time 24:00:00 supported by TIME columns, read the column as a string.
|
||||||
|
case PgOid.TIMETZ:
|
||||||
|
// In order to guarantee that we resolve TIMETZ columns with proper microsecond precision,
|
||||||
|
// read the column as a string instead and then re-parse inside the converter.
|
||||||
|
return rs.getString(columnIndex);
|
||||||
|
default:
|
||||||
|
Object x = rs.getObject(columnIndex);
|
||||||
|
if(x != null) {
|
||||||
|
LOGGER.trace("rs getobject returns class: {}; rs getObject value is: {}", x.getClass(), x);
|
||||||
|
}
|
||||||
|
return x;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
catch (SQLException e) {
|
||||||
|
// not a known type
|
||||||
|
return super.getColumnValue(rs, columnIndex, column);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Mutable context which is populated in the course of snapshotting.
|
||||||
|
*/
|
||||||
|
private static class PostgresSnapshotContext extends SnapshotContext {
|
||||||
|
|
||||||
|
public PostgresSnapshotContext(String catalogName) throws SQLException {
|
||||||
|
super(catalogName);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
@ -41,17 +41,15 @@ public Struct struct(SourceInfo sourceInfo) {
|
|||||||
Struct result = super.commonStruct(sourceInfo);
|
Struct result = super.commonStruct(sourceInfo);
|
||||||
result.put(SourceInfo.SCHEMA_NAME_KEY, sourceInfo.schemaName());
|
result.put(SourceInfo.SCHEMA_NAME_KEY, sourceInfo.schemaName());
|
||||||
result.put(SourceInfo.TABLE_NAME_KEY, sourceInfo.tableName());
|
result.put(SourceInfo.TABLE_NAME_KEY, sourceInfo.tableName());
|
||||||
// use the offset information without the snapshot part and usec part
|
if (sourceInfo.txId() != null) {
|
||||||
sourceInfo.offset().forEach((k, v) -> {
|
result.put(SourceInfo.TXID_KEY, sourceInfo.txId());
|
||||||
if (SourceInfo.TIMESTAMP_USEC_KEY.equals(k) ||
|
}
|
||||||
SourceInfo.LAST_SNAPSHOT_RECORD_KEY.equals(k) ||
|
if (sourceInfo.lsn() != null) {
|
||||||
SourceInfo.SNAPSHOT_KEY.equals(k)) {
|
result.put(SourceInfo.LSN_KEY, sourceInfo.lsn());
|
||||||
return;
|
}
|
||||||
}
|
if (sourceInfo.xmin() != null) {
|
||||||
else {
|
result.put(SourceInfo.XMIN_KEY, sourceInfo.xmin());
|
||||||
result.put(k, v);
|
}
|
||||||
}
|
|
||||||
});
|
|
||||||
return result;
|
return result;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -3,21 +3,15 @@
|
|||||||
*
|
*
|
||||||
* Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0
|
* Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0
|
||||||
*/
|
*/
|
||||||
|
|
||||||
package io.debezium.connector.postgresql;
|
package io.debezium.connector.postgresql;
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
import java.sql.SQLException;
|
import java.sql.SQLException;
|
||||||
import java.time.Instant;
|
import java.time.Instant;
|
||||||
import java.util.Iterator;
|
import java.util.Iterator;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Objects;
|
import java.util.Objects;
|
||||||
import java.util.concurrent.ExecutorService;
|
|
||||||
import java.util.concurrent.atomic.AtomicBoolean;
|
|
||||||
import java.util.concurrent.atomic.AtomicLong;
|
|
||||||
import java.util.concurrent.atomic.AtomicReference;
|
import java.util.concurrent.atomic.AtomicReference;
|
||||||
import java.util.function.Consumer;
|
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
import org.apache.kafka.connect.data.Schema;
|
import org.apache.kafka.connect.data.Schema;
|
||||||
@ -26,279 +20,172 @@
|
|||||||
import org.apache.kafka.connect.source.SourceRecord;
|
import org.apache.kafka.connect.source.SourceRecord;
|
||||||
import org.postgresql.jdbc.PgConnection;
|
import org.postgresql.jdbc.PgConnection;
|
||||||
import org.postgresql.replication.LogSequenceNumber;
|
import org.postgresql.replication.LogSequenceNumber;
|
||||||
|
import org.slf4j.Logger;
|
||||||
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
import io.debezium.annotation.ThreadSafe;
|
|
||||||
import io.debezium.connector.postgresql.connection.PostgresConnection;
|
import io.debezium.connector.postgresql.connection.PostgresConnection;
|
||||||
import io.debezium.connector.postgresql.connection.ReplicationConnection;
|
import io.debezium.connector.postgresql.connection.ReplicationConnection;
|
||||||
import io.debezium.connector.postgresql.connection.ReplicationMessage;
|
import io.debezium.connector.postgresql.connection.ReplicationMessage;
|
||||||
import io.debezium.connector.postgresql.connection.ReplicationStream;
|
import io.debezium.connector.postgresql.connection.ReplicationStream;
|
||||||
|
import io.debezium.connector.postgresql.spi.Snapshotter;
|
||||||
import io.debezium.data.Envelope;
|
import io.debezium.data.Envelope;
|
||||||
import io.debezium.function.BlockingConsumer;
|
import io.debezium.function.BlockingConsumer;
|
||||||
import io.debezium.function.Predicates;
|
import io.debezium.pipeline.ErrorHandler;
|
||||||
import io.debezium.heartbeat.Heartbeat;
|
import io.debezium.pipeline.EventDispatcher;
|
||||||
|
import io.debezium.pipeline.source.spi.StreamingChangeEventSource;
|
||||||
import io.debezium.relational.Column;
|
import io.debezium.relational.Column;
|
||||||
import io.debezium.relational.ColumnEditor;
|
import io.debezium.relational.ColumnEditor;
|
||||||
import io.debezium.relational.Table;
|
import io.debezium.relational.Table;
|
||||||
import io.debezium.relational.TableEditor;
|
import io.debezium.relational.TableEditor;
|
||||||
import io.debezium.relational.TableId;
|
import io.debezium.relational.TableId;
|
||||||
import io.debezium.relational.TableSchema;
|
import io.debezium.relational.TableSchema;
|
||||||
|
import io.debezium.schema.TopicSelector;
|
||||||
import io.debezium.util.Clock;
|
import io.debezium.util.Clock;
|
||||||
import io.debezium.util.LoggingContext;
|
|
||||||
import io.debezium.util.Metronome;
|
import io.debezium.util.Metronome;
|
||||||
import io.debezium.util.Strings;
|
import io.debezium.util.Strings;
|
||||||
import io.debezium.util.Threads;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* A {@link RecordsProducer} which creates {@link SourceRecord records} from a
|
|
||||||
* Postgres streaming replication connection and {@link ReplicationMessage
|
|
||||||
* messages}.
|
|
||||||
* <p>
|
|
||||||
* See the <a href=
|
|
||||||
* "https://jdbc.postgresql.org/documentation/head/replication.html">Physical
|
|
||||||
* and Logical replication API</a> to learn more about the underlying Postgres
|
|
||||||
* APIs.
|
|
||||||
*
|
*
|
||||||
* @author Horia Chiorean (hchiorea@redhat.com)
|
* @author Horia Chiorean (hchiorea@redhat.com), Jiri Pechanec
|
||||||
*/
|
*/
|
||||||
@ThreadSafe
|
public class PostgresStreamingChangeEventSource implements StreamingChangeEventSource {
|
||||||
public class RecordsStreamProducer extends RecordsProducer {
|
|
||||||
|
|
||||||
private static final String CONTEXT_NAME = "records-stream-producer";
|
private static final Logger LOGGER = LoggerFactory.getLogger(PostgresStreamingChangeEventSource.class);
|
||||||
|
|
||||||
private final ExecutorService executorService;
|
private final PostgresConnection connection;
|
||||||
private ReplicationConnection replicationConnection;
|
private final EventDispatcher<TableId> dispatcher;
|
||||||
private final AtomicReference<ReplicationStream> replicationStream;
|
private final ErrorHandler errorHandler;
|
||||||
private final AtomicBoolean cleanupExecuted = new AtomicBoolean();
|
private final Clock clock;
|
||||||
private final PostgresConnection metadataConnection;
|
private final PostgresSchema schema;
|
||||||
|
private final PostgresOffsetContext offsetContext;
|
||||||
|
private final PostgresConnectorConfig connectorConfig;
|
||||||
|
private final PostgresTaskContext taskContext;
|
||||||
|
private final ReplicationConnection replicationConnection;
|
||||||
|
private final AtomicReference<ReplicationStream> replicationStream = new AtomicReference<>();
|
||||||
private Long lastCompletelyProcessedLsn;
|
private Long lastCompletelyProcessedLsn;
|
||||||
|
private final Snapshotter snapshotter;
|
||||||
/**
|
|
||||||
* Serves as a message box between Kafka Connect main loop thread and stream producer thread.
|
|
||||||
* Kafka Connect thread sends the message with the value of LSN of the last transaction that was
|
|
||||||
* fully sent to Kafka and appropriate offsets committed.
|
|
||||||
* <p>
|
|
||||||
* Stream producer thread receives the LSN that is certain that was delivered to Kafka topic
|
|
||||||
* and flushes the LSN to PostgreSQL database server so the associated WAL segment can be released.
|
|
||||||
*/
|
|
||||||
private final AtomicLong lastCommittedLsn = new AtomicLong(-1);
|
|
||||||
private final Metronome pauseNoMessage;
|
private final Metronome pauseNoMessage;
|
||||||
|
|
||||||
private final Heartbeat heartbeat;
|
public PostgresStreamingChangeEventSource(PostgresConnectorConfig connectorConfig, Snapshotter snapshotter, PostgresOffsetContext offsetContext, PostgresConnection connection, EventDispatcher<TableId> dispatcher, ErrorHandler errorHandler, Clock clock, PostgresSchema schema, PostgresTaskContext taskContext, ReplicationConnection replicationConnection) {
|
||||||
|
this.connectorConfig = connectorConfig;
|
||||||
@FunctionalInterface
|
this.connection = connection;
|
||||||
public static interface PgConnectionSupplier {
|
this.dispatcher = dispatcher;
|
||||||
PgConnection get() throws SQLException;
|
this.errorHandler = errorHandler;
|
||||||
}
|
this.clock = clock;
|
||||||
|
this.schema = schema;
|
||||||
/**
|
this.offsetContext = (offsetContext != null) ? offsetContext :
|
||||||
* Creates new producer instance for the given task context
|
PostgresOffsetContext.initialContext(connectorConfig, connection, clock);
|
||||||
*
|
|
||||||
* @param taskContext a {@link PostgresTaskContext}, never null
|
|
||||||
* @param sourceInfo a {@link SourceInfo} instance to track stored offsets
|
|
||||||
* @param replicationConnection a {@link ReplicationConnection} that is used to perform actual replication
|
|
||||||
*/
|
|
||||||
public RecordsStreamProducer(PostgresTaskContext taskContext,
|
|
||||||
SourceInfo sourceInfo,
|
|
||||||
ReplicationConnection replicationConnection) {
|
|
||||||
super(taskContext, sourceInfo);
|
|
||||||
executorService = Threads.newSingleThreadExecutor(PostgresConnector.class, taskContext.config().getLogicalName(), CONTEXT_NAME);
|
|
||||||
this.replicationStream = new AtomicReference<>();
|
|
||||||
this.replicationConnection = replicationConnection;
|
|
||||||
|
|
||||||
heartbeat = Heartbeat.create(taskContext.config().getConfig(), taskContext.topicSelector().getHeartbeatTopic(),
|
|
||||||
taskContext.config().getLogicalName());
|
|
||||||
pauseNoMessage = Metronome.sleeper(taskContext.getConfig().getPollInterval(), Clock.SYSTEM);
|
pauseNoMessage = Metronome.sleeper(taskContext.getConfig().getPollInterval(), Clock.SYSTEM);
|
||||||
metadataConnection = taskContext.createConnection();
|
this.taskContext = taskContext;
|
||||||
}
|
this.snapshotter = snapshotter;
|
||||||
|
this.replicationConnection = replicationConnection;
|
||||||
// this maybe should only be used for testing?
|
|
||||||
public RecordsStreamProducer(PostgresTaskContext taskContext,
|
|
||||||
SourceInfo sourceInfo) {
|
|
||||||
this(taskContext, sourceInfo, null);
|
|
||||||
try {
|
|
||||||
this.replicationConnection = taskContext.createReplicationConnection(false);
|
|
||||||
}
|
|
||||||
catch (SQLException e) {
|
|
||||||
throw new ConnectException(e);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected synchronized void start(BlockingConsumer<ChangeEvent> eventConsumer, Consumer<Throwable> failureConsumer) {
|
public void execute(ChangeEventSourceContext context) throws InterruptedException {
|
||||||
LoggingContext.PreviousContext previousContext = taskContext.configureLoggingContext(CONTEXT_NAME);
|
if (!snapshotter.shouldStream()) {
|
||||||
|
LOGGER.info("Streaming is not enabled in currect configuration");
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
try {
|
try {
|
||||||
if (executorService.isShutdown()) {
|
if (offsetContext.hasLastKnownPosition()) {
|
||||||
logger.info("Streaming will not start, stop already requested");
|
|
||||||
return;
|
|
||||||
}
|
|
||||||
if (sourceInfo.hasLastKnownPosition()) {
|
|
||||||
// start streaming from the last recorded position in the offset
|
// start streaming from the last recorded position in the offset
|
||||||
Long lsn = sourceInfo.lsn();
|
Long lsn = offsetContext.lsn();
|
||||||
if (logger.isDebugEnabled()) {
|
if (LOGGER.isDebugEnabled()) {
|
||||||
logger.debug("retrieved latest position from stored offset '{}'", ReplicationConnection.format(lsn));
|
LOGGER.debug("retrieved latest position from stored offset '{}'", ReplicationConnection.format(lsn));
|
||||||
}
|
}
|
||||||
replicationStream.compareAndSet(null, replicationConnection.startStreaming(lsn));
|
replicationStream.compareAndSet(null, replicationConnection.startStreaming(lsn));
|
||||||
} else {
|
} else {
|
||||||
logger.info("no previous LSN found in Kafka, streaming from the latest xlogpos or flushed LSN...");
|
LOGGER.info("no previous LSN found in Kafka, streaming from the latest xlogpos or flushed LSN...");
|
||||||
replicationStream.compareAndSet(null, replicationConnection.startStreaming());
|
replicationStream.compareAndSet(null, replicationConnection.startStreaming());
|
||||||
}
|
}
|
||||||
|
|
||||||
// for large databases with many tables, we can timeout the slot while refreshing schema
|
|
||||||
// so we need to start a background thread that just responds to keep alive
|
|
||||||
replicationStream.get().startKeepAlive(Threads.newSingleThreadExecutor(PostgresConnector.class, taskContext.config().getLogicalName(), CONTEXT_NAME + "-keep-alive"));
|
|
||||||
// refresh the schema so we have a latest view of the DB tables
|
// refresh the schema so we have a latest view of the DB tables
|
||||||
taskContext.refreshSchema(metadataConnection, true);
|
taskContext.refreshSchema(connection, true);
|
||||||
taskContext.schema().assureNonEmptySchema();
|
|
||||||
|
|
||||||
this.lastCompletelyProcessedLsn = sourceInfo.lsn();
|
this.lastCompletelyProcessedLsn = offsetContext.lsn();
|
||||||
|
|
||||||
// the new thread will inherit it's parent MDC
|
final ReplicationStream stream = this.replicationStream.get();
|
||||||
executorService.submit(() -> streamChanges(eventConsumer, failureConsumer));
|
while (context.isRunning()) {
|
||||||
} catch (Throwable t) {
|
stream.readPending(message -> {
|
||||||
throw new ConnectException(t.getCause() != null ? t.getCause() : t);
|
final Long lsn = stream.lastReceivedLsn();
|
||||||
} finally {
|
if (message == null) {
|
||||||
previousContext.restore();
|
LOGGER.trace("Received empty message");
|
||||||
}
|
lastCompletelyProcessedLsn = lsn;
|
||||||
}
|
pauseNoMessage.pause();
|
||||||
|
return;
|
||||||
private void streamChanges(BlockingConsumer<ChangeEvent> consumer, Consumer<Throwable> failureConsumer) {
|
|
||||||
ReplicationStream stream = this.replicationStream.get();
|
|
||||||
// once we are streaming changes, we stop the keep alive, as the read loop
|
|
||||||
// will ensure that happens
|
|
||||||
stream.stopKeepAlive();
|
|
||||||
// run while we haven't been requested to stop
|
|
||||||
while (!Thread.currentThread().isInterrupted()) {
|
|
||||||
try {
|
|
||||||
flushLatestCommittedLsn(stream);
|
|
||||||
// this will block until a message is available
|
|
||||||
if (!stream.readPending(x -> process(x, stream.lastReceivedLsn(), consumer))) {
|
|
||||||
if (lastCompletelyProcessedLsn != null) {
|
|
||||||
heartbeat.heartbeat(sourceInfo.partition(), sourceInfo.offset(),
|
|
||||||
r -> consumer.accept(new ChangeEvent(r, lastCompletelyProcessedLsn)));
|
|
||||||
}
|
}
|
||||||
pauseNoMessage.pause();
|
if (message.isLastEventForLsn()) {
|
||||||
}
|
lastCompletelyProcessedLsn = lsn;
|
||||||
}
|
}
|
||||||
catch (SQLException e) {
|
|
||||||
Throwable cause = e.getCause();
|
final TableId tableId = PostgresSchema.parse(message.getTable());
|
||||||
if (cause != null && (cause instanceof IOException)) {
|
Objects.requireNonNull(tableId);
|
||||||
//TODO author=Horia Chiorean date=08/11/2016 description=this is because we can't safely close the stream atm
|
|
||||||
logger.warn("Closing replication stream due to db connection IO exception...");
|
offsetContext.updateWalPosition(lsn, lastCompletelyProcessedLsn, message.getCommitTime(), message.getTransactionId(), tableId, taskContext.getSlotXmin(connection));
|
||||||
} else {
|
dispatcher
|
||||||
logger.error("unexpected exception while streaming logical changes", e);
|
.dispatchDataChangeEvent(
|
||||||
}
|
tableId,
|
||||||
failureConsumer.accept(e);
|
new PostgresChangeRecordEmitter(
|
||||||
throw new ConnectException(e);
|
offsetContext,
|
||||||
}
|
clock,
|
||||||
catch (InterruptedException e) {
|
connectorConfig,
|
||||||
logger.info("Interrupted from sleep, producer termination requested");
|
schema,
|
||||||
Thread.currentThread().interrupt();
|
connection,
|
||||||
}
|
message
|
||||||
catch (Throwable e) {
|
)
|
||||||
logger.error("unexpected exception while streaming logical changes", e);
|
);
|
||||||
failureConsumer.accept(e);
|
});
|
||||||
throw new ConnectException(e);
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
catch (Throwable e) {
|
||||||
|
errorHandler.setProducerThrowable(e);
|
||||||
private void flushLatestCommittedLsn(ReplicationStream stream) throws SQLException {
|
|
||||||
final long newLsn = lastCommittedLsn.getAndSet(-1);
|
|
||||||
if (newLsn != -1) {
|
|
||||||
if (logger.isDebugEnabled()) {
|
|
||||||
logger.debug("Flushing LSN to server: {}", LogSequenceNumber.valueOf(newLsn));
|
|
||||||
}
|
|
||||||
// tell the server the point up to which we've processed data, so it can be free to recycle WAL segments
|
|
||||||
stream.flushLsn(newLsn);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
protected synchronized void commit(long lsn) {
|
|
||||||
LoggingContext.PreviousContext previousContext = taskContext.configureLoggingContext(CONTEXT_NAME);
|
|
||||||
try {
|
|
||||||
if (logger.isDebugEnabled()) {
|
|
||||||
logger.debug("Flushing of LSN '{}' requested", LogSequenceNumber.valueOf(lsn));
|
|
||||||
}
|
|
||||||
lastCommittedLsn.set(lsn);
|
|
||||||
}
|
}
|
||||||
finally {
|
finally {
|
||||||
previousContext.restore();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
protected synchronized void stop() {
|
|
||||||
LoggingContext.PreviousContext previousContext = taskContext.configureLoggingContext(CONTEXT_NAME);
|
|
||||||
|
|
||||||
try {
|
|
||||||
if (!cleanupExecuted.compareAndSet(false, true)) {
|
|
||||||
logger.debug("already stopped....");
|
|
||||||
return;
|
|
||||||
}
|
|
||||||
|
|
||||||
ReplicationStream stream = this.replicationStream.get();
|
|
||||||
// if we have a stream, ensure that it has been stopped
|
|
||||||
if (stream != null) {
|
|
||||||
try {
|
|
||||||
flushLatestCommittedLsn(stream);
|
|
||||||
}
|
|
||||||
catch (SQLException e) {
|
|
||||||
logger.error("Failed to execute the final LSN flush", e);
|
|
||||||
}
|
|
||||||
stream.stopKeepAlive();
|
|
||||||
}
|
|
||||||
|
|
||||||
closeConnections();
|
|
||||||
} finally {
|
|
||||||
replicationStream.set(null);
|
|
||||||
executorService.shutdownNow();
|
|
||||||
previousContext.restore();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
private void closeConnections() {
|
|
||||||
Exception closingException = null;
|
|
||||||
|
|
||||||
try {
|
|
||||||
if (replicationConnection != null) {
|
if (replicationConnection != null) {
|
||||||
logger.debug("stopping streaming...");
|
LOGGER.debug("stopping streaming...");
|
||||||
//TODO author=Horia Chiorean date=08/11/2016 description=Ideally we'd close the stream, but it's not reliable atm (see javadoc)
|
//TODO author=Horia Chiorean date=08/11/2016 description=Ideally we'd close the stream, but it's not reliable atm (see javadoc)
|
||||||
//replicationStream.close();
|
//replicationStream.close();
|
||||||
// close the connection - this should also disconnect the current stream even if it's blocking
|
// close the connection - this should also disconnect the current stream even if it's blocking
|
||||||
replicationConnection.close();
|
try {
|
||||||
}
|
replicationConnection.close();
|
||||||
}
|
}
|
||||||
catch(Exception e) {
|
catch (Exception e) {
|
||||||
closingException = e;
|
|
||||||
}
|
|
||||||
finally {
|
|
||||||
try {
|
|
||||||
if (metadataConnection != null) {
|
|
||||||
metadataConnection.close();
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
catch(Exception e) {
|
}
|
||||||
ConnectException rethrown = new ConnectException(e);
|
}
|
||||||
if (closingException != null) {
|
|
||||||
rethrown.addSuppressed(closingException);
|
@Override
|
||||||
|
public void commitOffset(Map<String, ?> offset) {
|
||||||
|
try {
|
||||||
|
ReplicationStream replicationStream = this.replicationStream.get();
|
||||||
|
final Long lsn = (Long) offset.get(PostgresOffsetContext.LAST_COMPLETELY_PROCESSED_LSN_KEY);
|
||||||
|
|
||||||
|
if (replicationStream != null && lsn != null) {
|
||||||
|
if (LOGGER.isDebugEnabled()) {
|
||||||
|
LOGGER.debug("Flushing LSN to server: {}", LogSequenceNumber.valueOf(lsn));
|
||||||
}
|
}
|
||||||
|
// tell the server the point up to which we've processed data, so it can be free to recycle WAL segments
|
||||||
throw rethrown;
|
replicationStream.flushLsn(lsn);
|
||||||
}
|
}
|
||||||
|
else {
|
||||||
if (closingException != null) {
|
LOGGER.debug("Streaming has already stopped, ignoring commit callback...");
|
||||||
throw new ConnectException(closingException);
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
catch (SQLException e) {
|
||||||
|
throw new ConnectException(e);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private void process(ReplicationMessage message, Long lsn, BlockingConsumer<ChangeEvent> consumer) throws SQLException, InterruptedException {
|
private void process(ReplicationMessage message, Long lsn, BlockingConsumer<ChangeEvent> consumer) throws SQLException, InterruptedException {
|
||||||
// in some cases we can get null if PG gives us back a message earlier than the latest reported flushed LSN.
|
// in some cases we can get null if PG gives us back a message earlier than the latest reported flushed LSN.
|
||||||
// WAL2JSON can also send empty changes for DDL, materialized views, etc. and the heartbeat still needs to fire.
|
// WAL2JSON can also send empty changes for DDL, materialized views, etc. and the heartbeat still needs to fire.
|
||||||
if (message == null) {
|
if (message == null) {
|
||||||
logger.trace("Received empty message");
|
LOGGER.trace("Received empty message");
|
||||||
lastCompletelyProcessedLsn = lsn;
|
lastCompletelyProcessedLsn = lsn;
|
||||||
|
// TODO heartbeat
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
if (message.isLastEventForLsn()) {
|
if (message.isLastEventForLsn()) {
|
||||||
@ -311,9 +198,9 @@ private void process(ReplicationMessage message, Long lsn, BlockingConsumer<Chan
|
|||||||
// update the source info with the coordinates for this message
|
// update the source info with the coordinates for this message
|
||||||
Instant commitTime = message.getCommitTime();
|
Instant commitTime = message.getCommitTime();
|
||||||
long txId = message.getTransactionId();
|
long txId = message.getTransactionId();
|
||||||
sourceInfo.update(lsn, commitTime, txId, tableId, taskContext.getSlotXmin(metadataConnection));
|
offsetContext.updateWalPosition(lsn, null, commitTime, txId, tableId, taskContext.getSlotXmin(connection));
|
||||||
if (logger.isDebugEnabled()) {
|
if (LOGGER.isDebugEnabled()) {
|
||||||
logger.debug("received new message at position {}\n{}", ReplicationConnection.format(lsn), message);
|
LOGGER.debug("received new message at position {}\n{}", ReplicationConnection.format(lsn), message);
|
||||||
}
|
}
|
||||||
|
|
||||||
TableSchema tableSchema = tableSchemaFor(tableId);
|
TableSchema tableSchema = tableSchemaFor(tableId);
|
||||||
@ -322,12 +209,12 @@ private void process(ReplicationMessage message, Long lsn, BlockingConsumer<Chan
|
|||||||
ReplicationMessage.Operation operation = message.getOperation();
|
ReplicationMessage.Operation operation = message.getOperation();
|
||||||
switch (operation) {
|
switch (operation) {
|
||||||
case INSERT: {
|
case INSERT: {
|
||||||
Object[] row = columnValues(message.getNewTupleList(), tableId, message.shouldSchemaBeSynchronized(), message.hasTypeMetadata());
|
Object[] row = columnValues(message.getNewTupleList(), tableId, true, message.hasTypeMetadata());
|
||||||
generateCreateRecord(tableId, row, consumer);
|
generateCreateRecord(tableId, row, consumer);
|
||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
case UPDATE: {
|
case UPDATE: {
|
||||||
Object[] newRow = columnValues(message.getNewTupleList(), tableId, message.shouldSchemaBeSynchronized(), message.hasTypeMetadata());
|
Object[] newRow = columnValues(message.getNewTupleList(), tableId, true, message.hasTypeMetadata());
|
||||||
Object[] oldRow = columnValues(message.getOldTupleList(), tableId, false, message.hasTypeMetadata());
|
Object[] oldRow = columnValues(message.getOldTupleList(), tableId, false, message.hasTypeMetadata());
|
||||||
generateUpdateRecord(tableId, oldRow, newRow, consumer);
|
generateUpdateRecord(tableId, oldRow, newRow, consumer);
|
||||||
break;
|
break;
|
||||||
@ -338,36 +225,40 @@ private void process(ReplicationMessage message, Long lsn, BlockingConsumer<Chan
|
|||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
default: {
|
default: {
|
||||||
logger.warn("unknown message operation: {}", operation);
|
LOGGER.warn("unknown message operation: {}", operation);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
if (message.isLastEventForLsn()) {
|
||||||
|
// TODO heartbeat
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
protected void generateCreateRecord(TableId tableId, Object[] rowData, BlockingConsumer<ChangeEvent> recordConsumer) throws InterruptedException {
|
protected void generateCreateRecord(TableId tableId, Object[] rowData, BlockingConsumer<ChangeEvent> recordConsumer) throws InterruptedException {
|
||||||
if (rowData == null || rowData.length == 0) {
|
if (rowData == null || rowData.length == 0) {
|
||||||
logger.warn("no new values found for table '{}' from update message at '{}'; skipping record", tableId, sourceInfo);
|
LOGGER.warn("no new values found for table '{}' from update message at '{}'; skipping record", tableId, offsetContext);
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
TableSchema tableSchema = schema().schemaFor(tableId);
|
TableSchema tableSchema = schema.schemaFor(tableId);
|
||||||
assert tableSchema != null;
|
assert tableSchema != null;
|
||||||
Object key = tableSchema.keyFromColumnData(rowData);
|
Object key = tableSchema.keyFromColumnData(rowData);
|
||||||
logger.trace("key value is: {}", key);
|
LOGGER.trace("key value is: {}", key);
|
||||||
Struct value = tableSchema.valueFromColumnData(rowData);
|
Struct value = tableSchema.valueFromColumnData(rowData);
|
||||||
if (value == null) {
|
if (value == null) {
|
||||||
logger.warn("no values found for table '{}' from create message at '{}'; skipping record", tableId, sourceInfo);
|
LOGGER.warn("no values found for table '{}' from create message at '{}'; skipping record", tableId, offsetContext);
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
Schema keySchema = tableSchema.keySchema();
|
Schema keySchema = tableSchema.keySchema();
|
||||||
Map<String, ?> partition = sourceInfo.partition();
|
Map<String, ?> partition = offsetContext.getPartition();
|
||||||
Map<String, ?> offset = sourceInfo.offset();
|
Map<String, ?> offset = offsetContext.getOffset();
|
||||||
String topicName = topicSelector().topicNameFor(tableId);
|
String topicName = topicSelector().topicNameFor(tableId);
|
||||||
Envelope envelope = tableSchema.getEnvelopeSchema();
|
Envelope envelope = tableSchema.getEnvelopeSchema();
|
||||||
|
|
||||||
SourceRecord record = new SourceRecord(partition, offset, topicName, null, keySchema, key, envelope.schema(),
|
SourceRecord record = new SourceRecord(partition, offset, topicName, null, keySchema, key, envelope.schema(),
|
||||||
envelope.create(value, sourceInfo.struct(), clock().currentTimeInMillis()));
|
envelope.create(value, offsetContext.getSourceInfo(), clock.currentTimeInMillis()));
|
||||||
if (logger.isDebugEnabled()) {
|
if (LOGGER.isDebugEnabled()) {
|
||||||
logger.debug("sending create event '{}' to topic '{}'", record, topicName);
|
LOGGER.debug("sending create event '{}' to topic '{}'", record, topicName);
|
||||||
}
|
}
|
||||||
recordConsumer.accept(new ChangeEvent(record, lastCompletelyProcessedLsn));
|
recordConsumer.accept(new ChangeEvent(record, lastCompletelyProcessedLsn));
|
||||||
}
|
}
|
||||||
@ -375,14 +266,14 @@ protected void generateCreateRecord(TableId tableId, Object[] rowData, BlockingC
|
|||||||
protected void generateUpdateRecord(TableId tableId, Object[] oldRowData, Object[] newRowData,
|
protected void generateUpdateRecord(TableId tableId, Object[] oldRowData, Object[] newRowData,
|
||||||
BlockingConsumer<ChangeEvent> recordConsumer) throws InterruptedException {
|
BlockingConsumer<ChangeEvent> recordConsumer) throws InterruptedException {
|
||||||
if (newRowData == null || newRowData.length == 0) {
|
if (newRowData == null || newRowData.length == 0) {
|
||||||
logger.warn("no values found for table '{}' from update message at '{}'; skipping record" , tableId, sourceInfo);
|
LOGGER.warn("no values found for table '{}' from update message at '{}'; skipping record" , tableId, offsetContext);
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
Schema oldKeySchema = null;
|
Schema oldKeySchema = null;
|
||||||
Struct oldValue = null;
|
Struct oldValue = null;
|
||||||
Object oldKey = null;
|
Object oldKey = null;
|
||||||
|
|
||||||
TableSchema tableSchema = schema().schemaFor(tableId);
|
TableSchema tableSchema = schema.schemaFor(tableId);
|
||||||
assert tableSchema != null;
|
assert tableSchema != null;
|
||||||
|
|
||||||
if (oldRowData != null && oldRowData.length > 0) {
|
if (oldRowData != null && oldRowData.length > 0) {
|
||||||
@ -395,11 +286,11 @@ protected void generateUpdateRecord(TableId tableId, Object[] oldRowData, Object
|
|||||||
Struct newValue = tableSchema.valueFromColumnData(newRowData);
|
Struct newValue = tableSchema.valueFromColumnData(newRowData);
|
||||||
|
|
||||||
Schema newKeySchema = tableSchema.keySchema();
|
Schema newKeySchema = tableSchema.keySchema();
|
||||||
Map<String, ?> partition = sourceInfo.partition();
|
Map<String, ?> partition = offsetContext.getPartition();
|
||||||
Map<String, ?> offset = sourceInfo.offset();
|
Map<String, ?> offset = offsetContext.getOffset();
|
||||||
String topicName = topicSelector().topicNameFor(tableId);
|
String topicName = topicSelector().topicNameFor(tableId);
|
||||||
Envelope envelope = tableSchema.getEnvelopeSchema();
|
Envelope envelope = tableSchema.getEnvelopeSchema();
|
||||||
Struct source = sourceInfo.struct();
|
Struct source = offsetContext.getSourceInfo();
|
||||||
|
|
||||||
if (oldKey != null && !Objects.equals(oldKey, newKey)) {
|
if (oldKey != null && !Objects.equals(oldKey, newKey)) {
|
||||||
// the primary key has changed, so we need to send a DELETE followed by a CREATE
|
// the primary key has changed, so we need to send a DELETE followed by a CREATE
|
||||||
@ -408,10 +299,10 @@ protected void generateUpdateRecord(TableId tableId, Object[] oldRowData, Object
|
|||||||
ChangeEvent changeEvent = new ChangeEvent(
|
ChangeEvent changeEvent = new ChangeEvent(
|
||||||
new SourceRecord(
|
new SourceRecord(
|
||||||
partition, offset, topicName, null, oldKeySchema, oldKey, envelope.schema(),
|
partition, offset, topicName, null, oldKeySchema, oldKey, envelope.schema(),
|
||||||
envelope.delete(oldValue, source, clock().currentTimeInMillis())),
|
envelope.delete(oldValue, source, clock.currentTimeInMillis())),
|
||||||
lastCompletelyProcessedLsn);
|
lastCompletelyProcessedLsn);
|
||||||
if (logger.isDebugEnabled()) {
|
if (LOGGER.isDebugEnabled()) {
|
||||||
logger.debug("sending delete event '{}' to topic '{}'", changeEvent.getRecord(), topicName);
|
LOGGER.debug("sending delete event '{}' to topic '{}'", changeEvent.getRecord(), topicName);
|
||||||
}
|
}
|
||||||
recordConsumer.accept(changeEvent);
|
recordConsumer.accept(changeEvent);
|
||||||
|
|
||||||
@ -420,8 +311,8 @@ protected void generateUpdateRecord(TableId tableId, Object[] oldRowData, Object
|
|||||||
changeEvent = new ChangeEvent(
|
changeEvent = new ChangeEvent(
|
||||||
new SourceRecord(partition, offset, topicName, null, oldKeySchema, oldKey, null, null),
|
new SourceRecord(partition, offset, topicName, null, oldKeySchema, oldKey, null, null),
|
||||||
lastCompletelyProcessedLsn);
|
lastCompletelyProcessedLsn);
|
||||||
if (logger.isDebugEnabled()) {
|
if (LOGGER.isDebugEnabled()) {
|
||||||
logger.debug("sending tombstone event '{}' to topic '{}'", changeEvent.getRecord(), topicName);
|
LOGGER.debug("sending tombstone event '{}' to topic '{}'", changeEvent.getRecord(), topicName);
|
||||||
}
|
}
|
||||||
recordConsumer.accept(changeEvent);
|
recordConsumer.accept(changeEvent);
|
||||||
}
|
}
|
||||||
@ -430,36 +321,36 @@ protected void generateUpdateRecord(TableId tableId, Object[] oldRowData, Object
|
|||||||
changeEvent = new ChangeEvent(
|
changeEvent = new ChangeEvent(
|
||||||
new SourceRecord(
|
new SourceRecord(
|
||||||
partition, offset, topicName, null, newKeySchema, newKey, envelope.schema(),
|
partition, offset, topicName, null, newKeySchema, newKey, envelope.schema(),
|
||||||
envelope.create(newValue, source, clock().currentTimeInMillis())),
|
envelope.create(newValue, source, clock.currentTimeInMillis())),
|
||||||
lastCompletelyProcessedLsn);
|
lastCompletelyProcessedLsn);
|
||||||
if (logger.isDebugEnabled()) {
|
if (LOGGER.isDebugEnabled()) {
|
||||||
logger.debug("sending create event '{}' to topic '{}'", changeEvent.getRecord(), topicName);
|
LOGGER.debug("sending create event '{}' to topic '{}'", changeEvent.getRecord(), topicName);
|
||||||
}
|
}
|
||||||
recordConsumer.accept(changeEvent);
|
recordConsumer.accept(changeEvent);
|
||||||
} else {
|
} else {
|
||||||
SourceRecord record = new SourceRecord(partition, offset, topicName, null,
|
SourceRecord record = new SourceRecord(partition, offset, topicName, null,
|
||||||
newKeySchema, newKey, envelope.schema(),
|
newKeySchema, newKey, envelope.schema(),
|
||||||
envelope.update(oldValue, newValue, source, clock().currentTimeInMillis()));
|
envelope.update(oldValue, newValue, source, clock.currentTimeInMillis()));
|
||||||
recordConsumer.accept(new ChangeEvent(record, lastCompletelyProcessedLsn));
|
recordConsumer.accept(new ChangeEvent(record, lastCompletelyProcessedLsn));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
protected void generateDeleteRecord(TableId tableId, Object[] oldRowData, BlockingConsumer<ChangeEvent> recordConsumer) throws InterruptedException {
|
protected void generateDeleteRecord(TableId tableId, Object[] oldRowData, BlockingConsumer<ChangeEvent> recordConsumer) throws InterruptedException {
|
||||||
if (oldRowData == null || oldRowData.length == 0) {
|
if (oldRowData == null || oldRowData.length == 0) {
|
||||||
logger.warn("no values found for table '{}' from delete message at '{}'; skipping record" , tableId, sourceInfo);
|
LOGGER.warn("no values found for table '{}' from delete message at '{}'; skipping record" , tableId, offsetContext);
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
TableSchema tableSchema = schema().schemaFor(tableId);
|
TableSchema tableSchema = schema.schemaFor(tableId);
|
||||||
assert tableSchema != null;
|
assert tableSchema != null;
|
||||||
Object key = tableSchema.keyFromColumnData(oldRowData);
|
Object key = tableSchema.keyFromColumnData(oldRowData);
|
||||||
Struct value = tableSchema.valueFromColumnData(oldRowData);
|
Struct value = tableSchema.valueFromColumnData(oldRowData);
|
||||||
if (value == null) {
|
if (value == null) {
|
||||||
logger.warn("ignoring delete message for table '{}' because it does not have a primary key defined and replica identity for the table is not FULL", tableId);
|
LOGGER.warn("ignoring delete message for table '{}' because it does not have a primary key defined and replica identity for the table is not FULL", tableId);
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
Schema keySchema = tableSchema.keySchema();
|
Schema keySchema = tableSchema.keySchema();
|
||||||
Map<String, ?> partition = sourceInfo.partition();
|
Map<String, ?> partition = offsetContext.getPartition();
|
||||||
Map<String, ?> offset = sourceInfo.offset();
|
Map<String, ?> offset = offsetContext.getOffset();
|
||||||
String topicName = topicSelector().topicNameFor(tableId);
|
String topicName = topicSelector().topicNameFor(tableId);
|
||||||
Envelope envelope = tableSchema.getEnvelopeSchema();
|
Envelope envelope = tableSchema.getEnvelopeSchema();
|
||||||
|
|
||||||
@ -468,10 +359,10 @@ protected void generateDeleteRecord(TableId tableId, Object[] oldRowData, Blocki
|
|||||||
new SourceRecord(
|
new SourceRecord(
|
||||||
partition, offset, topicName, null,
|
partition, offset, topicName, null,
|
||||||
keySchema, key, envelope.schema(),
|
keySchema, key, envelope.schema(),
|
||||||
envelope.delete(value, sourceInfo.struct(), clock().currentTimeInMillis())),
|
envelope.delete(value, offsetContext.getSourceInfo(), clock.currentTimeInMillis())),
|
||||||
lastCompletelyProcessedLsn);
|
lastCompletelyProcessedLsn);
|
||||||
if (logger.isDebugEnabled()) {
|
if (LOGGER.isDebugEnabled()) {
|
||||||
logger.debug("sending delete event '{}' to topic '{}'", changeEvent.getRecord(), topicName);
|
LOGGER.debug("sending delete event '{}' to topic '{}'", changeEvent.getRecord(), topicName);
|
||||||
}
|
}
|
||||||
recordConsumer.accept(changeEvent);
|
recordConsumer.accept(changeEvent);
|
||||||
|
|
||||||
@ -480,8 +371,8 @@ protected void generateDeleteRecord(TableId tableId, Object[] oldRowData, Blocki
|
|||||||
changeEvent = new ChangeEvent(
|
changeEvent = new ChangeEvent(
|
||||||
new SourceRecord(partition, offset, topicName, null, keySchema, key, null, null),
|
new SourceRecord(partition, offset, topicName, null, keySchema, key, null, null),
|
||||||
lastCompletelyProcessedLsn);
|
lastCompletelyProcessedLsn);
|
||||||
if (logger.isDebugEnabled()) {
|
if (LOGGER.isDebugEnabled()) {
|
||||||
logger.debug("sending tombstone event '{}' to topic '{}'", changeEvent.getRecord(), topicName);
|
LOGGER.debug("sending tombstone event '{}' to topic '{}'", changeEvent.getRecord(), topicName);
|
||||||
}
|
}
|
||||||
recordConsumer.accept(changeEvent);
|
recordConsumer.accept(changeEvent);
|
||||||
}
|
}
|
||||||
@ -492,40 +383,40 @@ private Object[] columnValues(List<ReplicationMessage.Column> columns, TableId t
|
|||||||
if (columns == null || columns.isEmpty()) {
|
if (columns == null || columns.isEmpty()) {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
Table table = schema().tableFor(tableId);
|
Table table = schema.tableFor(tableId);
|
||||||
assert table != null;
|
assert table != null;
|
||||||
|
|
||||||
// check if we need to refresh our local schema due to DB schema changes for this table
|
// check if we need to refresh our local schema due to DB schema changes for this table
|
||||||
if (refreshSchemaIfChanged && schemaChanged(columns, table, metadataInMessage)) {
|
if (refreshSchemaIfChanged && schemaChanged(columns, table, metadataInMessage)) {
|
||||||
// Refresh the schema so we get information about primary keys
|
try (final PostgresConnection connection = taskContext.createConnection()) {
|
||||||
schema().refresh(metadataConnection, tableId, taskContext.config().skipRefreshSchemaOnMissingToastableData());
|
// Refresh the schema so we get information about primary keys
|
||||||
// Update the schema with metadata coming from decoder message
|
schema.refresh(connection, tableId, taskContext.config().skipRefreshSchemaOnMissingToastableData());
|
||||||
if (metadataInMessage) {
|
// Update the schema with metadata coming from decoder message
|
||||||
schema().refresh(tableFromFromMessage(columns, schema().tableFor(tableId)));
|
if (metadataInMessage) {
|
||||||
|
schema.refresh(tableFromFromMessage(columns, schema.tableFor(tableId)));
|
||||||
|
}
|
||||||
|
table = schema.tableFor(tableId);
|
||||||
}
|
}
|
||||||
table = schema().tableFor(tableId);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// 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[columnsWithoutToasted.size() < schemaColumns.size() ? schemaColumns.size() : columnsWithoutToasted.size()];
|
Object[] values = new Object[columns.size() < schemaColumns.size() ? schemaColumns.size() : columns.size()];
|
||||||
|
|
||||||
for (ReplicationMessage.Column column : columnsWithoutToasted) {
|
for (ReplicationMessage.Column column : columns) {
|
||||||
//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);
|
||||||
if (tableColumn == null) {
|
if (tableColumn == null) {
|
||||||
logger.warn(
|
LOGGER.warn(
|
||||||
"Internal schema is out-of-sync with incoming decoder events; column {} will be omitted from the change event.",
|
"Internal schema is out-of-sync with incoming decoder events; column {} will be omitted from the change event.",
|
||||||
column.getName());
|
column.getName());
|
||||||
continue;
|
continue;
|
||||||
}
|
}
|
||||||
int position = tableColumn.position() - 1;
|
int position = tableColumn.position() - 1;
|
||||||
if (position < 0 || position >= values.length) {
|
if (position < 0 || position >= values.length) {
|
||||||
logger.warn(
|
LOGGER.warn(
|
||||||
"Internal schema is out-of-sync with incoming decoder events; column {} will be omitted from the change event.",
|
"Internal schema is out-of-sync with incoming decoder events; column {} will be omitted from the change event.",
|
||||||
column.getName());
|
column.getName());
|
||||||
continue;
|
continue;
|
||||||
@ -555,7 +446,7 @@ private boolean schemaChanged(List<ReplicationMessage.Column> columns, Table tab
|
|||||||
if (msgHasMissingColumns || msgHasAdditionalColumns) {
|
if (msgHasMissingColumns || msgHasAdditionalColumns) {
|
||||||
// the table metadata has less or more columns than the event, which means the table structure has changed,
|
// the table metadata has less or more columns than the event, which means the table structure has changed,
|
||||||
// so we need to trigger a refresh...
|
// so we need to trigger a refresh...
|
||||||
logger.info("Different column count {} present in the server message as schema in memory contains {}; refreshing table schema",
|
LOGGER.info("Different column count {} present in the server message as schema in memory contains {}; refreshing table schema",
|
||||||
replicationColumnCount,
|
replicationColumnCount,
|
||||||
tableColumnCount);
|
tableColumnCount);
|
||||||
return true;
|
return true;
|
||||||
@ -567,14 +458,14 @@ private boolean schemaChanged(List<ReplicationMessage.Column> columns, Table tab
|
|||||||
String columnName = message.getName();
|
String columnName = message.getName();
|
||||||
Column column = table.columnWithName(columnName);
|
Column column = table.columnWithName(columnName);
|
||||||
if (column == null) {
|
if (column == null) {
|
||||||
logger.info("found new column '{}' present in the server message which is not part of the table metadata; refreshing table schema", columnName);
|
LOGGER.info("found new column '{}' present in the server message which is not part of the table metadata; refreshing table schema", columnName);
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
else {
|
else {
|
||||||
final int localType = column.nativeType();
|
final int localType = column.nativeType();
|
||||||
final int incomingType = message.getType().getOid();
|
final int incomingType = message.getType().getOid();
|
||||||
if (localType != incomingType) {
|
if (localType != incomingType) {
|
||||||
logger.info("detected new type for column '{}', old type was {} ({}), new type is {} ({}); refreshing table schema", columnName, localType, column.typeName(),
|
LOGGER.info("detected new type for column '{}', old type was {} ({}), new type is {} ({}); refreshing table schema", columnName, localType, column.typeName(),
|
||||||
incomingType, message.getType().getName());
|
incomingType, message.getType().getName());
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
@ -582,21 +473,21 @@ private boolean schemaChanged(List<ReplicationMessage.Column> columns, Table tab
|
|||||||
final int localLength = column.length();
|
final int localLength = column.length();
|
||||||
final int incomingLength = message.getTypeMetadata().getLength();
|
final int incomingLength = message.getTypeMetadata().getLength();
|
||||||
if (localLength != incomingLength) {
|
if (localLength != incomingLength) {
|
||||||
logger.info("detected new length for column '{}', old length was {}, new length is {}; refreshing table schema", columnName, localLength,
|
LOGGER.info("detected new length for column '{}', old length was {}, new length is {}; refreshing table schema", columnName, localLength,
|
||||||
incomingLength);
|
incomingLength);
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
final int localScale = column.scale().get();
|
final int localScale = column.scale().get();
|
||||||
final int incomingScale = message.getTypeMetadata().getScale();
|
final int incomingScale = message.getTypeMetadata().getScale();
|
||||||
if (localScale != incomingScale) {
|
if (localScale != incomingScale) {
|
||||||
logger.info("detected new scale for column '{}', old scale was {}, new scale is {}; refreshing table schema", columnName, localScale,
|
LOGGER.info("detected new scale for column '{}', old scale was {}, new scale is {}; refreshing table schema", columnName, localScale,
|
||||||
incomingScale);
|
incomingScale);
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
final boolean localOptional = column.isOptional();
|
final boolean localOptional = column.isOptional();
|
||||||
final boolean incomingOptional = message.isOptional();
|
final boolean incomingOptional = message.isOptional();
|
||||||
if (localOptional != incomingOptional) {
|
if (localOptional != incomingOptional) {
|
||||||
logger.info("detected new optional status for column '{}', old value was {}, new value is {}; refreshing table schema", columnName, localOptional, incomingOptional);
|
LOGGER.info("detected new optional status for column '{}', old value was {}, new value is {}; refreshing table schema", columnName, localOptional, incomingOptional);
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -617,10 +508,10 @@ private boolean hasMissingUntoastedColumns(Table table, List<ReplicationMessage.
|
|||||||
.map(Column::name)
|
.map(Column::name)
|
||||||
.collect(Collectors.toList());
|
.collect(Collectors.toList());
|
||||||
|
|
||||||
List<String> toastableColumns = schema().getToastableColumnsForTableId(table.id());
|
List<String> toastableColumns = schema.getToastableColumnsForTableId(table.id());
|
||||||
|
|
||||||
if (logger.isDebugEnabled()) {
|
if (LOGGER.isDebugEnabled()) {
|
||||||
logger.debug("msg columns: '{}' --- missing columns: '{}' --- toastableColumns: '{}",
|
LOGGER.debug("msg columns: '{}' --- missing columns: '{}' --- toastableColumns: '{}",
|
||||||
String.join(",", msgColumnNames),
|
String.join(",", msgColumnNames),
|
||||||
String.join(",", missingColumnNames),
|
String.join(",", missingColumnNames),
|
||||||
String.join(",", toastableColumns));
|
String.join(",", toastableColumns));
|
||||||
@ -631,9 +522,8 @@ private boolean hasMissingUntoastedColumns(Table table, List<ReplicationMessage.
|
|||||||
}
|
}
|
||||||
|
|
||||||
private TableSchema tableSchemaFor(TableId tableId) throws SQLException {
|
private TableSchema tableSchemaFor(TableId tableId) throws SQLException {
|
||||||
PostgresSchema schema = schema();
|
|
||||||
if (schema.isFilteredOut(tableId)) {
|
if (schema.isFilteredOut(tableId)) {
|
||||||
logger.debug("table '{}' is filtered out, ignoring", tableId);
|
LOGGER.debug("table '{}' is filtered out, ignoring", tableId);
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
TableSchema tableSchema = schema.schemaFor(tableId);
|
TableSchema tableSchema = schema.schemaFor(tableId);
|
||||||
@ -642,19 +532,21 @@ private TableSchema tableSchemaFor(TableId tableId) throws SQLException {
|
|||||||
}
|
}
|
||||||
// we don't have a schema registered for this table, even though the filters would allow it...
|
// we don't have a schema registered for this table, even though the filters would allow it...
|
||||||
// which means that is a newly created table; so refresh our schema to get the definition for this table
|
// which means that is a newly created table; so refresh our schema to get the definition for this table
|
||||||
schema.refresh(metadataConnection, tableId, taskContext.config().skipRefreshSchemaOnMissingToastableData());
|
try (final PostgresConnection connection = taskContext.createConnection()) {
|
||||||
|
schema.refresh(connection, tableId, taskContext.config().skipRefreshSchemaOnMissingToastableData());
|
||||||
|
}
|
||||||
tableSchema = schema.schemaFor(tableId);
|
tableSchema = schema.schemaFor(tableId);
|
||||||
if (tableSchema == null) {
|
if (tableSchema == null) {
|
||||||
logger.warn("cannot load schema for table '{}'", tableId);
|
LOGGER.warn("cannot load schema for table '{}'", tableId);
|
||||||
return null;
|
return null;
|
||||||
} else {
|
} else {
|
||||||
logger.debug("refreshed DB schema to include table '{}'", tableId);
|
LOGGER.debug("refreshed DB schema to include table '{}'", tableId);
|
||||||
return tableSchema;
|
return tableSchema;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private synchronized PgConnection typeResolverConnection() throws SQLException {
|
private synchronized PgConnection typeResolverConnection() throws SQLException {
|
||||||
return (PgConnection) metadataConnection.connection();
|
return (PgConnection) connection.connection();
|
||||||
}
|
}
|
||||||
|
|
||||||
private Table tableFromFromMessage(List<ReplicationMessage.Column> columns, Table table) {
|
private Table tableFromFromMessage(List<ReplicationMessage.Column> columns, Table table) {
|
||||||
@ -679,7 +571,7 @@ private Table tableFromFromMessage(List<ReplicationMessage.Column> columns, Tabl
|
|||||||
while (itPkCandidates.hasNext()) {
|
while (itPkCandidates.hasNext()) {
|
||||||
final String candidateName = itPkCandidates.next();
|
final String candidateName = itPkCandidates.next();
|
||||||
if (!combinedTable.hasUniqueValues() && combinedTable.columnWithName(candidateName) == null) {
|
if (!combinedTable.hasUniqueValues() && combinedTable.columnWithName(candidateName) == null) {
|
||||||
logger.error("Potentional inconsistency in key for message {}", columns);
|
LOGGER.error("Potentional inconsistency in key for message {}", columns);
|
||||||
itPkCandidates.remove();
|
itPkCandidates.remove();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -687,8 +579,12 @@ private Table tableFromFromMessage(List<ReplicationMessage.Column> columns, Tabl
|
|||||||
return combinedTable.create();
|
return combinedTable.create();
|
||||||
}
|
}
|
||||||
|
|
||||||
// test-only
|
private TopicSelector<TableId> topicSelector() {
|
||||||
boolean isStreamingRunning() {
|
return taskContext.topicSelector();
|
||||||
return replicationStream.get() != null;
|
}
|
||||||
|
|
||||||
|
@FunctionalInterface
|
||||||
|
public static interface PgConnectionSupplier {
|
||||||
|
PgConnection get() throws SQLException;
|
||||||
}
|
}
|
||||||
}
|
}
|
@ -1,471 +0,0 @@
|
|||||||
/*
|
|
||||||
* Copyright Debezium Authors.
|
|
||||||
*
|
|
||||||
* Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0
|
|
||||||
*/
|
|
||||||
|
|
||||||
package io.debezium.connector.postgresql;
|
|
||||||
|
|
||||||
import java.sql.Connection;
|
|
||||||
import java.sql.ResultSet;
|
|
||||||
import java.sql.ResultSetMetaData;
|
|
||||||
import java.sql.SQLException;
|
|
||||||
import java.sql.Statement;
|
|
||||||
import java.time.Duration;
|
|
||||||
import java.util.Map;
|
|
||||||
import java.util.Optional;
|
|
||||||
import java.util.concurrent.CompletableFuture;
|
|
||||||
import java.util.concurrent.ExecutorService;
|
|
||||||
import java.util.concurrent.atomic.AtomicInteger;
|
|
||||||
import java.util.concurrent.atomic.AtomicReference;
|
|
||||||
import java.util.function.Consumer;
|
|
||||||
|
|
||||||
import org.apache.kafka.connect.data.Schema;
|
|
||||||
import org.apache.kafka.connect.data.Struct;
|
|
||||||
import org.apache.kafka.connect.errors.ConnectException;
|
|
||||||
import org.apache.kafka.connect.source.SourceRecord;
|
|
||||||
import org.postgresql.util.PGmoney;
|
|
||||||
|
|
||||||
import io.debezium.annotation.ThreadSafe;
|
|
||||||
import io.debezium.config.ConfigurationDefaults;
|
|
||||||
import io.debezium.connector.SnapshotRecord;
|
|
||||||
import io.debezium.connector.postgresql.connection.PostgresConnection;
|
|
||||||
import io.debezium.connector.postgresql.connection.ReplicationConnection;
|
|
||||||
import io.debezium.connector.postgresql.snapshot.SnapshotterWrapper;
|
|
||||||
import io.debezium.connector.postgresql.spi.SlotCreationResult;
|
|
||||||
import io.debezium.connector.postgresql.spi.Snapshotter;
|
|
||||||
import io.debezium.data.Envelope;
|
|
||||||
import io.debezium.data.SpecialValueDecimal;
|
|
||||||
import io.debezium.function.BlockingConsumer;
|
|
||||||
import io.debezium.heartbeat.Heartbeat;
|
|
||||||
import io.debezium.relational.Table;
|
|
||||||
import io.debezium.relational.TableId;
|
|
||||||
import io.debezium.relational.TableSchema;
|
|
||||||
import io.debezium.util.Clock;
|
|
||||||
import io.debezium.util.LoggingContext;
|
|
||||||
import io.debezium.util.Metronome;
|
|
||||||
import io.debezium.util.Strings;
|
|
||||||
import io.debezium.util.Threads;
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Producer of {@link org.apache.kafka.connect.source.SourceRecord source records} from a database snapshot. Once completed,
|
|
||||||
* this producer can optionally continue streaming records, using another {@link RecordsStreamProducer} instance.
|
|
||||||
*
|
|
||||||
* @author Horia Chiorean (hchiorea@redhat.com)
|
|
||||||
*/
|
|
||||||
@ThreadSafe
|
|
||||||
public class RecordsSnapshotProducer extends RecordsProducer {
|
|
||||||
|
|
||||||
private static final String CONTEXT_NAME = "records-snapshot-producer";
|
|
||||||
|
|
||||||
private final ExecutorService executorService;
|
|
||||||
private final Optional<RecordsStreamProducer> streamProducer;
|
|
||||||
|
|
||||||
private final AtomicReference<SourceRecord> currentRecord;
|
|
||||||
private final Snapshotter snapshotter;
|
|
||||||
private final SlotCreationResult slotCreatedInfo;
|
|
||||||
|
|
||||||
public RecordsSnapshotProducer(PostgresTaskContext taskContext,
|
|
||||||
SourceInfo sourceInfo,
|
|
||||||
SnapshotterWrapper snapWrapper) {
|
|
||||||
super(taskContext, sourceInfo);
|
|
||||||
executorService = Threads.newSingleThreadExecutor(PostgresConnector.class, taskContext.config().getLogicalName(), CONTEXT_NAME);
|
|
||||||
currentRecord = new AtomicReference<>();
|
|
||||||
this.snapshotter = snapWrapper.getSnapshotter();
|
|
||||||
if (snapshotter.shouldStream()) {
|
|
||||||
boolean shouldExport = snapshotter.exportSnapshot();
|
|
||||||
ReplicationConnection replConn;
|
|
||||||
try {
|
|
||||||
replConn = taskContext.createReplicationConnection(shouldExport);
|
|
||||||
// we need to create the slot before we start streaming if it doesn't exist
|
|
||||||
// otherwise we can't stream back changes happening while the snapshot is taking place
|
|
||||||
if (!snapWrapper.doesSlotExist()) {
|
|
||||||
slotCreatedInfo = replConn.createReplicationSlot().orElse(null);
|
|
||||||
}
|
|
||||||
else {
|
|
||||||
slotCreatedInfo = null;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
catch (SQLException ex) {
|
|
||||||
throw new ConnectException(ex);
|
|
||||||
}
|
|
||||||
streamProducer = Optional.of(new RecordsStreamProducer(taskContext, sourceInfo, replConn));
|
|
||||||
}
|
|
||||||
else {
|
|
||||||
streamProducer = Optional.empty();
|
|
||||||
slotCreatedInfo = null;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
protected void start(BlockingConsumer<ChangeEvent> eventConsumer, Consumer<Throwable> failureConsumer) {
|
|
||||||
// MDC should be in inherited from parent to child threads
|
|
||||||
LoggingContext.PreviousContext previousContext = taskContext.configureLoggingContext(CONTEXT_NAME);
|
|
||||||
try {
|
|
||||||
CompletableFuture.runAsync(this::delaySnapshotIfNeeded, executorService)
|
|
||||||
.thenRunAsync(() -> this.takeSnapshot(eventConsumer), executorService)
|
|
||||||
.thenRunAsync(() -> this.startStreaming(eventConsumer, failureConsumer), executorService)
|
|
||||||
.exceptionally(e -> {
|
|
||||||
logger.error("unexpected exception", e.getCause() != null ? e.getCause() : e);
|
|
||||||
// always stop to clean up data
|
|
||||||
stop();
|
|
||||||
failureConsumer.accept(e);
|
|
||||||
|
|
||||||
return null;
|
|
||||||
});
|
|
||||||
} finally {
|
|
||||||
previousContext.restore();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
private void delaySnapshotIfNeeded() {
|
|
||||||
Duration delay = taskContext.getConfig().getSnapshotDelay();
|
|
||||||
if (delay.isZero() || delay.isNegative()) {
|
|
||||||
return;
|
|
||||||
}
|
|
||||||
|
|
||||||
Threads.Timer timer = Threads.timer(Clock.SYSTEM, delay);
|
|
||||||
Metronome metronome = Metronome.parker(ConfigurationDefaults.RETURN_CONTROL_INTERVAL, Clock.SYSTEM);
|
|
||||||
|
|
||||||
while (!timer.expired()) {
|
|
||||||
try {
|
|
||||||
logger.info("The connector will wait for {}s before proceeding", timer.remaining().getSeconds());
|
|
||||||
metronome.pause();
|
|
||||||
}
|
|
||||||
catch (InterruptedException e) {
|
|
||||||
Thread.currentThread().interrupt();
|
|
||||||
logger.debug("Interrupted while awaiting initial snapshot delay");
|
|
||||||
return;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
private void startStreaming(BlockingConsumer<ChangeEvent> consumer, Consumer<Throwable> failureConsumer) {
|
|
||||||
try {
|
|
||||||
// and then start streaming if necessary
|
|
||||||
streamProducer.ifPresent(producer -> {
|
|
||||||
if (sourceInfo.lsn() != null) {
|
|
||||||
if (logger.isInfoEnabled()) {
|
|
||||||
logger.info("Snapshot finished, continuing streaming changes from {}", ReplicationConnection.format(sourceInfo.lsn()));
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
// still starting the stream producer, also if the connector has stopped already.
|
|
||||||
// otherwise the executor started in its constructor wouldn't be stopped. This logic
|
|
||||||
// will be obsolete when moving to the new framework classes.
|
|
||||||
producer.start(consumer, failureConsumer);
|
|
||||||
});
|
|
||||||
} finally {
|
|
||||||
// always cleanup our local data
|
|
||||||
cleanup();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
protected void commit(long lsn) {
|
|
||||||
streamProducer.ifPresent(x -> x.commit(lsn));
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
protected void stop() {
|
|
||||||
try {
|
|
||||||
streamProducer.ifPresent(RecordsStreamProducer::stop);
|
|
||||||
} finally {
|
|
||||||
cleanup();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
private void cleanup() {
|
|
||||||
currentRecord.set(null);
|
|
||||||
executorService.shutdownNow();
|
|
||||||
}
|
|
||||||
|
|
||||||
private void takeSnapshot(BlockingConsumer<ChangeEvent> consumer) {
|
|
||||||
if (executorService.isShutdown()) {
|
|
||||||
logger.info("Not taking snapshot as this task has been cancelled already");
|
|
||||||
return;
|
|
||||||
}
|
|
||||||
|
|
||||||
long snapshotStart = clock().currentTimeInMillis();
|
|
||||||
Connection jdbcConnection = null;
|
|
||||||
try (PostgresConnection connection = taskContext.createConnection()) {
|
|
||||||
jdbcConnection = connection.connection();
|
|
||||||
|
|
||||||
logger.info("Step 0: disabling autocommit");
|
|
||||||
connection.setAutoCommit(false);
|
|
||||||
|
|
||||||
Duration lockTimeout = taskContext.config().snapshotLockTimeout();
|
|
||||||
logger.info("Step 1: starting transaction and refreshing the DB schemas for database '{}' and user '{}'",
|
|
||||||
connection.database(), connection.username());
|
|
||||||
|
|
||||||
String transactionStatement = snapshotter.snapshotTransactionIsolationLevelStatement(slotCreatedInfo);
|
|
||||||
logger.info("opening transaction with statement {}", transactionStatement);
|
|
||||||
connection.executeWithoutCommitting(transactionStatement);
|
|
||||||
|
|
||||||
//next refresh the schema which will load all the tables taking the filters into account
|
|
||||||
PostgresSchema schema = schema();
|
|
||||||
schema.refresh(connection, false);
|
|
||||||
|
|
||||||
Optional<String> lockStatement = snapshotter.snapshotTableLockingStatement(lockTimeout, schema.tableIds());
|
|
||||||
if (lockStatement.isPresent()) {
|
|
||||||
logger.info("Step 2: locking each of the database tables, waiting a maximum of '{}' seconds for each lock",
|
|
||||||
lockTimeout.getSeconds());
|
|
||||||
connection.executeWithoutCommitting(lockStatement.get());
|
|
||||||
//now that we have the locks, refresh the schema
|
|
||||||
schema.refresh(connection, false);
|
|
||||||
}
|
|
||||||
else {
|
|
||||||
// if we are not in an exported snapshot, this may result in some inconsistencies.
|
|
||||||
// Let the user know
|
|
||||||
if (!snapshotter.exportSnapshot()) {
|
|
||||||
logger.warn("Step 2: skipping locking each table, this may result in inconsistent schema!");
|
|
||||||
}
|
|
||||||
else {
|
|
||||||
logger.info("Step 2: skipping locking each table in an exported snapshot");
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
long xlogStart = getTransactionStartLsn(connection);
|
|
||||||
long txId = connection.currentTransactionId().longValue();
|
|
||||||
if (logger.isInfoEnabled()) {
|
|
||||||
logger.info("\t read xlogStart at '{}' from transaction '{}'", ReplicationConnection.format(xlogStart), txId);
|
|
||||||
}
|
|
||||||
|
|
||||||
// and mark the start of the snapshot
|
|
||||||
sourceInfo.startSnapshot();
|
|
||||||
// use the old xmin, as we don't want to update it if in xmin recovery
|
|
||||||
sourceInfo.update(xlogStart, clock().currentTime(), txId, null, sourceInfo.xmin());
|
|
||||||
|
|
||||||
logger.info("Step 3: reading and exporting the contents of each table");
|
|
||||||
AtomicInteger rowsCounter = new AtomicInteger(0);
|
|
||||||
|
|
||||||
for(TableId tableId : schema.tableIds()) {
|
|
||||||
long exportStart = clock().currentTimeInMillis();
|
|
||||||
logger.info("\t exporting data from table '{}'", tableId);
|
|
||||||
try {
|
|
||||||
final Optional<String> selectStatement = snapshotter.buildSnapshotQuery(tableId);
|
|
||||||
if (!selectStatement.isPresent()) {
|
|
||||||
logger.warn("For table '{}' the select statement was not provided, skipping table", tableId);
|
|
||||||
}
|
|
||||||
else {
|
|
||||||
logger.info("For table '{}' using select statement: '{}'", tableId, selectStatement);
|
|
||||||
|
|
||||||
connection.queryWithBlockingConsumer(selectStatement.get(),
|
|
||||||
this::readTableStatement,
|
|
||||||
rs -> readTable(tableId, rs, consumer, rowsCounter));
|
|
||||||
if (logger.isInfoEnabled()) {
|
|
||||||
logger.info("\t finished exporting '{}' records for '{}'; total duration '{}'", rowsCounter.get(),
|
|
||||||
tableId, Strings.duration(clock().currentTimeInMillis() - exportStart));
|
|
||||||
}
|
|
||||||
rowsCounter.set(0);
|
|
||||||
}
|
|
||||||
} catch (SQLException e) {
|
|
||||||
throw new ConnectException(e);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
// finally commit the transaction to release all the locks...
|
|
||||||
logger.info("Step 4: committing transaction '{}'", txId);
|
|
||||||
jdbcConnection.commit();
|
|
||||||
|
|
||||||
SourceRecord currentRecord = this.currentRecord.get();
|
|
||||||
if (currentRecord != null) {
|
|
||||||
// process and send the last record after marking it as such
|
|
||||||
logger.info("Step 5: sending the last snapshot record");
|
|
||||||
sourceInfo.markLastSnapshotRecord();
|
|
||||||
|
|
||||||
// the sourceInfo element already has been baked into the record value, so
|
|
||||||
// update the "last_snapshot_marker" in there
|
|
||||||
changeSourceToLastSnapshotRecord(currentRecord);
|
|
||||||
this.currentRecord.set(new SourceRecord(currentRecord.sourcePartition(), sourceInfo.offset(),
|
|
||||||
currentRecord.topic(), currentRecord.kafkaPartition(),
|
|
||||||
currentRecord.keySchema(), currentRecord.key(),
|
|
||||||
currentRecord.valueSchema(), currentRecord.value()));
|
|
||||||
|
|
||||||
sendCurrentRecord(consumer);
|
|
||||||
}
|
|
||||||
|
|
||||||
// and complete the snapshot
|
|
||||||
sourceInfo.completeSnapshot();
|
|
||||||
if (logger.isInfoEnabled()) {
|
|
||||||
logger.info("Snapshot completed in '{}'", Strings.duration(clock().currentTimeInMillis() - snapshotStart));
|
|
||||||
}
|
|
||||||
Heartbeat
|
|
||||||
.create(
|
|
||||||
taskContext.config().getConfig(),
|
|
||||||
taskContext.topicSelector().getHeartbeatTopic(),
|
|
||||||
taskContext.config().getLogicalName()
|
|
||||||
)
|
|
||||||
.forcedBeat(
|
|
||||||
sourceInfo.partition(),
|
|
||||||
sourceInfo.offset(),
|
|
||||||
r -> consumer.accept(new ChangeEvent(r, sourceInfo.lsn())
|
|
||||||
)
|
|
||||||
);
|
|
||||||
|
|
||||||
taskContext.schema().assureNonEmptySchema();
|
|
||||||
}
|
|
||||||
catch (SQLException e) {
|
|
||||||
rollbackTransaction(jdbcConnection);
|
|
||||||
|
|
||||||
throw new ConnectException(e);
|
|
||||||
}
|
|
||||||
catch(InterruptedException e) {
|
|
||||||
Thread.interrupted();
|
|
||||||
rollbackTransaction(jdbcConnection);
|
|
||||||
|
|
||||||
if (logger.isWarnEnabled()) {
|
|
||||||
logger.warn("Snapshot aborted after '{}'", Strings.duration(clock().currentTimeInMillis() - snapshotStart));
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
private long getTransactionStartLsn(PostgresConnection connection) throws SQLException {
|
|
||||||
if (snapshotter.exportSnapshot() && slotCreatedInfo != null) {
|
|
||||||
// When performing an exported snapshot based on a newly created replication slot, the txLogStart position
|
|
||||||
// should be based on the replication slot snapshot transaction point. This is crucial so that if any
|
|
||||||
// SQL operations occur mid-snapshot that they'll be properly captured when streaming begins; otherwise
|
|
||||||
// they'll be lost.
|
|
||||||
return slotCreatedInfo.startLsn();
|
|
||||||
}
|
|
||||||
return connection.currentXLogLocation();
|
|
||||||
}
|
|
||||||
|
|
||||||
private void changeSourceToLastSnapshotRecord(SourceRecord currentRecord) {
|
|
||||||
final Struct envelope = (Struct) currentRecord.value();
|
|
||||||
final Struct source = (Struct) envelope.get(Envelope.FieldName.SOURCE);
|
|
||||||
if (source.schema().field(SourceInfo.LAST_SNAPSHOT_RECORD_KEY) != null && source.getBoolean(SourceInfo.LAST_SNAPSHOT_RECORD_KEY) != null) {
|
|
||||||
source.put(SourceInfo.LAST_SNAPSHOT_RECORD_KEY, true);
|
|
||||||
}
|
|
||||||
if (SnapshotRecord.fromSource(source) == SnapshotRecord.TRUE) {
|
|
||||||
SnapshotRecord.LAST.toSource(source);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
private void rollbackTransaction(Connection jdbcConnection) {
|
|
||||||
try {
|
|
||||||
if (jdbcConnection != null) {
|
|
||||||
jdbcConnection.rollback();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
catch (SQLException se) {
|
|
||||||
logger.error("Cannot rollback snapshot transaction", se);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
private Statement readTableStatement(Connection conn) throws SQLException {
|
|
||||||
int fetchSize = taskContext.config().getSnapshotFetchSize();
|
|
||||||
Statement statement = conn.createStatement(); // the default cursor is FORWARD_ONLY
|
|
||||||
statement.setFetchSize(fetchSize);
|
|
||||||
return statement;
|
|
||||||
}
|
|
||||||
|
|
||||||
private void readTable(TableId tableId, ResultSet rs,
|
|
||||||
BlockingConsumer<ChangeEvent> consumer,
|
|
||||||
AtomicInteger rowsCounter) throws SQLException, InterruptedException {
|
|
||||||
Table table = schema().tableFor(tableId);
|
|
||||||
assert table != null;
|
|
||||||
final int numColumns = table.columns().size();
|
|
||||||
final Object[] row = new Object[numColumns];
|
|
||||||
final ResultSetMetaData metaData = rs.getMetaData();
|
|
||||||
while (rs.next()) {
|
|
||||||
rowsCounter.incrementAndGet();
|
|
||||||
sendCurrentRecord(consumer);
|
|
||||||
for (int i = 0, j = 1; i != numColumns; ++i, ++j) {
|
|
||||||
row[i] = valueForColumn(rs, j, metaData);
|
|
||||||
}
|
|
||||||
generateReadRecord(tableId, row);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
private Object valueForColumn(ResultSet rs, int colIdx, ResultSetMetaData metaData) throws SQLException {
|
|
||||||
try {
|
|
||||||
final String columnTypeName = metaData.getColumnTypeName(colIdx);
|
|
||||||
final PostgresType type = taskContext.schema().getTypeRegistry().get(columnTypeName);
|
|
||||||
|
|
||||||
logger.trace("Type of incoming data is: {}", type.getOid());
|
|
||||||
logger.trace("ColumnTypeName is: {}", columnTypeName);
|
|
||||||
logger.trace("Type is: {}", type);
|
|
||||||
|
|
||||||
if (type.isArrayType()) {
|
|
||||||
return rs.getArray(colIdx);
|
|
||||||
}
|
|
||||||
|
|
||||||
switch (type.getOid()) {
|
|
||||||
case PgOid.MONEY:
|
|
||||||
//TODO author=Horia Chiorean date=14/11/2016 description=workaround for https://github.com/pgjdbc/pgjdbc/issues/100
|
|
||||||
return new PGmoney(rs.getString(colIdx)).val;
|
|
||||||
case PgOid.BIT:
|
|
||||||
return rs.getString(colIdx);
|
|
||||||
case PgOid.NUMERIC:
|
|
||||||
final String s = rs.getString(colIdx);
|
|
||||||
if (s == null) {
|
|
||||||
return s;
|
|
||||||
}
|
|
||||||
|
|
||||||
Optional<SpecialValueDecimal> value = PostgresValueConverter.toSpecialValue(s);
|
|
||||||
return value.isPresent() ? value.get() : new SpecialValueDecimal(rs.getBigDecimal(colIdx));
|
|
||||||
case PgOid.TIME:
|
|
||||||
// To handle time 24:00:00 supported by TIME columns, read the column as a string.
|
|
||||||
case PgOid.TIMETZ:
|
|
||||||
// In order to guarantee that we resolve TIMETZ columns with proper microsecond precision,
|
|
||||||
// read the column as a string instead and then re-parse inside the converter.
|
|
||||||
return rs.getString(colIdx);
|
|
||||||
default:
|
|
||||||
Object x = rs.getObject(colIdx);
|
|
||||||
if(x != null) {
|
|
||||||
logger.trace("rs getobject returns class: {}; rs getObject value is: {}", x.getClass(), x);
|
|
||||||
}
|
|
||||||
return x;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
catch (SQLException e) {
|
|
||||||
// not a known type
|
|
||||||
return rs.getObject(colIdx);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
protected void generateReadRecord(TableId tableId, Object[] rowData) {
|
|
||||||
// Clear the existing record to prevent reprocessing stale data.
|
|
||||||
currentRecord.set(null);
|
|
||||||
|
|
||||||
if (rowData.length == 0) {
|
|
||||||
return;
|
|
||||||
}
|
|
||||||
logger.trace("tableId value is: {}", tableId);
|
|
||||||
TableSchema tableSchema = schema().schemaFor(tableId);
|
|
||||||
assert tableSchema != null;
|
|
||||||
Object key = tableSchema.keyFromColumnData(rowData);
|
|
||||||
Struct value = tableSchema.valueFromColumnData(rowData);
|
|
||||||
|
|
||||||
if (value == null) {
|
|
||||||
logger.trace("Read event for null key with value {}", value);
|
|
||||||
return;
|
|
||||||
}
|
|
||||||
Schema keySchema = tableSchema.keySchema();
|
|
||||||
sourceInfo.update(clock().currentTimeAsInstant(), tableId);
|
|
||||||
Map<String, ?> partition = sourceInfo.partition();
|
|
||||||
Map<String, ?> offset = sourceInfo.offset();
|
|
||||||
String topicName = topicSelector().topicNameFor(tableId);
|
|
||||||
Envelope envelope = tableSchema.getEnvelopeSchema();
|
|
||||||
currentRecord.set(new SourceRecord(partition, offset, topicName, null, keySchema, key, envelope.schema(),
|
|
||||||
envelope.read(value, sourceInfo.struct(), clock().currentTimeInMillis())));
|
|
||||||
}
|
|
||||||
|
|
||||||
private void sendCurrentRecord(BlockingConsumer<ChangeEvent> consumer) throws InterruptedException {
|
|
||||||
SourceRecord record = currentRecord.get();
|
|
||||||
if (record == null) {
|
|
||||||
return;
|
|
||||||
}
|
|
||||||
if (logger.isDebugEnabled()) {
|
|
||||||
logger.debug("sending read event '{}'", record);
|
|
||||||
}
|
|
||||||
//send the last generated record
|
|
||||||
consumer.accept(new ChangeEvent(record, sourceInfo.lsn()));
|
|
||||||
}
|
|
||||||
|
|
||||||
// test-only
|
|
||||||
boolean isStreamingRunning() {
|
|
||||||
return streamProducer.isPresent() && streamProducer.get().isStreamingRunning();
|
|
||||||
}
|
|
||||||
}
|
|
@ -7,17 +7,12 @@
|
|||||||
package io.debezium.connector.postgresql;
|
package io.debezium.connector.postgresql;
|
||||||
|
|
||||||
import java.time.Instant;
|
import java.time.Instant;
|
||||||
import java.util.Collections;
|
|
||||||
import java.util.HashMap;
|
|
||||||
import java.util.Map;
|
|
||||||
|
|
||||||
import io.debezium.annotation.NotThreadSafe;
|
import io.debezium.annotation.NotThreadSafe;
|
||||||
import io.debezium.connector.AbstractSourceInfo;
|
|
||||||
import io.debezium.connector.SnapshotRecord;
|
import io.debezium.connector.SnapshotRecord;
|
||||||
|
import io.debezium.connector.common.BaseSourceInfo;
|
||||||
import io.debezium.connector.postgresql.connection.ReplicationConnection;
|
import io.debezium.connector.postgresql.connection.ReplicationConnection;
|
||||||
import io.debezium.connector.postgresql.spi.OffsetState;
|
|
||||||
import io.debezium.relational.TableId;
|
import io.debezium.relational.TableId;
|
||||||
import io.debezium.time.Conversions;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Information about the source of information, which for normal events contains information about the transaction id and the
|
* Information about the source of information, which for normal events contains information about the transaction id and the
|
||||||
@ -74,9 +69,8 @@
|
|||||||
* @author Horia Chiorean
|
* @author Horia Chiorean
|
||||||
*/
|
*/
|
||||||
@NotThreadSafe
|
@NotThreadSafe
|
||||||
public final class SourceInfo extends AbstractSourceInfo {
|
public final class SourceInfo extends BaseSourceInfo {
|
||||||
|
|
||||||
public static final String SERVER_PARTITION_KEY = "server";
|
|
||||||
public static final String TIMESTAMP_USEC_KEY = "ts_usec";
|
public static final String TIMESTAMP_USEC_KEY = "ts_usec";
|
||||||
public static final String TXID_KEY = "txId";
|
public static final String TXID_KEY = "txId";
|
||||||
public static final String XMIN_KEY = "xmin";
|
public static final String XMIN_KEY = "xmin";
|
||||||
@ -84,75 +78,17 @@ public final class SourceInfo extends AbstractSourceInfo {
|
|||||||
public static final String LAST_SNAPSHOT_RECORD_KEY = "last_snapshot_record";
|
public static final String LAST_SNAPSHOT_RECORD_KEY = "last_snapshot_record";
|
||||||
|
|
||||||
private final String dbName;
|
private final String dbName;
|
||||||
private final Map<String, String> sourcePartition;
|
|
||||||
|
|
||||||
private Long lsn;
|
private Long lsn;
|
||||||
private Long txId;
|
private Long txId;
|
||||||
private Long xmin;
|
private Long xmin;
|
||||||
private Instant timestamp;
|
private Instant timestamp;
|
||||||
private boolean snapshot = false;
|
|
||||||
private Boolean lastSnapshotRecord;
|
|
||||||
private String schemaName;
|
private String schemaName;
|
||||||
private String tableName;
|
private String tableName;
|
||||||
|
|
||||||
protected SourceInfo(PostgresConnectorConfig connectorConfig) {
|
protected SourceInfo(PostgresConnectorConfig connectorConfig) {
|
||||||
super(connectorConfig);
|
super(connectorConfig);
|
||||||
this.dbName = connectorConfig.databaseName();
|
this.dbName = connectorConfig.databaseName();
|
||||||
this.sourcePartition = Collections.singletonMap(SERVER_PARTITION_KEY, connectorConfig.getLogicalName());
|
|
||||||
}
|
|
||||||
|
|
||||||
protected void load(Map<String, Object> lastStoredOffset) {
|
|
||||||
this.lsn = ((Number) lastStoredOffset.get(LSN_KEY)).longValue();
|
|
||||||
this.txId = ((Number) lastStoredOffset.get(TXID_KEY)).longValue();
|
|
||||||
this.xmin = (Long) lastStoredOffset.get(XMIN_KEY);
|
|
||||||
this.timestamp = Conversions.toInstantFromMicros((Long) lastStoredOffset.get(TIMESTAMP_USEC_KEY));
|
|
||||||
this.snapshot = lastStoredOffset.containsKey(SNAPSHOT_KEY);
|
|
||||||
if (this.snapshot) {
|
|
||||||
this.lastSnapshotRecord = (Boolean) lastStoredOffset.get(LAST_SNAPSHOT_RECORD_KEY);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Get the Kafka Connect detail about the source "partition", which describes the portion of the source that we are
|
|
||||||
* consuming. Since we're streaming changes for a single database, the source partition specifies only the {@code serverName}
|
|
||||||
* as the value for the partition.
|
|
||||||
*
|
|
||||||
* @return the source partition information; never null
|
|
||||||
*/
|
|
||||||
public Map<String, String> partition() {
|
|
||||||
return sourcePartition;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Get the Kafka Connect detail about the source "offset", which describes the position within the source where we last
|
|
||||||
* have last read.
|
|
||||||
*
|
|
||||||
* @return a copy of the current offset; never null
|
|
||||||
*/
|
|
||||||
public Map<String, ?> offset() {
|
|
||||||
assert serverName() != null && dbName != null;
|
|
||||||
Map<String, Object> result = new HashMap<>();
|
|
||||||
if (timestamp != null) {
|
|
||||||
result.put(TIMESTAMP_USEC_KEY, Conversions.toEpochMicros(timestamp));
|
|
||||||
}
|
|
||||||
if (txId != null) {
|
|
||||||
result.put(TXID_KEY, txId);
|
|
||||||
}
|
|
||||||
if (lsn != null) {
|
|
||||||
result.put(LSN_KEY, lsn);
|
|
||||||
}
|
|
||||||
if (xmin != null) {
|
|
||||||
result.put(XMIN_KEY, xmin);
|
|
||||||
}
|
|
||||||
if (snapshot) {
|
|
||||||
result.put(SNAPSHOT_KEY, true);
|
|
||||||
result.put(LAST_SNAPSHOT_RECORD_KEY, lastSnapshotRecord);
|
|
||||||
}
|
|
||||||
return result;
|
|
||||||
}
|
|
||||||
|
|
||||||
public OffsetState asOffsetState() {
|
|
||||||
return new OffsetState(lsn, txId, xmin, timestamp, isSnapshotInEffect());
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -194,35 +130,6 @@ protected SourceInfo update(Instant timestamp, TableId tableId) {
|
|||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
protected SourceInfo markLastSnapshotRecord() {
|
|
||||||
this.lastSnapshotRecord = true;
|
|
||||||
return this;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Determine whether a snapshot is currently in effect, meaning it was started and has not completed.
|
|
||||||
*
|
|
||||||
* @return {@code true} if a snapshot is in effect, or {@code false} otherwise
|
|
||||||
*/
|
|
||||||
public boolean isSnapshotInEffect() {
|
|
||||||
return snapshot && (this.lastSnapshotRecord == null || !this.lastSnapshotRecord);
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Denote that a snapshot is being (or has been) started.
|
|
||||||
*/
|
|
||||||
protected void startSnapshot() {
|
|
||||||
this.snapshot = true;
|
|
||||||
this.lastSnapshotRecord = false;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Denote that a snapshot has completed successfully.
|
|
||||||
*/
|
|
||||||
protected void completeSnapshot() {
|
|
||||||
this.snapshot = false;
|
|
||||||
}
|
|
||||||
|
|
||||||
public Long lsn() {
|
public Long lsn() {
|
||||||
return this.lsn;
|
return this.lsn;
|
||||||
}
|
}
|
||||||
@ -249,19 +156,13 @@ protected Instant timestamp() {
|
|||||||
return timestamp;
|
return timestamp;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
protected Long txId() {
|
||||||
protected SnapshotRecord snapshot() {
|
return txId;
|
||||||
if (snapshot) {
|
|
||||||
if (lastSnapshotRecord) {
|
|
||||||
return SnapshotRecord.LAST;
|
|
||||||
}
|
|
||||||
return SnapshotRecord.TRUE;
|
|
||||||
}
|
|
||||||
return SnapshotRecord.FALSE;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
public boolean hasLastKnownPosition() {
|
@Override
|
||||||
return this.lsn != null;
|
public SnapshotRecord snapshot() {
|
||||||
|
return super.snapshot();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
@ -281,11 +182,7 @@ public String toString() {
|
|||||||
if (timestamp != null) {
|
if (timestamp != null) {
|
||||||
sb.append(", timestamp=").append(timestamp);
|
sb.append(", timestamp=").append(timestamp);
|
||||||
}
|
}
|
||||||
boolean snapshotInEffect = isSnapshotInEffect();
|
sb.append(", snapshot=").append(snapshot());
|
||||||
sb.append(", snapshot=").append(snapshotInEffect);
|
|
||||||
if (snapshotInEffect) {
|
|
||||||
sb.append(", last_snapshot_record=").append(lastSnapshotRecord);
|
|
||||||
}
|
|
||||||
if (schemaName != null) {
|
if (schemaName != null) {
|
||||||
sb.append(", schema=").append(schemaName);
|
sb.append(", schema=").append(schemaName);
|
||||||
}
|
}
|
||||||
|
@ -28,7 +28,7 @@ public boolean isToastedColumn() {
|
|||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Object getValue(RecordsStreamProducer.PgConnectionSupplier connection, boolean includeUnknownDatatypes) {
|
public Object getValue(PostgresStreamingChangeEventSource.PgConnectionSupplier connection, boolean includeUnknownDatatypes) {
|
||||||
throw new UnsupportedOperationException("A toasted column does not supply a value");
|
throw new UnsupportedOperationException("A toasted column does not supply a value");
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -484,7 +484,7 @@ private static TypeRegistry initTypeRegistry(Connection db) {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
catch (SQLException e) {
|
catch (SQLException e) {
|
||||||
throw new ConnectException("Could not intialize type registry", e);
|
throw new ConnectException("Database connection failed during intializiation of the type registry", e);
|
||||||
}
|
}
|
||||||
return typeRegistryBuilder.build();
|
return typeRegistryBuilder.build();
|
||||||
}
|
}
|
||||||
|
@ -22,8 +22,8 @@
|
|||||||
import org.postgresql.util.PGInterval;
|
import org.postgresql.util.PGInterval;
|
||||||
import org.postgresql.util.PGmoney;
|
import org.postgresql.util.PGmoney;
|
||||||
|
|
||||||
|
import io.debezium.connector.postgresql.PostgresStreamingChangeEventSource.PgConnectionSupplier;
|
||||||
import io.debezium.connector.postgresql.PostgresType;
|
import io.debezium.connector.postgresql.PostgresType;
|
||||||
import io.debezium.connector.postgresql.RecordsStreamProducer.PgConnectionSupplier;
|
|
||||||
import io.debezium.data.SpecialValueDecimal;
|
import io.debezium.data.SpecialValueDecimal;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -11,8 +11,8 @@
|
|||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
|
import io.debezium.connector.postgresql.PostgresStreamingChangeEventSource.PgConnectionSupplier;
|
||||||
import io.debezium.connector.postgresql.PostgresType;
|
import io.debezium.connector.postgresql.PostgresType;
|
||||||
import io.debezium.connector.postgresql.RecordsStreamProducer.PgConnectionSupplier;
|
|
||||||
import io.debezium.connector.postgresql.connection.ReplicationMessage.ColumnValue;
|
import io.debezium.connector.postgresql.connection.ReplicationMessage.ColumnValue;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -25,8 +25,8 @@
|
|||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
|
import io.debezium.connector.postgresql.PostgresStreamingChangeEventSource.PgConnectionSupplier;
|
||||||
import io.debezium.connector.postgresql.PostgresType;
|
import io.debezium.connector.postgresql.PostgresType;
|
||||||
import io.debezium.connector.postgresql.RecordsStreamProducer.PgConnectionSupplier;
|
|
||||||
import io.debezium.connector.postgresql.ToastedReplicationMessageColumn;
|
import io.debezium.connector.postgresql.ToastedReplicationMessageColumn;
|
||||||
import io.debezium.connector.postgresql.TypeRegistry;
|
import io.debezium.connector.postgresql.TypeRegistry;
|
||||||
import io.debezium.connector.postgresql.connection.AbstractMessageDecoder;
|
import io.debezium.connector.postgresql.connection.AbstractMessageDecoder;
|
||||||
|
@ -8,8 +8,8 @@
|
|||||||
import java.time.Instant;
|
import java.time.Instant;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
|
||||||
|
import io.debezium.connector.postgresql.PostgresStreamingChangeEventSource.PgConnectionSupplier;
|
||||||
import io.debezium.connector.postgresql.PostgresType;
|
import io.debezium.connector.postgresql.PostgresType;
|
||||||
import io.debezium.connector.postgresql.RecordsStreamProducer.PgConnectionSupplier;
|
|
||||||
import io.debezium.connector.postgresql.connection.ReplicationMessage;
|
import io.debezium.connector.postgresql.connection.ReplicationMessage;
|
||||||
import io.debezium.connector.postgresql.connection.ReplicationMessageColumnValueResolver;
|
import io.debezium.connector.postgresql.connection.ReplicationMessageColumnValueResolver;
|
||||||
|
|
||||||
|
@ -11,6 +11,8 @@
|
|||||||
|
|
||||||
import org.apache.kafka.connect.errors.ConnectException;
|
import org.apache.kafka.connect.errors.ConnectException;
|
||||||
import org.postgresql.replication.fluent.logical.ChainedLogicalStreamBuilder;
|
import org.postgresql.replication.fluent.logical.ChainedLogicalStreamBuilder;
|
||||||
|
import org.slf4j.Logger;
|
||||||
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
import com.google.protobuf.InvalidProtocolBufferException;
|
import com.google.protobuf.InvalidProtocolBufferException;
|
||||||
|
|
||||||
@ -29,6 +31,8 @@
|
|||||||
*/
|
*/
|
||||||
public class PgProtoMessageDecoder extends AbstractMessageDecoder {
|
public class PgProtoMessageDecoder extends AbstractMessageDecoder {
|
||||||
|
|
||||||
|
private static final Logger LOGGER = LoggerFactory.getLogger(PgProtoMessageDecoder.class);
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void processMessage(final ByteBuffer buffer, ReplicationMessageProcessor processor, TypeRegistry typeRegistry) throws SQLException, InterruptedException {
|
public void processMessage(final ByteBuffer buffer, ReplicationMessageProcessor processor, TypeRegistry typeRegistry) throws SQLException, InterruptedException {
|
||||||
try {
|
try {
|
||||||
@ -39,6 +43,7 @@ public void processMessage(final ByteBuffer buffer, ReplicationMessageProcessor
|
|||||||
final byte[] source = buffer.array();
|
final byte[] source = buffer.array();
|
||||||
final byte[] content = Arrays.copyOfRange(source, buffer.arrayOffset(), source.length);
|
final byte[] content = Arrays.copyOfRange(source, buffer.arrayOffset(), source.length);
|
||||||
final RowMessage message = PgProto.RowMessage.parseFrom(content);
|
final RowMessage message = PgProto.RowMessage.parseFrom(content);
|
||||||
|
LOGGER.trace("Received protobuf message from the server {}", message);
|
||||||
if (!message.getNewTypeinfoList().isEmpty() && message.getNewTupleCount() != message.getNewTypeinfoCount()) {
|
if (!message.getNewTypeinfoList().isEmpty() && message.getNewTupleCount() != message.getNewTypeinfoCount()) {
|
||||||
throw new ConnectException(String.format("Message from transaction {} has {} data columns but only {} of type info",
|
throw new ConnectException(String.format("Message from transaction {} has {} data columns but only {} of type info",
|
||||||
Integer.toUnsignedLong(message.getTransactionId()),
|
Integer.toUnsignedLong(message.getTransactionId()),
|
||||||
|
@ -26,9 +26,9 @@
|
|||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
import io.debezium.connector.postgresql.PgOid;
|
import io.debezium.connector.postgresql.PgOid;
|
||||||
|
import io.debezium.connector.postgresql.PostgresStreamingChangeEventSource.PgConnectionSupplier;
|
||||||
import io.debezium.connector.postgresql.PostgresType;
|
import io.debezium.connector.postgresql.PostgresType;
|
||||||
import io.debezium.connector.postgresql.PostgresValueConverter;
|
import io.debezium.connector.postgresql.PostgresValueConverter;
|
||||||
import io.debezium.connector.postgresql.RecordsStreamProducer.PgConnectionSupplier;
|
|
||||||
import io.debezium.connector.postgresql.TypeRegistry;
|
import io.debezium.connector.postgresql.TypeRegistry;
|
||||||
import io.debezium.connector.postgresql.connection.AbstractReplicationMessageColumn;
|
import io.debezium.connector.postgresql.connection.AbstractReplicationMessageColumn;
|
||||||
import io.debezium.connector.postgresql.connection.ReplicationMessage;
|
import io.debezium.connector.postgresql.connection.ReplicationMessage;
|
||||||
|
@ -47,8 +47,6 @@
|
|||||||
import org.apache.kafka.connect.data.SchemaBuilder;
|
import org.apache.kafka.connect.data.SchemaBuilder;
|
||||||
import org.apache.kafka.connect.data.Struct;
|
import org.apache.kafka.connect.data.Struct;
|
||||||
import org.apache.kafka.connect.source.SourceRecord;
|
import org.apache.kafka.connect.source.SourceRecord;
|
||||||
import org.awaitility.Awaitility;
|
|
||||||
import org.awaitility.Duration;
|
|
||||||
import org.junit.Rule;
|
import org.junit.Rule;
|
||||||
import org.junit.rules.TestRule;
|
import org.junit.rules.TestRule;
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
@ -65,7 +63,7 @@
|
|||||||
import io.debezium.data.geometry.Geography;
|
import io.debezium.data.geometry.Geography;
|
||||||
import io.debezium.data.geometry.Geometry;
|
import io.debezium.data.geometry.Geometry;
|
||||||
import io.debezium.data.geometry.Point;
|
import io.debezium.data.geometry.Point;
|
||||||
import io.debezium.function.BlockingConsumer;
|
import io.debezium.embedded.AbstractConnectorTest;
|
||||||
import io.debezium.junit.TestLogger;
|
import io.debezium.junit.TestLogger;
|
||||||
import io.debezium.relational.TableId;
|
import io.debezium.relational.TableId;
|
||||||
import io.debezium.time.Date;
|
import io.debezium.time.Date;
|
||||||
@ -76,14 +74,16 @@
|
|||||||
import io.debezium.time.Timestamp;
|
import io.debezium.time.Timestamp;
|
||||||
import io.debezium.time.ZonedTime;
|
import io.debezium.time.ZonedTime;
|
||||||
import io.debezium.time.ZonedTimestamp;
|
import io.debezium.time.ZonedTimestamp;
|
||||||
import io.debezium.util.VariableLatch;
|
import io.debezium.util.Clock;
|
||||||
|
import io.debezium.util.ElapsedTimeStrategy;
|
||||||
|
import io.debezium.util.Testing;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Base class for the integration tests for the different {@link RecordsProducer} instances
|
* Base class for the integration tests for the different {@link RecordsProducer} instances
|
||||||
*
|
*
|
||||||
* @author Horia Chiorean (hchiorea@redhat.com)
|
* @author Horia Chiorean (hchiorea@redhat.com)
|
||||||
*/
|
*/
|
||||||
public abstract class AbstractRecordsProducerTest {
|
public abstract class AbstractRecordsProducerTest extends AbstractConnectorTest {
|
||||||
|
|
||||||
private static final Logger LOGGER = LoggerFactory.getLogger(AbstractRecordsProducerTest.class);
|
private static final Logger LOGGER = LoggerFactory.getLogger(AbstractRecordsProducerTest.class);
|
||||||
|
|
||||||
@ -975,14 +975,14 @@ protected TestConsumer testConsumer(int expectedRecordsCount, String... topicPre
|
|||||||
return new TestConsumer(expectedRecordsCount, topicPrefixes);
|
return new TestConsumer(expectedRecordsCount, topicPrefixes);
|
||||||
}
|
}
|
||||||
|
|
||||||
protected static class TestConsumer implements BlockingConsumer<ChangeEvent> {
|
protected class TestConsumer {
|
||||||
private final ConcurrentLinkedQueue<SourceRecord> records;
|
private final ConcurrentLinkedQueue<SourceRecord> records;
|
||||||
private final VariableLatch latch;
|
private int expectedRecordsCount;
|
||||||
private final List<String> topicPrefixes;
|
private final List<String> topicPrefixes;
|
||||||
private boolean ignoreExtraRecords = false;
|
private boolean ignoreExtraRecords = false;
|
||||||
|
|
||||||
protected TestConsumer(int expectedRecordsCount, String... topicPrefixes) {
|
protected TestConsumer(int expectedRecordsCount, String... topicPrefixes) {
|
||||||
this.latch = new VariableLatch(expectedRecordsCount);
|
this.expectedRecordsCount = expectedRecordsCount;
|
||||||
this.records = new ConcurrentLinkedQueue<>();
|
this.records = new ConcurrentLinkedQueue<>();
|
||||||
this.topicPrefixes = Arrays.stream(topicPrefixes)
|
this.topicPrefixes = Arrays.stream(topicPrefixes)
|
||||||
.map(p -> TestHelper.TEST_SERVER + "." + p)
|
.map(p -> TestHelper.TEST_SERVER + "." + p)
|
||||||
@ -993,22 +993,33 @@ public void setIgnoreExtraRecords(boolean ignoreExtraRecords) {
|
|||||||
this.ignoreExtraRecords = ignoreExtraRecords;
|
this.ignoreExtraRecords = ignoreExtraRecords;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
public void accept(SourceRecord record) {
|
||||||
public void accept(ChangeEvent event) {
|
if (ignoreTopic(record.topic())) {
|
||||||
final SourceRecord record = event.getRecord();
|
|
||||||
if ( ignoreTopic(record.topic()) ) {
|
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
if (latch.getCount() == 0) {
|
if (records.size() >= expectedRecordsCount) {
|
||||||
if (ignoreExtraRecords) {
|
addRecord(record);
|
||||||
records.add(record);
|
if (!ignoreExtraRecords) {
|
||||||
} else {
|
|
||||||
fail("received more events than expected");
|
fail("received more events than expected");
|
||||||
}
|
}
|
||||||
} else {
|
}
|
||||||
records.add(record);
|
else {
|
||||||
latch.countDown();
|
addRecord(record);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private void addRecord(SourceRecord record) {
|
||||||
|
records.add(record);
|
||||||
|
if (Testing.Debug.isEnabled()) {
|
||||||
|
Testing.debug("Consumed record " + records.size() + " / " + expectedRecordsCount + " ("
|
||||||
|
+ (expectedRecordsCount - records.size()) + " more)");
|
||||||
|
Testing.debug(record);
|
||||||
|
}
|
||||||
|
else if (Testing.Print.isEnabled()) {
|
||||||
|
Testing.print("Consumed record " + records.size() + " / " + expectedRecordsCount + " ("
|
||||||
|
+ (expectedRecordsCount - records.size()) + " more)");
|
||||||
|
Testing.print(record);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -1027,8 +1038,7 @@ private boolean ignoreTopic(String topicName) {
|
|||||||
}
|
}
|
||||||
|
|
||||||
protected void expects(int expectedRecordsCount) {
|
protected void expects(int expectedRecordsCount) {
|
||||||
assert latch.getCount() == 0;
|
this.expectedRecordsCount = expectedRecordsCount;
|
||||||
this.latch.countUp(expectedRecordsCount);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
protected SourceRecord remove() {
|
protected SourceRecord remove() {
|
||||||
@ -1048,13 +1058,28 @@ protected void clear() {
|
|||||||
}
|
}
|
||||||
|
|
||||||
protected void await(long timeout, TimeUnit unit) throws InterruptedException {
|
protected void await(long timeout, TimeUnit unit) throws InterruptedException {
|
||||||
if (!latch.await(timeout, unit)) {
|
final ElapsedTimeStrategy timer = ElapsedTimeStrategy.constant(Clock.SYSTEM, unit.toMillis(timeout));
|
||||||
fail("Consumer is still expecting " + latch.getCount() + " records, as it received only " + records.size());
|
timer.hasElapsed();
|
||||||
|
while (!timer.hasElapsed()) {
|
||||||
|
final SourceRecord r = consumeRecord();
|
||||||
|
if (r != null) {
|
||||||
|
accept(r);
|
||||||
|
if (records.size() == expectedRecordsCount) {
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
if (records.size() != expectedRecordsCount) {
|
||||||
|
fail("Consumer is still expecting " + (expectedRecordsCount -records.size()) + " records, as it received only " + records.size());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
protected void waitForStreamingToStart(RecordsSnapshotProducer producer) throws InterruptedException {
|
protected void waitForStreamingToStart() throws InterruptedException {
|
||||||
Awaitility.await().atMost(Duration.FIVE_SECONDS).until(producer::isStreamingRunning);
|
waitForStreamingRunning("postgres", "test_server");
|
||||||
|
}
|
||||||
|
|
||||||
|
protected PostgresSchema getSchema() {
|
||||||
|
return null;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -469,10 +469,10 @@ private void startConnectorWithInitialSnapshotRecord() throws Exception {
|
|||||||
assertThat(recordsFromOutbox.size()).isEqualTo(1);
|
assertThat(recordsFromOutbox.size()).isEqualTo(1);
|
||||||
}
|
}
|
||||||
|
|
||||||
private void startConnectorWithNoSnapshot() {
|
private void startConnectorWithNoSnapshot() throws InterruptedException {
|
||||||
Configuration.Builder configBuilder = getConfigurationBuilder(SnapshotMode.NEVER);
|
Configuration.Builder configBuilder = getConfigurationBuilder(SnapshotMode.NEVER);
|
||||||
start(PostgresConnector.class, configBuilder.build());
|
start(PostgresConnector.class, configBuilder.build());
|
||||||
assertConnectorIsRunning();
|
waitForStreamingRunning("postgres", TestHelper.TEST_SERVER);
|
||||||
}
|
}
|
||||||
|
|
||||||
private static Configuration.Builder getConfigurationBuilder(SnapshotMode snapshotMode) {
|
private static Configuration.Builder getConfigurationBuilder(SnapshotMode snapshotMode) {
|
||||||
|
@ -319,6 +319,7 @@ public void shouldConsumeMessagesFromSnapshot() throws Exception {
|
|||||||
@Test
|
@Test
|
||||||
@FixFor("DBZ-997")
|
@FixFor("DBZ-997")
|
||||||
public void shouldReceiveChangesForChangePKColumnDefinition() throws Exception {
|
public void shouldReceiveChangesForChangePKColumnDefinition() throws Exception {
|
||||||
|
Testing.Print.enable();
|
||||||
final String slotName = "pkcolumndef" + new Random().nextInt(100);
|
final String slotName = "pkcolumndef" + new Random().nextInt(100);
|
||||||
TestHelper.create().dropReplicationSlot(slotName);
|
TestHelper.create().dropReplicationSlot(slotName);
|
||||||
try {
|
try {
|
||||||
@ -410,6 +411,7 @@ public void shouldReceiveChangesForChangePKColumnDefinition() throws Exception {
|
|||||||
// where there are limited logical replication slots configured.
|
// where there are limited logical replication slots configured.
|
||||||
stopConnector(null);
|
stopConnector(null);
|
||||||
TestHelper.create().dropReplicationSlot(slotName);
|
TestHelper.create().dropReplicationSlot(slotName);
|
||||||
|
throw t;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -536,14 +538,14 @@ public void shouldExecuteOnConnectStatements() throws Exception {
|
|||||||
start(PostgresConnector.class, configBuilder.build());
|
start(PostgresConnector.class, configBuilder.build());
|
||||||
assertConnectorIsRunning();
|
assertConnectorIsRunning();
|
||||||
|
|
||||||
SourceRecords actualRecords = consumeRecordsByTopic(7);
|
SourceRecords actualRecords = consumeRecordsByTopic(6);
|
||||||
assertKey(actualRecords.allRecordsInOrder().get(0), "pk", 1);
|
assertKey(actualRecords.allRecordsInOrder().get(0), "pk", 1);
|
||||||
assertKey(actualRecords.allRecordsInOrder().get(1), "pk", 2);
|
assertKey(actualRecords.allRecordsInOrder().get(1), "pk", 2);
|
||||||
|
|
||||||
// JdbcConnection#connection() is called multiple times during connector start-up,
|
// JdbcConnection#connection() is called multiple times during connector start-up,
|
||||||
// so the given statements will be executed multiple times, resulting in multiple
|
// so the given statements will be executed multiple times, resulting in multiple
|
||||||
// records; here we're interested just in the first insert for s2.a
|
// records; here we're interested just in the first insert for s2.a
|
||||||
assertValueField(actualRecords.allRecordsInOrder().get(6), "after/bb", "hello; world");
|
assertValueField(actualRecords.allRecordsInOrder().get(5), "after/bb", "hello; world");
|
||||||
|
|
||||||
stopConnector();
|
stopConnector();
|
||||||
TestHelper.dropDefaultReplicationSlot();
|
TestHelper.dropDefaultReplicationSlot();
|
||||||
|
@ -13,15 +13,15 @@
|
|||||||
|
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.concurrent.TimeUnit;
|
import java.util.concurrent.TimeUnit;
|
||||||
import java.util.function.Consumer;
|
|
||||||
|
|
||||||
import org.apache.kafka.connect.source.SourceRecord;
|
import org.apache.kafka.connect.source.SourceRecord;
|
||||||
import org.junit.After;
|
|
||||||
import org.junit.Before;
|
import org.junit.Before;
|
||||||
import org.junit.Rule;
|
import org.junit.Rule;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
import org.junit.rules.TestRule;
|
import org.junit.rules.TestRule;
|
||||||
|
|
||||||
|
import io.debezium.config.Configuration;
|
||||||
|
import io.debezium.connector.postgresql.PostgresConnectorConfig.SnapshotMode;
|
||||||
import io.debezium.connector.postgresql.connection.PostgresConnection;
|
import io.debezium.connector.postgresql.connection.PostgresConnection;
|
||||||
import io.debezium.connector.postgresql.connection.ReplicationConnection;
|
import io.debezium.connector.postgresql.connection.ReplicationConnection;
|
||||||
import io.debezium.connector.postgresql.junit.SkipTestDependingOnDecoderPluginNameRule;
|
import io.debezium.connector.postgresql.junit.SkipTestDependingOnDecoderPluginNameRule;
|
||||||
@ -30,7 +30,6 @@
|
|||||||
import io.debezium.doc.FixFor;
|
import io.debezium.doc.FixFor;
|
||||||
import io.debezium.junit.ConditionalFail;
|
import io.debezium.junit.ConditionalFail;
|
||||||
import io.debezium.relational.TableId;
|
import io.debezium.relational.TableId;
|
||||||
import io.debezium.schema.TopicSelector;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Integration test to verify PostGIS types defined in public schema.
|
* Integration test to verify PostGIS types defined in public schema.
|
||||||
@ -39,9 +38,7 @@
|
|||||||
*/
|
*/
|
||||||
public class PublicGeometryIT extends AbstractRecordsProducerTest {
|
public class PublicGeometryIT extends AbstractRecordsProducerTest {
|
||||||
|
|
||||||
private RecordsStreamProducer recordsProducer;
|
|
||||||
private TestConsumer consumer;
|
private TestConsumer consumer;
|
||||||
private final Consumer<Throwable> blackHole = t -> {};
|
|
||||||
|
|
||||||
@Rule
|
@Rule
|
||||||
public final TestRule skip = new SkipTestDependingOnDecoderPluginNameRule();
|
public final TestRule skip = new SkipTestDependingOnDecoderPluginNameRule();
|
||||||
@ -63,26 +60,14 @@ public void before() throws Exception {
|
|||||||
"CREATE TABLE public.postgis_array_table (pk SERIAL, ga GEOMETRY[], gann GEOMETRY[] NOT NULL, PRIMARY KEY(pk));",
|
"CREATE TABLE public.postgis_array_table (pk SERIAL, ga GEOMETRY[], gann GEOMETRY[] NOT NULL, PRIMARY KEY(pk));",
|
||||||
"CREATE TABLE public.dummy_table (pk SERIAL, PRIMARY KEY(pk));"
|
"CREATE TABLE public.dummy_table (pk SERIAL, PRIMARY KEY(pk));"
|
||||||
);
|
);
|
||||||
PostgresConnectorConfig config = new PostgresConnectorConfig(TestHelper.defaultConfig()
|
setupRecordsProducer(TestHelper.defaultConfig());
|
||||||
.with(PostgresConnectorConfig.INCLUDE_UNKNOWN_DATATYPES, false)
|
|
||||||
.build());
|
|
||||||
setupRecordsProducer(config);
|
|
||||||
}
|
|
||||||
|
|
||||||
@After
|
|
||||||
public void after() throws Exception {
|
|
||||||
if (recordsProducer != null) {
|
|
||||||
recordsProducer.stop();
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test(timeout = 30000)
|
@Test(timeout = 30000)
|
||||||
@FixFor("DBZ-1144")
|
@FixFor("DBZ-1144")
|
||||||
public void shouldReceiveChangesForInsertsWithPostgisTypes() throws Exception {
|
public void shouldReceiveChangesForInsertsWithPostgisTypes() throws Exception {
|
||||||
consumer = testConsumer(1, "public"); // spatial_ref_sys produces a ton of records in the postgis schema
|
consumer = testConsumer(1, "public");
|
||||||
consumer.setIgnoreExtraRecords(true);
|
waitForStreamingToStart();
|
||||||
recordsProducer.start(consumer, blackHole);
|
|
||||||
|
|
||||||
// need to wait for all the spatial_ref_sys to flow through and be ignored.
|
// need to wait for all the spatial_ref_sys to flow through and be ignored.
|
||||||
// this exceeds the normal 2s timeout.
|
// this exceeds the normal 2s timeout.
|
||||||
TestHelper.execute("INSERT INTO public.dummy_table DEFAULT VALUES;");
|
TestHelper.execute("INSERT INTO public.dummy_table DEFAULT VALUES;");
|
||||||
@ -104,19 +89,12 @@ public void shouldReceiveChangesForInsertsWithPostgisTypes() throws Exception {
|
|||||||
assertInsert(INSERT_POSTGIS_ARRAY_TYPES_IN_PUBLIC_STMT, 1, schemaAndValuesForPostgisArrayTypes());
|
assertInsert(INSERT_POSTGIS_ARRAY_TYPES_IN_PUBLIC_STMT, 1, schemaAndValuesForPostgisArrayTypes());
|
||||||
}
|
}
|
||||||
|
|
||||||
private void setupRecordsProducer(PostgresConnectorConfig config) {
|
private void setupRecordsProducer(Configuration.Builder config) {
|
||||||
if (recordsProducer != null) {
|
start(PostgresConnector.class, config
|
||||||
recordsProducer.stop();
|
.with(PostgresConnectorConfig.SNAPSHOT_MODE, SnapshotMode.NEVER)
|
||||||
}
|
.build()
|
||||||
|
|
||||||
TopicSelector<TableId> selector = PostgresTopicSelector.create(config);
|
|
||||||
|
|
||||||
PostgresTaskContext context = new PostgresTaskContext(
|
|
||||||
config,
|
|
||||||
TestHelper.getSchema(config),
|
|
||||||
selector
|
|
||||||
);
|
);
|
||||||
recordsProducer = new RecordsStreamProducer(context, new SourceInfo(config));
|
assertConnectorIsRunning();
|
||||||
}
|
}
|
||||||
|
|
||||||
private void assertInsert(String statement, Integer pk, List<SchemaAndValueField> expectedSchemaAndValuesByColumn) {
|
private void assertInsert(String statement, Integer pk, List<SchemaAndValueField> expectedSchemaAndValuesByColumn) {
|
||||||
|
@ -26,27 +26,23 @@
|
|||||||
import org.apache.kafka.connect.data.Struct;
|
import org.apache.kafka.connect.data.Struct;
|
||||||
import org.apache.kafka.connect.source.SourceRecord;
|
import org.apache.kafka.connect.source.SourceRecord;
|
||||||
import org.fest.assertions.Assertions;
|
import org.fest.assertions.Assertions;
|
||||||
import org.junit.After;
|
|
||||||
import org.junit.Before;
|
import org.junit.Before;
|
||||||
import org.junit.Rule;
|
import org.junit.Rule;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
import org.junit.rules.TestRule;
|
import org.junit.rules.TestRule;
|
||||||
|
|
||||||
|
import io.debezium.config.Configuration;
|
||||||
|
import io.debezium.connector.postgresql.PostgresConnectorConfig.SnapshotMode;
|
||||||
import io.debezium.connector.postgresql.junit.SkipTestDependingOnDatabaseVersionRule;
|
import io.debezium.connector.postgresql.junit.SkipTestDependingOnDatabaseVersionRule;
|
||||||
import io.debezium.connector.postgresql.junit.SkipWhenDatabaseVersionLessThan;
|
import io.debezium.connector.postgresql.junit.SkipWhenDatabaseVersionLessThan;
|
||||||
import io.debezium.connector.postgresql.snapshot.AlwaysSnapshotter;
|
|
||||||
import io.debezium.connector.postgresql.snapshot.InitialOnlySnapshotter;
|
|
||||||
import io.debezium.connector.postgresql.snapshot.SnapshotterWrapper;
|
|
||||||
import io.debezium.connector.postgresql.spi.Snapshotter;
|
|
||||||
import io.debezium.data.Envelope;
|
import io.debezium.data.Envelope;
|
||||||
import io.debezium.data.VerifyRecord;
|
import io.debezium.data.VerifyRecord;
|
||||||
import io.debezium.doc.FixFor;
|
import io.debezium.doc.FixFor;
|
||||||
import io.debezium.heartbeat.Heartbeat;
|
import io.debezium.heartbeat.Heartbeat;
|
||||||
import io.debezium.jdbc.TemporalPrecisionMode;
|
import io.debezium.jdbc.TemporalPrecisionMode;
|
||||||
import io.debezium.relational.RelationalDatabaseConnectorConfig.DecimalHandlingMode;
|
import io.debezium.relational.RelationalDatabaseConnectorConfig.DecimalHandlingMode;
|
||||||
import io.debezium.relational.TableId;
|
|
||||||
import io.debezium.schema.TopicSelector;
|
|
||||||
import io.debezium.util.Collect;
|
import io.debezium.util.Collect;
|
||||||
|
import io.debezium.util.Testing;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Integration test for {@link RecordsSnapshotProducerIT}
|
* Integration test for {@link RecordsSnapshotProducerIT}
|
||||||
@ -58,53 +54,29 @@ public class RecordsSnapshotProducerIT extends AbstractRecordsProducerTest {
|
|||||||
@Rule
|
@Rule
|
||||||
public final TestRule skip = new SkipTestDependingOnDatabaseVersionRule();
|
public final TestRule skip = new SkipTestDependingOnDatabaseVersionRule();
|
||||||
|
|
||||||
private RecordsSnapshotProducer snapshotProducer;
|
|
||||||
private PostgresTaskContext context;
|
|
||||||
|
|
||||||
@Before
|
@Before
|
||||||
public void before() throws Exception {
|
public void before() throws Exception {
|
||||||
TestHelper.dropAllSchemas();
|
TestHelper.dropAllSchemas();
|
||||||
TestHelper.executeDDL("init_postgis.ddl");
|
TestHelper.executeDDL("init_postgis.ddl");
|
||||||
TestHelper.executeDDL("postgres_create_tables.ddl");
|
TestHelper.executeDDL("postgres_create_tables.ddl");
|
||||||
TestHelper.executeDDL("postgis_create_tables.ddl");
|
TestHelper.executeDDL("postgis_create_tables.ddl");
|
||||||
|
|
||||||
PostgresConnectorConfig config = new PostgresConnectorConfig(TestHelper.defaultConfig()
|
|
||||||
.build());
|
|
||||||
TopicSelector<TableId> selector = PostgresTopicSelector.create(config);
|
|
||||||
context = new PostgresTaskContext(
|
|
||||||
config,
|
|
||||||
TestHelper.getSchema(config),
|
|
||||||
selector
|
|
||||||
);
|
|
||||||
// Testing.Print.enable();
|
|
||||||
}
|
|
||||||
|
|
||||||
@After
|
|
||||||
public void after() throws Exception {
|
|
||||||
if (snapshotProducer != null) {
|
|
||||||
snapshotProducer.stop();
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void shouldGenerateSnapshotsForDefaultDatatypes() throws Exception {
|
public void shouldGenerateSnapshotsForDefaultDatatypes() throws Exception {
|
||||||
PostgresConnectorConfig config = new PostgresConnectorConfig(TestHelper.defaultConfig()
|
|
||||||
.build());
|
|
||||||
snapshotProducer = buildNoStreamProducer(context, config);
|
|
||||||
|
|
||||||
TestConsumer consumer = testConsumer(ALL_STMTS.size(), "public", "Quoted__");
|
|
||||||
|
|
||||||
//insert data for each of different supported types
|
//insert data for each of different supported types
|
||||||
String statementsBuilder = ALL_STMTS.stream().collect(Collectors.joining(";" + System.lineSeparator())) + ";";
|
String statementsBuilder = ALL_STMTS.stream().collect(Collectors.joining(";" + System.lineSeparator())) + ";";
|
||||||
TestHelper.execute(statementsBuilder);
|
TestHelper.execute(statementsBuilder);
|
||||||
|
|
||||||
//then start the producer and validate all records are there
|
//then start the producer and validate all records are there
|
||||||
snapshotProducer.start(consumer, e -> {});
|
buildNoStreamProducer(TestHelper.defaultConfig());
|
||||||
|
|
||||||
|
TestConsumer consumer = testConsumer(ALL_STMTS.size(), "public", "Quoted__");
|
||||||
consumer.await(TestHelper.waitTimeForRecords() * 30, TimeUnit.SECONDS);
|
consumer.await(TestHelper.waitTimeForRecords() * 30, TimeUnit.SECONDS);
|
||||||
|
|
||||||
Map<String, List<SchemaAndValueField>> expectedValuesByTopicName = super.schemaAndValuesByTopicName();
|
Map<String, List<SchemaAndValueField>> expectedValuesByTopicName = super.schemaAndValuesByTopicName();
|
||||||
consumer.process(record -> assertReadRecord(record, expectedValuesByTopicName));
|
consumer.process(record -> assertReadRecord(record, expectedValuesByTopicName));
|
||||||
|
Testing.Print.enable();
|
||||||
// check the offset information for each record
|
// check the offset information for each record
|
||||||
while (!consumer.isEmpty()) {
|
while (!consumer.isEmpty()) {
|
||||||
SourceRecord record = consumer.remove();
|
SourceRecord record = consumer.remove();
|
||||||
@ -115,25 +87,14 @@ public void shouldGenerateSnapshotsForDefaultDatatypes() throws Exception {
|
|||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void shouldGenerateSnapshotsForCustomDatatypes() throws Exception {
|
public void shouldGenerateSnapshotsForCustomDatatypes() throws Exception {
|
||||||
final PostgresConnectorConfig config = new PostgresConnectorConfig(
|
|
||||||
TestHelper.defaultConfig()
|
|
||||||
.with(PostgresConnectorConfig.SNAPSHOT_MODE, PostgresConnectorConfig.SnapshotMode.INITIAL)
|
|
||||||
.with(PostgresConnectorConfig.INCLUDE_UNKNOWN_DATATYPES, true)
|
|
||||||
.build()
|
|
||||||
);
|
|
||||||
context = new PostgresTaskContext(
|
|
||||||
config,
|
|
||||||
TestHelper.getSchema(config),
|
|
||||||
PostgresTopicSelector.create(config)
|
|
||||||
);
|
|
||||||
snapshotProducer = buildNoStreamProducer(context, config);
|
|
||||||
|
|
||||||
final TestConsumer consumer = testConsumer(1, "public");
|
|
||||||
|
|
||||||
TestHelper.execute(INSERT_CUSTOM_TYPES_STMT);
|
TestHelper.execute(INSERT_CUSTOM_TYPES_STMT);
|
||||||
|
|
||||||
//then start the producer and validate all records are there
|
//then start the producer and validate all records are there
|
||||||
snapshotProducer.start(consumer, e -> {});
|
buildNoStreamProducer(TestHelper.defaultConfig()
|
||||||
|
.with(PostgresConnectorConfig.INCLUDE_UNKNOWN_DATATYPES, true)
|
||||||
|
);
|
||||||
|
|
||||||
|
final TestConsumer consumer = testConsumer(1, "public");
|
||||||
consumer.await(TestHelper.waitTimeForRecords() * 30, TimeUnit.SECONDS);
|
consumer.await(TestHelper.waitTimeForRecords() * 30, TimeUnit.SECONDS);
|
||||||
|
|
||||||
final Map<String, List<SchemaAndValueField>> expectedValuesByTopicName = Collect.hashMapOf("public.custom_table", schemasAndValuesForCustomTypes());
|
final Map<String, List<SchemaAndValueField>> expectedValuesByTopicName = Collect.hashMapOf("public.custom_table", schemasAndValuesForCustomTypes());
|
||||||
@ -146,14 +107,6 @@ public void shouldGenerateSnapshotAndContinueStreaming() throws Exception {
|
|||||||
TestHelper.dropAllSchemas();
|
TestHelper.dropAllSchemas();
|
||||||
TestHelper.executeDDL("postgres_create_tables.ddl");
|
TestHelper.executeDDL("postgres_create_tables.ddl");
|
||||||
|
|
||||||
PostgresConnectorConfig config = new PostgresConnectorConfig(TestHelper.defaultConfig().build());
|
|
||||||
TopicSelector<TableId> selector = PostgresTopicSelector.create(config);
|
|
||||||
context = new PostgresTaskContext(
|
|
||||||
config,
|
|
||||||
TestHelper.getSchema(config),
|
|
||||||
selector
|
|
||||||
);
|
|
||||||
|
|
||||||
String insertStmt = "INSERT INTO s1.a (aa) VALUES (1);" +
|
String insertStmt = "INSERT INTO s1.a (aa) VALUES (1);" +
|
||||||
"INSERT INTO s2.a (aa) VALUES (1);";
|
"INSERT INTO s2.a (aa) VALUES (1);";
|
||||||
|
|
||||||
@ -164,42 +117,43 @@ public void shouldGenerateSnapshotAndContinueStreaming() throws Exception {
|
|||||||
insertStmt;
|
insertStmt;
|
||||||
TestHelper.execute(statements);
|
TestHelper.execute(statements);
|
||||||
|
|
||||||
snapshotProducer = buildWithStreamProducer(context, config);
|
buildWithStreamProducer(TestHelper.defaultConfig());
|
||||||
|
|
||||||
TestConsumer consumer = testConsumer(2, "s1", "s2");
|
TestConsumer consumer = testConsumer(2, "s1", "s2");
|
||||||
snapshotProducer.start(consumer, e -> {});
|
|
||||||
|
|
||||||
// first make sure we get the initial records from both schemas...
|
// first make sure we get the initial records from both schemas...
|
||||||
consumer.await(TestHelper.waitTimeForRecords(), TimeUnit.SECONDS);
|
consumer.await(TestHelper.waitTimeForRecords(), TimeUnit.SECONDS);
|
||||||
consumer.clear();
|
consumer.clear();
|
||||||
|
|
||||||
// then insert some more data and check that we get it back
|
// then insert some more data and check that we get it back
|
||||||
waitForStreamingToStart(snapshotProducer);
|
waitForStreamingToStart();
|
||||||
TestHelper.execute(insertStmt);
|
TestHelper.execute(insertStmt);
|
||||||
consumer.expects(2);
|
consumer.expects(2);
|
||||||
consumer.await(TestHelper.waitTimeForRecords(), TimeUnit.SECONDS);
|
consumer.await(TestHelper.waitTimeForRecords() * 30, TimeUnit.SECONDS);
|
||||||
|
|
||||||
SourceRecord first = consumer.remove();
|
SourceRecord first = consumer.remove();
|
||||||
VerifyRecord.isValidInsert(first, PK_FIELD, 2);
|
VerifyRecord.isValidInsert(first, PK_FIELD, 2);
|
||||||
assertEquals(topicName("s1.a"), first.topic());
|
assertEquals(topicName("s1.a"), first.topic());
|
||||||
assertRecordOffsetAndSnapshotSource(first, false, false);
|
assertRecordOffsetAndSnapshotSource(first, false, false);
|
||||||
assertSourceInfo(first, "test_database", "s1", "a");
|
assertSourceInfo(first, TestHelper.TEST_DATABASE, "s1", "a");
|
||||||
|
|
||||||
SourceRecord second = consumer.remove();
|
SourceRecord second = consumer.remove();
|
||||||
VerifyRecord.isValidInsert(second, PK_FIELD, 2);
|
VerifyRecord.isValidInsert(second, PK_FIELD, 2);
|
||||||
assertEquals(topicName("s2.a"), second.topic());
|
assertEquals(topicName("s2.a"), second.topic());
|
||||||
assertRecordOffsetAndSnapshotSource(second, false, false);
|
assertRecordOffsetAndSnapshotSource(second, false, false);
|
||||||
assertSourceInfo(second, "test_database", "s2", "a");
|
assertSourceInfo(second, TestHelper.TEST_DATABASE, "s2", "a");
|
||||||
|
|
||||||
// now shut down the producers and insert some more records
|
// now shut down the producers and insert some more records
|
||||||
snapshotProducer.stop();
|
stopConnector();
|
||||||
|
assertConnectorNotRunning();
|
||||||
TestHelper.execute(insertStmt);
|
TestHelper.execute(insertStmt);
|
||||||
|
|
||||||
// start a new producer back up, take a new snapshot (we expect all the records to be read back)
|
// start a new producer back up, take a new snapshot (we expect all the records to be read back)
|
||||||
int expectedRecordsCount = 6;
|
int expectedRecordsCount = 6;
|
||||||
|
buildWithStreamProducer(TestHelper.defaultConfig());
|
||||||
|
|
||||||
consumer = testConsumer(expectedRecordsCount, "s1", "s2");
|
consumer = testConsumer(expectedRecordsCount, "s1", "s2");
|
||||||
snapshotProducer = buildWithStreamProducer(context, config);
|
consumer.await(TestHelper.waitTimeForRecords() * 30, TimeUnit.SECONDS);
|
||||||
snapshotProducer.start(consumer, e -> {});
|
|
||||||
consumer.await(TestHelper.waitTimeForRecords(), TimeUnit.SECONDS);
|
|
||||||
|
|
||||||
AtomicInteger counter = new AtomicInteger(0);
|
AtomicInteger counter = new AtomicInteger(0);
|
||||||
consumer.process(record -> {
|
consumer.process(record -> {
|
||||||
@ -212,7 +166,7 @@ public void shouldGenerateSnapshotAndContinueStreaming() throws Exception {
|
|||||||
consumer.clear();
|
consumer.clear();
|
||||||
|
|
||||||
// now insert two more records and check that we only get those back from the stream
|
// now insert two more records and check that we only get those back from the stream
|
||||||
waitForStreamingToStart(snapshotProducer);
|
waitForStreamingToStart();
|
||||||
TestHelper.execute(insertStmt);
|
TestHelper.execute(insertStmt);
|
||||||
consumer.expects(2);
|
consumer.expects(2);
|
||||||
|
|
||||||
@ -220,12 +174,12 @@ public void shouldGenerateSnapshotAndContinueStreaming() throws Exception {
|
|||||||
first = consumer.remove();
|
first = consumer.remove();
|
||||||
VerifyRecord.isValidInsert(first, PK_FIELD, 4);
|
VerifyRecord.isValidInsert(first, PK_FIELD, 4);
|
||||||
assertRecordOffsetAndSnapshotSource(first, false, false);
|
assertRecordOffsetAndSnapshotSource(first, false, false);
|
||||||
assertSourceInfo(first, "test_database", "s1", "a");
|
assertSourceInfo(first, TestHelper.TEST_DATABASE, "s1", "a");
|
||||||
|
|
||||||
second = consumer.remove();
|
second = consumer.remove();
|
||||||
VerifyRecord.isValidInsert(second, PK_FIELD, 4);
|
VerifyRecord.isValidInsert(second, PK_FIELD, 4);
|
||||||
assertRecordOffsetAndSnapshotSource(second, false, false);
|
assertRecordOffsetAndSnapshotSource(second, false, false);
|
||||||
assertSourceInfo(second, "test_database", "s2", "a");
|
assertSourceInfo(second, TestHelper.TEST_DATABASE, "s2", "a");
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
@ -235,23 +189,13 @@ public void shouldGenerateSnapshotAndSendHeartBeat() throws Exception {
|
|||||||
TestHelper.dropAllSchemas();
|
TestHelper.dropAllSchemas();
|
||||||
TestHelper.execute("CREATE TABLE t1 (pk SERIAL, aa integer, PRIMARY KEY(pk)); INSERT INTO t1 VALUES (default, 11)");
|
TestHelper.execute("CREATE TABLE t1 (pk SERIAL, aa integer, PRIMARY KEY(pk)); INSERT INTO t1 VALUES (default, 11)");
|
||||||
|
|
||||||
PostgresConnectorConfig config = new PostgresConnectorConfig(
|
buildWithStreamProducer(TestHelper.defaultConfig()
|
||||||
TestHelper.defaultConfig()
|
.with(PostgresConnectorConfig.SNAPSHOT_MODE, PostgresConnectorConfig.SnapshotMode.INITIAL)
|
||||||
.with(PostgresConnectorConfig.SNAPSHOT_MODE, PostgresConnectorConfig.SnapshotMode.INITIAL)
|
.with(PostgresConnectorConfig.INCLUDE_SCHEMA_CHANGES, true)
|
||||||
.with(PostgresConnectorConfig.INCLUDE_SCHEMA_CHANGES, true)
|
.with(Heartbeat.HEARTBEAT_INTERVAL, 300_000)
|
||||||
.with(Heartbeat.HEARTBEAT_INTERVAL, 300_000)
|
|
||||||
.build()
|
|
||||||
);
|
|
||||||
TopicSelector<TableId> selector = PostgresTopicSelector.create(config);
|
|
||||||
context = new PostgresTaskContext(
|
|
||||||
config,
|
|
||||||
TestHelper.getSchema(config),
|
|
||||||
selector
|
|
||||||
);
|
);
|
||||||
|
|
||||||
snapshotProducer = buildWithStreamProducer(context, config);
|
|
||||||
TestConsumer consumer = testConsumer(2);
|
TestConsumer consumer = testConsumer(2);
|
||||||
snapshotProducer.start(consumer, e -> {});
|
|
||||||
|
|
||||||
// Make sure we get the table schema record and heartbeat record
|
// Make sure we get the table schema record and heartbeat record
|
||||||
consumer.await(TestHelper.waitTimeForRecords(), TimeUnit.SECONDS);
|
consumer.await(TestHelper.waitTimeForRecords(), TimeUnit.SECONDS);
|
||||||
@ -262,21 +206,6 @@ public void shouldGenerateSnapshotAndSendHeartBeat() throws Exception {
|
|||||||
final SourceRecord second = consumer.remove();
|
final SourceRecord second = consumer.remove();
|
||||||
assertThat(second.topic()).startsWith("__debezium-heartbeat");
|
assertThat(second.topic()).startsWith("__debezium-heartbeat");
|
||||||
assertRecordOffsetAndSnapshotSource(second, false, false);
|
assertRecordOffsetAndSnapshotSource(second, false, false);
|
||||||
|
|
||||||
// now shut down the producers and insert some more records
|
|
||||||
snapshotProducer.stop();
|
|
||||||
}
|
|
||||||
|
|
||||||
private RecordsSnapshotProducer buildNoStreamProducer(PostgresTaskContext ctx, PostgresConnectorConfig config) {
|
|
||||||
Snapshotter sn = new InitialOnlySnapshotter();
|
|
||||||
SnapshotterWrapper snw = new SnapshotterWrapper(sn, config, null, null);
|
|
||||||
return new RecordsSnapshotProducer(ctx, TestHelper.sourceInfo(), snw);
|
|
||||||
}
|
|
||||||
|
|
||||||
private RecordsSnapshotProducer buildWithStreamProducer(PostgresTaskContext ctx, PostgresConnectorConfig config) {
|
|
||||||
Snapshotter sn = new AlwaysSnapshotter();
|
|
||||||
SnapshotterWrapper snw = new SnapshotterWrapper(sn, config, null, null);
|
|
||||||
return new RecordsSnapshotProducer(ctx, TestHelper.sourceInfo(), snw);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private void assertReadRecord(SourceRecord record, Map<String, List<SchemaAndValueField>> expectedValuesByTopicName) {
|
private void assertReadRecord(SourceRecord record, Map<String, List<SchemaAndValueField>> expectedValuesByTopicName) {
|
||||||
@ -290,28 +219,16 @@ private void assertReadRecord(SourceRecord record, Map<String, List<SchemaAndVal
|
|||||||
@Test
|
@Test
|
||||||
@FixFor("DBZ-342")
|
@FixFor("DBZ-342")
|
||||||
public void shouldGenerateSnapshotsForDefaultDatatypesAdaptiveMicroseconds() throws Exception {
|
public void shouldGenerateSnapshotsForDefaultDatatypesAdaptiveMicroseconds() throws Exception {
|
||||||
PostgresConnectorConfig config = new PostgresConnectorConfig(
|
|
||||||
TestHelper.defaultConfig()
|
|
||||||
.with(PostgresConnectorConfig.TIME_PRECISION_MODE, TemporalPrecisionMode.ADAPTIVE_TIME_MICROSECONDS)
|
|
||||||
.build());
|
|
||||||
|
|
||||||
TopicSelector<TableId> selector = PostgresTopicSelector.create(config);
|
|
||||||
context = new PostgresTaskContext(
|
|
||||||
config,
|
|
||||||
TestHelper.getSchema(config),
|
|
||||||
selector
|
|
||||||
);
|
|
||||||
|
|
||||||
snapshotProducer = buildNoStreamProducer(context, config);
|
|
||||||
|
|
||||||
TestConsumer consumer = testConsumer(ALL_STMTS.size(), "public", "Quoted__");
|
|
||||||
|
|
||||||
//insert data for each of different supported types
|
//insert data for each of different supported types
|
||||||
String statementsBuilder = ALL_STMTS.stream().collect(Collectors.joining(";" + System.lineSeparator())) + ";";
|
String statementsBuilder = ALL_STMTS.stream().collect(Collectors.joining(";" + System.lineSeparator())) + ";";
|
||||||
TestHelper.execute(statementsBuilder);
|
TestHelper.execute(statementsBuilder);
|
||||||
|
|
||||||
//then start the producer and validate all records are there
|
//then start the producer and validate all records are there
|
||||||
snapshotProducer.start(consumer, e -> {});
|
buildNoStreamProducer(TestHelper.defaultConfig()
|
||||||
|
.with(PostgresConnectorConfig.TIME_PRECISION_MODE, TemporalPrecisionMode.ADAPTIVE_TIME_MICROSECONDS)
|
||||||
|
);
|
||||||
|
|
||||||
|
TestConsumer consumer = testConsumer(ALL_STMTS.size(), "public", "Quoted__");
|
||||||
consumer.await(TestHelper.waitTimeForRecords() * 30, TimeUnit.SECONDS);
|
consumer.await(TestHelper.waitTimeForRecords() * 30, TimeUnit.SECONDS);
|
||||||
|
|
||||||
Map<String, List<SchemaAndValueField>> expectedValuesByTopicName = super.schemaAndValuesByTopicNameAdaptiveTimeMicroseconds();
|
Map<String, List<SchemaAndValueField>> expectedValuesByTopicName = super.schemaAndValuesByTopicNameAdaptiveTimeMicroseconds();
|
||||||
@ -332,27 +249,15 @@ public void shouldGenerateSnapshotsForDecimalDatatypesUsingStringEncoding() thro
|
|||||||
TestHelper.dropAllSchemas();
|
TestHelper.dropAllSchemas();
|
||||||
TestHelper.executeDDL("postgres_create_tables.ddl");
|
TestHelper.executeDDL("postgres_create_tables.ddl");
|
||||||
|
|
||||||
PostgresConnectorConfig config = new PostgresConnectorConfig(
|
|
||||||
TestHelper.defaultConfig()
|
|
||||||
.with(PostgresConnectorConfig.DECIMAL_HANDLING_MODE, DecimalHandlingMode.STRING)
|
|
||||||
.build());
|
|
||||||
|
|
||||||
TopicSelector<TableId> selector = PostgresTopicSelector.create(config);
|
|
||||||
context = new PostgresTaskContext(
|
|
||||||
config,
|
|
||||||
TestHelper.getSchema(config),
|
|
||||||
selector
|
|
||||||
);
|
|
||||||
|
|
||||||
snapshotProducer = buildNoStreamProducer(context, config);
|
|
||||||
|
|
||||||
TestConsumer consumer = testConsumer(1, "public", "Quoted_\"");
|
|
||||||
|
|
||||||
//insert data for each of different supported types
|
//insert data for each of different supported types
|
||||||
TestHelper.execute(INSERT_NUMERIC_DECIMAL_TYPES_STMT);
|
TestHelper.execute(INSERT_NUMERIC_DECIMAL_TYPES_STMT);
|
||||||
|
|
||||||
//then start the producer and validate all records are there
|
//then start the producer and validate all records are there
|
||||||
snapshotProducer.start(consumer, e -> {});
|
buildNoStreamProducer(TestHelper.defaultConfig()
|
||||||
|
.with(PostgresConnectorConfig.DECIMAL_HANDLING_MODE, DecimalHandlingMode.STRING)
|
||||||
|
);
|
||||||
|
|
||||||
|
TestConsumer consumer = testConsumer(1, "public", "Quoted__");
|
||||||
consumer.await(TestHelper.waitTimeForRecords() * 30, TimeUnit.SECONDS);
|
consumer.await(TestHelper.waitTimeForRecords() * 30, TimeUnit.SECONDS);
|
||||||
|
|
||||||
Map<String, List<SchemaAndValueField>> expectedValuesByTopicName = super.schemaAndValuesByTopicNameStringEncodedDecimals();
|
Map<String, List<SchemaAndValueField>> expectedValuesByTopicName = super.schemaAndValuesByTopicNameStringEncodedDecimals();
|
||||||
@ -386,21 +291,6 @@ public void shouldGenerateSnapshotsForPartitionedTables() throws Exception {
|
|||||||
|
|
||||||
TestHelper.execute(ddl);
|
TestHelper.execute(ddl);
|
||||||
|
|
||||||
PostgresConnectorConfig config = new PostgresConnectorConfig(
|
|
||||||
TestHelper.defaultConfig()
|
|
||||||
.build());
|
|
||||||
|
|
||||||
TopicSelector<TableId> selector = PostgresTopicSelector.create(config);
|
|
||||||
context = new PostgresTaskContext(
|
|
||||||
config,
|
|
||||||
TestHelper.getSchema(config),
|
|
||||||
selector
|
|
||||||
);
|
|
||||||
|
|
||||||
snapshotProducer = buildNoStreamProducer(context, config);
|
|
||||||
|
|
||||||
TestConsumer consumer = testConsumer(1 + 2 * 30); // Every record comes once from partitioned table and from partition
|
|
||||||
|
|
||||||
// add 1 record to `first_table`. To reproduce the bug we must process at
|
// add 1 record to `first_table`. To reproduce the bug we must process at
|
||||||
// least one row before processing the partitioned table.
|
// least one row before processing the partitioned table.
|
||||||
TestHelper.execute("INSERT INTO first_table (pk, user_id) VALUES (1000, 1);");
|
TestHelper.execute("INSERT INTO first_table (pk, user_id) VALUES (1000, 1);");
|
||||||
@ -414,7 +304,9 @@ public void shouldGenerateSnapshotsForPartitionedTables() throws Exception {
|
|||||||
"FROM generate_series(1, 20);");
|
"FROM generate_series(1, 20);");
|
||||||
|
|
||||||
// then start the producer and validate all records are there
|
// then start the producer and validate all records are there
|
||||||
snapshotProducer.start(consumer, e -> {});
|
buildNoStreamProducer(TestHelper.defaultConfig());
|
||||||
|
|
||||||
|
TestConsumer consumer = testConsumer(1 + 2 * 30); // Every record comes once from partitioned table and from partition
|
||||||
consumer.await(TestHelper.waitTimeForRecords() * 30, TimeUnit.SECONDS);
|
consumer.await(TestHelper.waitTimeForRecords() * 30, TimeUnit.SECONDS);
|
||||||
|
|
||||||
Set<Integer> ids = new HashSet<>();
|
Set<Integer> ids = new HashSet<>();
|
||||||
@ -459,27 +351,13 @@ public void shouldGenerateSnapshotsForHstores() throws Exception {
|
|||||||
TestHelper.dropAllSchemas();
|
TestHelper.dropAllSchemas();
|
||||||
TestHelper.executeDDL("postgres_create_tables.ddl");
|
TestHelper.executeDDL("postgres_create_tables.ddl");
|
||||||
|
|
||||||
PostgresConnectorConfig config = new PostgresConnectorConfig(
|
|
||||||
TestHelper.defaultConfig()
|
|
||||||
.with(PostgresConnectorConfig.HSTORE_HANDLING_MODE, PostgresConnectorConfig.HStoreHandlingMode.JSON)
|
|
||||||
.build());
|
|
||||||
|
|
||||||
TopicSelector<TableId> selector = PostgresTopicSelector.create(config);
|
|
||||||
context = new PostgresTaskContext(
|
|
||||||
config,
|
|
||||||
TestHelper.getSchema(config),
|
|
||||||
selector
|
|
||||||
);
|
|
||||||
|
|
||||||
snapshotProducer = buildNoStreamProducer(context, config);
|
|
||||||
|
|
||||||
TestConsumer consumer = testConsumer(1, "public", "Quoted_\"");
|
|
||||||
|
|
||||||
//insert data for each of different supported types
|
//insert data for each of different supported types
|
||||||
TestHelper.execute(INSERT_HSTORE_TYPE_STMT);
|
TestHelper.execute(INSERT_HSTORE_TYPE_STMT);
|
||||||
|
|
||||||
//then start the producer and validate all records are there
|
//then start the producer and validate all records are there
|
||||||
snapshotProducer.start(consumer, e -> {});
|
buildNoStreamProducer(TestHelper.defaultConfig());
|
||||||
|
|
||||||
|
TestConsumer consumer = testConsumer(1, "public", "Quoted__");
|
||||||
consumer.await(TestHelper.waitTimeForRecords() * 30, TimeUnit.SECONDS);
|
consumer.await(TestHelper.waitTimeForRecords() * 30, TimeUnit.SECONDS);
|
||||||
|
|
||||||
|
|
||||||
@ -491,24 +369,12 @@ public void shouldGenerateSnapshotsForHstores() throws Exception {
|
|||||||
@Test
|
@Test
|
||||||
@FixFor("DBZ-1163")
|
@FixFor("DBZ-1163")
|
||||||
public void shouldGenerateSnapshotForATableWithoutPrimaryKey() throws Exception {
|
public void shouldGenerateSnapshotForATableWithoutPrimaryKey() throws Exception {
|
||||||
final PostgresConnectorConfig config = new PostgresConnectorConfig(
|
|
||||||
TestHelper.defaultConfig()
|
|
||||||
.with(PostgresConnectorConfig.SNAPSHOT_MODE, PostgresConnectorConfig.SnapshotMode.INITIAL)
|
|
||||||
.build()
|
|
||||||
);
|
|
||||||
context = new PostgresTaskContext(
|
|
||||||
config,
|
|
||||||
TestHelper.getSchema(config),
|
|
||||||
PostgresTopicSelector.create(config)
|
|
||||||
);
|
|
||||||
snapshotProducer = buildNoStreamProducer(context, config);
|
|
||||||
|
|
||||||
final TestConsumer consumer = testConsumer(1, "public");
|
|
||||||
|
|
||||||
TestHelper.execute("insert into table_without_pk values(1, 1000)");
|
TestHelper.execute("insert into table_without_pk values(1, 1000)");
|
||||||
|
|
||||||
//then start the producer and validate all records are there
|
//then start the producer and validate all records are there
|
||||||
snapshotProducer.start(consumer, e -> {});
|
buildNoStreamProducer(TestHelper.defaultConfig());
|
||||||
|
|
||||||
|
TestConsumer consumer = testConsumer(1, "public", "Quoted__");
|
||||||
consumer.await(TestHelper.waitTimeForRecords() * 30, TimeUnit.SECONDS);
|
consumer.await(TestHelper.waitTimeForRecords() * 30, TimeUnit.SECONDS);
|
||||||
|
|
||||||
List<SchemaAndValueField> schemaAndValueFields = Arrays.asList(
|
List<SchemaAndValueField> schemaAndValueFields = Arrays.asList(
|
||||||
@ -531,17 +397,13 @@ public void shouldGenerateSnapshotForMacaddr8Datatype() throws Exception {
|
|||||||
TestHelper.dropAllSchemas();
|
TestHelper.dropAllSchemas();
|
||||||
TestHelper.execute("CREATE TABLE macaddr8_table(pk SERIAL, m MACADDR8, PRIMARY KEY(pk));");
|
TestHelper.execute("CREATE TABLE macaddr8_table(pk SERIAL, m MACADDR8, PRIMARY KEY(pk));");
|
||||||
|
|
||||||
PostgresConnectorConfig config = new PostgresConnectorConfig(TestHelper.defaultConfig()
|
|
||||||
.build());
|
|
||||||
snapshotProducer = buildNoStreamProducer(context, config);
|
|
||||||
|
|
||||||
final TestConsumer consumer = testConsumer(1, "public");
|
|
||||||
|
|
||||||
// insert macaddr8 data
|
// insert macaddr8 data
|
||||||
TestHelper.execute(INSERT_MACADDR8_TYPE_STMT);
|
TestHelper.execute(INSERT_MACADDR8_TYPE_STMT);
|
||||||
|
|
||||||
// then start the producer and validate the record are there
|
// then start the producer and validate the record are there
|
||||||
snapshotProducer.start(consumer, e -> {});
|
buildNoStreamProducer(TestHelper.defaultConfig());
|
||||||
|
|
||||||
|
TestConsumer consumer = testConsumer(1, "public");
|
||||||
consumer.await(TestHelper.waitTimeForRecords() * 30, TimeUnit.SECONDS);
|
consumer.await(TestHelper.waitTimeForRecords() * 30, TimeUnit.SECONDS);
|
||||||
|
|
||||||
final Map<String, List<SchemaAndValueField>> expectedValueByTopicName = Collect.hashMapOf("public.macaddr8_table", schemaAndValueForMacaddr8Type());
|
final Map<String, List<SchemaAndValueField>> expectedValueByTopicName = Collect.hashMapOf("public.macaddr8_table", schemaAndValueForMacaddr8Type());
|
||||||
@ -555,16 +417,12 @@ public void shouldGenerateSnapshotForTwentyFourHourTime() throws Exception {
|
|||||||
TestHelper.dropAllSchemas();
|
TestHelper.dropAllSchemas();
|
||||||
TestHelper.executeDDL("postgres_create_tables.ddl");
|
TestHelper.executeDDL("postgres_create_tables.ddl");
|
||||||
|
|
||||||
PostgresConnectorConfig config = new PostgresConnectorConfig(TestHelper.defaultConfig()
|
|
||||||
.build());
|
|
||||||
snapshotProducer = buildNoStreamProducer(context, config);
|
|
||||||
|
|
||||||
final TestConsumer consumer = testConsumer(1, "public");
|
|
||||||
|
|
||||||
// insert data and time data
|
// insert data and time data
|
||||||
TestHelper.execute(INSERT_DATE_TIME_TYPES_STMT);
|
TestHelper.execute(INSERT_DATE_TIME_TYPES_STMT);
|
||||||
|
|
||||||
snapshotProducer.start(consumer, e -> {});
|
buildNoStreamProducer(TestHelper.defaultConfig());
|
||||||
|
|
||||||
|
TestConsumer consumer = testConsumer(1, "public");
|
||||||
consumer.await(TestHelper.waitTimeForRecords() * 30, TimeUnit.SECONDS);
|
consumer.await(TestHelper.waitTimeForRecords() * 30, TimeUnit.SECONDS);
|
||||||
|
|
||||||
final Map<String, List<SchemaAndValueField>> expectedValueByTopicName = Collect.hashMapOf("public.time_table", schemaAndValuesForDateTimeTypes());
|
final Map<String, List<SchemaAndValueField>> expectedValueByTopicName = Collect.hashMapOf("public.time_table", schemaAndValuesForDateTimeTypes());
|
||||||
@ -579,19 +437,36 @@ public void shouldNotSnapshotMaterializedViews() throws Exception {
|
|||||||
TestHelper.execute("CREATE TABLE mv_real_table (pk SERIAL, i integer, s VARCHAR(50), PRIMARY KEY(pk));");
|
TestHelper.execute("CREATE TABLE mv_real_table (pk SERIAL, i integer, s VARCHAR(50), PRIMARY KEY(pk));");
|
||||||
TestHelper.execute("CREATE MATERIALIZED VIEW mv (pk, s) AS SELECT mrv.pk, mrv.s FROM mv_real_table mrv WITH DATA;");
|
TestHelper.execute("CREATE MATERIALIZED VIEW mv (pk, s) AS SELECT mrv.pk, mrv.s FROM mv_real_table mrv WITH DATA;");
|
||||||
|
|
||||||
PostgresConnectorConfig config = new PostgresConnectorConfig(TestHelper.defaultConfig().build());
|
|
||||||
snapshotProducer = buildNoStreamProducer(context, config);
|
|
||||||
|
|
||||||
final TestConsumer consumer = testConsumer(1, "public");
|
|
||||||
|
|
||||||
// insert data
|
// insert data
|
||||||
TestHelper.execute("INSERT INTO mv_real_table (i,s) VALUES (1,'1');");
|
TestHelper.execute("INSERT INTO mv_real_table (i,s) VALUES (1,'1');");
|
||||||
TestHelper.execute("REFRESH MATERIALIZED VIEW mv WITH DATA;");
|
TestHelper.execute("REFRESH MATERIALIZED VIEW mv WITH DATA;");
|
||||||
|
|
||||||
snapshotProducer.start(consumer, e -> {});
|
buildNoStreamProducer(TestHelper.defaultConfig());
|
||||||
|
|
||||||
|
TestConsumer consumer = testConsumer(1, "public");
|
||||||
consumer.await(TestHelper.waitTimeForRecords() * 30, TimeUnit.SECONDS);
|
consumer.await(TestHelper.waitTimeForRecords() * 30, TimeUnit.SECONDS);
|
||||||
|
|
||||||
final Map<String, List<SchemaAndValueField>> expectedValueByTopicName = Collect.hashMapOf("public.mv_real_table", schemaAndValueForMaterializedViewBaseType());
|
final Map<String, List<SchemaAndValueField>> expectedValueByTopicName = Collect.hashMapOf("public.mv_real_table", schemaAndValueForMaterializedViewBaseType());
|
||||||
consumer.process(record ->assertReadRecord(record, expectedValueByTopicName));
|
consumer.process(record ->assertReadRecord(record, expectedValueByTopicName));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private void buildNoStreamProducer(Configuration.Builder config) {
|
||||||
|
start(PostgresConnector.class, config
|
||||||
|
.with(PostgresConnectorConfig.SNAPSHOT_MODE, SnapshotMode.INITIAL_ONLY)
|
||||||
|
.with(PostgresConnectorConfig.SNAPSHOT_MODE_CLASS, CustomTestSnapshot.class.getName())
|
||||||
|
.with(PostgresConnectorConfig.DROP_SLOT_ON_STOP, Boolean.FALSE)
|
||||||
|
.build()
|
||||||
|
);
|
||||||
|
assertConnectorIsRunning();
|
||||||
|
}
|
||||||
|
|
||||||
|
private void buildWithStreamProducer(Configuration.Builder config) {
|
||||||
|
start(PostgresConnector.class, config
|
||||||
|
.with(PostgresConnectorConfig.SNAPSHOT_MODE, SnapshotMode.ALWAYS)
|
||||||
|
.with(PostgresConnectorConfig.SNAPSHOT_MODE_CLASS, CustomTestSnapshot.class.getName())
|
||||||
|
.with(PostgresConnectorConfig.DROP_SLOT_ON_STOP, Boolean.FALSE)
|
||||||
|
.build()
|
||||||
|
);
|
||||||
|
assertConnectorIsRunning();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
@ -8,7 +8,9 @@
|
|||||||
|
|
||||||
import static io.debezium.connector.postgresql.TestHelper.PK_FIELD;
|
import static io.debezium.connector.postgresql.TestHelper.PK_FIELD;
|
||||||
import static io.debezium.connector.postgresql.TestHelper.topicName;
|
import static io.debezium.connector.postgresql.TestHelper.topicName;
|
||||||
|
import static io.debezium.connector.postgresql.junit.SkipWhenDecoderPluginNameIs.DecoderPluginName.DECODERBUFS;
|
||||||
import static io.debezium.connector.postgresql.junit.SkipWhenDecoderPluginNameIs.DecoderPluginName.PGOUTPUT;
|
import static io.debezium.connector.postgresql.junit.SkipWhenDecoderPluginNameIs.DecoderPluginName.PGOUTPUT;
|
||||||
|
import static io.debezium.connector.postgresql.junit.SkipWhenDecoderPluginNameIsNot.DecoderPluginName.WAL2JSON;
|
||||||
import static junit.framework.TestCase.assertEquals;
|
import static junit.framework.TestCase.assertEquals;
|
||||||
import static org.fest.assertions.Assertions.assertThat;
|
import static org.fest.assertions.Assertions.assertThat;
|
||||||
import static org.junit.Assert.assertFalse;
|
import static org.junit.Assert.assertFalse;
|
||||||
@ -19,29 +21,26 @@
|
|||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.concurrent.TimeUnit;
|
import java.util.concurrent.TimeUnit;
|
||||||
import java.util.function.Consumer;
|
import java.util.function.Function;
|
||||||
|
|
||||||
import static io.debezium.connector.postgresql.junit.SkipWhenDecoderPluginNameIsNot.DecoderPluginName.WAL2JSON;
|
|
||||||
import static io.debezium.connector.postgresql.junit.SkipWhenDecoderPluginNameIs.DecoderPluginName.DECODERBUFS;
|
|
||||||
|
|
||||||
import io.debezium.config.Configuration;
|
|
||||||
import io.debezium.connector.postgresql.junit.SkipTestDependingOnDecoderPluginNameRule;
|
|
||||||
import io.debezium.connector.postgresql.junit.SkipWhenDecoderPluginNameIs;
|
|
||||||
import io.debezium.connector.postgresql.junit.SkipWhenDecoderPluginNameIsNot;
|
|
||||||
import org.apache.commons.lang3.RandomStringUtils;
|
import org.apache.commons.lang3.RandomStringUtils;
|
||||||
import org.apache.kafka.connect.data.Decimal;
|
import org.apache.kafka.connect.data.Decimal;
|
||||||
import org.apache.kafka.connect.data.SchemaBuilder;
|
import org.apache.kafka.connect.data.SchemaBuilder;
|
||||||
import org.apache.kafka.connect.data.Struct;
|
import org.apache.kafka.connect.data.Struct;
|
||||||
import org.apache.kafka.connect.source.SourceRecord;
|
import org.apache.kafka.connect.source.SourceRecord;
|
||||||
import org.fest.assertions.Assertions;
|
import org.fest.assertions.Assertions;
|
||||||
import org.junit.After;
|
|
||||||
import org.junit.Before;
|
import org.junit.Before;
|
||||||
import org.junit.Rule;
|
import org.junit.Rule;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
import org.junit.rules.TestRule;
|
import org.junit.rules.TestRule;
|
||||||
|
|
||||||
import io.debezium.config.CommonConnectorConfig;
|
import io.debezium.config.CommonConnectorConfig;
|
||||||
|
import io.debezium.config.Configuration;
|
||||||
import io.debezium.connector.postgresql.PostgresConnectorConfig.SchemaRefreshMode;
|
import io.debezium.connector.postgresql.PostgresConnectorConfig.SchemaRefreshMode;
|
||||||
|
import io.debezium.connector.postgresql.PostgresConnectorConfig.SnapshotMode;
|
||||||
|
import io.debezium.connector.postgresql.junit.SkipTestDependingOnDecoderPluginNameRule;
|
||||||
|
import io.debezium.connector.postgresql.junit.SkipWhenDecoderPluginNameIs;
|
||||||
|
import io.debezium.connector.postgresql.junit.SkipWhenDecoderPluginNameIsNot;
|
||||||
import io.debezium.data.Envelope;
|
import io.debezium.data.Envelope;
|
||||||
import io.debezium.data.VariableScaleDecimal;
|
import io.debezium.data.VariableScaleDecimal;
|
||||||
import io.debezium.data.VerifyRecord;
|
import io.debezium.data.VerifyRecord;
|
||||||
@ -52,7 +51,6 @@
|
|||||||
import io.debezium.junit.ShouldFailWhen;
|
import io.debezium.junit.ShouldFailWhen;
|
||||||
import io.debezium.relational.Table;
|
import io.debezium.relational.Table;
|
||||||
import io.debezium.relational.TableId;
|
import io.debezium.relational.TableId;
|
||||||
import io.debezium.schema.TopicSelector;
|
|
||||||
import io.debezium.util.Testing;
|
import io.debezium.util.Testing;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -63,9 +61,7 @@
|
|||||||
*/
|
*/
|
||||||
public class RecordsStreamProducerIT extends AbstractRecordsProducerTest {
|
public class RecordsStreamProducerIT extends AbstractRecordsProducerTest {
|
||||||
|
|
||||||
private RecordsStreamProducer recordsProducer;
|
|
||||||
private TestConsumer consumer;
|
private TestConsumer consumer;
|
||||||
private final Consumer<Throwable> blackHole = t -> {};
|
|
||||||
|
|
||||||
@Rule
|
@Rule
|
||||||
public final TestRule skip = new SkipTestDependingOnDecoderPluginNameRule();
|
public final TestRule skip = new SkipTestDependingOnDecoderPluginNameRule();
|
||||||
@ -97,25 +93,44 @@ public void before() throws Exception {
|
|||||||
.with("assumeMinServerVersion.set", "9.4");
|
.with("assumeMinServerVersion.set", "9.4");
|
||||||
}
|
}
|
||||||
|
|
||||||
PostgresConnectorConfig config = new PostgresConnectorConfig(configBuilder.build());
|
Testing.Print.enable();
|
||||||
setupRecordsProducer(config);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@After
|
private void startConnector(Function<Configuration.Builder, Configuration.Builder> customConfig, boolean waitForSnapshot) throws InterruptedException {
|
||||||
public void after() throws Exception {
|
start(PostgresConnector.class, new PostgresConnectorConfig(customConfig.apply(TestHelper.defaultConfig()
|
||||||
if (recordsProducer != null) {
|
.with(PostgresConnectorConfig.INCLUDE_UNKNOWN_DATATYPES, false)
|
||||||
recordsProducer.stop();
|
.with(PostgresConnectorConfig.SCHEMA_BLACKLIST, "postgis")
|
||||||
|
.with(PostgresConnectorConfig.SNAPSHOT_MODE, SnapshotMode.INITIAL))
|
||||||
|
.build()).getConfig()
|
||||||
|
);
|
||||||
|
assertConnectorIsRunning();
|
||||||
|
waitForStreamingToStart();
|
||||||
|
|
||||||
|
if (waitForSnapshot) {
|
||||||
|
// Wait for snapshot to be in progress
|
||||||
|
consumer = testConsumer(1);
|
||||||
|
consumer.await(TestHelper.waitTimeForRecords(), TimeUnit.SECONDS);
|
||||||
|
consumer.remove();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private void startConnector(Function<Configuration.Builder, Configuration.Builder> customConfig) throws InterruptedException {
|
||||||
|
startConnector(customConfig, true);
|
||||||
|
}
|
||||||
|
|
||||||
|
private void startConnector() throws InterruptedException {
|
||||||
|
startConnector(Function.identity(), true);
|
||||||
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void shouldReceiveChangesForInsertsWithDifferentDataTypes() throws Exception {
|
public void shouldReceiveChangesForInsertsWithDifferentDataTypes() throws Exception {
|
||||||
TestHelper.executeDDL("postgres_create_tables.ddl");
|
TestHelper.executeDDL("postgres_create_tables.ddl");
|
||||||
|
startConnector();
|
||||||
|
|
||||||
consumer = testConsumer(1);
|
consumer = testConsumer(1);
|
||||||
recordsProducer.start(consumer, blackHole);
|
|
||||||
|
|
||||||
//numerical types
|
//numerical types
|
||||||
|
consumer.expects(1);
|
||||||
assertInsert(INSERT_NUMERIC_TYPES_STMT, 1, schemasAndValuesForNumericType());
|
assertInsert(INSERT_NUMERIC_TYPES_STMT, 1, schemasAndValuesForNumericType());
|
||||||
|
|
||||||
//numerical decimal types
|
//numerical decimal types
|
||||||
@ -157,12 +172,10 @@ public void shouldReceiveChangesAfterConnectionRestart() throws Exception {
|
|||||||
TestHelper.dropDefaultReplicationSlot();
|
TestHelper.dropDefaultReplicationSlot();
|
||||||
TestHelper.dropPublication();
|
TestHelper.dropPublication();
|
||||||
|
|
||||||
PostgresConnectorConfig config = new PostgresConnectorConfig(TestHelper.defaultConfig()
|
startConnector(config -> config
|
||||||
.with(PostgresConnectorConfig.INCLUDE_UNKNOWN_DATATYPES, true)
|
.with(PostgresConnectorConfig.INCLUDE_UNKNOWN_DATATYPES, true)
|
||||||
.with(PostgresConnectorConfig.SCHEMA_BLACKLIST, "postgis")
|
.with(PostgresConnectorConfig.SCHEMA_BLACKLIST, "postgis")
|
||||||
.build());
|
);
|
||||||
|
|
||||||
setupRecordsProducer(config);
|
|
||||||
|
|
||||||
TestHelper.execute("CREATE TABLE t0 (pk SERIAL, d INTEGER, PRIMARY KEY(pk));");
|
TestHelper.execute("CREATE TABLE t0 (pk SERIAL, d INTEGER, PRIMARY KEY(pk));");
|
||||||
|
|
||||||
@ -170,7 +183,6 @@ public void shouldReceiveChangesAfterConnectionRestart() throws Exception {
|
|||||||
|
|
||||||
// Start the producer and wait; the wait is to guarantee that the stream thread is polling
|
// Start the producer and wait; the wait is to guarantee that the stream thread is polling
|
||||||
// This appears to be a potential race condition problem
|
// This appears to be a potential race condition problem
|
||||||
recordsProducer.start(consumer, blackHole);
|
|
||||||
TimeUnit.SECONDS.sleep(TestHelper.waitTimeForRecords());
|
TimeUnit.SECONDS.sleep(TestHelper.waitTimeForRecords());
|
||||||
|
|
||||||
// Insert new row and verify inserted
|
// Insert new row and verify inserted
|
||||||
@ -178,19 +190,21 @@ public void shouldReceiveChangesAfterConnectionRestart() throws Exception {
|
|||||||
assertRecordInserted("public.t0", PK_FIELD, 1);
|
assertRecordInserted("public.t0", PK_FIELD, 1);
|
||||||
|
|
||||||
// simulate the connector is stopped
|
// simulate the connector is stopped
|
||||||
recordsProducer.stop();
|
stopConnector();
|
||||||
|
|
||||||
setupRecordsProducer(config);
|
|
||||||
|
|
||||||
// Alter schema offline
|
// Alter schema offline
|
||||||
TestHelper.execute("ALTER TABLE t0 ADD COLUMN d2 INTEGER;");
|
TestHelper.execute("ALTER TABLE t0 ADD COLUMN d2 INTEGER;");
|
||||||
TestHelper.execute("ALTER TABLE t0 ALTER COLUMN d SET NOT NULL;");
|
TestHelper.execute("ALTER TABLE t0 ALTER COLUMN d SET NOT NULL;");
|
||||||
|
|
||||||
consumer = testConsumer(1);
|
|
||||||
|
|
||||||
// Start the producer and wait; the wait is to guarantee the stream thread is polling
|
// Start the producer and wait; the wait is to guarantee the stream thread is polling
|
||||||
// This appears to be a potential race condition problem
|
// This appears to be a potential race condition problem
|
||||||
recordsProducer.start(consumer, blackHole);
|
startConnector(config -> config
|
||||||
|
.with(PostgresConnectorConfig.INCLUDE_UNKNOWN_DATATYPES, true)
|
||||||
|
.with(PostgresConnectorConfig.SCHEMA_BLACKLIST, "postgis"),
|
||||||
|
false
|
||||||
|
);
|
||||||
|
consumer = testConsumer(1);
|
||||||
TimeUnit.SECONDS.sleep(TestHelper.waitTimeForRecords());
|
TimeUnit.SECONDS.sleep(TestHelper.waitTimeForRecords());
|
||||||
|
|
||||||
// Insert new row and verify inserted
|
// Insert new row and verify inserted
|
||||||
@ -200,19 +214,11 @@ public void shouldReceiveChangesAfterConnectionRestart() throws Exception {
|
|||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void shouldReceiveChangesForInsertsCustomTypes() throws Exception {
|
public void shouldReceiveChangesForInsertsCustomTypes() throws Exception {
|
||||||
PostgresConnectorConfig config = new PostgresConnectorConfig(TestHelper.defaultConfig()
|
|
||||||
.with(PostgresConnectorConfig.INCLUDE_UNKNOWN_DATATYPES, true)
|
|
||||||
.with(PostgresConnectorConfig.SCHEMA_BLACKLIST, "postgis")
|
|
||||||
.build());
|
|
||||||
setupRecordsProducer(config);
|
|
||||||
TestHelper.executeDDL("postgres_create_tables.ddl");
|
TestHelper.executeDDL("postgres_create_tables.ddl");
|
||||||
|
|
||||||
consumer = testConsumer(1);
|
startConnector(config -> config.with(PostgresConnectorConfig.INCLUDE_UNKNOWN_DATATYPES, true));
|
||||||
recordsProducer.start(consumer, blackHole);
|
|
||||||
|
|
||||||
// custom types + null value
|
// custom types + null value
|
||||||
assertInsert(INSERT_CUSTOM_TYPES_STMT, 1, schemasAndValuesForCustomTypes());
|
assertInsert(INSERT_CUSTOM_TYPES_STMT, 1, schemasAndValuesForCustomTypes());
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
@ -261,17 +267,15 @@ public void shouldProcessNotNullColumnsAdaptiveMsDateTypes() throws Exception {
|
|||||||
}
|
}
|
||||||
|
|
||||||
private Struct testProcessNotNullColumns(TemporalPrecisionMode temporalMode) throws Exception {
|
private Struct testProcessNotNullColumns(TemporalPrecisionMode temporalMode) throws Exception {
|
||||||
PostgresConnectorConfig config = new PostgresConnectorConfig(TestHelper.defaultConfig()
|
TestHelper.executeDDL("postgres_create_tables.ddl");
|
||||||
|
|
||||||
|
startConnector(config -> config
|
||||||
.with(PostgresConnectorConfig.INCLUDE_UNKNOWN_DATATYPES, true)
|
.with(PostgresConnectorConfig.INCLUDE_UNKNOWN_DATATYPES, true)
|
||||||
.with(PostgresConnectorConfig.SCHEMA_BLACKLIST, "postgis")
|
.with(PostgresConnectorConfig.SCHEMA_BLACKLIST, "postgis")
|
||||||
.with(PostgresConnectorConfig.TIME_PRECISION_MODE, temporalMode)
|
.with(PostgresConnectorConfig.TIME_PRECISION_MODE, temporalMode)
|
||||||
.build());
|
);
|
||||||
setupRecordsProducer(config);
|
|
||||||
TestHelper.executeDDL("postgres_create_tables.ddl");
|
|
||||||
|
|
||||||
consumer = testConsumer(1);
|
|
||||||
recordsProducer.start(consumer, blackHole);
|
|
||||||
|
|
||||||
|
consumer.expects(1);
|
||||||
executeAndWait("INSERT INTO not_null_table VALUES (default, 30, '2019-02-10 11:34:58', '2019-02-10 11:35:00', '10:20:11', '10:20:12', '2019-02-01', '$20', B'101')");
|
executeAndWait("INSERT INTO not_null_table VALUES (default, 30, '2019-02-10 11:34:58', '2019-02-10 11:35:00', '10:20:11', '10:20:12', '2019-02-01', '$20', B'101')");
|
||||||
consumer.remove();
|
consumer.remove();
|
||||||
|
|
||||||
@ -286,9 +290,10 @@ private Struct testProcessNotNullColumns(TemporalPrecisionMode temporalMode) thr
|
|||||||
@Test(timeout = 30000)
|
@Test(timeout = 30000)
|
||||||
public void shouldReceiveChangesForInsertsWithPostgisTypes() throws Exception {
|
public void shouldReceiveChangesForInsertsWithPostgisTypes() throws Exception {
|
||||||
TestHelper.executeDDL("postgis_create_tables.ddl");
|
TestHelper.executeDDL("postgis_create_tables.ddl");
|
||||||
|
|
||||||
|
startConnector();
|
||||||
consumer = testConsumer(1, "public"); // spatial_ref_sys produces a ton of records in the postgis schema
|
consumer = testConsumer(1, "public"); // spatial_ref_sys produces a ton of records in the postgis schema
|
||||||
consumer.setIgnoreExtraRecords(true);
|
consumer.setIgnoreExtraRecords(true);
|
||||||
recordsProducer.start(consumer, blackHole);
|
|
||||||
|
|
||||||
// need to wait for all the spatial_ref_sys to flow through and be ignored.
|
// need to wait for all the spatial_ref_sys to flow through and be ignored.
|
||||||
// this exceeds the normal 2s timeout.
|
// this exceeds the normal 2s timeout.
|
||||||
@ -312,9 +317,10 @@ public void shouldReceiveChangesForInsertsWithPostgisTypes() throws Exception {
|
|||||||
@Test(timeout = 30000)
|
@Test(timeout = 30000)
|
||||||
public void shouldReceiveChangesForInsertsWithPostgisArrayTypes() throws Exception {
|
public void shouldReceiveChangesForInsertsWithPostgisArrayTypes() throws Exception {
|
||||||
TestHelper.executeDDL("postgis_create_tables.ddl");
|
TestHelper.executeDDL("postgis_create_tables.ddl");
|
||||||
|
|
||||||
|
startConnector();
|
||||||
consumer = testConsumer(1, "public"); // spatial_ref_sys produces a ton of records in the postgis schema
|
consumer = testConsumer(1, "public"); // spatial_ref_sys produces a ton of records in the postgis schema
|
||||||
consumer.setIgnoreExtraRecords(true);
|
consumer.setIgnoreExtraRecords(true);
|
||||||
recordsProducer.start(consumer, blackHole);
|
|
||||||
|
|
||||||
// need to wait for all the spatial_ref_sys to flow through and be ignored.
|
// need to wait for all the spatial_ref_sys to flow through and be ignored.
|
||||||
// this exceeds the normal 2s timeout.
|
// this exceeds the normal 2s timeout.
|
||||||
@ -341,8 +347,7 @@ public void shouldReceiveChangesForInsertsWithPostgisArrayTypes() throws Excepti
|
|||||||
public void shouldReceiveChangesForInsertsWithQuotedNames() throws Exception {
|
public void shouldReceiveChangesForInsertsWithQuotedNames() throws Exception {
|
||||||
TestHelper.executeDDL("postgres_create_tables.ddl");
|
TestHelper.executeDDL("postgres_create_tables.ddl");
|
||||||
|
|
||||||
consumer = testConsumer(1);
|
startConnector();
|
||||||
recordsProducer.start(consumer, blackHole);
|
|
||||||
|
|
||||||
// Quoted column name
|
// Quoted column name
|
||||||
assertInsert(INSERT_QUOTED_TYPES_STMT, 1, schemasAndValuesForQuotedTypes());
|
assertInsert(INSERT_QUOTED_TYPES_STMT, 1, schemasAndValuesForQuotedTypes());
|
||||||
@ -352,19 +357,17 @@ public void shouldReceiveChangesForInsertsWithQuotedNames() throws Exception {
|
|||||||
public void shouldReceiveChangesForInsertsWithArrayTypes() throws Exception {
|
public void shouldReceiveChangesForInsertsWithArrayTypes() throws Exception {
|
||||||
TestHelper.executeDDL("postgres_create_tables.ddl");
|
TestHelper.executeDDL("postgres_create_tables.ddl");
|
||||||
|
|
||||||
consumer = testConsumer(1);
|
startConnector();
|
||||||
recordsProducer.start(consumer, blackHole);
|
|
||||||
|
|
||||||
assertInsert(INSERT_ARRAY_TYPES_STMT, 1, schemasAndValuesForArrayTypes());
|
assertInsert(INSERT_ARRAY_TYPES_STMT, 1, schemasAndValuesForArrayTypes());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
@FixFor("DBZ-1029")
|
@FixFor("DBZ-1029")
|
||||||
public void shouldReceiveChangesForInsertsIndependentOfReplicaIdentity() {
|
public void shouldReceiveChangesForInsertsIndependentOfReplicaIdentity() throws Exception {
|
||||||
// insert statement should not be affected by replica identity settings in any way
|
// insert statement should not be affected by replica identity settings in any way
|
||||||
|
|
||||||
consumer = testConsumer(1);
|
startConnector();
|
||||||
recordsProducer.start(consumer, blackHole);
|
|
||||||
|
|
||||||
TestHelper.execute("ALTER TABLE test_table REPLICA IDENTITY DEFAULT;");
|
TestHelper.execute("ALTER TABLE test_table REPLICA IDENTITY DEFAULT;");
|
||||||
String statement = "INSERT INTO test_table (text) VALUES ('pk_and_default');";
|
String statement = "INSERT INTO test_table (text) VALUES ('pk_and_default');";
|
||||||
@ -391,8 +394,7 @@ public void shouldReceiveChangesForInsertsIndependentOfReplicaIdentity() {
|
|||||||
public void shouldReceiveChangesForNullInsertsWithArrayTypes() throws Exception {
|
public void shouldReceiveChangesForNullInsertsWithArrayTypes() throws Exception {
|
||||||
TestHelper.executeDDL("postgres_create_tables.ddl");
|
TestHelper.executeDDL("postgres_create_tables.ddl");
|
||||||
|
|
||||||
consumer = testConsumer(1);
|
startConnector();
|
||||||
recordsProducer.start(consumer, blackHole);
|
|
||||||
|
|
||||||
assertInsert(INSERT_ARRAY_TYPES_WITH_NULL_VALUES_STMT, 1, schemasAndValuesForArrayTypesWithNullValues());
|
assertInsert(INSERT_ARRAY_TYPES_WITH_NULL_VALUES_STMT, 1, schemasAndValuesForArrayTypesWithNullValues());
|
||||||
}
|
}
|
||||||
@ -402,8 +404,9 @@ public void shouldReceiveChangesForNewTable() throws Exception {
|
|||||||
String statement = "CREATE SCHEMA s1;" +
|
String statement = "CREATE SCHEMA s1;" +
|
||||||
"CREATE TABLE s1.a (pk SERIAL, aa integer, PRIMARY KEY(pk));" +
|
"CREATE TABLE s1.a (pk SERIAL, aa integer, PRIMARY KEY(pk));" +
|
||||||
"INSERT INTO s1.a (aa) VALUES (11);";
|
"INSERT INTO s1.a (aa) VALUES (11);";
|
||||||
consumer = testConsumer(1);
|
|
||||||
recordsProducer.start(consumer, blackHole);
|
startConnector();
|
||||||
|
|
||||||
executeAndWait(statement);
|
executeAndWait(statement);
|
||||||
assertRecordInserted("s1.a", PK_FIELD, 1);
|
assertRecordInserted("s1.a", PK_FIELD, 1);
|
||||||
}
|
}
|
||||||
@ -413,8 +416,8 @@ public void shouldReceiveChangesForRenamedTable() throws Exception {
|
|||||||
String statement = "DROP TABLE IF EXISTS renamed_test_table;" +
|
String statement = "DROP TABLE IF EXISTS renamed_test_table;" +
|
||||||
"ALTER TABLE test_table RENAME TO renamed_test_table;" +
|
"ALTER TABLE test_table RENAME TO renamed_test_table;" +
|
||||||
"INSERT INTO renamed_test_table (text) VALUES ('new');";
|
"INSERT INTO renamed_test_table (text) VALUES ('new');";
|
||||||
consumer = testConsumer(1);
|
startConnector();
|
||||||
recordsProducer.start(consumer, blackHole);
|
|
||||||
executeAndWait(statement);
|
executeAndWait(statement);
|
||||||
assertRecordInserted("public.renamed_test_table", PK_FIELD, 2);
|
assertRecordInserted("public.renamed_test_table", PK_FIELD, 2);
|
||||||
}
|
}
|
||||||
@ -422,8 +425,7 @@ public void shouldReceiveChangesForRenamedTable() throws Exception {
|
|||||||
@Test
|
@Test
|
||||||
@SkipWhenDecoderPluginNameIs(value = PGOUTPUT, reason = "An update on a table with no primary key and default replica throws PSQLException as tables must have a PK")
|
@SkipWhenDecoderPluginNameIs(value = PGOUTPUT, reason = "An update on a table with no primary key and default replica throws PSQLException as tables must have a PK")
|
||||||
public void shouldReceiveChangesForUpdates() throws Exception {
|
public void shouldReceiveChangesForUpdates() throws Exception {
|
||||||
consumer = testConsumer(1);
|
startConnector();
|
||||||
recordsProducer.start(consumer, blackHole);
|
|
||||||
executeAndWait("UPDATE test_table set text='update' WHERE pk=1");
|
executeAndWait("UPDATE test_table set text='update' WHERE pk=1");
|
||||||
|
|
||||||
// the update record should be the last record
|
// the update record should be the last record
|
||||||
@ -480,8 +482,8 @@ public void shouldReceiveChangesForUpdatesWithColumnChanges() throws Exception {
|
|||||||
"ALTER TABLE test_table REPLICA IDENTITY FULL;" +
|
"ALTER TABLE test_table REPLICA IDENTITY FULL;" +
|
||||||
"UPDATE test_table SET uvc ='aa' WHERE pk = 1;";
|
"UPDATE test_table SET uvc ='aa' WHERE pk = 1;";
|
||||||
|
|
||||||
|
startConnector();
|
||||||
consumer = testConsumer(1);
|
consumer = testConsumer(1);
|
||||||
recordsProducer.start(consumer, blackHole);
|
|
||||||
executeAndWait(statements);
|
executeAndWait(statements);
|
||||||
|
|
||||||
// the update should be the last record
|
// the update should be the last record
|
||||||
@ -551,8 +553,8 @@ public void shouldReceiveChangesForUpdatesWithColumnChanges() throws Exception {
|
|||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void shouldReceiveChangesForUpdatesWithPKChanges() throws Exception {
|
public void shouldReceiveChangesForUpdatesWithPKChanges() throws Exception {
|
||||||
|
startConnector();
|
||||||
consumer = testConsumer(3);
|
consumer = testConsumer(3);
|
||||||
recordsProducer.start(consumer, blackHole);
|
|
||||||
executeAndWait("UPDATE test_table SET text = 'update', pk = 2");
|
executeAndWait("UPDATE test_table SET text = 'update', pk = 2");
|
||||||
|
|
||||||
String topicName = topicName("public.test_table");
|
String topicName = topicName("public.test_table");
|
||||||
@ -576,14 +578,11 @@ public void shouldReceiveChangesForUpdatesWithPKChanges() throws Exception {
|
|||||||
@Test
|
@Test
|
||||||
@FixFor("DBZ-582")
|
@FixFor("DBZ-582")
|
||||||
public void shouldReceiveChangesForUpdatesWithPKChangesWithoutTombstone() throws Exception {
|
public void shouldReceiveChangesForUpdatesWithPKChangesWithoutTombstone() throws Exception {
|
||||||
PostgresConnectorConfig config = new PostgresConnectorConfig(TestHelper.defaultConfig()
|
startConnector(config -> config
|
||||||
.with(PostgresConnectorConfig.INCLUDE_UNKNOWN_DATATYPES, true)
|
.with(PostgresConnectorConfig.INCLUDE_UNKNOWN_DATATYPES, true)
|
||||||
.with(CommonConnectorConfig.TOMBSTONES_ON_DELETE, false)
|
.with(CommonConnectorConfig.TOMBSTONES_ON_DELETE, false)
|
||||||
.build()
|
|
||||||
);
|
);
|
||||||
setupRecordsProducer(config);
|
|
||||||
consumer = testConsumer(2);
|
consumer = testConsumer(2);
|
||||||
recordsProducer.start(consumer, blackHole);
|
|
||||||
|
|
||||||
executeAndWait("UPDATE test_table SET text = 'update', pk = 2");
|
executeAndWait("UPDATE test_table SET text = 'update', pk = 2");
|
||||||
|
|
||||||
@ -605,8 +604,8 @@ public void shouldReceiveChangesForDefaultValues() throws Exception {
|
|||||||
String statements = "ALTER TABLE test_table REPLICA IDENTITY FULL;" +
|
String statements = "ALTER TABLE test_table REPLICA IDENTITY FULL;" +
|
||||||
"ALTER TABLE test_table ADD COLUMN default_column TEXT DEFAULT 'default';" +
|
"ALTER TABLE test_table ADD COLUMN default_column TEXT DEFAULT 'default';" +
|
||||||
"INSERT INTO test_table (text) VALUES ('update');";
|
"INSERT INTO test_table (text) VALUES ('update');";
|
||||||
|
startConnector();
|
||||||
consumer = testConsumer(1);
|
consumer = testConsumer(1);
|
||||||
recordsProducer.start(consumer, blackHole);
|
|
||||||
executeAndWait(statements);
|
executeAndWait(statements);
|
||||||
|
|
||||||
SourceRecord insertRecord = consumer.remove();
|
SourceRecord insertRecord = consumer.remove();
|
||||||
@ -625,8 +624,8 @@ public void shouldReceiveChangesForTypeConstraints() throws Exception {
|
|||||||
"ALTER TABLE test_table REPLICA IDENTITY FULL;" +
|
"ALTER TABLE test_table REPLICA IDENTITY FULL;" +
|
||||||
"UPDATE test_table SET num_val = 123.45 WHERE pk = 1;";
|
"UPDATE test_table SET num_val = 123.45 WHERE pk = 1;";
|
||||||
|
|
||||||
|
startConnector();
|
||||||
consumer = testConsumer(1);
|
consumer = testConsumer(1);
|
||||||
recordsProducer.start(consumer, blackHole);
|
|
||||||
executeAndWait(statements);
|
executeAndWait(statements);
|
||||||
|
|
||||||
// the update should be the last record
|
// the update should be the last record
|
||||||
@ -720,8 +719,9 @@ public void shouldReceiveChangesForDeletes() throws Exception {
|
|||||||
// add a new entry and remove both
|
// add a new entry and remove both
|
||||||
String statements = "INSERT INTO test_table (text) VALUES ('insert2');" +
|
String statements = "INSERT INTO test_table (text) VALUES ('insert2');" +
|
||||||
"DELETE FROM test_table WHERE pk > 0;";
|
"DELETE FROM test_table WHERE pk > 0;";
|
||||||
|
|
||||||
|
startConnector();
|
||||||
consumer = testConsumer(5);
|
consumer = testConsumer(5);
|
||||||
recordsProducer.start(consumer, blackHole);
|
|
||||||
executeAndWait(statements);
|
executeAndWait(statements);
|
||||||
|
|
||||||
|
|
||||||
@ -753,18 +753,14 @@ record = consumer.remove();
|
|||||||
@Test
|
@Test
|
||||||
@FixFor("DBZ-582")
|
@FixFor("DBZ-582")
|
||||||
public void shouldReceiveChangesForDeletesWithoutTombstone() throws Exception {
|
public void shouldReceiveChangesForDeletesWithoutTombstone() throws Exception {
|
||||||
PostgresConnectorConfig config = new PostgresConnectorConfig(TestHelper.defaultConfig()
|
|
||||||
.with(PostgresConnectorConfig.INCLUDE_UNKNOWN_DATATYPES, true)
|
|
||||||
.with(CommonConnectorConfig.TOMBSTONES_ON_DELETE, false)
|
|
||||||
.build()
|
|
||||||
);
|
|
||||||
setupRecordsProducer(config);
|
|
||||||
|
|
||||||
// add a new entry and remove both
|
// add a new entry and remove both
|
||||||
String statements = "INSERT INTO test_table (text) VALUES ('insert2');" +
|
String statements = "INSERT INTO test_table (text) VALUES ('insert2');" +
|
||||||
"DELETE FROM test_table WHERE pk > 0;";
|
"DELETE FROM test_table WHERE pk > 0;";
|
||||||
|
startConnector(config -> config
|
||||||
|
.with(PostgresConnectorConfig.INCLUDE_UNKNOWN_DATATYPES, true)
|
||||||
|
.with(CommonConnectorConfig.TOMBSTONES_ON_DELETE, false)
|
||||||
|
);
|
||||||
consumer = testConsumer(3);
|
consumer = testConsumer(3);
|
||||||
recordsProducer.start(consumer, blackHole);
|
|
||||||
executeAndWait(statements);
|
executeAndWait(statements);
|
||||||
|
|
||||||
|
|
||||||
@ -786,19 +782,16 @@ record = consumer.remove();
|
|||||||
@Test
|
@Test
|
||||||
@SkipWhenDecoderPluginNameIs(value = PGOUTPUT, reason = "A delete on a table with no primary key and default replica throws PSQLException as tables must have a PK")
|
@SkipWhenDecoderPluginNameIs(value = PGOUTPUT, reason = "A delete on a table with no primary key and default replica throws PSQLException as tables must have a PK")
|
||||||
public void shouldReceiveChangesForDeletesDependingOnReplicaIdentity() throws Exception {
|
public void shouldReceiveChangesForDeletesDependingOnReplicaIdentity() throws Exception {
|
||||||
PostgresConnectorConfig config = new PostgresConnectorConfig(TestHelper.defaultConfig()
|
|
||||||
.with(PostgresConnectorConfig.INCLUDE_UNKNOWN_DATATYPES, true)
|
|
||||||
.with(CommonConnectorConfig.TOMBSTONES_ON_DELETE, false)
|
|
||||||
.build()
|
|
||||||
);
|
|
||||||
setupRecordsProducer(config);
|
|
||||||
String topicName = topicName("public.test_table");
|
String topicName = topicName("public.test_table");
|
||||||
|
|
||||||
// With PK we should get delete event with default level of replica identity
|
// With PK we should get delete event with default level of replica identity
|
||||||
String statement = "ALTER TABLE test_table REPLICA IDENTITY DEFAULT;" +
|
String statement = "ALTER TABLE test_table REPLICA IDENTITY DEFAULT;" +
|
||||||
"DELETE FROM test_table WHERE pk = 1;";
|
"DELETE FROM test_table WHERE pk = 1;";
|
||||||
|
startConnector(config -> config
|
||||||
|
.with(PostgresConnectorConfig.INCLUDE_UNKNOWN_DATATYPES, true)
|
||||||
|
.with(CommonConnectorConfig.TOMBSTONES_ON_DELETE, false)
|
||||||
|
);
|
||||||
consumer = testConsumer(1);
|
consumer = testConsumer(1);
|
||||||
recordsProducer.start(consumer, blackHole);
|
|
||||||
executeAndWait(statement);
|
executeAndWait(statement);
|
||||||
SourceRecord record = consumer.remove();
|
SourceRecord record = consumer.remove();
|
||||||
assertEquals(topicName, record.topic());
|
assertEquals(topicName, record.topic());
|
||||||
@ -828,15 +821,9 @@ record = consumer.remove();
|
|||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void shouldReceiveNumericTypeAsDouble() throws Exception {
|
public void shouldReceiveNumericTypeAsDouble() throws Exception {
|
||||||
PostgresConnectorConfig config = new PostgresConnectorConfig(TestHelper.defaultConfig()
|
|
||||||
.with(PostgresConnectorConfig.DECIMAL_HANDLING_MODE, PostgresConnectorConfig.DecimalHandlingMode.DOUBLE)
|
|
||||||
.build());
|
|
||||||
setupRecordsProducer(config);
|
|
||||||
|
|
||||||
TestHelper.executeDDL("postgres_create_tables.ddl");
|
TestHelper.executeDDL("postgres_create_tables.ddl");
|
||||||
|
|
||||||
consumer = testConsumer(1);
|
startConnector(config -> config.with(PostgresConnectorConfig.DECIMAL_HANDLING_MODE, PostgresConnectorConfig.DecimalHandlingMode.DOUBLE));
|
||||||
recordsProducer.start(consumer, blackHole);
|
|
||||||
|
|
||||||
assertInsert(INSERT_NUMERIC_DECIMAL_TYPES_STMT, 1, schemasAndValuesForDoubleEncodedNumericTypes());
|
assertInsert(INSERT_NUMERIC_DECIMAL_TYPES_STMT, 1, schemasAndValuesForDoubleEncodedNumericTypes());
|
||||||
}
|
}
|
||||||
@ -844,15 +831,9 @@ public void shouldReceiveNumericTypeAsDouble() throws Exception {
|
|||||||
@Test
|
@Test
|
||||||
@FixFor("DBZ-611")
|
@FixFor("DBZ-611")
|
||||||
public void shouldReceiveNumericTypeAsString() throws Exception {
|
public void shouldReceiveNumericTypeAsString() throws Exception {
|
||||||
PostgresConnectorConfig config = new PostgresConnectorConfig(TestHelper.defaultConfig()
|
|
||||||
.with(PostgresConnectorConfig.DECIMAL_HANDLING_MODE, PostgresConnectorConfig.DecimalHandlingMode.STRING)
|
|
||||||
.build());
|
|
||||||
setupRecordsProducer(config);
|
|
||||||
|
|
||||||
TestHelper.executeDDL("postgres_create_tables.ddl");
|
TestHelper.executeDDL("postgres_create_tables.ddl");
|
||||||
|
|
||||||
consumer = testConsumer(1);
|
startConnector(config -> config.with(PostgresConnectorConfig.DECIMAL_HANDLING_MODE, PostgresConnectorConfig.DecimalHandlingMode.STRING));
|
||||||
recordsProducer.start(consumer, blackHole);
|
|
||||||
|
|
||||||
assertInsert(INSERT_NUMERIC_DECIMAL_TYPES_STMT, 1, schemasAndValuesForStringEncodedNumericTypes());
|
assertInsert(INSERT_NUMERIC_DECIMAL_TYPES_STMT, 1, schemasAndValuesForStringEncodedNumericTypes());
|
||||||
}
|
}
|
||||||
@ -860,13 +841,9 @@ public void shouldReceiveNumericTypeAsString() throws Exception {
|
|||||||
@Test
|
@Test
|
||||||
@FixFor("DBZ-898")
|
@FixFor("DBZ-898")
|
||||||
public void shouldReceiveHStoreTypeWithSingleValueAsMap() throws Exception {
|
public void shouldReceiveHStoreTypeWithSingleValueAsMap() throws Exception {
|
||||||
PostgresConnectorConfig config = new PostgresConnectorConfig(TestHelper.defaultConfig()
|
|
||||||
.with(PostgresConnectorConfig.HSTORE_HANDLING_MODE, PostgresConnectorConfig.HStoreHandlingMode.MAP)
|
|
||||||
.build());
|
|
||||||
setupRecordsProducer(config);
|
|
||||||
TestHelper.executeDDL("postgres_create_tables.ddl");
|
TestHelper.executeDDL("postgres_create_tables.ddl");
|
||||||
consumer = testConsumer(1);
|
|
||||||
recordsProducer.start(consumer, blackHole);
|
startConnector(config -> config.with(PostgresConnectorConfig.HSTORE_HANDLING_MODE, PostgresConnectorConfig.HStoreHandlingMode.MAP));
|
||||||
|
|
||||||
assertInsert(INSERT_HSTORE_TYPE_STMT, 1, schemaAndValueFieldForMapEncodedHStoreType());
|
assertInsert(INSERT_HSTORE_TYPE_STMT, 1, schemaAndValueFieldForMapEncodedHStoreType());
|
||||||
}
|
}
|
||||||
@ -874,13 +851,9 @@ public void shouldReceiveHStoreTypeWithSingleValueAsMap() throws Exception {
|
|||||||
@Test
|
@Test
|
||||||
@FixFor("DBZ-898")
|
@FixFor("DBZ-898")
|
||||||
public void shouldReceiveHStoreTypeWithMultipleValuesAsMap() throws Exception {
|
public void shouldReceiveHStoreTypeWithMultipleValuesAsMap() throws Exception {
|
||||||
PostgresConnectorConfig config = new PostgresConnectorConfig(TestHelper.defaultConfig()
|
|
||||||
.with(PostgresConnectorConfig.HSTORE_HANDLING_MODE, PostgresConnectorConfig.HStoreHandlingMode.MAP)
|
|
||||||
.build());
|
|
||||||
setupRecordsProducer(config);
|
|
||||||
TestHelper.executeDDL("postgres_create_tables.ddl");
|
TestHelper.executeDDL("postgres_create_tables.ddl");
|
||||||
consumer = testConsumer(1);
|
|
||||||
recordsProducer.start(consumer, blackHole);
|
startConnector(config -> config.with(PostgresConnectorConfig.HSTORE_HANDLING_MODE, PostgresConnectorConfig.HStoreHandlingMode.MAP));
|
||||||
|
|
||||||
assertInsert(INSERT_HSTORE_TYPE_WITH_MULTIPLE_VALUES_STMT, 1, schemaAndValueFieldForMapEncodedHStoreTypeWithMultipleValues());
|
assertInsert(INSERT_HSTORE_TYPE_WITH_MULTIPLE_VALUES_STMT, 1, schemaAndValueFieldForMapEncodedHStoreTypeWithMultipleValues());
|
||||||
}
|
}
|
||||||
@ -888,13 +861,9 @@ public void shouldReceiveHStoreTypeWithMultipleValuesAsMap() throws Exception {
|
|||||||
@Test
|
@Test
|
||||||
@FixFor("DBZ-898")
|
@FixFor("DBZ-898")
|
||||||
public void shouldReceiveHStoreTypeWithNullValuesAsMap() throws Exception {
|
public void shouldReceiveHStoreTypeWithNullValuesAsMap() throws Exception {
|
||||||
PostgresConnectorConfig config = new PostgresConnectorConfig(TestHelper.defaultConfig()
|
|
||||||
.with(PostgresConnectorConfig.HSTORE_HANDLING_MODE, PostgresConnectorConfig.HStoreHandlingMode.MAP)
|
|
||||||
.build());
|
|
||||||
setupRecordsProducer(config);
|
|
||||||
TestHelper.executeDDL("postgres_create_tables.ddl");
|
TestHelper.executeDDL("postgres_create_tables.ddl");
|
||||||
consumer = testConsumer(1);
|
|
||||||
recordsProducer.start(consumer, blackHole);
|
startConnector(config -> config.with(PostgresConnectorConfig.HSTORE_HANDLING_MODE, PostgresConnectorConfig.HStoreHandlingMode.MAP));
|
||||||
|
|
||||||
assertInsert(INSERT_HSTORE_TYPE_WITH_NULL_VALUES_STMT, 1, schemaAndValueFieldForMapEncodedHStoreTypeWithNullValues());
|
assertInsert(INSERT_HSTORE_TYPE_WITH_NULL_VALUES_STMT, 1, schemaAndValueFieldForMapEncodedHStoreTypeWithNullValues());
|
||||||
}
|
}
|
||||||
@ -902,13 +871,9 @@ public void shouldReceiveHStoreTypeWithNullValuesAsMap() throws Exception {
|
|||||||
@Test
|
@Test
|
||||||
@FixFor("DBZ-898")
|
@FixFor("DBZ-898")
|
||||||
public void shouldReceiveHStoreTypeWithSpecialCharactersInValuesAsMap() throws Exception {
|
public void shouldReceiveHStoreTypeWithSpecialCharactersInValuesAsMap() throws Exception {
|
||||||
PostgresConnectorConfig config = new PostgresConnectorConfig(TestHelper.defaultConfig()
|
|
||||||
.with(PostgresConnectorConfig.HSTORE_HANDLING_MODE, PostgresConnectorConfig.HStoreHandlingMode.MAP)
|
|
||||||
.build());
|
|
||||||
setupRecordsProducer(config);
|
|
||||||
TestHelper.executeDDL("postgres_create_tables.ddl");
|
TestHelper.executeDDL("postgres_create_tables.ddl");
|
||||||
consumer = testConsumer(1);
|
|
||||||
recordsProducer.start(consumer, blackHole);
|
startConnector(config -> config.with(PostgresConnectorConfig.HSTORE_HANDLING_MODE, PostgresConnectorConfig.HStoreHandlingMode.MAP));
|
||||||
|
|
||||||
assertInsert(INSERT_HSTORE_TYPE_WITH_SPECIAL_CHAR_STMT, 1, schemaAndValueFieldForMapEncodedHStoreTypeWithSpecialCharacters());
|
assertInsert(INSERT_HSTORE_TYPE_WITH_SPECIAL_CHAR_STMT, 1, schemaAndValueFieldForMapEncodedHStoreTypeWithSpecialCharacters());
|
||||||
}
|
}
|
||||||
@ -916,13 +881,10 @@ public void shouldReceiveHStoreTypeWithSpecialCharactersInValuesAsMap() throws E
|
|||||||
@Test
|
@Test
|
||||||
@FixFor("DBZ-898")
|
@FixFor("DBZ-898")
|
||||||
public void shouldReceiveHStoreTypeAsJsonString() throws Exception {
|
public void shouldReceiveHStoreTypeAsJsonString() throws Exception {
|
||||||
PostgresConnectorConfig config = new PostgresConnectorConfig(TestHelper.defaultConfig()
|
|
||||||
.with(PostgresConnectorConfig.HSTORE_HANDLING_MODE, PostgresConnectorConfig.HStoreHandlingMode.JSON)
|
|
||||||
.build());
|
|
||||||
setupRecordsProducer(config);
|
|
||||||
TestHelper.executeDDL("postgres_create_tables.ddl");
|
TestHelper.executeDDL("postgres_create_tables.ddl");
|
||||||
consumer = testConsumer(1);
|
consumer = testConsumer(1);
|
||||||
recordsProducer.start(consumer, blackHole);
|
|
||||||
|
startConnector(config -> config.with(PostgresConnectorConfig.HSTORE_HANDLING_MODE, PostgresConnectorConfig.HStoreHandlingMode.JSON));
|
||||||
|
|
||||||
assertInsert(INSERT_HSTORE_TYPE_STMT, 1, schemaAndValueFieldForJsonEncodedHStoreType());
|
assertInsert(INSERT_HSTORE_TYPE_STMT, 1, schemaAndValueFieldForJsonEncodedHStoreType());
|
||||||
}
|
}
|
||||||
@ -930,13 +892,9 @@ public void shouldReceiveHStoreTypeAsJsonString() throws Exception {
|
|||||||
@Test
|
@Test
|
||||||
@FixFor("DBZ-898")
|
@FixFor("DBZ-898")
|
||||||
public void shouldReceiveHStoreTypeWithMultipleValuesAsJsonString() throws Exception {
|
public void shouldReceiveHStoreTypeWithMultipleValuesAsJsonString() throws Exception {
|
||||||
PostgresConnectorConfig config = new PostgresConnectorConfig(TestHelper.defaultConfig()
|
|
||||||
.with(PostgresConnectorConfig.HSTORE_HANDLING_MODE, PostgresConnectorConfig.HStoreHandlingMode.JSON)
|
|
||||||
.build());
|
|
||||||
setupRecordsProducer(config);
|
|
||||||
TestHelper.executeDDL("postgres_create_tables.ddl");
|
TestHelper.executeDDL("postgres_create_tables.ddl");
|
||||||
consumer = testConsumer(1);
|
|
||||||
recordsProducer.start(consumer, blackHole);
|
startConnector(config -> config.with(PostgresConnectorConfig.HSTORE_HANDLING_MODE, PostgresConnectorConfig.HStoreHandlingMode.JSON));
|
||||||
|
|
||||||
assertInsert(INSERT_HSTORE_TYPE_WITH_MULTIPLE_VALUES_STMT, 1, schemaAndValueFieldForJsonEncodedHStoreTypeWithMultipleValues());
|
assertInsert(INSERT_HSTORE_TYPE_WITH_MULTIPLE_VALUES_STMT, 1, schemaAndValueFieldForJsonEncodedHStoreTypeWithMultipleValues());
|
||||||
}
|
}
|
||||||
@ -944,13 +902,9 @@ public void shouldReceiveHStoreTypeWithMultipleValuesAsJsonString() throws Excep
|
|||||||
@Test
|
@Test
|
||||||
@FixFor("DBZ-898")
|
@FixFor("DBZ-898")
|
||||||
public void shouldReceiveHStoreTypeWithSpecialValuesInJsonString() throws Exception {
|
public void shouldReceiveHStoreTypeWithSpecialValuesInJsonString() throws Exception {
|
||||||
PostgresConnectorConfig config = new PostgresConnectorConfig(TestHelper.defaultConfig()
|
|
||||||
.with(PostgresConnectorConfig.HSTORE_HANDLING_MODE, PostgresConnectorConfig.HStoreHandlingMode.JSON)
|
|
||||||
.build());
|
|
||||||
setupRecordsProducer(config);
|
|
||||||
TestHelper.executeDDL("postgres_create_tables.ddl");
|
TestHelper.executeDDL("postgres_create_tables.ddl");
|
||||||
consumer = testConsumer(1);
|
|
||||||
recordsProducer.start(consumer, blackHole);
|
startConnector(config -> config.with(PostgresConnectorConfig.HSTORE_HANDLING_MODE, PostgresConnectorConfig.HStoreHandlingMode.JSON));
|
||||||
|
|
||||||
assertInsert(INSERT_HSTORE_TYPE_WITH_SPECIAL_CHAR_STMT, 1, schemaAndValueFieldForJsonEncodedHStoreTypeWithSpcialCharacters());
|
assertInsert(INSERT_HSTORE_TYPE_WITH_SPECIAL_CHAR_STMT, 1, schemaAndValueFieldForJsonEncodedHStoreTypeWithSpcialCharacters());
|
||||||
}
|
}
|
||||||
@ -958,13 +912,9 @@ public void shouldReceiveHStoreTypeWithSpecialValuesInJsonString() throws Except
|
|||||||
@Test
|
@Test
|
||||||
@FixFor("DBZ-898")
|
@FixFor("DBZ-898")
|
||||||
public void shouldReceiveHStoreTypeWithNullValuesAsJsonString() throws Exception {
|
public void shouldReceiveHStoreTypeWithNullValuesAsJsonString() throws Exception {
|
||||||
PostgresConnectorConfig config = new PostgresConnectorConfig(TestHelper.defaultConfig()
|
|
||||||
.with(PostgresConnectorConfig.HSTORE_HANDLING_MODE, PostgresConnectorConfig.HStoreHandlingMode.JSON)
|
|
||||||
.build());
|
|
||||||
setupRecordsProducer(config);
|
|
||||||
TestHelper.executeDDL("postgres_create_tables.ddl");
|
TestHelper.executeDDL("postgres_create_tables.ddl");
|
||||||
consumer = testConsumer(1);
|
|
||||||
recordsProducer.start(consumer, blackHole);
|
startConnector(config -> config.with(PostgresConnectorConfig.HSTORE_HANDLING_MODE, PostgresConnectorConfig.HStoreHandlingMode.JSON));
|
||||||
|
|
||||||
assertInsert(INSERT_HSTORE_TYPE_WITH_NULL_VALUES_STMT, 1, schemaAndValueFieldForJsonEncodedHStoreTypeWithNullValues());
|
assertInsert(INSERT_HSTORE_TYPE_WITH_NULL_VALUES_STMT, 1, schemaAndValueFieldForJsonEncodedHStoreTypeWithNullValues());
|
||||||
}
|
}
|
||||||
@ -977,8 +927,8 @@ public void shouldProcessIntervalDelete() throws Exception {
|
|||||||
"INSERT INTO table_with_interval VALUES (default, 'Bar', default);" +
|
"INSERT INTO table_with_interval VALUES (default, 'Bar', default);" +
|
||||||
"DELETE FROM table_with_interval WHERE id = 1;";
|
"DELETE FROM table_with_interval WHERE id = 1;";
|
||||||
|
|
||||||
consumer = testConsumer(4);
|
startConnector();
|
||||||
recordsProducer.start(consumer, blackHole);
|
consumer.expects(4);
|
||||||
executeAndWait(statements);
|
executeAndWait(statements);
|
||||||
|
|
||||||
final String topicPrefix = "public.table_with_interval";
|
final String topicPrefix = "public.table_with_interval";
|
||||||
@ -998,32 +948,12 @@ record = consumer.remove();
|
|||||||
VerifyRecord.isValidTombstone(record, pk, 1);
|
VerifyRecord.isValidTombstone(record, pk, 1);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
|
||||||
@FixFor("DBZ-501")
|
|
||||||
public void shouldNotStartAfterStop() throws Exception {
|
|
||||||
recordsProducer.stop();
|
|
||||||
recordsProducer.start(consumer, blackHole);
|
|
||||||
|
|
||||||
// Need to remove record created in @Before
|
|
||||||
PostgresConnectorConfig config = new PostgresConnectorConfig(TestHelper.defaultConfig().with(PostgresConnectorConfig.INCLUDE_UNKNOWN_DATATYPES, true).build());
|
|
||||||
setupRecordsProducer(config);
|
|
||||||
|
|
||||||
consumer = testConsumer(1);
|
|
||||||
recordsProducer.start(consumer, blackHole);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
@FixFor("DBZ-644")
|
@FixFor("DBZ-644")
|
||||||
public void shouldPropagateSourceColumnTypeToSchemaParameter() throws Exception {
|
public void shouldPropagateSourceColumnTypeToSchemaParameter() throws Exception {
|
||||||
PostgresConnectorConfig config = new PostgresConnectorConfig(TestHelper.defaultConfig()
|
|
||||||
.with("column.propagate.source.type", ".*vc.*")
|
|
||||||
.build());
|
|
||||||
setupRecordsProducer(config);
|
|
||||||
|
|
||||||
TestHelper.executeDDL("postgres_create_tables.ddl");
|
TestHelper.executeDDL("postgres_create_tables.ddl");
|
||||||
|
|
||||||
consumer = testConsumer(1);
|
startConnector(config -> config.with("column.propagate.source.type", ".*vc.*"));
|
||||||
recordsProducer.start(consumer, blackHole);
|
|
||||||
|
|
||||||
assertInsert(INSERT_STRING_TYPES_STMT, 1, schemasAndValuesForStringTypesWithSourceColumnTypeInfo());
|
assertInsert(INSERT_STRING_TYPES_STMT, 1, schemasAndValuesForStringTypesWithSourceColumnTypeInfo());
|
||||||
}
|
}
|
||||||
@ -1031,16 +961,12 @@ public void shouldPropagateSourceColumnTypeToSchemaParameter() throws Exception
|
|||||||
@Test
|
@Test
|
||||||
@FixFor("DBZ-1073")
|
@FixFor("DBZ-1073")
|
||||||
public void shouldPropagateSourceColumnTypeScaleToSchemaParameter() throws Exception {
|
public void shouldPropagateSourceColumnTypeScaleToSchemaParameter() throws Exception {
|
||||||
PostgresConnectorConfig config = new PostgresConnectorConfig(TestHelper.defaultConfig()
|
|
||||||
.with("column.propagate.source.type", ".*(d|dzs)")
|
|
||||||
.with(PostgresConnectorConfig.DECIMAL_HANDLING_MODE, PostgresConnectorConfig.DecimalHandlingMode.DOUBLE)
|
|
||||||
.build());
|
|
||||||
setupRecordsProducer(config);
|
|
||||||
|
|
||||||
TestHelper.executeDDL("postgres_create_tables.ddl");
|
TestHelper.executeDDL("postgres_create_tables.ddl");
|
||||||
|
|
||||||
consumer = testConsumer(1);
|
startConnector(config -> config
|
||||||
recordsProducer.start(consumer, blackHole);
|
.with("column.propagate.source.type", ".*(d|dzs)")
|
||||||
|
.with(PostgresConnectorConfig.DECIMAL_HANDLING_MODE, PostgresConnectorConfig.DecimalHandlingMode.DOUBLE)
|
||||||
|
);
|
||||||
|
|
||||||
assertInsert(INSERT_NUMERIC_DECIMAL_TYPES_STMT, 1, schemasAndValuesForNumericTypesWithSourceColumnTypeInfo());
|
assertInsert(INSERT_NUMERIC_DECIMAL_TYPES_STMT, 1, schemasAndValuesForNumericTypesWithSourceColumnTypeInfo());
|
||||||
}
|
}
|
||||||
@ -1050,13 +976,11 @@ public void shouldPropagateSourceColumnTypeScaleToSchemaParameter() throws Excep
|
|||||||
public void shouldReceiveHeartbeatAlsoWhenChangingNonWhitelistedTable() throws Exception {
|
public void shouldReceiveHeartbeatAlsoWhenChangingNonWhitelistedTable() throws Exception {
|
||||||
// the low heartbeat interval should make sure that a heartbeat message is emitted after each change record
|
// the low heartbeat interval should make sure that a heartbeat message is emitted after each change record
|
||||||
// received from Postgres
|
// received from Postgres
|
||||||
Testing.Print.enable();
|
startConnector(config -> config
|
||||||
PostgresConnectorConfig config = new PostgresConnectorConfig(TestHelper.defaultConfig()
|
|
||||||
.with(Heartbeat.HEARTBEAT_INTERVAL, "1")
|
.with(Heartbeat.HEARTBEAT_INTERVAL, "1")
|
||||||
.with(PostgresConnectorConfig.POLL_INTERVAL_MS, "50")
|
.with(PostgresConnectorConfig.POLL_INTERVAL_MS, "50")
|
||||||
.with(PostgresConnectorConfig.TABLE_WHITELIST, "s1\\.b")
|
.with(PostgresConnectorConfig.TABLE_WHITELIST, "s1\\.b")
|
||||||
.build());
|
);
|
||||||
setupRecordsProducer(config);
|
|
||||||
|
|
||||||
String statement = "CREATE SCHEMA s1;" +
|
String statement = "CREATE SCHEMA s1;" +
|
||||||
"CREATE TABLE s1.a (pk SERIAL, aa integer, PRIMARY KEY(pk));" +
|
"CREATE TABLE s1.a (pk SERIAL, aa integer, PRIMARY KEY(pk));" +
|
||||||
@ -1068,7 +992,6 @@ public void shouldReceiveHeartbeatAlsoWhenChangingNonWhitelistedTable() throws E
|
|||||||
final int expectedHeartbeats = 5;
|
final int expectedHeartbeats = 5;
|
||||||
consumer = testConsumer(1 + expectedHeartbeats);
|
consumer = testConsumer(1 + expectedHeartbeats);
|
||||||
consumer.setIgnoreExtraRecords(true);
|
consumer.setIgnoreExtraRecords(true);
|
||||||
recordsProducer.start(consumer, blackHole);
|
|
||||||
executeAndWait(statement);
|
executeAndWait(statement);
|
||||||
|
|
||||||
// change record for s1.b and heartbeats
|
// change record for s1.b and heartbeats
|
||||||
@ -1082,17 +1005,15 @@ public void shouldReceiveHeartbeatAlsoWhenChangingNonWhitelistedTable() throws E
|
|||||||
@FixFor("DBZ-911")
|
@FixFor("DBZ-911")
|
||||||
@SkipWhenDecoderPluginNameIs(value = PGOUTPUT, reason = "Decoder synchronizes all schema columns when processing relation messages")
|
@SkipWhenDecoderPluginNameIs(value = PGOUTPUT, reason = "Decoder synchronizes all schema columns when processing relation messages")
|
||||||
public void shouldNotRefreshSchemaOnUnchangedToastedData() throws Exception {
|
public void shouldNotRefreshSchemaOnUnchangedToastedData() throws Exception {
|
||||||
PostgresConnectorConfig config = new PostgresConnectorConfig(TestHelper.defaultConfig()
|
startConnector(config -> config
|
||||||
.with(PostgresConnectorConfig.SCHEMA_REFRESH_MODE, PostgresConnectorConfig.SchemaRefreshMode.COLUMNS_DIFF_EXCLUDE_UNCHANGED_TOAST)
|
.with(PostgresConnectorConfig.SCHEMA_REFRESH_MODE, PostgresConnectorConfig.SchemaRefreshMode.COLUMNS_DIFF_EXCLUDE_UNCHANGED_TOAST)
|
||||||
.build());
|
);
|
||||||
setupRecordsProducer(config);
|
|
||||||
|
|
||||||
String toastedValue = RandomStringUtils.randomAlphanumeric(10000);
|
String toastedValue = RandomStringUtils.randomAlphanumeric(10000);
|
||||||
|
|
||||||
// inserting a toasted value should /always/ produce a correct record
|
// inserting a toasted value should /always/ produce a correct record
|
||||||
String statement = "ALTER TABLE test_table ADD COLUMN not_toast integer; INSERT INTO test_table (not_toast, text) values (10, '" + toastedValue + "')";
|
String statement = "ALTER TABLE test_table ADD COLUMN not_toast integer; INSERT INTO test_table (not_toast, text) values (10, '" + toastedValue + "')";
|
||||||
consumer = testConsumer(1);
|
consumer = testConsumer(1);
|
||||||
recordsProducer.start(consumer, blackHole);
|
|
||||||
executeAndWait(statement);
|
executeAndWait(statement);
|
||||||
|
|
||||||
SourceRecord record = consumer.remove();
|
SourceRecord record = consumer.remove();
|
||||||
@ -1110,7 +1031,7 @@ public void shouldNotRefreshSchemaOnUnchangedToastedData() throws Exception {
|
|||||||
|
|
||||||
consumer.expects(1);
|
consumer.expects(1);
|
||||||
executeAndWait(statement);
|
executeAndWait(statement);
|
||||||
Table tbl = recordsProducer.schema().tableFor(TableId.parse("public.test_table"));
|
Table tbl = getSchema().tableFor(TableId.parse("public.test_table"));
|
||||||
assertEquals(Arrays.asList("pk", "text", "not_toast"), tbl.retrieveColumnNames());
|
assertEquals(Arrays.asList("pk", "text", "not_toast"), tbl.retrieveColumnNames());
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -1118,17 +1039,15 @@ public void shouldNotRefreshSchemaOnUnchangedToastedData() throws Exception {
|
|||||||
@FixFor("DBZ-911")
|
@FixFor("DBZ-911")
|
||||||
@SkipWhenDecoderPluginNameIsNot(value = SkipWhenDecoderPluginNameIsNot.DecoderPluginName.PGOUTPUT, reason = "Decoder synchronizes all schema columns when processing relation messages")
|
@SkipWhenDecoderPluginNameIsNot(value = SkipWhenDecoderPluginNameIsNot.DecoderPluginName.PGOUTPUT, reason = "Decoder synchronizes all schema columns when processing relation messages")
|
||||||
public void shouldRefreshSchemaOnUnchangedToastedDataWhenSchemaChanged() throws Exception {
|
public void shouldRefreshSchemaOnUnchangedToastedDataWhenSchemaChanged() throws Exception {
|
||||||
PostgresConnectorConfig config = new PostgresConnectorConfig(TestHelper.defaultConfig()
|
startConnector(config -> config
|
||||||
.with(PostgresConnectorConfig.SCHEMA_REFRESH_MODE, PostgresConnectorConfig.SchemaRefreshMode.COLUMNS_DIFF_EXCLUDE_UNCHANGED_TOAST)
|
.with(PostgresConnectorConfig.SCHEMA_REFRESH_MODE, PostgresConnectorConfig.SchemaRefreshMode.COLUMNS_DIFF_EXCLUDE_UNCHANGED_TOAST)
|
||||||
.build());
|
);
|
||||||
setupRecordsProducer(config);
|
|
||||||
|
|
||||||
String toastedValue = RandomStringUtils.randomAlphanumeric(10000);
|
String toastedValue = RandomStringUtils.randomAlphanumeric(10000);
|
||||||
|
|
||||||
// inserting a toasted value should /always/ produce a correct record
|
// inserting a toasted value should /always/ produce a correct record
|
||||||
String statement = "ALTER TABLE test_table ADD COLUMN not_toast integer; INSERT INTO test_table (not_toast, text) values (10, '" + toastedValue + "')";
|
String statement = "ALTER TABLE test_table ADD COLUMN not_toast integer; INSERT INTO test_table (not_toast, text) values (10, '" + toastedValue + "')";
|
||||||
consumer = testConsumer(1);
|
consumer = testConsumer(1);
|
||||||
recordsProducer.start(consumer, blackHole);
|
|
||||||
executeAndWait(statement);
|
executeAndWait(statement);
|
||||||
|
|
||||||
SourceRecord record = consumer.remove();
|
SourceRecord record = consumer.remove();
|
||||||
@ -1146,7 +1065,7 @@ public void shouldRefreshSchemaOnUnchangedToastedDataWhenSchemaChanged() throws
|
|||||||
|
|
||||||
consumer.expects(1);
|
consumer.expects(1);
|
||||||
executeAndWait(statement);
|
executeAndWait(statement);
|
||||||
Table tbl = recordsProducer.schema().tableFor(TableId.parse("public.test_table"));
|
Table tbl = getSchema().tableFor(TableId.parse("public.test_table"));
|
||||||
|
|
||||||
assertEquals(Arrays.asList("pk", "not_toast"), tbl.retrieveColumnNames());
|
assertEquals(Arrays.asList("pk", "not_toast"), tbl.retrieveColumnNames());
|
||||||
}
|
}
|
||||||
@ -1154,10 +1073,9 @@ public void shouldRefreshSchemaOnUnchangedToastedDataWhenSchemaChanged() throws
|
|||||||
@Test
|
@Test
|
||||||
@FixFor("DBZ-842")
|
@FixFor("DBZ-842")
|
||||||
public void shouldNotPropagateUnchangedToastedData() throws Exception {
|
public void shouldNotPropagateUnchangedToastedData() throws Exception {
|
||||||
PostgresConnectorConfig config = new PostgresConnectorConfig(TestHelper.defaultConfig()
|
startConnector(config -> config
|
||||||
.with(PostgresConnectorConfig.SCHEMA_REFRESH_MODE, PostgresConnectorConfig.SchemaRefreshMode.COLUMNS_DIFF_EXCLUDE_UNCHANGED_TOAST)
|
.with(PostgresConnectorConfig.SCHEMA_REFRESH_MODE, PostgresConnectorConfig.SchemaRefreshMode.COLUMNS_DIFF_EXCLUDE_UNCHANGED_TOAST)
|
||||||
.build());
|
);
|
||||||
setupRecordsProducer(config);
|
|
||||||
|
|
||||||
final String toastedValue1 = RandomStringUtils.randomAlphanumeric(10000);
|
final String toastedValue1 = RandomStringUtils.randomAlphanumeric(10000);
|
||||||
final String toastedValue2 = RandomStringUtils.randomAlphanumeric(10000);
|
final String toastedValue2 = RandomStringUtils.randomAlphanumeric(10000);
|
||||||
@ -1172,7 +1090,6 @@ public void shouldNotPropagateUnchangedToastedData() throws Exception {
|
|||||||
+ "INSERT INTO test_table (not_toast, text, mandatory_text) values (10, '" + toastedValue1 + "', '" + toastedValue1 + "');"
|
+ "INSERT INTO test_table (not_toast, text, mandatory_text) values (10, '" + toastedValue1 + "', '" + toastedValue1 + "');"
|
||||||
+ "INSERT INTO test_table (not_toast, text, mandatory_text) values (10, '" + toastedValue2 + "', '" + toastedValue2 + "');";
|
+ "INSERT INTO test_table (not_toast, text, mandatory_text) values (10, '" + toastedValue2 + "', '" + toastedValue2 + "');";
|
||||||
consumer = testConsumer(2);
|
consumer = testConsumer(2);
|
||||||
recordsProducer.start(consumer, blackHole);
|
|
||||||
executeAndWait(statement);
|
executeAndWait(statement);
|
||||||
|
|
||||||
// after record should contain the toasted value
|
// after record should contain the toasted value
|
||||||
@ -1194,7 +1111,7 @@ public void shouldNotPropagateUnchangedToastedData() throws Exception {
|
|||||||
consumer.expects(6);
|
consumer.expects(6);
|
||||||
executeAndWait(statement);
|
executeAndWait(statement);
|
||||||
consumer.process(record -> {
|
consumer.process(record -> {
|
||||||
Table tbl = recordsProducer.schema().tableFor(TableId.parse("public.test_table"));
|
Table tbl = getSchema().tableFor(TableId.parse("public.test_table"));
|
||||||
assertEquals(Arrays.asList("pk", "text", "not_toast", "mandatory_text"), tbl.retrieveColumnNames());
|
assertEquals(Arrays.asList("pk", "text", "not_toast", "mandatory_text"), tbl.retrieveColumnNames());
|
||||||
});
|
});
|
||||||
assertRecordSchemaAndValues(Arrays.asList(
|
assertRecordSchemaAndValues(Arrays.asList(
|
||||||
@ -1238,8 +1155,8 @@ public void shouldReceiveChangesForTableWithoutPrimaryKey() throws Exception{
|
|||||||
"ALTER TABLE test_table REPLICA IDENTITY FULL"
|
"ALTER TABLE test_table REPLICA IDENTITY FULL"
|
||||||
);
|
);
|
||||||
|
|
||||||
|
startConnector(Function.identity(), false);
|
||||||
consumer = testConsumer(1);
|
consumer = testConsumer(1);
|
||||||
recordsProducer.start(consumer, blackHole);
|
|
||||||
|
|
||||||
// INSERT
|
// INSERT
|
||||||
String statement = "INSERT INTO test_table (text) VALUES ('a');";
|
String statement = "INSERT INTO test_table (text) VALUES ('a');";
|
||||||
@ -1291,10 +1208,9 @@ public void shouldReceiveChangesForTableWithoutPrimaryKey() throws Exception{
|
|||||||
public void testPassingStreamParams() throws Exception {
|
public void testPassingStreamParams() throws Exception {
|
||||||
// Verify that passing stream parameters works by using the WAL2JSON add-tables parameter which acts as a
|
// Verify that passing stream parameters works by using the WAL2JSON add-tables parameter which acts as a
|
||||||
// whitelist.
|
// whitelist.
|
||||||
PostgresConnectorConfig config = new PostgresConnectorConfig(TestHelper.defaultConfig()
|
startConnector(config -> config
|
||||||
.with(PostgresConnectorConfig.STREAM_PARAMS, "add-tables=s1.should_stream")
|
.with(PostgresConnectorConfig.STREAM_PARAMS, "add-tables=s1.should_stream")
|
||||||
.build());
|
);
|
||||||
setupRecordsProducer(config);
|
|
||||||
String statement = "CREATE SCHEMA s1;" +
|
String statement = "CREATE SCHEMA s1;" +
|
||||||
"CREATE TABLE s1.should_stream (pk SERIAL, aa integer, PRIMARY KEY(pk));" +
|
"CREATE TABLE s1.should_stream (pk SERIAL, aa integer, PRIMARY KEY(pk));" +
|
||||||
"CREATE TABLE s1.should_not_stream (pk SERIAL, aa integer, PRIMARY KEY(pk));" +
|
"CREATE TABLE s1.should_not_stream (pk SERIAL, aa integer, PRIMARY KEY(pk));" +
|
||||||
@ -1304,7 +1220,6 @@ public void testPassingStreamParams() throws Exception {
|
|||||||
|
|
||||||
// Verify only one record made it
|
// Verify only one record made it
|
||||||
consumer = testConsumer(1);
|
consumer = testConsumer(1);
|
||||||
recordsProducer.start(consumer, blackHole);
|
|
||||||
executeAndWait(statement);
|
executeAndWait(statement);
|
||||||
|
|
||||||
// Verify the record that made it was from the whitelisted table
|
// Verify the record that made it was from the whitelisted table
|
||||||
@ -1317,10 +1232,9 @@ public void testPassingStreamParams() throws Exception {
|
|||||||
@SkipWhenDecoderPluginNameIsNot(value = WAL2JSON, reason = "WAL2JSON specific: Pass multiple stream parameters and values verifying they work")
|
@SkipWhenDecoderPluginNameIsNot(value = WAL2JSON, reason = "WAL2JSON specific: Pass multiple stream parameters and values verifying they work")
|
||||||
public void testPassingStreamMultipleParams() throws Exception {
|
public void testPassingStreamMultipleParams() throws Exception {
|
||||||
// Verify that passing multiple stream parameters and multiple parameter values works.
|
// Verify that passing multiple stream parameters and multiple parameter values works.
|
||||||
PostgresConnectorConfig config = new PostgresConnectorConfig(TestHelper.defaultConfig()
|
startConnector(config -> config
|
||||||
.with(PostgresConnectorConfig.STREAM_PARAMS, "add-tables=s1.should_stream,s2.*;filter-tables=s2.should_not_stream")
|
.with(PostgresConnectorConfig.STREAM_PARAMS, "add-tables=s1.should_stream,s2.*;filter-tables=s2.should_not_stream")
|
||||||
.build());
|
);
|
||||||
setupRecordsProducer(config);
|
|
||||||
String statement = "CREATE SCHEMA s1;" + "CREATE SCHEMA s2;" +
|
String statement = "CREATE SCHEMA s1;" + "CREATE SCHEMA s2;" +
|
||||||
"CREATE TABLE s1.should_stream (pk SERIAL, aa integer, PRIMARY KEY(pk));" +
|
"CREATE TABLE s1.should_stream (pk SERIAL, aa integer, PRIMARY KEY(pk));" +
|
||||||
"CREATE TABLE s2.should_stream (pk SERIAL, aa integer, PRIMARY KEY(pk));" +
|
"CREATE TABLE s2.should_stream (pk SERIAL, aa integer, PRIMARY KEY(pk));" +
|
||||||
@ -1334,7 +1248,6 @@ public void testPassingStreamMultipleParams() throws Exception {
|
|||||||
|
|
||||||
// Verify only the whitelisted record from s1 and s2 made it.
|
// Verify only the whitelisted record from s1 and s2 made it.
|
||||||
consumer = testConsumer(2);
|
consumer = testConsumer(2);
|
||||||
recordsProducer.start(consumer, blackHole);
|
|
||||||
executeAndWait(statement);
|
executeAndWait(statement);
|
||||||
|
|
||||||
// Verify the record that made it was from the whitelisted table
|
// Verify the record that made it was from the whitelisted table
|
||||||
@ -1373,14 +1286,10 @@ public void shouldReceiveChangesForReplicaIdentityFullTableWithToastedValueTable
|
|||||||
public void testEmptyChangesProducesHeartbeat() throws Exception {
|
public void testEmptyChangesProducesHeartbeat() throws Exception {
|
||||||
// the low heartbeat interval should make sure that a heartbeat message is emitted after each change record
|
// the low heartbeat interval should make sure that a heartbeat message is emitted after each change record
|
||||||
// received from Postgres
|
// received from Postgres
|
||||||
PostgresConnectorConfig config = new PostgresConnectorConfig(TestHelper.defaultConfig()
|
startConnector(config -> config.with(Heartbeat.HEARTBEAT_INTERVAL, "1"));
|
||||||
.with(Heartbeat.HEARTBEAT_INTERVAL, "1")
|
|
||||||
.build());
|
|
||||||
setupRecordsProducer(config);
|
|
||||||
|
|
||||||
// Expecting 1 heartbeat + 1 data change
|
// Expecting 1 heartbeat + 1 data change
|
||||||
consumer = testConsumer(1 + 1);
|
consumer.expects(1 + 1);
|
||||||
recordsProducer.start(consumer, blackHole);
|
|
||||||
|
|
||||||
executeAndWait(
|
executeAndWait(
|
||||||
"DROP TABLE IF EXISTS test_table;" +
|
"DROP TABLE IF EXISTS test_table;" +
|
||||||
@ -1403,13 +1312,7 @@ public void testEmptyChangesProducesHeartbeat() throws Exception {
|
|||||||
@Test
|
@Test
|
||||||
@FixFor("DBZ-1082")
|
@FixFor("DBZ-1082")
|
||||||
public void shouldHaveNoXminWhenNotEnabled() throws Exception {
|
public void shouldHaveNoXminWhenNotEnabled() throws Exception {
|
||||||
// Verify that passing multiple stream parameters and multiple parameter values works.
|
startConnector(config -> config.with(PostgresConnectorConfig.XMIN_FETCH_INTERVAL, "0"));
|
||||||
PostgresConnectorConfig config = new PostgresConnectorConfig(TestHelper.defaultConfig()
|
|
||||||
.with(PostgresConnectorConfig.XMIN_FETCH_INTERVAL, "0")
|
|
||||||
.build());
|
|
||||||
setupRecordsProducer(config);
|
|
||||||
consumer = testConsumer(1);
|
|
||||||
recordsProducer.start(consumer, blackHole);
|
|
||||||
|
|
||||||
TestHelper.execute("ALTER TABLE test_table REPLICA IDENTITY DEFAULT;");
|
TestHelper.execute("ALTER TABLE test_table REPLICA IDENTITY DEFAULT;");
|
||||||
String statement = "INSERT INTO test_table (text) VALUES ('no_xmin');";
|
String statement = "INSERT INTO test_table (text) VALUES ('no_xmin');";
|
||||||
@ -1428,13 +1331,7 @@ public void shouldHaveNoXminWhenNotEnabled() throws Exception {
|
|||||||
@Test
|
@Test
|
||||||
@FixFor("DBZ-1082")
|
@FixFor("DBZ-1082")
|
||||||
public void shouldHaveXminWhenEnabled() throws Exception {
|
public void shouldHaveXminWhenEnabled() throws Exception {
|
||||||
// Verify that passing multiple stream parameters and multiple parameter values works.
|
startConnector(config -> config.with(PostgresConnectorConfig.XMIN_FETCH_INTERVAL, "10"));
|
||||||
PostgresConnectorConfig config = new PostgresConnectorConfig(TestHelper.defaultConfig()
|
|
||||||
.with(PostgresConnectorConfig.XMIN_FETCH_INTERVAL, "10")
|
|
||||||
.build());
|
|
||||||
setupRecordsProducer(config);
|
|
||||||
consumer = testConsumer(1);
|
|
||||||
recordsProducer.start(consumer, blackHole);
|
|
||||||
|
|
||||||
TestHelper.execute("ALTER TABLE test_table REPLICA IDENTITY DEFAULT;");
|
TestHelper.execute("ALTER TABLE test_table REPLICA IDENTITY DEFAULT;");
|
||||||
String statement = "INSERT INTO test_table (text) VALUES ('with_xmin');";
|
String statement = "INSERT INTO test_table (text) VALUES ('with_xmin');";
|
||||||
@ -1451,11 +1348,6 @@ public void shouldHaveXminWhenEnabled() throws Exception {
|
|||||||
}
|
}
|
||||||
|
|
||||||
private void testReceiveChangesForReplicaIdentityFullTableWithToastedValue(PostgresConnectorConfig.SchemaRefreshMode mode, boolean tablesBeforeStart) throws Exception{
|
private void testReceiveChangesForReplicaIdentityFullTableWithToastedValue(PostgresConnectorConfig.SchemaRefreshMode mode, boolean tablesBeforeStart) throws Exception{
|
||||||
PostgresConnectorConfig config = new PostgresConnectorConfig(TestHelper.defaultConfig()
|
|
||||||
.with(PostgresConnectorConfig.SCHEMA_REFRESH_MODE, mode)
|
|
||||||
.build());
|
|
||||||
setupRecordsProducer(config);
|
|
||||||
|
|
||||||
if (tablesBeforeStart) {
|
if (tablesBeforeStart) {
|
||||||
TestHelper.execute(
|
TestHelper.execute(
|
||||||
"DROP TABLE IF EXISTS test_table;",
|
"DROP TABLE IF EXISTS test_table;",
|
||||||
@ -1464,8 +1356,7 @@ private void testReceiveChangesForReplicaIdentityFullTableWithToastedValue(Postg
|
|||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
consumer = testConsumer(1);
|
startConnector(config -> config.with(PostgresConnectorConfig.SCHEMA_REFRESH_MODE, mode));
|
||||||
recordsProducer.start(consumer, blackHole);
|
|
||||||
|
|
||||||
final String toastedValue = RandomStringUtils.randomAlphanumeric(10000);
|
final String toastedValue = RandomStringUtils.randomAlphanumeric(10000);
|
||||||
|
|
||||||
@ -1515,8 +1406,6 @@ private void testReceiveChangesForReplicaIdentityFullTableWithToastedValue(Postg
|
|||||||
new SchemaAndValueField("not_toast", SchemaBuilder.OPTIONAL_INT32_SCHEMA, 20)
|
new SchemaAndValueField("not_toast", SchemaBuilder.OPTIONAL_INT32_SCHEMA, 20)
|
||||||
), updatedRecord, Envelope.FieldName.AFTER);
|
), updatedRecord, Envelope.FieldName.AFTER);
|
||||||
}
|
}
|
||||||
|
|
||||||
recordsProducer.stop();
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private void assertHeartBeatRecordInserted() {
|
private void assertHeartBeatRecordInserted() {
|
||||||
@ -1532,21 +1421,6 @@ private void assertHeartBeatRecordInserted() {
|
|||||||
assertThat(value.getInt64("ts_ms")).isLessThanOrEqualTo(Instant.now().toEpochMilli());
|
assertThat(value.getInt64("ts_ms")).isLessThanOrEqualTo(Instant.now().toEpochMilli());
|
||||||
}
|
}
|
||||||
|
|
||||||
private void setupRecordsProducer(PostgresConnectorConfig config) {
|
|
||||||
if (recordsProducer != null) {
|
|
||||||
recordsProducer.stop();
|
|
||||||
}
|
|
||||||
|
|
||||||
TopicSelector<TableId> selector = PostgresTopicSelector.create(config);
|
|
||||||
|
|
||||||
PostgresTaskContext context = new PostgresTaskContext(
|
|
||||||
config,
|
|
||||||
TestHelper.getSchema(config),
|
|
||||||
selector
|
|
||||||
);
|
|
||||||
recordsProducer = new RecordsStreamProducer(context, new SourceInfo(config));
|
|
||||||
}
|
|
||||||
|
|
||||||
private void assertInsert(String statement, List<SchemaAndValueField> expectedSchemaAndValuesByColumn) {
|
private void assertInsert(String statement, List<SchemaAndValueField> expectedSchemaAndValuesByColumn) {
|
||||||
assertInsert(statement, null, expectedSchemaAndValuesByColumn);
|
assertInsert(statement, null, expectedSchemaAndValuesByColumn);
|
||||||
}
|
}
|
||||||
|
@ -13,17 +13,13 @@
|
|||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.concurrent.TimeUnit;
|
import java.util.concurrent.TimeUnit;
|
||||||
|
|
||||||
import io.debezium.connector.postgresql.snapshot.InitialOnlySnapshotter;
|
|
||||||
import io.debezium.connector.postgresql.snapshot.SnapshotterWrapper;
|
|
||||||
import io.debezium.connector.postgresql.spi.Snapshotter;
|
|
||||||
import org.apache.kafka.connect.source.SourceRecord;
|
import org.apache.kafka.connect.source.SourceRecord;
|
||||||
import org.fest.assertions.Assertions;
|
import org.fest.assertions.Assertions;
|
||||||
import org.junit.After;
|
import org.junit.Before;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
|
||||||
import io.debezium.config.Configuration;
|
import io.debezium.config.Configuration;
|
||||||
import io.debezium.relational.TableId;
|
import io.debezium.connector.postgresql.PostgresConnectorConfig.SnapshotMode;
|
||||||
import io.debezium.schema.TopicSelector;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Integration test for {@link io.debezium.connector.postgresql.PostgresConnectorConfig.SNAPSHOT_SELECT_STATEMENT_OVERRIDES_BY_TABLE}
|
* Integration test for {@link io.debezium.connector.postgresql.PostgresConnectorConfig.SNAPSHOT_SELECT_STATEMENT_OVERRIDES_BY_TABLE}
|
||||||
@ -49,43 +45,23 @@ public class SnapshotWithOverridesProducerIT extends AbstractRecordsProducerTest
|
|||||||
"INSERT INTO over.t2 VALUES (102);" +
|
"INSERT INTO over.t2 VALUES (102);" +
|
||||||
"INSERT INTO over.t2 VALUES (103);";
|
"INSERT INTO over.t2 VALUES (103);";
|
||||||
|
|
||||||
private RecordsSnapshotProducer snapshotProducer;
|
@Before
|
||||||
private PostgresTaskContext context;
|
public void before() throws SQLException {
|
||||||
private PostgresConnectorConfig config;
|
|
||||||
|
|
||||||
public void before(Configuration overrides) throws SQLException {
|
|
||||||
TestHelper.dropAllSchemas();
|
TestHelper.dropAllSchemas();
|
||||||
|
|
||||||
config = new PostgresConnectorConfig(TestHelper.defaultConfig().with(overrides).build());
|
|
||||||
TopicSelector<TableId> selector = PostgresTopicSelector.create(config);
|
|
||||||
context = new PostgresTaskContext(
|
|
||||||
config,
|
|
||||||
TestHelper.getSchema(config),
|
|
||||||
selector
|
|
||||||
);
|
|
||||||
}
|
|
||||||
|
|
||||||
@After
|
|
||||||
public void after() throws Exception {
|
|
||||||
if (snapshotProducer != null) {
|
|
||||||
snapshotProducer.stop();
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void shouldUseOverriddenSelectStatementDuringSnapshotting() throws Exception {
|
public void shouldUseOverriddenSelectStatementDuringSnapshotting() throws Exception {
|
||||||
before(Configuration.create()
|
TestHelper.execute(STATEMENTS);
|
||||||
|
|
||||||
|
buildProducer(TestHelper.defaultConfig()
|
||||||
.with(PostgresConnectorConfig.SNAPSHOT_SELECT_STATEMENT_OVERRIDES_BY_TABLE, "over.t1")
|
.with(PostgresConnectorConfig.SNAPSHOT_SELECT_STATEMENT_OVERRIDES_BY_TABLE, "over.t1")
|
||||||
.with(PostgresConnectorConfig.SNAPSHOT_SELECT_STATEMENT_OVERRIDES_BY_TABLE.name() + ".over.t1", "SELECT * FROM over.t1 WHERE pk > 100")
|
.with(PostgresConnectorConfig.SNAPSHOT_SELECT_STATEMENT_OVERRIDES_BY_TABLE.name() + ".over.t1", "SELECT * FROM over.t1 WHERE pk > 100")
|
||||||
.build());
|
);
|
||||||
snapshotProducer = buildStreamProducer(context, config);
|
|
||||||
|
|
||||||
final int expectedRecordsCount = 3 + 6;
|
final int expectedRecordsCount = 3 + 6;
|
||||||
|
|
||||||
TestHelper.execute(STATEMENTS);
|
|
||||||
TestConsumer consumer = testConsumer(expectedRecordsCount, "over");
|
TestConsumer consumer = testConsumer(expectedRecordsCount, "over");
|
||||||
|
|
||||||
snapshotProducer.start(consumer, e -> {});
|
|
||||||
consumer.await(TestHelper.waitTimeForRecords(), TimeUnit.SECONDS);
|
consumer.await(TestHelper.waitTimeForRecords(), TimeUnit.SECONDS);
|
||||||
|
|
||||||
final Map<String, List<SourceRecord>> recordsByTopic = recordsByTopic(expectedRecordsCount, consumer);
|
final Map<String, List<SourceRecord>> recordsByTopic = recordsByTopic(expectedRecordsCount, consumer);
|
||||||
@ -95,19 +71,17 @@ public void shouldUseOverriddenSelectStatementDuringSnapshotting() throws Except
|
|||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void shouldUseMultipleOverriddenSelectStatementsDuringSnapshotting() throws Exception {
|
public void shouldUseMultipleOverriddenSelectStatementsDuringSnapshotting() throws Exception {
|
||||||
before(Configuration.create()
|
TestHelper.execute(STATEMENTS);
|
||||||
|
|
||||||
|
buildProducer(TestHelper.defaultConfig()
|
||||||
.with(PostgresConnectorConfig.SNAPSHOT_SELECT_STATEMENT_OVERRIDES_BY_TABLE, "over.t1,over.t2")
|
.with(PostgresConnectorConfig.SNAPSHOT_SELECT_STATEMENT_OVERRIDES_BY_TABLE, "over.t1,over.t2")
|
||||||
.with(PostgresConnectorConfig.SNAPSHOT_SELECT_STATEMENT_OVERRIDES_BY_TABLE.name() + ".over.t1", "SELECT * FROM over.t1 WHERE pk > 101")
|
.with(PostgresConnectorConfig.SNAPSHOT_SELECT_STATEMENT_OVERRIDES_BY_TABLE.name() + ".over.t1", "SELECT * FROM over.t1 WHERE pk > 101")
|
||||||
.with(PostgresConnectorConfig.SNAPSHOT_SELECT_STATEMENT_OVERRIDES_BY_TABLE.name() + ".over.t2", "SELECT * FROM over.t2 WHERE pk > 100")
|
.with(PostgresConnectorConfig.SNAPSHOT_SELECT_STATEMENT_OVERRIDES_BY_TABLE.name() + ".over.t2", "SELECT * FROM over.t2 WHERE pk > 100")
|
||||||
.build());
|
);
|
||||||
snapshotProducer = buildStreamProducer(context, config);
|
|
||||||
|
|
||||||
final int expectedRecordsCount = 2 + 3;
|
final int expectedRecordsCount = 2 + 3;
|
||||||
|
|
||||||
TestHelper.execute(STATEMENTS);
|
|
||||||
TestConsumer consumer = testConsumer(expectedRecordsCount, "over");
|
TestConsumer consumer = testConsumer(expectedRecordsCount, "over");
|
||||||
|
|
||||||
snapshotProducer.start(consumer, e -> {});
|
|
||||||
consumer.await(TestHelper.waitTimeForRecords(), TimeUnit.SECONDS);
|
consumer.await(TestHelper.waitTimeForRecords(), TimeUnit.SECONDS);
|
||||||
|
|
||||||
final Map<String, List<SourceRecord>> recordsByTopic = recordsByTopic(expectedRecordsCount, consumer);
|
final Map<String, List<SourceRecord>> recordsByTopic = recordsByTopic(expectedRecordsCount, consumer);
|
||||||
@ -125,9 +99,11 @@ private Map<String, List<SourceRecord>> recordsByTopic(final int expectedRecords
|
|||||||
return recordsByTopic;
|
return recordsByTopic;
|
||||||
}
|
}
|
||||||
|
|
||||||
private RecordsSnapshotProducer buildStreamProducer(PostgresTaskContext ctx, PostgresConnectorConfig config) {
|
private void buildProducer(Configuration.Builder config) {
|
||||||
Snapshotter sn = new InitialOnlySnapshotter();
|
start(PostgresConnector.class, config
|
||||||
SnapshotterWrapper snw = new SnapshotterWrapper(sn, config, null, null);
|
.with(PostgresConnectorConfig.SNAPSHOT_MODE, SnapshotMode.INITIAL_ONLY)
|
||||||
return new RecordsSnapshotProducer(ctx, TestHelper.sourceInfo(), snw);
|
.build()
|
||||||
|
);
|
||||||
|
assertConnectorIsRunning();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -38,7 +38,7 @@
|
|||||||
public final class TestHelper {
|
public final class TestHelper {
|
||||||
|
|
||||||
protected static final String TEST_SERVER = "test_server";
|
protected static final String TEST_SERVER = "test_server";
|
||||||
protected static final String TEST_DATABASE = "test_database";
|
protected static final String TEST_DATABASE = "postgres";
|
||||||
protected static final String PK_FIELD = "pk";
|
protected static final String PK_FIELD = "pk";
|
||||||
private static final String TEST_PROPERTY_PREFIX = "debezium.test.";
|
private static final String TEST_PROPERTY_PREFIX = "debezium.test.";
|
||||||
private static final Logger LOGGER = LoggerFactory.getLogger(TestHelper.class);
|
private static final Logger LOGGER = LoggerFactory.getLogger(TestHelper.class);
|
||||||
|
@ -9,6 +9,7 @@ log4j.rootLogger=INFO, stdout
|
|||||||
|
|
||||||
# Set up the default logging to be INFO level, then override specific units
|
# Set up the default logging to be INFO level, then override specific units
|
||||||
log4j.logger.io.debezium=INFO
|
log4j.logger.io.debezium=INFO
|
||||||
|
log4j.logger.io.debezium.connector.postgresql=INFO
|
||||||
log4j.logger.io.debezium.embedded.EmbeddedEngine$EmbeddedConfig=WARN
|
log4j.logger.io.debezium.embedded.EmbeddedEngine$EmbeddedConfig=WARN
|
||||||
#log4j.logger.io.debezium.embedded.EmbeddedEngine=DEBUG
|
#log4j.logger.io.debezium.embedded.EmbeddedEngine=DEBUG
|
||||||
#log4j.logger.io.debezium.connector.postgresql.RecordsStreamProducer=DEBUG
|
#log4j.logger.io.debezium.connector.postgresql.RecordsStreamProducer=DEBUG
|
||||||
|
@ -108,14 +108,6 @@ public void setChangePosition(TxLogPosition position, int eventCount) {
|
|||||||
sourceInfo.setChangeLsn(position.getInTxLsn());
|
sourceInfo.setChangeLsn(position.getInTxLsn());
|
||||||
}
|
}
|
||||||
|
|
||||||
public void setSourceTime(Instant instant) {
|
|
||||||
sourceInfo.setSourceTime(instant);
|
|
||||||
}
|
|
||||||
|
|
||||||
public void setTableId(TableId tableId) {
|
|
||||||
sourceInfo.setTableId(tableId);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public boolean isSnapshotRunning() {
|
public boolean isSnapshotRunning() {
|
||||||
return sourceInfo.isSnapshot() && !snapshotCompleted;
|
return sourceInfo.isSnapshot() && !snapshotCompleted;
|
||||||
@ -186,4 +178,10 @@ public String toString() {
|
|||||||
public void markLastSnapshotRecord() {
|
public void markLastSnapshotRecord() {
|
||||||
sourceInfo.setSnapshot(SnapshotRecord.LAST);
|
sourceInfo.setSnapshot(SnapshotRecord.LAST);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void event(TableId tableId, Instant timestamp) {
|
||||||
|
sourceInfo.setSourceTime(timestamp);
|
||||||
|
sourceInfo.setTableId(tableId);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
@ -10,7 +10,7 @@
|
|||||||
import java.sql.SQLException;
|
import java.sql.SQLException;
|
||||||
import java.sql.Savepoint;
|
import java.sql.Savepoint;
|
||||||
import java.sql.Statement;
|
import java.sql.Statement;
|
||||||
import java.time.Instant;
|
import java.util.Optional;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
@ -20,16 +20,15 @@
|
|||||||
import io.debezium.connector.sqlserver.SqlServerConnectorConfig.SnapshotIsolationMode;
|
import io.debezium.connector.sqlserver.SqlServerConnectorConfig.SnapshotIsolationMode;
|
||||||
import io.debezium.pipeline.EventDispatcher;
|
import io.debezium.pipeline.EventDispatcher;
|
||||||
import io.debezium.pipeline.source.spi.SnapshotProgressListener;
|
import io.debezium.pipeline.source.spi.SnapshotProgressListener;
|
||||||
import io.debezium.pipeline.spi.ChangeRecordEmitter;
|
|
||||||
import io.debezium.pipeline.spi.OffsetContext;
|
import io.debezium.pipeline.spi.OffsetContext;
|
||||||
import io.debezium.relational.HistorizedRelationalSnapshotChangeEventSource;
|
import io.debezium.relational.RelationalSnapshotChangeEventSource;
|
||||||
import io.debezium.relational.Table;
|
import io.debezium.relational.Table;
|
||||||
import io.debezium.relational.TableId;
|
import io.debezium.relational.TableId;
|
||||||
import io.debezium.schema.SchemaChangeEvent;
|
import io.debezium.schema.SchemaChangeEvent;
|
||||||
import io.debezium.schema.SchemaChangeEvent.SchemaChangeEventType;
|
import io.debezium.schema.SchemaChangeEvent.SchemaChangeEventType;
|
||||||
import io.debezium.util.Clock;
|
import io.debezium.util.Clock;
|
||||||
|
|
||||||
public class SqlServerSnapshotChangeEventSource extends HistorizedRelationalSnapshotChangeEventSource {
|
public class SqlServerSnapshotChangeEventSource extends RelationalSnapshotChangeEventSource {
|
||||||
|
|
||||||
private static final Logger LOGGER = LoggerFactory.getLogger(SqlServerSnapshotChangeEventSource.class);
|
private static final Logger LOGGER = LoggerFactory.getLogger(SqlServerSnapshotChangeEventSource.class);
|
||||||
|
|
||||||
@ -199,15 +198,8 @@ protected void complete(SnapshotContext snapshotContext) {
|
|||||||
* @return a valid query string
|
* @return a valid query string
|
||||||
*/
|
*/
|
||||||
@Override
|
@Override
|
||||||
protected String getSnapshotSelect(SnapshotContext snapshotContext, TableId tableId) {
|
protected Optional<String> getSnapshotSelect(SnapshotContext snapshotContext, TableId tableId) {
|
||||||
return String.format("SELECT * FROM [%s].[%s]", tableId.schema(), tableId.table());
|
return Optional.of(String.format("SELECT * FROM [%s].[%s]", tableId.schema(), tableId.table()));
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
protected ChangeRecordEmitter getChangeRecordEmitter(SnapshotContext snapshotContext, TableId tableId, Object[] row) {
|
|
||||||
((SqlServerOffsetContext) snapshotContext.offset).setSourceTime(Instant.ofEpochMilli(getClock().currentTimeInMillis()));
|
|
||||||
((SqlServerOffsetContext) snapshotContext.offset).setTableId(tableId);
|
|
||||||
return new SnapshotChangeRecordEmitter(snapshotContext.offset, row, getClock());
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -222,8 +222,7 @@ public void execute(ChangeEventSourceContext context) throws InterruptedExceptio
|
|||||||
final Object[] dataNext = (operation == SqlServerChangeRecordEmitter.OP_UPDATE_BEFORE) ? tableWithSmallestLsn.getData() : null;
|
final Object[] dataNext = (operation == SqlServerChangeRecordEmitter.OP_UPDATE_BEFORE) ? tableWithSmallestLsn.getData() : null;
|
||||||
|
|
||||||
offsetContext.setChangePosition(tableWithSmallestLsn.getChangePosition(), eventCount);
|
offsetContext.setChangePosition(tableWithSmallestLsn.getChangePosition(), eventCount);
|
||||||
offsetContext.setSourceTime(metadataConnection.timestampOfLsn(tableWithSmallestLsn.getChangePosition().getCommitLsn()));
|
offsetContext.event(tableWithSmallestLsn.getChangeTable().getSourceTableId(), metadataConnection.timestampOfLsn(tableWithSmallestLsn.getChangePosition().getCommitLsn()));
|
||||||
offsetContext.setTableId(tableWithSmallestLsn.getChangeTable().getSourceTableId());
|
|
||||||
|
|
||||||
dispatcher
|
dispatcher
|
||||||
.dispatchDataChangeEvent(
|
.dispatchDataChangeEvent(
|
||||||
|
@ -87,6 +87,7 @@ public void takeSnapshotInExclusiveMode() throws Exception {
|
|||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void takeSnapshotInSnapshotMode() throws Exception {
|
public void takeSnapshotInSnapshotMode() throws Exception {
|
||||||
|
Testing.Print.enable();
|
||||||
takeSnapshot(SnapshotIsolationMode.SNAPSHOT);
|
takeSnapshot(SnapshotIsolationMode.SNAPSHOT);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -6,6 +6,7 @@
|
|||||||
package io.debezium.pipeline;
|
package io.debezium.pipeline;
|
||||||
|
|
||||||
import java.util.Objects;
|
import java.util.Objects;
|
||||||
|
import java.util.Optional;
|
||||||
import java.util.function.Supplier;
|
import java.util.function.Supplier;
|
||||||
|
|
||||||
import org.apache.kafka.connect.data.Schema;
|
import org.apache.kafka.connect.data.Schema;
|
||||||
@ -15,7 +16,9 @@
|
|||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
import io.debezium.config.CommonConnectorConfig;
|
import io.debezium.config.CommonConnectorConfig;
|
||||||
|
import io.debezium.connector.SnapshotRecord;
|
||||||
import io.debezium.connector.base.ChangeEventQueue;
|
import io.debezium.connector.base.ChangeEventQueue;
|
||||||
|
import io.debezium.data.Envelope;
|
||||||
import io.debezium.data.Envelope.Operation;
|
import io.debezium.data.Envelope.Operation;
|
||||||
import io.debezium.heartbeat.Heartbeat;
|
import io.debezium.heartbeat.Heartbeat;
|
||||||
import io.debezium.pipeline.source.spi.DataChangeEventListener;
|
import io.debezium.pipeline.source.spi.DataChangeEventListener;
|
||||||
@ -54,6 +57,7 @@ public class EventDispatcher<T extends DataCollectionId> {
|
|||||||
private final Heartbeat heartbeat;
|
private final Heartbeat heartbeat;
|
||||||
private DataChangeEventListener eventListener = DataChangeEventListener.NO_OP;
|
private DataChangeEventListener eventListener = DataChangeEventListener.NO_OP;
|
||||||
private final boolean emitTombstonesOnDelete;
|
private final boolean emitTombstonesOnDelete;
|
||||||
|
private InconsistentSchemaHandler<T> inconsistentSchemaHandler = this::errorOnMissingSchema;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Change event receiver for events dispatched from a streaming change event source.
|
* Change event receiver for events dispatched from a streaming change event source.
|
||||||
@ -85,8 +89,7 @@ public void dispatchSnapshotEvent(T dataCollectionId, ChangeRecordEmitter change
|
|||||||
|
|
||||||
// TODO handle as per inconsistent schema info option
|
// TODO handle as per inconsistent schema info option
|
||||||
if (dataCollectionSchema == null) {
|
if (dataCollectionSchema == null) {
|
||||||
eventListener.onErroneousEvent("source = " + dataCollectionId);
|
errorOnMissingSchema(dataCollectionId, changeRecordEmitter);
|
||||||
throw new IllegalArgumentException("No metadata registered for captured table " + dataCollectionId);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
changeRecordEmitter.emitChangeRecords(dataCollectionSchema, new Receiver() {
|
changeRecordEmitter.emitChangeRecords(dataCollectionSchema, new Receiver() {
|
||||||
@ -122,9 +125,13 @@ public void dispatchDataChangeEvent(T dataCollectionId, ChangeRecordEmitter chan
|
|||||||
|
|
||||||
// TODO handle as per inconsistent schema info option
|
// TODO handle as per inconsistent schema info option
|
||||||
if (dataCollectionSchema == null) {
|
if (dataCollectionSchema == null) {
|
||||||
eventListener.onErroneousEvent("source = " + dataCollectionId);
|
final Optional<DataCollectionSchema> replacementSchema = inconsistentSchemaHandler.handle(dataCollectionId, changeRecordEmitter);
|
||||||
throw new IllegalArgumentException("No metadata registered for captured table " + dataCollectionId);
|
if (!replacementSchema.isPresent()) {
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
dataCollectionSchema = replacementSchema.get();
|
||||||
}
|
}
|
||||||
|
dataCollectionSchema = changeRecordEmitter.synchronizeTableSchema(dataCollectionSchema);
|
||||||
|
|
||||||
changeRecordEmitter.emitChangeRecords(dataCollectionSchema, new Receiver() {
|
changeRecordEmitter.emitChangeRecords(dataCollectionSchema, new Receiver() {
|
||||||
|
|
||||||
@ -144,6 +151,19 @@ public void changeRecord(DataCollectionSchema schema, Operation operation, Objec
|
|||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public void setInconsistentSchemaHandler(InconsistentSchemaHandler<T> inconsistentSchemaHandler) {
|
||||||
|
this.inconsistentSchemaHandler = inconsistentSchemaHandler;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Optional<DataCollectionSchema> errorOnMissingSchema(T dataCollectionId, ChangeRecordEmitter changeRecordEmitter) {
|
||||||
|
eventListener.onErroneousEvent("source = " + dataCollectionId);
|
||||||
|
throw new IllegalArgumentException("No metadata registered for captured table " + dataCollectionId);
|
||||||
|
}
|
||||||
|
|
||||||
|
public Optional<DataCollectionSchema> ignoreMissingSchema(T dataCollectionId, ChangeRecordEmitter changeRecordEmitter) {
|
||||||
|
return Optional.empty();
|
||||||
|
}
|
||||||
|
|
||||||
public void dispatchSchemaChangeEvent(T dataCollectionId, SchemaChangeEventEmitter schemaChangeEventEmitter) throws InterruptedException {
|
public void dispatchSchemaChangeEvent(T dataCollectionId, SchemaChangeEventEmitter schemaChangeEventEmitter) throws InterruptedException {
|
||||||
if(!filter.isIncluded(dataCollectionId)) {
|
if(!filter.isIncluded(dataCollectionId)) {
|
||||||
LOGGER.trace("Filtering schema change event for {}", dataCollectionId);
|
LOGGER.trace("Filtering schema change event for {}", dataCollectionId);
|
||||||
@ -227,7 +247,6 @@ public void changeRecord(DataCollectionSchema dataCollectionSchema, Operation op
|
|||||||
bufferedEvent = () -> {
|
bufferedEvent = () -> {
|
||||||
SourceRecord record = new SourceRecord(offsetContext.getPartition(), offsetContext.getOffset(),
|
SourceRecord record = new SourceRecord(offsetContext.getPartition(), offsetContext.getOffset(),
|
||||||
topicName, null, keySchema, key, dataCollectionSchema.getEnvelopeSchema().schema(), value);
|
topicName, null, keySchema, key, dataCollectionSchema.getEnvelopeSchema().schema(), value);
|
||||||
|
|
||||||
return changeEventCreator.createDataChangeEvent(record);
|
return changeEventCreator.createDataChangeEvent(record);
|
||||||
};
|
};
|
||||||
}
|
}
|
||||||
@ -235,7 +254,19 @@ public void changeRecord(DataCollectionSchema dataCollectionSchema, Operation op
|
|||||||
@Override
|
@Override
|
||||||
public void completeSnapshot() throws InterruptedException {
|
public void completeSnapshot() throws InterruptedException {
|
||||||
if(bufferedEvent != null) {
|
if(bufferedEvent != null) {
|
||||||
queue.enqueue(bufferedEvent.get());
|
// It is possible that the last snapshotted table was empty
|
||||||
|
// this way we ensure that the last event is always marked as last
|
||||||
|
// even if it originates form non-last table
|
||||||
|
final DataChangeEvent event = bufferedEvent.get();
|
||||||
|
final Struct envelope = (Struct) event.getRecord().value();
|
||||||
|
if (envelope.schema().field(Envelope.FieldName.SOURCE) != null) {
|
||||||
|
final Struct source = envelope.getStruct(Envelope.FieldName.SOURCE);
|
||||||
|
final SnapshotRecord snapshot = SnapshotRecord.fromSource(source);
|
||||||
|
if (snapshot == SnapshotRecord.TRUE) {
|
||||||
|
SnapshotRecord.LAST.toSource(source);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
queue.enqueue(event);
|
||||||
bufferedEvent = null;
|
bufferedEvent = null;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -257,4 +288,17 @@ public void schemaChangeEvent(SchemaChangeEvent event) throws InterruptedExcepti
|
|||||||
public void setEventListener(DataChangeEventListener eventListener) {
|
public void setEventListener(DataChangeEventListener eventListener) {
|
||||||
this.eventListener = eventListener;
|
this.eventListener = eventListener;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Reaction to an incoming change event for which schema is not found
|
||||||
|
*/
|
||||||
|
@FunctionalInterface
|
||||||
|
public static interface InconsistentSchemaHandler<T extends DataCollectionId> {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return collection schema if the schema was updated and event can be processed, {@code empty} to skip the processing
|
||||||
|
*/
|
||||||
|
Optional<DataCollectionSchema> handle(T dataCollectionId, ChangeRecordEmitter changeRecordEmitter);
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -29,6 +29,10 @@ public interface ChangeRecordEmitter {
|
|||||||
*/
|
*/
|
||||||
OffsetContext getOffset();
|
OffsetContext getOffset();
|
||||||
|
|
||||||
|
default DataCollectionSchema synchronizeTableSchema(DataCollectionSchema dataCollectionSchema) {
|
||||||
|
return dataCollectionSchema;
|
||||||
|
}
|
||||||
|
|
||||||
public interface Receiver {
|
public interface Receiver {
|
||||||
void changeRecord(DataCollectionSchema schema, Operation operation, Object key, Struct value, OffsetContext offset) throws InterruptedException;
|
void changeRecord(DataCollectionSchema schema, Operation operation, Object key, Struct value, OffsetContext offset) throws InterruptedException;
|
||||||
}
|
}
|
||||||
|
@ -5,11 +5,14 @@
|
|||||||
*/
|
*/
|
||||||
package io.debezium.pipeline.spi;
|
package io.debezium.pipeline.spi;
|
||||||
|
|
||||||
|
import java.time.Instant;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
|
||||||
import org.apache.kafka.connect.data.Schema;
|
import org.apache.kafka.connect.data.Schema;
|
||||||
import org.apache.kafka.connect.data.Struct;
|
import org.apache.kafka.connect.data.Struct;
|
||||||
|
|
||||||
|
import io.debezium.relational.TableId;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Keeps track of the current offset within the source DB's change stream. This reflects in the offset as committed to
|
* Keeps track of the current offset within the source DB's change stream. This reflects in the offset as committed to
|
||||||
* Kafka and in the source info block contained within CDC messages themselves.
|
* Kafka and in the source info block contained within CDC messages themselves.
|
||||||
@ -57,4 +60,11 @@ interface Loader {
|
|||||||
* Signals that a snapshot has been completed, which should reflect in an updated offset state.
|
* Signals that a snapshot has been completed, which should reflect in an updated offset state.
|
||||||
*/
|
*/
|
||||||
void postSnapshotCompletion();
|
void postSnapshotCompletion();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Records the name of the table and the timestamp of the last event
|
||||||
|
* @param tableId
|
||||||
|
* @param timestamp
|
||||||
|
*/
|
||||||
|
void event(TableId tableId, Instant timestamp);
|
||||||
}
|
}
|
||||||
|
@ -89,8 +89,9 @@ private void emitUpdateRecord(Receiver receiver, TableSchema tableSchema)
|
|||||||
Struct newValue = tableSchema.valueFromColumnData(newColumnValues);
|
Struct newValue = tableSchema.valueFromColumnData(newColumnValues);
|
||||||
Struct oldValue = tableSchema.valueFromColumnData(oldColumnValues);
|
Struct oldValue = tableSchema.valueFromColumnData(oldColumnValues);
|
||||||
|
|
||||||
// regular update
|
// some configurations does not provide old values in case of updates
|
||||||
if (Objects.equals(oldKey, newKey)) {
|
// in this case we handle all updates as regular ones
|
||||||
|
if (oldKey == null || Objects.equals(oldKey, newKey)) {
|
||||||
Struct envelope = tableSchema.getEnvelopeSchema().update(oldValue, newValue, offsetContext.getSourceInfo(), clock.currentTimeInMillis());
|
Struct envelope = tableSchema.getEnvelopeSchema().update(oldValue, newValue, offsetContext.getSourceInfo(), clock.currentTimeInMillis());
|
||||||
receiver.changeRecord(tableSchema, Operation.UPDATE, newKey, envelope, offsetContext);
|
receiver.changeRecord(tableSchema, Operation.UPDATE, newKey, envelope, offsetContext);
|
||||||
}
|
}
|
||||||
|
@ -29,6 +29,8 @@
|
|||||||
* @author Gunnar Morling
|
* @author Gunnar Morling
|
||||||
*/
|
*/
|
||||||
public abstract class RelationalDatabaseConnectorConfig extends CommonConnectorConfig {
|
public abstract class RelationalDatabaseConnectorConfig extends CommonConnectorConfig {
|
||||||
|
private static final String TABLE_BLACKLIST_NAME = "table.blacklist";
|
||||||
|
private static final String TABLE_WHITELIST_NAME = "table.whitelist";
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The set of predefined DecimalHandlingMode options or aliases.
|
* The set of predefined DecimalHandlingMode options or aliases.
|
||||||
@ -127,7 +129,7 @@ public static DecimalHandlingMode parse(String value, String defaultValue) {
|
|||||||
* {@code <databaseName>.<schemaName>.<tableName>}. May not be used with {@link #TABLE_BLACKLIST}, and superseded by database
|
* {@code <databaseName>.<schemaName>.<tableName>}. May not be used with {@link #TABLE_BLACKLIST}, and superseded by database
|
||||||
* inclusions/exclusions.
|
* inclusions/exclusions.
|
||||||
*/
|
*/
|
||||||
public static final Field TABLE_WHITELIST = Field.create("table.whitelist")
|
public static final Field TABLE_WHITELIST = Field.create(TABLE_WHITELIST_NAME)
|
||||||
.withDisplayName("Included tables")
|
.withDisplayName("Included tables")
|
||||||
.withType(Type.LIST)
|
.withType(Type.LIST)
|
||||||
.withWidth(Width.LONG)
|
.withWidth(Width.LONG)
|
||||||
@ -140,7 +142,7 @@ public static DecimalHandlingMode parse(String value, String defaultValue) {
|
|||||||
* monitoring. Fully-qualified names for tables are of the form {@code <databaseName>.<tableName>} or
|
* monitoring. Fully-qualified names for tables are of the form {@code <databaseName>.<tableName>} or
|
||||||
* {@code <databaseName>.<schemaName>.<tableName>}. May not be used with {@link #TABLE_WHITELIST}.
|
* {@code <databaseName>.<schemaName>.<tableName>}. May not be used with {@link #TABLE_WHITELIST}.
|
||||||
*/
|
*/
|
||||||
public static final Field TABLE_BLACKLIST = Field.create("table.blacklist")
|
public static final Field TABLE_BLACKLIST = Field.create(TABLE_BLACKLIST_NAME)
|
||||||
.withDisplayName("Excluded tables")
|
.withDisplayName("Excluded tables")
|
||||||
.withType(Type.STRING)
|
.withType(Type.STRING)
|
||||||
.withWidth(Width.LONG)
|
.withWidth(Width.LONG)
|
||||||
@ -192,6 +194,31 @@ public static DecimalHandlingMode parse(String value, String defaultValue) {
|
|||||||
"The value of those properties is the select statement to use when retrieving data from the specific table during snapshotting. " +
|
"The value of those properties is the select statement to use when retrieving data from the specific table during snapshotting. " +
|
||||||
"A possible use case for large append-only tables is setting a specific point where to start (resume) snapshotting, in case a previous snapshotting was interrupted.");
|
"A possible use case for large append-only tables is setting a specific point where to start (resume) snapshotting, in case a previous snapshotting was interrupted.");
|
||||||
|
|
||||||
|
/**
|
||||||
|
* A comma-separated list of regular expressions that match schema names to be monitored.
|
||||||
|
* May not be used with {@link #SCHEMA_BLACKLIST}.
|
||||||
|
*/
|
||||||
|
public static final Field SCHEMA_WHITELIST = Field.create("schema.whitelist")
|
||||||
|
.withDisplayName("Schemas")
|
||||||
|
.withType(Type.LIST)
|
||||||
|
.withWidth(Width.LONG)
|
||||||
|
.withImportance(Importance.HIGH)
|
||||||
|
.withDependents(TABLE_WHITELIST_NAME)
|
||||||
|
.withDescription("The schemas for which events should be captured");
|
||||||
|
|
||||||
|
/**
|
||||||
|
* A comma-separated list of regular expressions that match schema names to be excluded from monitoring.
|
||||||
|
* May not be used with {@link #SCHEMA_WHITELIST}.
|
||||||
|
*/
|
||||||
|
public static final Field SCHEMA_BLACKLIST = Field.create("schema.blacklist")
|
||||||
|
.withDisplayName("Exclude Schemas")
|
||||||
|
.withType(Type.STRING)
|
||||||
|
.withWidth(Width.LONG)
|
||||||
|
.withImportance(Importance.MEDIUM)
|
||||||
|
.withValidation(RelationalDatabaseConnectorConfig::validateSchemaBlacklist)
|
||||||
|
.withInvisibleRecommender()
|
||||||
|
.withDescription("The schemas for which events must not be captured");
|
||||||
|
|
||||||
private final RelationalTableFilters tableFilters;
|
private final RelationalTableFilters tableFilters;
|
||||||
|
|
||||||
protected RelationalDatabaseConnectorConfig(Configuration config, String logicalName, TableFilter systemTablesFilter,
|
protected RelationalDatabaseConnectorConfig(Configuration config, String logicalName, TableFilter systemTablesFilter,
|
||||||
@ -254,4 +281,14 @@ public Map<TableId, String> getSnapshotSelectOverridesByTable() {
|
|||||||
|
|
||||||
return Collections.unmodifiableMap(snapshotSelectOverridesByTable);
|
return Collections.unmodifiableMap(snapshotSelectOverridesByTable);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private static int validateSchemaBlacklist(Configuration config, Field field, Field.ValidationOutput problems) {
|
||||||
|
String whitelist = config.getString(SCHEMA_WHITELIST);
|
||||||
|
String blacklist = config.getString(SCHEMA_BLACKLIST);
|
||||||
|
if (whitelist != null && blacklist != null) {
|
||||||
|
problems.accept(SCHEMA_BLACKLIST, blacklist, "Schema whitelist is already specified");
|
||||||
|
return 1;
|
||||||
|
}
|
||||||
|
return 0;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
@ -11,9 +11,11 @@
|
|||||||
import java.sql.SQLException;
|
import java.sql.SQLException;
|
||||||
import java.sql.Statement;
|
import java.sql.Statement;
|
||||||
import java.time.Duration;
|
import java.time.Duration;
|
||||||
|
import java.time.Instant;
|
||||||
import java.util.HashSet;
|
import java.util.HashSet;
|
||||||
import java.util.Iterator;
|
import java.util.Iterator;
|
||||||
import java.util.LinkedHashSet;
|
import java.util.LinkedHashSet;
|
||||||
|
import java.util.Optional;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
import java.util.regex.Pattern;
|
import java.util.regex.Pattern;
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
@ -42,7 +44,7 @@
|
|||||||
import io.debezium.util.Threads.Timer;
|
import io.debezium.util.Threads.Timer;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Base class for {@link SnapshotChangeEventSource} for relational databases with a schema history.
|
* Base class for {@link SnapshotChangeEventSource} for relational databases with or without a schema history.
|
||||||
* <p>
|
* <p>
|
||||||
* A transaction is managed by this base class, sub-classes shouldn't rollback or commit this transaction. They are free
|
* A transaction is managed by this base class, sub-classes shouldn't rollback or commit this transaction. They are free
|
||||||
* to use nested transactions or savepoints, though.
|
* to use nested transactions or savepoints, though.
|
||||||
@ -51,9 +53,9 @@
|
|||||||
*/
|
*/
|
||||||
// TODO Mostly, this should be usable for Postgres as well; only the aspect of managing the schema history will have to
|
// TODO Mostly, this should be usable for Postgres as well; only the aspect of managing the schema history will have to
|
||||||
// be made optional based on the connector
|
// be made optional based on the connector
|
||||||
public abstract class HistorizedRelationalSnapshotChangeEventSource implements SnapshotChangeEventSource {
|
public abstract class RelationalSnapshotChangeEventSource implements SnapshotChangeEventSource {
|
||||||
|
|
||||||
private static final Logger LOGGER = LoggerFactory.getLogger(HistorizedRelationalSnapshotChangeEventSource.class);
|
private static final Logger LOGGER = LoggerFactory.getLogger(RelationalSnapshotChangeEventSource.class);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Interval for showing a log statement with the progress while scanning a single table.
|
* Interval for showing a log statement with the progress while scanning a single table.
|
||||||
@ -65,10 +67,10 @@ public abstract class HistorizedRelationalSnapshotChangeEventSource implements S
|
|||||||
private final JdbcConnection jdbcConnection;
|
private final JdbcConnection jdbcConnection;
|
||||||
private final HistorizedRelationalDatabaseSchema schema;
|
private final HistorizedRelationalDatabaseSchema schema;
|
||||||
private final EventDispatcher<TableId> dispatcher;
|
private final EventDispatcher<TableId> dispatcher;
|
||||||
private final Clock clock;
|
protected final Clock clock;
|
||||||
private final SnapshotProgressListener snapshotProgressListener;
|
private final SnapshotProgressListener snapshotProgressListener;
|
||||||
|
|
||||||
public HistorizedRelationalSnapshotChangeEventSource(RelationalDatabaseConnectorConfig connectorConfig,
|
public RelationalSnapshotChangeEventSource(RelationalDatabaseConnectorConfig connectorConfig,
|
||||||
OffsetContext previousOffset, JdbcConnection jdbcConnection, HistorizedRelationalDatabaseSchema schema,
|
OffsetContext previousOffset, JdbcConnection jdbcConnection, HistorizedRelationalDatabaseSchema schema,
|
||||||
EventDispatcher<TableId> dispatcher, Clock clock, SnapshotProgressListener snapshotProgressListener) {
|
EventDispatcher<TableId> dispatcher, Clock clock, SnapshotProgressListener snapshotProgressListener) {
|
||||||
this.connectorConfig = connectorConfig;
|
this.connectorConfig = connectorConfig;
|
||||||
@ -80,6 +82,12 @@ public HistorizedRelationalSnapshotChangeEventSource(RelationalDatabaseConnector
|
|||||||
this.snapshotProgressListener = snapshotProgressListener;
|
this.snapshotProgressListener = snapshotProgressListener;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public RelationalSnapshotChangeEventSource(RelationalDatabaseConnectorConfig connectorConfig,
|
||||||
|
OffsetContext previousOffset, JdbcConnection jdbcConnection,
|
||||||
|
EventDispatcher<TableId> dispatcher, Clock clock, SnapshotProgressListener snapshotProgressListener) {
|
||||||
|
this(connectorConfig, previousOffset, jdbcConnection, null, dispatcher, clock, snapshotProgressListener);
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public SnapshotResult execute(ChangeEventSourceContext context) throws InterruptedException {
|
public SnapshotResult execute(ChangeEventSourceContext context) throws InterruptedException {
|
||||||
SnapshottingTask snapshottingTask = getSnapshottingTask(previousOffset);
|
SnapshottingTask snapshottingTask = getSnapshottingTask(previousOffset);
|
||||||
@ -299,7 +307,9 @@ private void createSchemaChangeEventsForTables(ChangeEventSourceContext sourceCo
|
|||||||
|
|
||||||
Table table = snapshotContext.tables.forTable(tableId);
|
Table table = snapshotContext.tables.forTable(tableId);
|
||||||
|
|
||||||
schema.applySchemaChange(getCreateTableEvent(snapshotContext, table));
|
if (schema != null) {
|
||||||
|
schema.applySchemaChange(getCreateTableEvent(snapshotContext, table));
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -339,11 +349,15 @@ private void createDataEventsForTable(ChangeEventSourceContext sourceContext, Sn
|
|||||||
long exportStart = clock.currentTimeInMillis();
|
long exportStart = clock.currentTimeInMillis();
|
||||||
LOGGER.info("\t Exporting data from table '{}'", table.id());
|
LOGGER.info("\t Exporting data from table '{}'", table.id());
|
||||||
|
|
||||||
final String selectStatement = determineSnapshotSelect(snapshotContext, table.id());
|
final Optional<String> selectStatement = determineSnapshotSelect(snapshotContext, table.id());
|
||||||
LOGGER.info("\t For table '{}' using select statement: '{}'", table.id(), selectStatement);
|
if (!selectStatement.isPresent()) {
|
||||||
|
LOGGER.warn("For table '{}' the select statement was not provided, skipping table", table.id());
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
LOGGER.info("\t For table '{}' using select statement: '{}'", table.id(), selectStatement.get());
|
||||||
|
|
||||||
try (Statement statement = readTableStatement();
|
try (Statement statement = readTableStatement();
|
||||||
ResultSet rs = statement.executeQuery(selectStatement)) {
|
ResultSet rs = statement.executeQuery(selectStatement.get())) {
|
||||||
|
|
||||||
Column[] columns = getColumnsForResultSet(table, rs);
|
Column[] columns = getColumnsForResultSet(table, rs);
|
||||||
final int numColumns = table.columns().size();
|
final int numColumns = table.columns().size();
|
||||||
@ -375,10 +389,13 @@ private void createDataEventsForTable(ChangeEventSourceContext sourceContext, Sn
|
|||||||
if (snapshotContext.lastTable && snapshotContext.lastRecordInTable) {
|
if (snapshotContext.lastTable && snapshotContext.lastRecordInTable) {
|
||||||
snapshotContext.offset.markLastSnapshotRecord();
|
snapshotContext.offset.markLastSnapshotRecord();
|
||||||
}
|
}
|
||||||
dispatcher.dispatchSnapshotEvent(table.id(), getChangeRecordEmitter(snapshotContext, table.id(), row),
|
dispatcher.dispatchSnapshotEvent(table.id(), getChangeRecordEmitter(snapshotContext, table.id(), row), snapshotReceiver);
|
||||||
snapshotReceiver);
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
else if (snapshotContext.lastTable) {
|
||||||
|
// if the last table does not contain any records we still need to mark the last processed event as the last one
|
||||||
|
snapshotContext.offset.markLastSnapshotRecord();
|
||||||
|
}
|
||||||
|
|
||||||
LOGGER.info("\t Finished exporting {} records for table '{}'; total duration '{}'", rows,
|
LOGGER.info("\t Finished exporting {} records for table '{}'; total duration '{}'", rows,
|
||||||
table.id(), Strings.duration(clock.currentTimeInMillis() - exportStart));
|
table.id(), Strings.duration(clock.currentTimeInMillis() - exportStart));
|
||||||
@ -396,16 +413,19 @@ private Timer getTableScanLogTimer() {
|
|||||||
/**
|
/**
|
||||||
* Returns a {@link ChangeRecordEmitter} producing the change records for the given table row.
|
* Returns a {@link ChangeRecordEmitter} producing the change records for the given table row.
|
||||||
*/
|
*/
|
||||||
protected abstract ChangeRecordEmitter getChangeRecordEmitter(SnapshotContext snapshotContext, TableId tableId, Object[] row);
|
protected ChangeRecordEmitter getChangeRecordEmitter(SnapshotContext snapshotContext, TableId tableId, Object[] row) {
|
||||||
|
snapshotContext.offset.event(tableId, Instant.ofEpochMilli(getClock().currentTimeInMillis()));
|
||||||
|
return new SnapshotChangeRecordEmitter(snapshotContext.offset, row, getClock());
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Returns a valid query string for the specified table, either given by the user via snapshot select overrides or
|
* Returns a valid query string for the specified table, either given by the user via snapshot select overrides or
|
||||||
* defaulting to a statement provided by the DB-specific change event source.
|
* defaulting to a statement provided by the DB-specific change event source.
|
||||||
*
|
*
|
||||||
* @param tableId the table to generate a query for
|
* @param tableId the table to generate a query for
|
||||||
* @return a valid query string
|
* @return a valid query string or empty if table will not be snapshotted
|
||||||
*/
|
*/
|
||||||
private String determineSnapshotSelect(SnapshotContext snapshotContext, TableId tableId) {
|
private Optional<String> determineSnapshotSelect(SnapshotContext snapshotContext, TableId tableId) {
|
||||||
String overriddenSelect = connectorConfig.getSnapshotSelectOverridesByTable().get(tableId);
|
String overriddenSelect = connectorConfig.getSnapshotSelectOverridesByTable().get(tableId);
|
||||||
|
|
||||||
// try without catalog id, as this might or might not be populated based on the given connector
|
// try without catalog id, as this might or might not be populated based on the given connector
|
||||||
@ -413,16 +433,17 @@ private String determineSnapshotSelect(SnapshotContext snapshotContext, TableId
|
|||||||
overriddenSelect = connectorConfig.getSnapshotSelectOverridesByTable().get(new TableId(null, tableId.schema(), tableId.table()));
|
overriddenSelect = connectorConfig.getSnapshotSelectOverridesByTable().get(new TableId(null, tableId.schema(), tableId.table()));
|
||||||
}
|
}
|
||||||
|
|
||||||
return overriddenSelect != null ? overriddenSelect : getSnapshotSelect(snapshotContext, tableId);
|
return overriddenSelect != null ? Optional.of(overriddenSelect) : getSnapshotSelect(snapshotContext, tableId);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Returns the SELECT statement to be used for scanning the given table
|
* Returns the SELECT statement to be used for scanning the given table or empty value if
|
||||||
|
* the table will be streamed from but not snapshotted
|
||||||
*/
|
*/
|
||||||
// TODO Should it be Statement or similar?
|
// TODO Should it be Statement or similar?
|
||||||
// TODO Handle override option generically; a problem will be how to handle the dynamic part (Oracle's "... as of
|
// TODO Handle override option generically; a problem will be how to handle the dynamic part (Oracle's "... as of
|
||||||
// scn xyz")
|
// scn xyz")
|
||||||
protected abstract String getSnapshotSelect(SnapshotContext snapshotContext, TableId tableId);
|
protected abstract Optional<String> getSnapshotSelect(SnapshotContext snapshotContext, TableId tableId);
|
||||||
|
|
||||||
private Column[] getColumnsForResultSet(Table table, ResultSet rs) throws SQLException {
|
private Column[] getColumnsForResultSet(Table table, ResultSet rs) throws SQLException {
|
||||||
ResultSetMetaData metaData = rs.getMetaData();
|
ResultSetMetaData metaData = rs.getMetaData();
|
||||||
@ -435,7 +456,7 @@ private Column[] getColumnsForResultSet(Table table, ResultSet rs) throws SQLExc
|
|||||||
return columns;
|
return columns;
|
||||||
}
|
}
|
||||||
|
|
||||||
private Object getColumnValue(ResultSet rs, int columnIndex, Column column) throws SQLException {
|
protected Object getColumnValue(ResultSet rs, int columnIndex, Column column) throws SQLException {
|
||||||
return rs.getObject(columnIndex);
|
return rs.getObject(columnIndex);
|
||||||
}
|
}
|
||||||
|
|
@ -21,7 +21,8 @@ public RelationalTableFilters(Configuration config, TableFilter systemTablesFilt
|
|||||||
Predicate<TableId> predicate = Selectors.tableSelector()
|
Predicate<TableId> predicate = Selectors.tableSelector()
|
||||||
// .includeDatabases(config.getString(RelationalDatabaseConnectorConfig.DATABASE_WHITELIST))
|
// .includeDatabases(config.getString(RelationalDatabaseConnectorConfig.DATABASE_WHITELIST))
|
||||||
// .excludeDatabases(config.getString(RelationalDatabaseConnectorConfig.DATABASE_BLACKLIST))
|
// .excludeDatabases(config.getString(RelationalDatabaseConnectorConfig.DATABASE_BLACKLIST))
|
||||||
.includeTables(config.getString(RelationalDatabaseConnectorConfig.TABLE_WHITELIST), tableIdMapper)
|
.includeSchemas(config.getString(RelationalDatabaseConnectorConfig.SCHEMA_WHITELIST))
|
||||||
|
.excludeSchemas(config.getString(RelationalDatabaseConnectorConfig.SCHEMA_BLACKLIST)) .includeTables(config.getString(RelationalDatabaseConnectorConfig.TABLE_WHITELIST), tableIdMapper)
|
||||||
.excludeTables(config.getString(RelationalDatabaseConnectorConfig.TABLE_BLACKLIST), tableIdMapper)
|
.excludeTables(config.getString(RelationalDatabaseConnectorConfig.TABLE_BLACKLIST), tableIdMapper)
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
|
@ -3,11 +3,10 @@
|
|||||||
*
|
*
|
||||||
* Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0
|
* Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0
|
||||||
*/
|
*/
|
||||||
package io.debezium.connector.sqlserver;
|
package io.debezium.relational;
|
||||||
|
|
||||||
import io.debezium.data.Envelope.Operation;
|
import io.debezium.data.Envelope.Operation;
|
||||||
import io.debezium.pipeline.spi.OffsetContext;
|
import io.debezium.pipeline.spi.OffsetContext;
|
||||||
import io.debezium.relational.RelationalChangeRecordEmitter;
|
|
||||||
import io.debezium.util.Clock;
|
import io.debezium.util.Clock;
|
||||||
|
|
||||||
/**
|
/**
|
@ -5,21 +5,20 @@
|
|||||||
*/
|
*/
|
||||||
package io.debezium.relational;
|
package io.debezium.relational;
|
||||||
|
|
||||||
|
import java.util.Arrays;
|
||||||
import java.util.Objects;
|
import java.util.Objects;
|
||||||
import java.util.function.Function;
|
import java.util.function.Function;
|
||||||
|
|
||||||
import org.apache.kafka.connect.data.Schema;
|
import org.apache.kafka.connect.data.Schema;
|
||||||
import org.apache.kafka.connect.data.Struct;
|
import org.apache.kafka.connect.data.Struct;
|
||||||
|
import org.slf4j.Logger;
|
||||||
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
import io.debezium.annotation.Immutable;
|
import io.debezium.annotation.Immutable;
|
||||||
import io.debezium.data.Envelope;
|
import io.debezium.data.Envelope;
|
||||||
import io.debezium.data.SchemaUtil;
|
import io.debezium.data.SchemaUtil;
|
||||||
import io.debezium.schema.DataCollectionSchema;
|
import io.debezium.schema.DataCollectionSchema;
|
||||||
|
|
||||||
|
|
||||||
import org.slf4j.Logger;
|
|
||||||
import org.slf4j.LoggerFactory;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Defines the Kafka Connect {@link Schema} functionality associated with a given {@link Table table definition}, and which can
|
* Defines the Kafka Connect {@link Schema} functionality associated with a given {@link Table table definition}, and which can
|
||||||
* be used to send rows of data that match the table definition to Kafka Connect.
|
* be used to send rows of data that match the table definition to Kafka Connect.
|
||||||
@ -128,8 +127,7 @@ public Envelope getEnvelopeSchema() {
|
|||||||
*/
|
*/
|
||||||
public Object keyFromColumnData(Object[] columnData) {
|
public Object keyFromColumnData(Object[] columnData) {
|
||||||
if (logger.isTraceEnabled()) {
|
if (logger.isTraceEnabled()) {
|
||||||
logger.trace("columnData from current stack: {}", columnData);
|
logger.trace("columnData from current stack: {}", Arrays.toString(columnData));
|
||||||
logger.trace("key from column data stack: ", new Throwable());
|
|
||||||
}
|
}
|
||||||
return columnData == null ? null : keyGenerator.apply(columnData);
|
return columnData == null ? null : keyGenerator.apply(columnData);
|
||||||
}
|
}
|
||||||
|
@ -246,4 +246,8 @@ public static Instant toInstant(long epochNanos) {
|
|||||||
public static Instant toInstantFromMicros(long epochMicros) {
|
public static Instant toInstantFromMicros(long epochMicros) {
|
||||||
return toInstant(TimeUnit.MICROSECONDS.toNanos(epochMicros));
|
return toInstant(TimeUnit.MICROSECONDS.toNanos(epochMicros));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public static Instant toInstantFromMillis(long epochMillis) {
|
||||||
|
return toInstant(TimeUnit.MILLISECONDS.toNanos(epochMillis));
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
@ -8,7 +8,9 @@
|
|||||||
import static org.fest.assertions.Assertions.assertThat;
|
import static org.fest.assertions.Assertions.assertThat;
|
||||||
import static org.junit.Assert.fail;
|
import static org.junit.Assert.fail;
|
||||||
|
|
||||||
|
import java.lang.management.ManagementFactory;
|
||||||
import java.nio.file.Path;
|
import java.nio.file.Path;
|
||||||
|
import java.time.Duration;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
import java.util.Collection;
|
import java.util.Collection;
|
||||||
@ -27,6 +29,10 @@
|
|||||||
import java.util.function.Consumer;
|
import java.util.function.Consumer;
|
||||||
import java.util.function.Predicate;
|
import java.util.function.Predicate;
|
||||||
|
|
||||||
|
import javax.management.InstanceNotFoundException;
|
||||||
|
import javax.management.MBeanServer;
|
||||||
|
import javax.management.ObjectName;
|
||||||
|
|
||||||
import org.apache.kafka.common.config.Config;
|
import org.apache.kafka.common.config.Config;
|
||||||
import org.apache.kafka.common.config.ConfigValue;
|
import org.apache.kafka.common.config.ConfigValue;
|
||||||
import org.apache.kafka.connect.data.Field;
|
import org.apache.kafka.connect.data.Field;
|
||||||
@ -44,6 +50,7 @@
|
|||||||
import org.apache.kafka.connect.storage.FileOffsetBackingStore;
|
import org.apache.kafka.connect.storage.FileOffsetBackingStore;
|
||||||
import org.apache.kafka.connect.storage.OffsetStorageReaderImpl;
|
import org.apache.kafka.connect.storage.OffsetStorageReaderImpl;
|
||||||
import org.junit.After;
|
import org.junit.After;
|
||||||
|
import org.junit.Assert;
|
||||||
import org.junit.Before;
|
import org.junit.Before;
|
||||||
import org.junit.Rule;
|
import org.junit.Rule;
|
||||||
import org.junit.rules.TestRule;
|
import org.junit.rules.TestRule;
|
||||||
@ -59,7 +66,9 @@
|
|||||||
import io.debezium.junit.SkipTestRule;
|
import io.debezium.junit.SkipTestRule;
|
||||||
import io.debezium.junit.TestLogger;
|
import io.debezium.junit.TestLogger;
|
||||||
import io.debezium.relational.history.HistoryRecord;
|
import io.debezium.relational.history.HistoryRecord;
|
||||||
|
import io.debezium.util.Clock;
|
||||||
import io.debezium.util.LoggingContext;
|
import io.debezium.util.LoggingContext;
|
||||||
|
import io.debezium.util.Metronome;
|
||||||
import io.debezium.util.Testing;
|
import io.debezium.util.Testing;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -290,10 +299,14 @@ public void taskStarted() {
|
|||||||
logger.error("Stopping connector after record as requested");
|
logger.error("Stopping connector after record as requested");
|
||||||
throw new ConnectException("Stopping connector after record as requested");
|
throw new ConnectException("Stopping connector after record as requested");
|
||||||
}
|
}
|
||||||
try {
|
// Test stopped the connector, remaining records are ignored
|
||||||
consumedLines.put(record);
|
if (!engine.isRunning() || Thread.currentThread().isInterrupted()) {
|
||||||
} catch (InterruptedException e) {
|
return;
|
||||||
Thread.interrupted();
|
}
|
||||||
|
while (!consumedLines.offer(record)) {
|
||||||
|
if (!engine.isRunning() || Thread.currentThread().isInterrupted()) {
|
||||||
|
return;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
})
|
})
|
||||||
.using(this.getClass().getClassLoader())
|
.using(this.getClass().getClassLoader())
|
||||||
@ -540,7 +553,7 @@ protected void assertConnectorIsRunning() {
|
|||||||
* Assert that the connector is NOT currently running.
|
* Assert that the connector is NOT currently running.
|
||||||
*/
|
*/
|
||||||
protected void assertConnectorNotRunning() {
|
protected void assertConnectorNotRunning() {
|
||||||
assertThat(engine.isRunning()).isFalse();
|
assertThat(engine != null && engine.isRunning()).isFalse();
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -733,4 +746,54 @@ protected <T> Map<Map<String, T>, Map<String, Object>> readLastCommittedOffsets(
|
|||||||
offsetStore.stop();
|
offsetStore.stop();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public static void waitForSnapshotToBeCompleted(String connector, String server) throws InterruptedException {
|
||||||
|
int waitForSeconds = 60;
|
||||||
|
final MBeanServer mbeanServer = ManagementFactory.getPlatformMBeanServer();
|
||||||
|
final Metronome metronome = Metronome.sleeper(Duration.ofSeconds(1), Clock.system());
|
||||||
|
|
||||||
|
while (true) {
|
||||||
|
if (waitForSeconds-- <= 0) {
|
||||||
|
Assert.fail("Snapshot was not completed on time");
|
||||||
|
}
|
||||||
|
try {
|
||||||
|
final boolean completed = (boolean) mbeanServer.getAttribute(new ObjectName("debezium." + connector + ":type=connector-metrics,context=snapshot,server=" + server), "SnapshotCompleted");
|
||||||
|
if (completed) {
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
catch (InstanceNotFoundException e) {
|
||||||
|
Testing.print("Metrics has not started yet");
|
||||||
|
}
|
||||||
|
catch (Exception e) {
|
||||||
|
throw new IllegalStateException(e);
|
||||||
|
}
|
||||||
|
metronome.pause();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public static void waitForStreamingRunning(String connector, String server) throws InterruptedException {
|
||||||
|
int waitForSeconds = 60;
|
||||||
|
final MBeanServer mbeanServer = ManagementFactory.getPlatformMBeanServer();
|
||||||
|
final Metronome metronome = Metronome.sleeper(Duration.ofSeconds(1), Clock.system());
|
||||||
|
|
||||||
|
while (true) {
|
||||||
|
if (waitForSeconds-- <= 0) {
|
||||||
|
Assert.fail("Streaming was not started on time");
|
||||||
|
}
|
||||||
|
try {
|
||||||
|
final boolean completed = (boolean) mbeanServer.getAttribute(new ObjectName("debezium." + connector + ":type=connector-metrics,context=streaming,server=" + server), "Connected");
|
||||||
|
if (completed) {
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
catch (InstanceNotFoundException e) {
|
||||||
|
Testing.print("Metrics has not started yet");
|
||||||
|
}
|
||||||
|
catch (Exception e) {
|
||||||
|
throw new IllegalStateException(e);
|
||||||
|
}
|
||||||
|
metronome.pause();
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
Loading…
Reference in New Issue
Block a user