DBZ-777 PostgreSQL connector rewrite to the standard framework

This commit is contained in:
Jiri Pechanec 2019-08-14 11:21:23 +02:00 committed by Gunnar Morling
parent 89ae4b2151
commit f4246df6e4
44 changed files with 1860 additions and 1641 deletions

View File

@ -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;
} }
} }

View File

@ -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
);
}
}

View File

@ -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();
}
}

View File

@ -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());
}
}}

View File

@ -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;
} }
} }

View File

@ -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));
}
}

View File

@ -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);
}
}

View File

@ -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);
}
}
}

View File

@ -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;
} }
} }

View File

@ -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;
} }
} }

View File

@ -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();
}
}

View File

@ -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);
} }

View File

@ -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");
} }
} }

View File

@ -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();
} }

View File

@ -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;
/** /**

View File

@ -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;
/** /**

View File

@ -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;

View File

@ -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;

View File

@ -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()),

View File

@ -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;

View File

@ -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;
} }
} }

View File

@ -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) {

View File

@ -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();

View File

@ -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) {

View File

@ -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();
}
} }

View File

@ -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);
} }

View File

@ -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();
} }
} }

View File

@ -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);

View File

@ -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

View File

@ -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);
}
} }

View File

@ -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());
} }
/** /**

View File

@ -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(

View File

@ -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);
} }

View File

@ -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);
}
} }

View File

@ -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;
} }

View File

@ -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);
} }

View File

@ -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);
} }

View File

@ -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;
}
} }

View File

@ -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);
} }

View File

@ -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();

View File

@ -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;
/** /**

View File

@ -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);
} }

View File

@ -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));
}
} }

View File

@ -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();
}
}
} }