From 3767b1f5568ca016c7195e30d42c0bdca320dfe8 Mon Sep 17 00:00:00 2001 From: Hai Yan Date: Sat, 28 Dec 2024 23:47:52 +0800 Subject: [PATCH 01/10] First working version Signed-off-by: Hai Yan --- data-prepper-plugins/rds-source/build.gradle | 1 + .../plugins/source/rds/RdsService.java | 37 +- .../source/rds/configuration/EngineType.java | 3 +- .../rds/datatype/postgres/ColumnType.java | 55 +++ .../source/rds/leader/LeaderScheduler.java | 8 +- .../rds/stream/BinlogClientWrapper.java | 26 ++ .../rds/stream/BinlogEventListener.java | 22 +- .../rds/stream/LogicalReplicationClient.java | 128 +++++++ .../PostgresReplicationEventProcessor.java | 338 ++++++++++++++++++ .../rds/stream/ReplicationLogClient.java | 10 + .../stream/ReplicationLogClientFactory.java | 71 ++++ .../source/rds/stream/StreamScheduler.java | 8 +- .../source/rds/stream/StreamWorker.java | 19 +- .../rds/stream/StreamWorkerTaskRefresher.java | 30 +- shared-config/log4j2.properties | 9 + 15 files changed, 709 insertions(+), 56 deletions(-) create mode 100644 data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/datatype/postgres/ColumnType.java create mode 100644 data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/BinlogClientWrapper.java create mode 100644 data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationClient.java create mode 100644 data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/PostgresReplicationEventProcessor.java create mode 100644 data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/ReplicationLogClient.java create mode 100644 data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/ReplicationLogClientFactory.java diff --git a/data-prepper-plugins/rds-source/build.gradle b/data-prepper-plugins/rds-source/build.gradle index 77f1022f63..1b325457bf 100644 --- a/data-prepper-plugins/rds-source/build.gradle +++ b/data-prepper-plugins/rds-source/build.gradle @@ -23,6 +23,7 @@ dependencies { implementation 'com.zendesk:mysql-binlog-connector-java:0.29.2' implementation 'com.mysql:mysql-connector-j:8.4.0' + implementation 'org.postgresql:postgresql:42.7.4' compileOnly 'org.projectlombok:lombok:1.18.20' annotationProcessor 'org.projectlombok:lombok:1.18.20' diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsService.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsService.java index 5dad3cb3c6..2659f74382 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsService.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsService.java @@ -14,6 +14,7 @@ import org.opensearch.dataprepper.model.plugin.PluginConfigObservable; import org.opensearch.dataprepper.model.record.Record; import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourceCoordinator; +import org.opensearch.dataprepper.plugins.source.rds.configuration.EngineType; import org.opensearch.dataprepper.plugins.source.rds.export.DataFileScheduler; import org.opensearch.dataprepper.plugins.source.rds.export.ExportScheduler; import org.opensearch.dataprepper.plugins.source.rds.export.ExportTaskManager; @@ -28,7 +29,7 @@ import org.opensearch.dataprepper.plugins.source.rds.schema.ConnectionManager; import org.opensearch.dataprepper.plugins.source.rds.schema.QueryManager; import org.opensearch.dataprepper.plugins.source.rds.schema.SchemaManager; -import org.opensearch.dataprepper.plugins.source.rds.stream.BinlogClientFactory; +import org.opensearch.dataprepper.plugins.source.rds.stream.ReplicationLogClientFactory; import org.opensearch.dataprepper.plugins.source.rds.stream.StreamScheduler; import org.opensearch.dataprepper.plugins.source.rds.utils.IdentifierShortener; import org.slf4j.Logger; @@ -37,6 +38,7 @@ import software.amazon.awssdk.services.s3.S3Client; import java.util.ArrayList; +import java.util.Collections; import java.util.List; import java.util.Map; import java.util.concurrent.ExecutorService; @@ -102,8 +104,13 @@ public void start(Buffer> buffer) { final DbMetadata dbMetadata = rdsApiStrategy.describeDb(sourceConfig.getDbIdentifier()); final String s3PathPrefix = getS3PathPrefix(); final SchemaManager schemaManager = getSchemaManager(sourceConfig, dbMetadata); - final Map> tableColumnDataTypeMap = getColumnDataTypeMap(schemaManager); - final DbTableMetadata dbTableMetadata = new DbTableMetadata(dbMetadata, tableColumnDataTypeMap); + DbTableMetadata dbTableMetadata; + if (sourceConfig.getEngine() == EngineType.MYSQL) { + final Map> tableColumnDataTypeMap = getColumnDataTypeMap(schemaManager); + dbTableMetadata = new DbTableMetadata(dbMetadata, tableColumnDataTypeMap); + } else { + dbTableMetadata = new DbTableMetadata(dbMetadata, Collections.emptyMap()); + } leaderScheduler = new LeaderScheduler( sourceCoordinator, sourceConfig, s3PathPrefix, schemaManager, dbTableMetadata); @@ -121,21 +128,23 @@ public void start(Buffer> buffer) { } if (sourceConfig.isStreamEnabled()) { - BinlogClientFactory binaryLogClientFactory = new BinlogClientFactory(sourceConfig, rdsClient, dbMetadata); + ReplicationLogClientFactory replicationLogClientFactory = new ReplicationLogClientFactory(sourceConfig, rdsClient, dbMetadata); if (sourceConfig.isTlsEnabled()) { - binaryLogClientFactory.setSSLMode(SSLMode.REQUIRED); + replicationLogClientFactory.setSSLMode(SSLMode.REQUIRED); } else { - binaryLogClientFactory.setSSLMode(SSLMode.DISABLED); + replicationLogClientFactory.setSSLMode(SSLMode.DISABLED); } streamScheduler = new StreamScheduler( - sourceCoordinator, sourceConfig, s3PathPrefix, binaryLogClientFactory, buffer, pluginMetrics, acknowledgementSetManager, pluginConfigObservable); + sourceCoordinator, sourceConfig, s3PathPrefix, replicationLogClientFactory, buffer, pluginMetrics, acknowledgementSetManager, pluginConfigObservable); runnableList.add(streamScheduler); - resyncScheduler = new ResyncScheduler( - sourceCoordinator, sourceConfig, getQueryManager(sourceConfig, dbMetadata), s3PathPrefix, buffer, pluginMetrics, acknowledgementSetManager); - runnableList.add(resyncScheduler); + if (sourceConfig.getEngine() == EngineType.MYSQL) { + resyncScheduler = new ResyncScheduler( + sourceCoordinator, sourceConfig, getQueryManager(sourceConfig, dbMetadata), s3PathPrefix, buffer, pluginMetrics, acknowledgementSetManager); + runnableList.add(resyncScheduler); + } } executor = Executors.newFixedThreadPool(runnableList.size()); @@ -165,8 +174,8 @@ public void shutdown() { private SchemaManager getSchemaManager(final RdsSourceConfig sourceConfig, final DbMetadata dbMetadata) { final ConnectionManager connectionManager = new ConnectionManager( - dbMetadata.getEndpoint(), - dbMetadata.getPort(), + "127.0.0.1", + 5432, sourceConfig.getAuthenticationConfig().getUsername(), sourceConfig.getAuthenticationConfig().getPassword(), sourceConfig.isTlsEnabled()); @@ -177,8 +186,8 @@ private QueryManager getQueryManager(final RdsSourceConfig sourceConfig, final D final String readerEndpoint = dbMetadata.getReaderEndpoint() != null ? dbMetadata.getReaderEndpoint() : dbMetadata.getEndpoint(); final int readerPort = dbMetadata.getReaderPort() == 0 ? dbMetadata.getPort() : dbMetadata.getReaderPort(); final ConnectionManager readerConnectionManager = new ConnectionManager( - readerEndpoint, - readerPort, + "127.0.0.1", + 5432, sourceConfig.getAuthenticationConfig().getUsername(), sourceConfig.getAuthenticationConfig().getPassword(), sourceConfig.isTlsEnabled()); diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/configuration/EngineType.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/configuration/EngineType.java index f75ec32bfe..20f7f3b534 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/configuration/EngineType.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/configuration/EngineType.java @@ -13,7 +13,8 @@ public enum EngineType { - MYSQL("mysql"); + MYSQL("mysql"), + POSTGRES("postgres"); private static final Map ENGINE_TYPE_MAP = Arrays.stream(EngineType.values()) .collect(Collectors.toMap( diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/datatype/postgres/ColumnType.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/datatype/postgres/ColumnType.java new file mode 100644 index 0000000000..0f959cc033 --- /dev/null +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/datatype/postgres/ColumnType.java @@ -0,0 +1,55 @@ +package org.opensearch.dataprepper.plugins.source.rds.datatype.postgres; + +import java.util.HashMap; +import java.util.Map; + +public enum ColumnType { + BOOLEAN(16, "boolean"), + SMALLINT(21, "smallint"), + INTEGER(23, "integer"), + BIGINT(20, "bigint"), + REAL(700, "real"), + DOUBLE_PRECISION(701, "double precision"), + NUMERIC(1700, "numeric"), + TEXT(25, "text"), + VARCHAR(1043, "varchar"), + DATE(1082, "date"), + TIME(1083, "time"), + TIMESTAMP(1114, "timestamp"), + TIMESTAMPTZ(1184, "timestamptz"), + UUID(2950, "uuid"), + JSON(114, "json"), + JSONB(3802, "jsonb"); + + private final int typeId; + private final String typeName; + + private static final Map TYPE_ID_MAP = new HashMap<>(); + + static { + for (ColumnType type : values()) { + TYPE_ID_MAP.put(type.typeId, type); + } + } + + ColumnType(int typeId, String typeName) { + this.typeId = typeId; + this.typeName = typeName; + } + + public int getTypeId() { + return typeId; + } + + public String getTypeName() { + return typeName; + } + + public static ColumnType getByTypeId(int typeId) { + return TYPE_ID_MAP.get(typeId); + } + + public static String getTypeNameByEnum(ColumnType columnType) { + return columnType.getTypeName(); + } +} diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/leader/LeaderScheduler.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/leader/LeaderScheduler.java index 3f7e6d5cb2..82001f2509 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/leader/LeaderScheduler.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/leader/LeaderScheduler.java @@ -8,6 +8,7 @@ import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourceCoordinator; import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourcePartition; import org.opensearch.dataprepper.plugins.source.rds.RdsSourceConfig; +import org.opensearch.dataprepper.plugins.source.rds.configuration.EngineType; import org.opensearch.dataprepper.plugins.source.rds.coordination.partition.ExportPartition; import org.opensearch.dataprepper.plugins.source.rds.coordination.partition.GlobalState; import org.opensearch.dataprepper.plugins.source.rds.coordination.partition.LeaderPartition; @@ -159,10 +160,13 @@ private Map> getPrimaryKeyMap() { private void createStreamPartition(RdsSourceConfig sourceConfig) { final StreamProgressState progressState = new StreamProgressState(); progressState.setWaitForExport(sourceConfig.isExportEnabled()); - getCurrentBinlogPosition().ifPresent(progressState::setCurrentPosition); - progressState.setForeignKeyRelations(schemaManager.getForeignKeyRelations(sourceConfig.getTableNames())); + if (sourceConfig.getEngine() == EngineType.MYSQL) { + getCurrentBinlogPosition().ifPresent(progressState::setCurrentPosition); + progressState.setForeignKeyRelations(schemaManager.getForeignKeyRelations(sourceConfig.getTableNames())); + } StreamPartition streamPartition = new StreamPartition(sourceConfig.getDbIdentifier(), progressState); sourceCoordinator.createPartition(streamPartition); + } private Optional getCurrentBinlogPosition() { diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/BinlogClientWrapper.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/BinlogClientWrapper.java new file mode 100644 index 0000000000..20e8613ddf --- /dev/null +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/BinlogClientWrapper.java @@ -0,0 +1,26 @@ +package org.opensearch.dataprepper.plugins.source.rds.stream; + +import com.github.shyiko.mysql.binlog.BinaryLogClient; + +import java.io.IOException; + +public class BinlogClientWrapper implements ReplicationLogClient { + + private final BinaryLogClient binlogClient; + + public BinlogClientWrapper(final BinaryLogClient binlogClient) { + this.binlogClient = binlogClient; + } + + public void connect() throws IOException { + binlogClient.connect(); + } + + public void disconnect() throws IOException { + binlogClient.disconnect(); + } + + public BinaryLogClient getBinlogClient() { + return binlogClient; + } +} diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/BinlogEventListener.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/BinlogEventListener.java index 4491a7c643..09cbf0434f 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/BinlogEventListener.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/BinlogEventListener.java @@ -6,14 +6,7 @@ package org.opensearch.dataprepper.plugins.source.rds.stream; import com.github.shyiko.mysql.binlog.BinaryLogClient; -import com.github.shyiko.mysql.binlog.event.DeleteRowsEventData; -import com.github.shyiko.mysql.binlog.event.EventHeaderV4; -import com.github.shyiko.mysql.binlog.event.EventType; -import com.github.shyiko.mysql.binlog.event.RotateEventData; -import com.github.shyiko.mysql.binlog.event.TableMapEventData; -import com.github.shyiko.mysql.binlog.event.TableMapEventMetadata; -import com.github.shyiko.mysql.binlog.event.UpdateRowsEventData; -import com.github.shyiko.mysql.binlog.event.WriteRowsEventData; +import com.github.shyiko.mysql.binlog.event.*; import io.micrometer.core.instrument.Counter; import io.micrometer.core.instrument.DistributionSummary; import io.micrometer.core.instrument.Timer; @@ -30,24 +23,19 @@ import org.opensearch.dataprepper.plugins.source.rds.RdsSourceConfig; import org.opensearch.dataprepper.plugins.source.rds.converter.StreamRecordConverter; import org.opensearch.dataprepper.plugins.source.rds.coordination.partition.StreamPartition; -import org.opensearch.dataprepper.plugins.source.rds.model.BinlogCoordinate; -import org.opensearch.dataprepper.plugins.source.rds.model.DbTableMetadata; -import org.opensearch.dataprepper.plugins.source.rds.model.TableMetadata; import org.opensearch.dataprepper.plugins.source.rds.datatype.DataTypeHelper; import org.opensearch.dataprepper.plugins.source.rds.datatype.MySQLDataType; +import org.opensearch.dataprepper.plugins.source.rds.model.BinlogCoordinate; +import org.opensearch.dataprepper.plugins.source.rds.model.DbTableMetadata; import org.opensearch.dataprepper.plugins.source.rds.model.ParentTable; +import org.opensearch.dataprepper.plugins.source.rds.model.TableMetadata; import org.opensearch.dataprepper.plugins.source.rds.resync.CascadingActionDetector; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.Serializable; import java.time.Duration; -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; +import java.util.*; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.function.Consumer; diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationClient.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationClient.java new file mode 100644 index 0000000000..c52149cf81 --- /dev/null +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationClient.java @@ -0,0 +1,128 @@ +package org.opensearch.dataprepper.plugins.source.rds.stream; + +import org.postgresql.PGConnection; +import org.postgresql.PGProperty; +import org.postgresql.replication.LogSequenceNumber; +import org.postgresql.replication.PGReplicationConnection; +import org.postgresql.replication.PGReplicationStream; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.nio.ByteBuffer; +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.Statement; +import java.util.Properties; +import java.util.concurrent.TimeUnit; + +public class LogicalReplicationClient implements ReplicationLogClient { + + private static final Logger LOG = LoggerFactory.getLogger(LogicalReplicationClient.class); + private static final String URL_FORMAT = "jdbc:postgresql://%s:%d/%s"; + private final String endpoint; + private final int port; + private final String username; + private final String password; + private final String database; + + private final String jdbcUrl; + private Properties props; + + private PostgresReplicationEventProcessor eventProcessor; + + public LogicalReplicationClient(final String endpoint, + final int port, + final String username, + final String password, + final String database) { +// this.endpoint = endpoint; +// this.port = port; + this.endpoint = "127.0.0.1"; + this.port = 5432; + this.username = username; + this.password = password; + this.database = database; + jdbcUrl = String.format(URL_FORMAT, this.endpoint, this.port, this.database); + props = new Properties(); + } + + public void setEventProcessor(PostgresReplicationEventProcessor eventProcessor) { + this.eventProcessor = eventProcessor; + } + + public void connect() { + PGProperty.USER.set(props, username); + if (!password.isEmpty()) { + PGProperty.PASSWORD.set(props, password); + } + PGProperty.ASSUME_MIN_SERVER_VERSION.set(props, "9.4"); // This is required + PGProperty.REPLICATION.set(props, "database"); // This is also required + PGProperty.PREFER_QUERY_MODE.set(props, "simple"); // This is also required + + LOG.info("Connect to server with JDBC URL: {}", jdbcUrl); + + try (Connection conn = DriverManager.getConnection(jdbcUrl, props)) { + // TODO: remove hard-coded tables + final String createPublicationStatement = "CREATE PUBLICATION my_publication FOR TABLE cars, houses;"; + try (Statement stmt = conn.createStatement()) { + stmt.executeUpdate(createPublicationStatement); + LOG.info("Publication created successfully."); + } catch (Exception e) { + LOG.info("Publication might already exist: {}", e.getMessage()); + } + + PGConnection pgConnection = conn.unwrap(PGConnection.class); + + // Get the replication connection + PGReplicationConnection replicationConnection = pgConnection.getReplicationAPI(); + try { + replicationConnection + .createReplicationSlot() + .logical() + .withSlotName("my_replication_slot") + .withOutputPlugin("pgoutput") + .make(); + LOG.info("Replication slot created successfully."); + } catch (Exception e) { + LOG.info("Replication slot might already exist: {}", e.getMessage()); + } + + // Create a replication stream + PGReplicationStream stream = pgConnection.getReplicationAPI() + .replicationStream() + .logical() + .withSlotName("my_replication_slot") + .withSlotOption("proto_version", "1") + .withSlotOption("publication_names", "my_publication") + .start(); + + if (eventProcessor != null) { + while (true) { + // Read changes + ByteBuffer msg = stream.readPending(); + + if (msg == null) { + TimeUnit.MILLISECONDS.sleep(10L); + continue; + } + + // decode and convert events to Data Prepper events + eventProcessor.process(msg); + + // Acknowledge receiving the message + LogSequenceNumber lsn = stream.getLastReceiveLSN(); + stream.setFlushedLSN(lsn); + stream.setAppliedLSN(lsn); + } + } + } catch (Exception e) { + e.printStackTrace(); + } + } + + public void disconnect() { + + } + + +} diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/PostgresReplicationEventProcessor.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/PostgresReplicationEventProcessor.java new file mode 100644 index 0000000000..9f06707dbf --- /dev/null +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/PostgresReplicationEventProcessor.java @@ -0,0 +1,338 @@ +package org.opensearch.dataprepper.plugins.source.rds.stream; + +import org.opensearch.dataprepper.buffer.common.BufferAccumulator; +import org.opensearch.dataprepper.model.buffer.Buffer; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.event.JacksonEvent; +import org.opensearch.dataprepper.model.opensearch.OpenSearchBulkActions; +import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.plugins.source.rds.RdsSourceConfig; +import org.opensearch.dataprepper.plugins.source.rds.converter.StreamRecordConverter; +import org.opensearch.dataprepper.plugins.source.rds.coordination.partition.StreamPartition; +import org.opensearch.dataprepper.plugins.source.rds.datatype.postgres.ColumnType; +import org.opensearch.dataprepper.plugins.source.rds.model.TableMetadata; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.nio.ByteBuffer; +import java.time.Duration; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class PostgresReplicationEventProcessor { + + private static final Logger LOG = LoggerFactory.getLogger(PostgresReplicationEventProcessor.class); + + static final Duration BUFFER_TIMEOUT = Duration.ofSeconds(60); + static final int DEFAULT_BUFFER_BATCH_SIZE = 1_000; + + private final StreamPartition streamPartition; + private final StreamRecordConverter recordConverter; + private final Buffer> buffer; + private final BufferAccumulator> bufferAccumulator; + private final List pipelineEvents; + + private long currentLsn; + private long currentEventTimestamp; + + private Map tableMetadataMap; + + public PostgresReplicationEventProcessor(final StreamPartition streamPartition, + final RdsSourceConfig sourceConfig, + final Buffer> buffer, + final String s3Prefix) { + this.streamPartition = streamPartition; + recordConverter = new StreamRecordConverter(s3Prefix, sourceConfig.getPartitionCount()); + this.buffer = buffer; + bufferAccumulator = BufferAccumulator.create(buffer, DEFAULT_BUFFER_BATCH_SIZE, BUFFER_TIMEOUT); + + tableMetadataMap = new HashMap<>(); + pipelineEvents = new ArrayList<>(); + } + + public void process(ByteBuffer msg) { + // Message processing logic: + // If it's a BEGIN, note its LSN + // If it's a RELATION, update table metadata map + // If it's INSERT/UPDATE/DELETE, prepare events + // If it's a COMMIT, convert all prepared events and send to buffer + char messageType = (char) msg.get(); + if (messageType == 'B') { + processBeginMessage(msg); + } else if (messageType == 'R') { + processRelationMessage(msg); + } else if (messageType == 'I') { + processInsertMessage(msg); + } else if (messageType == 'U') { + processUpdateMessage(msg); + } else if (messageType == 'D') { + processDeleteMessage(msg); + } else if (messageType == 'C') { + processCommitMessage(msg); + } + } + + private void processCommitMessage(ByteBuffer msg) { + int flag = msg.get(); + long commitLsn = msg.getLong(); + long endLsn = msg.getLong(); + long epochMicro = msg.getLong(); + + if (currentLsn != commitLsn) { + // This shouldn't happen + LOG.warn("Commit LSN does not match current LSN, skipping"); + pipelineEvents.clear(); + return; + } + + writeToBuffer(bufferAccumulator); + LOG.info("Processed a COMMIT message with Flag: {} CommitLsn: {} EndLsn: {} Timestamp: {}", flag, commitLsn, endLsn, epochMicro); + } + + private void writeToBuffer(BufferAccumulator> bufferAccumulator) { + for (Event pipelineEvent : pipelineEvents) { + addToBufferAccumulator(bufferAccumulator, new Record<>(pipelineEvent)); + } + + flushBufferAccumulator(bufferAccumulator, pipelineEvents.size()); + pipelineEvents.clear(); + } + + private void addToBufferAccumulator(final BufferAccumulator> bufferAccumulator, final Record record) { + try { + bufferAccumulator.add(record); + } catch (Exception e) { + LOG.error("Failed to add event to buffer", e); + } + } + + private void flushBufferAccumulator(BufferAccumulator> bufferAccumulator, int eventCount) { + try { + bufferAccumulator.flush(); + } catch (Exception e) { + // this will only happen if writing to buffer gets interrupted from shutdown, + // otherwise bufferAccumulator will keep retrying with backoff + LOG.error("Failed to flush buffer", e); + } + } + + private void processDeleteMessage(ByteBuffer msg) { + int tableId = msg.getInt(); + char typeId = (char) msg.get(); + + final TableMetadata tableMetadata = tableMetadataMap.get((long)tableId); + final List columnNames = tableMetadata.getColumnNames(); + final List primaryKeys = tableMetadata.getPrimaryKeys(); + final long eventTimestampMillis = currentEventTimestamp; + + short numberOfColumns = msg.getShort(); + System.out.println("Column length is: " + numberOfColumns); + Map rowDataMap = new HashMap<>(); + for (int i = 0; i < numberOfColumns; i++) { + char type = (char) msg.get(); + String columnName = columnNames.get(i); + if (type == 'n') { + rowDataMap.put(columnName, null); + } else if (type == 't') { + int length = msg.getInt(); + byte[] bytes = new byte[length]; + msg.get(bytes); + rowDataMap.put(columnName, new String(bytes)); + } else if (type == 'u') { + // TODO: put original value here; this should only happen if not all column data is present in + // the UPDATE message + rowDataMap.put(columnName, "UNCHANGED"); + } else { + LOG.warn("Unknown column type: {}", type); + } + } + + final Event dataPrepperEvent = JacksonEvent.builder() + .withEventType("event") + .withData(rowDataMap) + .build(); + + final Event pipelineEvent = recordConverter.convert( + dataPrepperEvent, + tableMetadata.getDatabaseName(), + tableMetadata.getTableName(), + OpenSearchBulkActions.INDEX, + primaryKeys, + eventTimestampMillis, + eventTimestampMillis, + null); + pipelineEvents.add(pipelineEvent); + + LOG.info("Processed a DELETE message with RelationId: " + tableId + " TypeId: " + typeId + + " DeletedColumnValues: " + rowDataMap); + } + + private void processUpdateMessage(ByteBuffer msg) { + int tableId = msg.getInt(); + char typeId = (char) msg.get(); + + final TableMetadata tableMetadata = tableMetadataMap.get((long)tableId); + final List columnNames = tableMetadata.getColumnNames(); + final List primaryKeys = tableMetadata.getPrimaryKeys(); + final long eventTimestampMillis = currentEventTimestamp; + + if (typeId == 'K') { + // TODO + } else if (typeId == 'O') { + // TODO + } else if (typeId == 'N') { + Map rowDataMap = new HashMap<>(); + short numberOfColumns = msg.getShort(); + for (int i = 0; i < numberOfColumns; i++) { + char type = (char) msg.get(); + String columnName = columnNames.get(i); + if (type == 'n') { + rowDataMap.put(columnName, null); + } else if (type == 't') { + int length = msg.getInt(); + byte[] bytes = new byte[length]; + msg.get(bytes); + rowDataMap.put(columnName, new String(bytes)); + } else if (type == 'u') { + // TODO: put original value here; this should only happen if not all column data is present in + // the UPDATE message + rowDataMap.put(columnName, "UNCHANGED"); + } else { + LOG.warn("Unknown column type: {}", type); + } + } + + final Event dataPrepperEvent = JacksonEvent.builder() + .withEventType("event") + .withData(rowDataMap) + .build(); + + final Event pipelineEvent = recordConverter.convert( + dataPrepperEvent, + tableMetadata.getDatabaseName(), + tableMetadata.getTableName(), + OpenSearchBulkActions.INDEX, + primaryKeys, + eventTimestampMillis, + eventTimestampMillis, + null); + pipelineEvents.add(pipelineEvent); + + LOG.info("Processed an UPDATE message with RelationId: {} UpdatedColumnValues: {}", tableId, rowDataMap); + } + } + + private void processInsertMessage(ByteBuffer msg) { + int tableId = msg.getInt(); + char n_char = (char) msg.get(); // Skip the 'N' character + short numberOfColumns = msg.getShort(); + + final TableMetadata tableMetadata = tableMetadataMap.get((long)tableId); + final List columnNames = tableMetadata.getColumnNames(); + final List primaryKeys = tableMetadata.getPrimaryKeys(); + final long eventTimestampMillis = currentEventTimestamp; + + Map rowDataMap = new HashMap<>(); + for (int i = 0; i < numberOfColumns; i++) { + char type = (char) msg.get(); + if (type == 'n') { + rowDataMap.put(columnNames.get(i), null); + } else if (type == 't') { + int length = msg.getInt(); + byte[] bytes = new byte[length]; + msg.get(bytes); + rowDataMap.put(columnNames.get(i), new String(bytes)); + } else if (type == 'u') { + rowDataMap.put(columnNames.get(i), "UNCHANGED"); + } else { + LOG.warn("Unknown column type: {}", type); + } + } + + final Event dataPrepperEvent = JacksonEvent.builder() + .withEventType("event") + .withData(rowDataMap) + .build(); + + final Event pipelineEvent = recordConverter.convert( + dataPrepperEvent, + tableMetadata.getDatabaseName(), + tableMetadata.getTableName(), + OpenSearchBulkActions.INDEX, + primaryKeys, + eventTimestampMillis, + eventTimestampMillis, + null); + pipelineEvents.add(pipelineEvent); + + LOG.info("Processed an INSERT message with RelationOid: {} ColumnValues: {}", tableId, rowDataMap); + } + + private void processRelationMessage(ByteBuffer msg) { + int tableId = msg.getInt(); + // null terminated string + String schemaName = getNullTerminatedString(msg); + String tableName = getNullTerminatedString(msg); + int replicaId = msg.get(); + short numberOfColumns = msg.getShort(); + + List columnNames = new ArrayList<>(); + List columnsInfo = new ArrayList<>(); + for (int i = 0; i < numberOfColumns; i++) { + String columnInfo; + int flag = msg.get(); // 1 indicates this column is part of the replica identity + // null terminated string + String columnName = getNullTerminatedString(msg); + ColumnType columnType = ColumnType.getByTypeId(msg.getInt()); + String columnTypeName = columnType.getTypeName(); + int typeModifier = msg.getInt(); + if (columnType == ColumnType.VARCHAR) { + int varcharLength = typeModifier - 4; + columnInfo = columnName + " (varchar with length " + varcharLength + ")"; + } else if (columnType == ColumnType.NUMERIC) { + int precision = (typeModifier - 4) >> 16; + int scale = (typeModifier - 4) & 0xFFFF; + columnInfo = columnName + " (numeric with precision " + precision + ", scale " + scale + ")"; + } else { + columnInfo = columnName + " (" + columnTypeName + ")"; + } + columnsInfo.add(columnInfo); + columnNames.add(columnName); + } + + // TODO: get primary key beforehand + final TableMetadata tableMetadata = new TableMetadata( + tableName, schemaName, columnNames, List.of("id")); + + tableMetadataMap.put((long) tableId, tableMetadata); + + LOG.info("Processed an Relation message with RelationId: {} Namespace: {} RelationName: {} ReplicaId: {} ColumnsInfo: {}", tableId, schemaName, tableName, replicaId, columnsInfo); + } + + private void processBeginMessage(ByteBuffer msg) { + currentLsn = msg.getLong(); + long epochMicro = msg.getLong(); + currentEventTimestamp = convertPostgresEventTimestamp(epochMicro); + int transaction_xid = msg.getInt(); + + LOG.info("Processed BEGIN message with LSN: {}, Timestamp: {}, TransactionId: {}", currentLsn, currentEventTimestamp, transaction_xid); + } + + private long convertPostgresEventTimestamp(long postgresMicro) { + // Offset in microseconds between 1970-01-01 and 2000-01-01 + long offsetMicro = 946684800L * 1_000_000L; + return (postgresMicro + offsetMicro) / 1000; + } + + private String getNullTerminatedString(ByteBuffer msg) { + StringBuilder sb = new StringBuilder(); + while (msg.hasRemaining()) { + byte b = msg.get(); + if (b == 0) break; // Stop at null terminator + sb.append((char) b); + } + return sb.toString(); + } +} diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/ReplicationLogClient.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/ReplicationLogClient.java new file mode 100644 index 0000000000..29d20f5913 --- /dev/null +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/ReplicationLogClient.java @@ -0,0 +1,10 @@ +package org.opensearch.dataprepper.plugins.source.rds.stream; + +import java.io.IOException; + +public interface ReplicationLogClient { + + void connect() throws IOException; + + void disconnect() throws IOException; +} diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/ReplicationLogClientFactory.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/ReplicationLogClientFactory.java new file mode 100644 index 0000000000..a5baaa2b59 --- /dev/null +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/ReplicationLogClientFactory.java @@ -0,0 +1,71 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.rds.stream; + +import com.github.shyiko.mysql.binlog.BinaryLogClient; +import com.github.shyiko.mysql.binlog.event.deserialization.EventDeserializer; +import com.github.shyiko.mysql.binlog.network.SSLMode; +import org.opensearch.dataprepper.plugins.source.rds.RdsSourceConfig; +import org.opensearch.dataprepper.plugins.source.rds.configuration.EngineType; +import org.opensearch.dataprepper.plugins.source.rds.model.DbMetadata; +import software.amazon.awssdk.services.rds.RdsClient; + +public class ReplicationLogClientFactory { + + private final RdsClient rdsClient; + private final DbMetadata dbMetadata; + private final EngineType engineType; + private String username; + private String password; + private String database; + private SSLMode sslMode = SSLMode.REQUIRED; + + public ReplicationLogClientFactory(final RdsSourceConfig sourceConfig, + final RdsClient rdsClient, + final DbMetadata dbMetadata) { + this.rdsClient = rdsClient; + this.dbMetadata = dbMetadata; + engineType = sourceConfig.getEngine(); + username = sourceConfig.getAuthenticationConfig().getUsername(); + password = sourceConfig.getAuthenticationConfig().getPassword(); + database = "my_db"; // TODO: extact database name from source config + } + + public ReplicationLogClient create() { + if (engineType == EngineType.MYSQL) { + BinaryLogClient binaryLogClient = new BinaryLogClient( + dbMetadata.getEndpoint(), + dbMetadata.getPort(), + username, + password); + binaryLogClient.setSSLMode(sslMode); + final EventDeserializer eventDeserializer = new EventDeserializer(); + eventDeserializer.setCompatibilityMode( + EventDeserializer.CompatibilityMode.DATE_AND_TIME_AS_LONG + ); + binaryLogClient.setEventDeserializer(eventDeserializer); + return new BinlogClientWrapper(binaryLogClient); + } else if (engineType == EngineType.POSTGRES) { + return new LogicalReplicationClient( + dbMetadata.getEndpoint(), + dbMetadata.getPort(), + username, + password, + database); + } else { + throw new UnsupportedOperationException("Unsupported engine type: " + engineType); + } + } + + public void setSSLMode(SSLMode sslMode) { + this.sslMode = sslMode; + } + + public void setCredentials(String username, String password) { + this.username = username; + this.password = password; + } +} diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamScheduler.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamScheduler.java index 31c2900607..7d638931a3 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamScheduler.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamScheduler.java @@ -34,7 +34,7 @@ public class StreamScheduler implements Runnable { private final EnhancedSourceCoordinator sourceCoordinator; private final RdsSourceConfig sourceConfig; private final String s3Prefix; - private final BinlogClientFactory binlogClientFactory; + private ReplicationLogClientFactory replicationLogClientFactory; private final Buffer> buffer; private final PluginMetrics pluginMetrics; private final AcknowledgementSetManager acknowledgementSetManager; @@ -46,7 +46,7 @@ public class StreamScheduler implements Runnable { public StreamScheduler(final EnhancedSourceCoordinator sourceCoordinator, final RdsSourceConfig sourceConfig, final String s3Prefix, - final BinlogClientFactory binlogClientFactory, + final ReplicationLogClientFactory replicationLogClientFactory, final Buffer> buffer, final PluginMetrics pluginMetrics, final AcknowledgementSetManager acknowledgementSetManager, @@ -54,7 +54,7 @@ public StreamScheduler(final EnhancedSourceCoordinator sourceCoordinator, this.sourceCoordinator = sourceCoordinator; this.sourceConfig = sourceConfig; this.s3Prefix = s3Prefix; - this.binlogClientFactory = binlogClientFactory; + this.replicationLogClientFactory = replicationLogClientFactory; this.buffer = buffer; this.pluginMetrics = pluginMetrics; this.acknowledgementSetManager = acknowledgementSetManager; @@ -80,7 +80,7 @@ public void run() { final StreamCheckpointer streamCheckpointer = new StreamCheckpointer(sourceCoordinator, streamPartition, pluginMetrics); streamWorkerTaskRefresher = StreamWorkerTaskRefresher.create( - sourceCoordinator, streamPartition, streamCheckpointer, s3Prefix, binlogClientFactory, buffer, + sourceCoordinator, streamPartition, streamCheckpointer, s3Prefix, replicationLogClientFactory, buffer, () -> Executors.newSingleThreadExecutor(BackgroundThreadFactory.defaultExecutorThreadFactory("rds-source-stream-worker")), acknowledgementSetManager, pluginMetrics); diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamWorker.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamWorker.java index 0b92e19d85..36e4d369b6 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamWorker.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamWorker.java @@ -22,21 +22,21 @@ public class StreamWorker { private static final int DEFAULT_EXPORT_COMPLETE_WAIT_INTERVAL_MILLIS = 60_000; private final EnhancedSourceCoordinator sourceCoordinator; - private final BinaryLogClient binaryLogClient; + private final ReplicationLogClient replicationLogClient; private final PluginMetrics pluginMetrics; StreamWorker(final EnhancedSourceCoordinator sourceCoordinator, - final BinaryLogClient binaryLogClient, + final ReplicationLogClient replicationLogClient, final PluginMetrics pluginMetrics) { this.sourceCoordinator = sourceCoordinator; - this.binaryLogClient = binaryLogClient; + this.replicationLogClient = replicationLogClient; this.pluginMetrics = pluginMetrics; } public static StreamWorker create(final EnhancedSourceCoordinator sourceCoordinator, - final BinaryLogClient binaryLogClient, + final ReplicationLogClient replicationLogClient, final PluginMetrics pluginMetrics) { - return new StreamWorker(sourceCoordinator, binaryLogClient, pluginMetrics); + return new StreamWorker(sourceCoordinator, replicationLogClient, pluginMetrics); } public void processStream(final StreamPartition streamPartition) { @@ -51,16 +51,18 @@ public void processStream(final StreamPartition streamPartition) { } } - setStartBinlogPosition(streamPartition); + if (replicationLogClient instanceof BinaryLogClient) { + setStartBinlogPosition(streamPartition); + } try { LOG.info("Connect to database to read change events."); - binaryLogClient.connect(); + replicationLogClient.connect(); } catch (Exception e) { throw new RuntimeException(e); } finally { try { - binaryLogClient.disconnect(); + replicationLogClient.disconnect(); } catch (Exception e) { LOG.error("Binary log client failed to disconnect.", e); } @@ -90,6 +92,7 @@ private void setStartBinlogPosition(final StreamPartition streamPartition) { final String binlogFilename = startBinlogPosition.getBinlogFilename(); final long binlogPosition = startBinlogPosition.getBinlogPosition(); LOG.debug("Will start binlog stream from binlog file {} and position {}.", binlogFilename, binlogPosition); + BinaryLogClient binaryLogClient = (BinaryLogClient) replicationLogClient; binaryLogClient.setBinlogFilename(binlogFilename); binaryLogClient.setBinlogPosition(binlogPosition); } diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamWorkerTaskRefresher.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamWorkerTaskRefresher.java index dcc64354d0..a7d6bf478e 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamWorkerTaskRefresher.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamWorkerTaskRefresher.java @@ -16,6 +16,7 @@ import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourceCoordinator; import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourcePartition; import org.opensearch.dataprepper.plugins.source.rds.RdsSourceConfig; +import org.opensearch.dataprepper.plugins.source.rds.configuration.EngineType; import org.opensearch.dataprepper.plugins.source.rds.coordination.partition.GlobalState; import org.opensearch.dataprepper.plugins.source.rds.coordination.partition.StreamPartition; import org.opensearch.dataprepper.plugins.source.rds.model.DbTableMetadata; @@ -38,7 +39,7 @@ public class StreamWorkerTaskRefresher implements PluginConfigObserver> buffer; private final Supplier executorServiceSupplier; private final PluginMetrics pluginMetrics; @@ -53,7 +54,7 @@ public StreamWorkerTaskRefresher(final EnhancedSourceCoordinator sourceCoordinat final StreamPartition streamPartition, final StreamCheckpointer streamCheckpointer, final String s3Prefix, - final BinlogClientFactory binlogClientFactory, + final ReplicationLogClientFactory replicationLogClientFactory, final Buffer> buffer, final Supplier executorServiceSupplier, final AcknowledgementSetManager acknowledgementSetManager, @@ -67,7 +68,7 @@ public StreamWorkerTaskRefresher(final EnhancedSourceCoordinator sourceCoordinat executorService = executorServiceSupplier.get(); this.pluginMetrics = pluginMetrics; this.acknowledgementSetManager = acknowledgementSetManager; - this.binlogClientFactory = binlogClientFactory; + this.replicationLogClientFactory = replicationLogClientFactory; this.credentialsChangeCounter = pluginMetrics.counter(CREDENTIALS_CHANGED); this.taskRefreshErrorsCounter = pluginMetrics.counter(TASK_REFRESH_ERRORS); } @@ -76,7 +77,7 @@ public static StreamWorkerTaskRefresher create(final EnhancedSourceCoordinator s final StreamPartition streamPartition, final StreamCheckpointer streamCheckpointer, final String s3Prefix, - final BinlogClientFactory binlogClientFactory, + final ReplicationLogClientFactory binlogClientFactory, final Buffer> buffer, final Supplier executorServiceSupplier, final AcknowledgementSetManager acknowledgementSetManager, @@ -98,7 +99,7 @@ public void update(RdsSourceConfig sourceConfig) { try { executorService.shutdownNow(); executorService = executorServiceSupplier.get(); - binlogClientFactory.setCredentials( + replicationLogClientFactory.setCredentials( sourceConfig.getAuthenticationConfig().getUsername(), sourceConfig.getAuthenticationConfig().getPassword()); refreshTask(sourceConfig); @@ -117,13 +118,22 @@ public void shutdown() { } private void refreshTask(RdsSourceConfig sourceConfig) { - final BinaryLogClient binaryLogClient = binlogClientFactory.create(); final DbTableMetadata dbTableMetadata = getDBTableMetadata(streamPartition); final CascadingActionDetector cascadeActionDetector = new CascadingActionDetector(sourceCoordinator); - binaryLogClient.registerEventListener(BinlogEventListener.create( - streamPartition, buffer, sourceConfig, s3Prefix, pluginMetrics, binaryLogClient, - streamCheckpointer, acknowledgementSetManager, dbTableMetadata, cascadeActionDetector)); - final StreamWorker streamWorker = StreamWorker.create(sourceCoordinator, binaryLogClient, pluginMetrics); + + final ReplicationLogClient replicationLogClient = replicationLogClientFactory.create(); + if (sourceConfig.getEngine() == EngineType.MYSQL) { + final BinaryLogClient binaryLogClient = (BinaryLogClient) replicationLogClient; + binaryLogClient.registerEventListener(BinlogEventListener.create( + streamPartition, buffer, sourceConfig, s3Prefix, pluginMetrics, binaryLogClient, + streamCheckpointer, acknowledgementSetManager, dbTableMetadata, cascadeActionDetector)); + } else { + final LogicalReplicationClient logicalReplicationClient = (LogicalReplicationClient) replicationLogClient; + logicalReplicationClient.setEventProcessor(new PostgresReplicationEventProcessor( + streamPartition, sourceConfig, buffer, s3Prefix + )); + } + final StreamWorker streamWorker = StreamWorker.create(sourceCoordinator, replicationLogClient, pluginMetrics); executorService.submit(() -> streamWorker.processStream(streamPartition)); } diff --git a/shared-config/log4j2.properties b/shared-config/log4j2.properties index 7fbab57e95..0278fc095d 100644 --- a/shared-config/log4j2.properties +++ b/shared-config/log4j2.properties @@ -31,3 +31,12 @@ logger.plugins.level = info logger.springframework.name = org.springframework logger.springframework.level = info + +logger.rds.name = org.opensearch.dataprepper.plugins.source.rds +logger.rds.level = debug + +logger.s3source.name = org.opensearch.dataprepper.plugins.source.s3 +logger.s3source.level = warn + +logger.sourcecoord.name = org.opensearch.dataprepper.core.sourcecoordination.LeaseBasedSourceCoordinator +logger.sourcecoord.level = warn From 5f72e591ec8a28cb0585b1e250fed1ad57169007 Mon Sep 17 00:00:00 2001 From: Hai Yan Date: Mon, 6 Jan 2025 00:17:31 +0800 Subject: [PATCH 02/10] More progress and update existing unit tests Signed-off-by: Hai Yan --- .../plugins/source/rds/RdsService.java | 33 ++- .../state/StreamProgressState.java | 18 ++ .../source/rds/leader/LeaderScheduler.java | 25 +- .../source/rds/schema/MySqlSchemaManager.java | 186 +++++++++++++ .../rds/schema/PostgresConnectionManager.java | 59 +++++ .../rds/schema/PostgresSchemaManager.java | 59 +++++ .../source/rds/schema/SchemaManager.java | 187 +------------- .../rds/stream/BinlogEventListener.java | 7 +- .../rds/stream/LogicalReplicationClient.java | 62 ++--- .../PostgresReplicationEventProcessor.java | 244 ++++++------------ .../stream/ReplicationLogClientFactory.java | 67 +++-- .../source/rds/stream/StreamWorker.java | 18 +- .../rds/stream/StreamWorkerTaskRefresher.java | 4 +- .../plugins/source/rds/RdsServiceTest.java | 2 + .../rds/leader/LeaderSchedulerTest.java | 4 +- ...rTest.java => MySqlSchemaManagerTest.java} | 30 +-- .../rds/stream/StreamSchedulerTest.java | 8 +- .../stream/StreamWorkerTaskRefresherTest.java | 44 ++-- .../source/rds/stream/StreamWorkerTest.java | 12 +- 19 files changed, 608 insertions(+), 461 deletions(-) create mode 100644 data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/schema/MySqlSchemaManager.java create mode 100644 data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/schema/PostgresConnectionManager.java create mode 100644 data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/schema/PostgresSchemaManager.java rename data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/schema/{SchemaManagerTest.java => MySqlSchemaManagerTest.java} (94%) diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsService.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsService.java index 2659f74382..e81158f5d2 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsService.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsService.java @@ -28,7 +28,10 @@ import org.opensearch.dataprepper.plugins.source.rds.resync.ResyncScheduler; import org.opensearch.dataprepper.plugins.source.rds.schema.ConnectionManager; import org.opensearch.dataprepper.plugins.source.rds.schema.QueryManager; +import org.opensearch.dataprepper.plugins.source.rds.schema.MySqlSchemaManager; import org.opensearch.dataprepper.plugins.source.rds.schema.SchemaManager; +import org.opensearch.dataprepper.plugins.source.rds.schema.PostgresConnectionManager; +import org.opensearch.dataprepper.plugins.source.rds.schema.PostgresSchemaManager; import org.opensearch.dataprepper.plugins.source.rds.stream.ReplicationLogClientFactory; import org.opensearch.dataprepper.plugins.source.rds.stream.StreamScheduler; import org.opensearch.dataprepper.plugins.source.rds.utils.IdentifierShortener; @@ -103,10 +106,12 @@ public void start(Buffer> buffer) { new ClusterApiStrategy(rdsClient) : new InstanceApiStrategy(rdsClient); final DbMetadata dbMetadata = rdsApiStrategy.describeDb(sourceConfig.getDbIdentifier()); final String s3PathPrefix = getS3PathPrefix(); + final SchemaManager schemaManager = getSchemaManager(sourceConfig, dbMetadata); DbTableMetadata dbTableMetadata; if (sourceConfig.getEngine() == EngineType.MYSQL) { - final Map> tableColumnDataTypeMap = getColumnDataTypeMap(schemaManager); + final Map> tableColumnDataTypeMap = getColumnDataTypeMap( + (MySqlSchemaManager) schemaManager); dbTableMetadata = new DbTableMetadata(dbMetadata, tableColumnDataTypeMap); } else { dbTableMetadata = new DbTableMetadata(dbMetadata, Collections.emptyMap()); @@ -173,13 +178,29 @@ public void shutdown() { } private SchemaManager getSchemaManager(final RdsSourceConfig sourceConfig, final DbMetadata dbMetadata) { - final ConnectionManager connectionManager = new ConnectionManager( + // For MySQL + if (sourceConfig.getEngine() == EngineType.MYSQL) { + final ConnectionManager connectionManager = new ConnectionManager( + "127.0.0.1", + 5432, + sourceConfig.getAuthenticationConfig().getUsername(), + sourceConfig.getAuthenticationConfig().getPassword(), + sourceConfig.isTlsEnabled()); + return new MySqlSchemaManager(connectionManager); + } + // For Postgres + final PostgresConnectionManager connectionManager = new PostgresConnectionManager( "127.0.0.1", 5432, sourceConfig.getAuthenticationConfig().getUsername(), sourceConfig.getAuthenticationConfig().getPassword(), - sourceConfig.isTlsEnabled()); - return new SchemaManager(connectionManager); + sourceConfig.isTlsEnabled(), + getDatabaseName(sourceConfig.getTableNames())); + return new PostgresSchemaManager(connectionManager); + } + + private String getDatabaseName(List tableNames) { + return tableNames.get(0).split("\\.")[0]; } private QueryManager getQueryManager(final RdsSourceConfig sourceConfig, final DbMetadata dbMetadata) { @@ -212,13 +233,11 @@ private String getS3PathPrefix() { return s3PathPrefix; } - private Map> getColumnDataTypeMap(final SchemaManager schemaManager) { + private Map> getColumnDataTypeMap(final MySqlSchemaManager schemaManager) { return sourceConfig.getTableNames().stream() .collect(Collectors.toMap( fullTableName -> fullTableName, fullTableName -> schemaManager.getColumnDataTypes(fullTableName.split("\\.")[0], fullTableName.split("\\.")[1]) )); } - - } diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/state/StreamProgressState.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/state/StreamProgressState.java index 1f751e2087..d9174ef538 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/state/StreamProgressState.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/state/StreamProgressState.java @@ -16,6 +16,12 @@ public class StreamProgressState { @JsonProperty("currentPosition") private BinlogCoordinate currentPosition; + @JsonProperty("currentLsn") + private String currentLsn; + + @JsonProperty("replicationSlotName") + private String replicationSlotName; + @JsonProperty("waitForExport") private boolean waitForExport = false; @@ -26,10 +32,22 @@ public BinlogCoordinate getCurrentPosition() { return currentPosition; } + public String getCurrentLsn() { + return currentLsn; + } + + public String getReplicationSlotName() { + return replicationSlotName; + } + public void setCurrentPosition(BinlogCoordinate currentPosition) { this.currentPosition = currentPosition; } + public void setReplicationSlotName(String replicationSlotName) { + this.replicationSlotName = replicationSlotName; + } + public boolean shouldWaitForExport() { return waitForExport; } diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/leader/LeaderScheduler.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/leader/LeaderScheduler.java index 82001f2509..3f3bf19684 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/leader/LeaderScheduler.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/leader/LeaderScheduler.java @@ -18,7 +18,9 @@ import org.opensearch.dataprepper.plugins.source.rds.coordination.state.StreamProgressState; import org.opensearch.dataprepper.plugins.source.rds.model.BinlogCoordinate; import org.opensearch.dataprepper.plugins.source.rds.model.DbTableMetadata; +import org.opensearch.dataprepper.plugins.source.rds.schema.MySqlSchemaManager; import org.opensearch.dataprepper.plugins.source.rds.schema.SchemaManager; +import org.opensearch.dataprepper.plugins.source.rds.schema.PostgresSchemaManager; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -26,6 +28,7 @@ import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.UUID; import java.util.stream.Collectors; import static org.opensearch.dataprepper.plugins.source.rds.RdsService.S3_PATH_DELIMITER; @@ -153,7 +156,7 @@ private Map> getPrimaryKeyMap() { return sourceConfig.getTableNames().stream() .collect(Collectors.toMap( fullTableName -> fullTableName, - fullTableName -> schemaManager.getPrimaryKeys(fullTableName.split("\\.")[0], fullTableName.split("\\.")[1]) + fullTableName -> ((MySqlSchemaManager)schemaManager).getPrimaryKeys(fullTableName.split("\\.")[0], fullTableName.split("\\.")[1]) )); } @@ -162,16 +165,30 @@ private void createStreamPartition(RdsSourceConfig sourceConfig) { progressState.setWaitForExport(sourceConfig.isExportEnabled()); if (sourceConfig.getEngine() == EngineType.MYSQL) { getCurrentBinlogPosition().ifPresent(progressState::setCurrentPosition); - progressState.setForeignKeyRelations(schemaManager.getForeignKeyRelations(sourceConfig.getTableNames())); + progressState.setForeignKeyRelations(((MySqlSchemaManager)schemaManager).getForeignKeyRelations(sourceConfig.getTableNames())); + } else { + // Postgres + // Create replication slot, which will mark the starting point for stream + final String publicationName = generatePublicationName(); + final String slotName = generateReplicationSlotName(); + ((PostgresSchemaManager)schemaManager).createLogicalReplicationSlot(sourceConfig.getTableNames(), publicationName, slotName); + progressState.setReplicationSlotName(slotName); } StreamPartition streamPartition = new StreamPartition(sourceConfig.getDbIdentifier(), progressState); sourceCoordinator.createPartition(streamPartition); - } private Optional getCurrentBinlogPosition() { - Optional binlogCoordinate = schemaManager.getCurrentBinaryLogPosition(); + Optional binlogCoordinate = ((MySqlSchemaManager)schemaManager).getCurrentBinaryLogPosition(); LOG.debug("Current binlog position: {}", binlogCoordinate.orElse(null)); return binlogCoordinate; } + + private String generatePublicationName() { + return "data_prepper_publication_" + UUID.randomUUID().toString().substring(0, 8); + } + + private String generateReplicationSlotName() { + return "data_prepper_slot_" + UUID.randomUUID().toString().substring(0, 8); + } } diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/schema/MySqlSchemaManager.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/schema/MySqlSchemaManager.java new file mode 100644 index 0000000000..5421c7e7fb --- /dev/null +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/schema/MySqlSchemaManager.java @@ -0,0 +1,186 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.rds.schema; + +import org.opensearch.dataprepper.plugins.source.rds.model.BinlogCoordinate; +import org.opensearch.dataprepper.plugins.source.rds.model.ForeignKeyAction; +import org.opensearch.dataprepper.plugins.source.rds.model.ForeignKeyRelation; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.sql.Connection; +import java.sql.DatabaseMetaData; +import java.sql.ResultSet; +import java.sql.Statement; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; + +public class MySqlSchemaManager implements SchemaManager { + private static final Logger LOG = LoggerFactory.getLogger(MySqlSchemaManager.class); + + static final String[] TABLE_TYPES = new String[]{"TABLE"}; + static final String COLUMN_NAME = "COLUMN_NAME"; + static final String BINLOG_STATUS_QUERY = "SHOW MASTER STATUS"; + static final String BINLOG_FILE = "File"; + static final String BINLOG_POSITION = "Position"; + static final int NUM_OF_RETRIES = 3; + static final int BACKOFF_IN_MILLIS = 500; + static final String TYPE_NAME = "TYPE_NAME"; + static final String FKTABLE_NAME = "FKTABLE_NAME"; + static final String FKCOLUMN_NAME = "FKCOLUMN_NAME"; + static final String PKTABLE_NAME = "PKTABLE_NAME"; + static final String PKCOLUMN_NAME = "PKCOLUMN_NAME"; + static final String UPDATE_RULE = "UPDATE_RULE"; + static final String DELETE_RULE = "DELETE_RULE"; + static final String COLUMN_DEF = "COLUMN_DEF"; + private final ConnectionManager connectionManager; + + public MySqlSchemaManager(ConnectionManager connectionManager) { + this.connectionManager = connectionManager; + } + + public List getPrimaryKeys(final String database, final String table) { + int retry = 0; + while (retry <= NUM_OF_RETRIES) { + final List primaryKeys = new ArrayList<>(); + try (final Connection connection = connectionManager.getConnection()) { + try (final ResultSet rs = connection.getMetaData().getPrimaryKeys(database, null, table)) { + while (rs.next()) { + primaryKeys.add(rs.getString(COLUMN_NAME)); + } + return primaryKeys; + } + } catch (Exception e) { + LOG.error("Failed to get primary keys for table {}, retrying", table, e); + } + applyBackoff(); + retry++; + } + LOG.warn("Failed to get primary keys for table {}", table); + return List.of(); + } + + public Map getColumnDataTypes(final String database, final String tableName) { + final Map columnsToDataType = new HashMap<>(); + for (int retry = 0; retry <= NUM_OF_RETRIES; retry++) { + try (Connection connection = connectionManager.getConnection()) { + final DatabaseMetaData metaData = connection.getMetaData(); + + // Retrieve column metadata + try (ResultSet columns = metaData.getColumns(database, null, tableName, null)) { + while (columns.next()) { + columnsToDataType.put( + columns.getString(COLUMN_NAME), + columns.getString(TYPE_NAME) + ); + } + } + } catch (final Exception e) { + LOG.error("Failed to get dataTypes for database {} table {}, retrying", database, tableName, e); + if (retry == NUM_OF_RETRIES) { + throw new RuntimeException(String.format("Failed to get dataTypes for database %s table %s after " + + "%d retries", database, tableName, retry), e); + } + } + applyBackoff(); + } + return columnsToDataType; + } + + public Optional getCurrentBinaryLogPosition() { + int retry = 0; + while (retry <= NUM_OF_RETRIES) { + try (final Connection connection = connectionManager.getConnection()) { + final Statement statement = connection.createStatement(); + final ResultSet rs = statement.executeQuery(BINLOG_STATUS_QUERY); + if (rs.next()) { + return Optional.of(new BinlogCoordinate(rs.getString(BINLOG_FILE), rs.getLong(BINLOG_POSITION))); + } + } catch (Exception e) { + LOG.error("Failed to get current binary log position, retrying", e); + } + applyBackoff(); + retry++; + } + LOG.warn("Failed to get current binary log position"); + return Optional.empty(); + } + + /** + * Get the foreign key relations associated with the given tables. + * + * @param tableNames the table names + * @return the foreign key relations + */ + public List getForeignKeyRelations(List tableNames) { + int retry = 0; + while (retry <= NUM_OF_RETRIES) { + try (final Connection connection = connectionManager.getConnection()) { + final List foreignKeyRelations = new ArrayList<>(); + DatabaseMetaData metaData = connection.getMetaData(); + for (final String tableName : tableNames) { + String database = tableName.split("\\.")[0]; + String table = tableName.split("\\.")[1]; + ResultSet tableResult = metaData.getTables(database, null, table, TABLE_TYPES); + while (tableResult.next()) { + ResultSet foreignKeys = metaData.getImportedKeys(database, null, table); + + while (foreignKeys.next()) { + String fkTableName = foreignKeys.getString(FKTABLE_NAME); + String fkColumnName = foreignKeys.getString(FKCOLUMN_NAME); + String pkTableName = foreignKeys.getString(PKTABLE_NAME); + String pkColumnName = foreignKeys.getString(PKCOLUMN_NAME); + ForeignKeyAction updateAction = ForeignKeyAction.getActionFromMetadata(foreignKeys.getShort(UPDATE_RULE)); + ForeignKeyAction deleteAction = ForeignKeyAction.getActionFromMetadata(foreignKeys.getShort(DELETE_RULE)); + + Object defaultValue = null; + if (updateAction == ForeignKeyAction.SET_DEFAULT || deleteAction == ForeignKeyAction.SET_DEFAULT) { + // Get column default + ResultSet columnResult = metaData.getColumns(database, null, table, fkColumnName); + + if (columnResult.next()) { + defaultValue = columnResult.getObject(COLUMN_DEF); + } + } + + ForeignKeyRelation foreignKeyRelation = ForeignKeyRelation.builder() + .databaseName(database) + .parentTableName(pkTableName) + .referencedKeyName(pkColumnName) + .childTableName(fkTableName) + .foreignKeyName(fkColumnName) + .foreignKeyDefaultValue(defaultValue) + .updateAction(updateAction) + .deleteAction(deleteAction) + .build(); + + foreignKeyRelations.add(foreignKeyRelation); + } + } + } + + return foreignKeyRelations; + } catch (Exception e) { + LOG.error("Failed to scan foreign key references, retrying", e); + } + applyBackoff(); + retry++; + } + LOG.warn("Failed to scan foreign key references"); + return List.of(); + } + + private void applyBackoff() { + try { + Thread.sleep(BACKOFF_IN_MILLIS); + } catch (final InterruptedException e){ + Thread.currentThread().interrupt(); + } + } +} diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/schema/PostgresConnectionManager.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/schema/PostgresConnectionManager.java new file mode 100644 index 0000000000..2acb833b87 --- /dev/null +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/schema/PostgresConnectionManager.java @@ -0,0 +1,59 @@ +package org.opensearch.dataprepper.plugins.source.rds.schema; + +import org.postgresql.PGProperty; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.SQLException; +import java.util.Properties; + +public class PostgresConnectionManager { + private static final Logger LOG = LoggerFactory.getLogger(PostgresConnectionManager.class); + + static final String URL_FORMAT = "jdbc:postgresql://%s:%d/%s"; + + private final String endpoint; + private final int port; + private final String username; + private final String password; + private final boolean requireSSL; + private final String database; + + public PostgresConnectionManager(String endpoint, int port, String username, String password, boolean requireSSL, String database) { + this.endpoint = endpoint; + this.port = port; + this.username = username; + this.password = password; + this.requireSSL = requireSSL; + this.database = database; + } + + public Connection getConnection() throws SQLException { + final Properties props = new Properties(); + PGProperty.USER.set(props, username); + if (!password.isEmpty()) { + PGProperty.PASSWORD.set(props, password); + } + PGProperty.ASSUME_MIN_SERVER_VERSION.set(props, "9.4"); // This is required + PGProperty.REPLICATION.set(props, "database"); // This is also required + PGProperty.PREFER_QUERY_MODE.set(props, "simple"); + + if (requireSSL) { + PGProperty.SSL.set(props, "true"); + PGProperty.SSL_MODE.set(props, "require"); + } else { + PGProperty.SSL.set(props, "false"); + } + + final String jdbcUrl = String.format(URL_FORMAT, this.endpoint, this.port, this.database); + LOG.debug("Connecting to JDBC URL: {}", jdbcUrl); + return doGetConnection(jdbcUrl, props); + } + + // VisibleForTesting + Connection doGetConnection(String jdbcUrl, Properties props) throws SQLException { + return DriverManager.getConnection(jdbcUrl, props); + } +} diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/schema/PostgresSchemaManager.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/schema/PostgresSchemaManager.java new file mode 100644 index 0000000000..366aa80a46 --- /dev/null +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/schema/PostgresSchemaManager.java @@ -0,0 +1,59 @@ +package org.opensearch.dataprepper.plugins.source.rds.schema; + +import org.postgresql.PGConnection; +import org.postgresql.replication.PGReplicationConnection; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.sql.Connection; +import java.sql.PreparedStatement; +import java.util.List; + +public class PostgresSchemaManager implements SchemaManager { + private static final Logger LOG = LoggerFactory.getLogger(PostgresSchemaManager.class); + private final PostgresConnectionManager connectionManager; + + public PostgresSchemaManager(PostgresConnectionManager connectionManager) { + this.connectionManager = connectionManager; + } + + public void createLogicalReplicationSlot(final List tableNames, final String publicationName, final String slotName) { + StringBuilder createPublicationStatementBuilder = new StringBuilder("CREATE PUBLICATION ") + .append(publicationName) + .append(" FOR TABLE "); + for (int i = 0; i < tableNames.size(); i++) { + createPublicationStatementBuilder.append(tableNames.get(i)); + if (i < tableNames.size() - 1) { + createPublicationStatementBuilder.append(", "); + } + } + createPublicationStatementBuilder.append(";"); + final String createPublicationStatement = createPublicationStatementBuilder.toString(); + + try (Connection conn = connectionManager.getConnection()) { + try { + PreparedStatement statement = conn.prepareStatement(createPublicationStatement); + statement.executeUpdate(); + } catch (Exception e) { + LOG.info("Failed to create publication: {}", e.getMessage()); + } + + PGConnection pgConnection = conn.unwrap(PGConnection.class); + + // Create replication slot + PGReplicationConnection replicationConnection = pgConnection.getReplicationAPI(); + try { + replicationConnection.createReplicationSlot() + .logical() + .withSlotName(slotName) + .withOutputPlugin("pgoutput") + .make(); + LOG.info("Replication slot created successfully."); + } catch (Exception e) { + LOG.info("Failed to create replication slot: {}", e.getMessage()); + } + } catch (Exception e) { + LOG.error("Exception when creating replication slot. ", e); + } + } +} diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/schema/SchemaManager.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/schema/SchemaManager.java index bbe01ba160..9f6b53979d 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/schema/SchemaManager.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/schema/SchemaManager.java @@ -1,186 +1,7 @@ -/* - * Copyright OpenSearch Contributors - * SPDX-License-Identifier: Apache-2.0 - */ - package org.opensearch.dataprepper.plugins.source.rds.schema; -import org.opensearch.dataprepper.plugins.source.rds.model.BinlogCoordinate; -import org.opensearch.dataprepper.plugins.source.rds.model.ForeignKeyAction; -import org.opensearch.dataprepper.plugins.source.rds.model.ForeignKeyRelation; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.sql.Connection; -import java.sql.DatabaseMetaData; -import java.sql.ResultSet; -import java.sql.Statement; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Optional; - -public class SchemaManager { - private static final Logger LOG = LoggerFactory.getLogger(SchemaManager.class); - - static final String[] TABLE_TYPES = new String[]{"TABLE"}; - static final String COLUMN_NAME = "COLUMN_NAME"; - static final String BINLOG_STATUS_QUERY = "SHOW MASTER STATUS"; - static final String BINLOG_FILE = "File"; - static final String BINLOG_POSITION = "Position"; - static final int NUM_OF_RETRIES = 3; - static final int BACKOFF_IN_MILLIS = 500; - static final String TYPE_NAME = "TYPE_NAME"; - static final String FKTABLE_NAME = "FKTABLE_NAME"; - static final String FKCOLUMN_NAME = "FKCOLUMN_NAME"; - static final String PKTABLE_NAME = "PKTABLE_NAME"; - static final String PKCOLUMN_NAME = "PKCOLUMN_NAME"; - static final String UPDATE_RULE = "UPDATE_RULE"; - static final String DELETE_RULE = "DELETE_RULE"; - static final String COLUMN_DEF = "COLUMN_DEF"; - private final ConnectionManager connectionManager; - - public SchemaManager(ConnectionManager connectionManager) { - this.connectionManager = connectionManager; - } - - public List getPrimaryKeys(final String database, final String table) { - int retry = 0; - while (retry <= NUM_OF_RETRIES) { - final List primaryKeys = new ArrayList<>(); - try (final Connection connection = connectionManager.getConnection()) { - try (final ResultSet rs = connection.getMetaData().getPrimaryKeys(database, null, table)) { - while (rs.next()) { - primaryKeys.add(rs.getString(COLUMN_NAME)); - } - return primaryKeys; - } - } catch (Exception e) { - LOG.error("Failed to get primary keys for table {}, retrying", table, e); - } - applyBackoff(); - retry++; - } - LOG.warn("Failed to get primary keys for table {}", table); - return List.of(); - } - - public Map getColumnDataTypes(final String database, final String tableName) { - final Map columnsToDataType = new HashMap<>(); - for (int retry = 0; retry <= NUM_OF_RETRIES; retry++) { - try (Connection connection = connectionManager.getConnection()) { - final DatabaseMetaData metaData = connection.getMetaData(); - - // Retrieve column metadata - try (ResultSet columns = metaData.getColumns(database, null, tableName, null)) { - while (columns.next()) { - columnsToDataType.put( - columns.getString(COLUMN_NAME), - columns.getString(TYPE_NAME) - ); - } - } - } catch (final Exception e) { - LOG.error("Failed to get dataTypes for database {} table {}, retrying", database, tableName, e); - if (retry == NUM_OF_RETRIES) { - throw new RuntimeException(String.format("Failed to get dataTypes for database %s table %s after " + - "%d retries", database, tableName, retry), e); - } - } - applyBackoff(); - } - return columnsToDataType; - } - - public Optional getCurrentBinaryLogPosition() { - int retry = 0; - while (retry <= NUM_OF_RETRIES) { - try (final Connection connection = connectionManager.getConnection()) { - final Statement statement = connection.createStatement(); - final ResultSet rs = statement.executeQuery(BINLOG_STATUS_QUERY); - if (rs.next()) { - return Optional.of(new BinlogCoordinate(rs.getString(BINLOG_FILE), rs.getLong(BINLOG_POSITION))); - } - } catch (Exception e) { - LOG.error("Failed to get current binary log position, retrying", e); - } - applyBackoff(); - retry++; - } - LOG.warn("Failed to get current binary log position"); - return Optional.empty(); - } - - /** - * Get the foreign key relations associated with the given tables. - * - * @param tableNames the table names - * @return the foreign key relations - */ - public List getForeignKeyRelations(List tableNames) { - int retry = 0; - while (retry <= NUM_OF_RETRIES) { - try (final Connection connection = connectionManager.getConnection()) { - final List foreignKeyRelations = new ArrayList<>(); - DatabaseMetaData metaData = connection.getMetaData(); - for (final String tableName : tableNames) { - String database = tableName.split("\\.")[0]; - String table = tableName.split("\\.")[1]; - ResultSet tableResult = metaData.getTables(database, null, table, TABLE_TYPES); - while (tableResult.next()) { - ResultSet foreignKeys = metaData.getImportedKeys(database, null, table); - - while (foreignKeys.next()) { - String fkTableName = foreignKeys.getString(FKTABLE_NAME); - String fkColumnName = foreignKeys.getString(FKCOLUMN_NAME); - String pkTableName = foreignKeys.getString(PKTABLE_NAME); - String pkColumnName = foreignKeys.getString(PKCOLUMN_NAME); - ForeignKeyAction updateAction = ForeignKeyAction.getActionFromMetadata(foreignKeys.getShort(UPDATE_RULE)); - ForeignKeyAction deleteAction = ForeignKeyAction.getActionFromMetadata(foreignKeys.getShort(DELETE_RULE)); - - Object defaultValue = null; - if (updateAction == ForeignKeyAction.SET_DEFAULT || deleteAction == ForeignKeyAction.SET_DEFAULT) { - // Get column default - ResultSet columnResult = metaData.getColumns(database, null, table, fkColumnName); - - if (columnResult.next()) { - defaultValue = columnResult.getObject(COLUMN_DEF); - } - } - - ForeignKeyRelation foreignKeyRelation = ForeignKeyRelation.builder() - .databaseName(database) - .parentTableName(pkTableName) - .referencedKeyName(pkColumnName) - .childTableName(fkTableName) - .foreignKeyName(fkColumnName) - .foreignKeyDefaultValue(defaultValue) - .updateAction(updateAction) - .deleteAction(deleteAction) - .build(); - - foreignKeyRelations.add(foreignKeyRelation); - } - } - } - - return foreignKeyRelations; - } catch (Exception e) { - LOG.error("Failed to scan foreign key references, retrying", e); - } - applyBackoff(); - retry++; - } - LOG.warn("Failed to scan foreign key references"); - return List.of(); - } - - private void applyBackoff() { - try { - Thread.sleep(BACKOFF_IN_MILLIS); - } catch (final InterruptedException e){ - Thread.currentThread().interrupt(); - } - } +/** + * Interface for manager classes that are used to get metadata of a database, such as table schemas + */ +public interface SchemaManager { } diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/BinlogEventListener.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/BinlogEventListener.java index 09cbf0434f..37debf481c 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/BinlogEventListener.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/BinlogEventListener.java @@ -35,7 +35,12 @@ import java.io.Serializable; import java.time.Duration; -import java.util.*; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.function.Consumer; diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationClient.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationClient.java index c52149cf81..2319eb1368 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationClient.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationClient.java @@ -3,15 +3,14 @@ import org.postgresql.PGConnection; import org.postgresql.PGProperty; import org.postgresql.replication.LogSequenceNumber; -import org.postgresql.replication.PGReplicationConnection; import org.postgresql.replication.PGReplicationStream; +import org.postgresql.replication.fluent.logical.ChainedLogicalStreamBuilder; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.nio.ByteBuffer; import java.sql.Connection; import java.sql.DriverManager; -import java.sql.Statement; import java.util.Properties; import java.util.concurrent.TimeUnit; @@ -24,17 +23,20 @@ public class LogicalReplicationClient implements ReplicationLogClient { private final String username; private final String password; private final String database; - private final String jdbcUrl; + private final String replicationSlotName; private Properties props; - + private LogSequenceNumber startLsn; private PostgresReplicationEventProcessor eventProcessor; + private volatile boolean disconnectRequested = false; + public LogicalReplicationClient(final String endpoint, final int port, final String username, final String password, - final String database) { + final String database, + final String replicationSlotName) { // this.endpoint = endpoint; // this.port = port; this.endpoint = "127.0.0.1"; @@ -42,6 +44,7 @@ public LogicalReplicationClient(final String endpoint, this.username = username; this.password = password; this.database = database; + this.replicationSlotName = replicationSlotName; jdbcUrl = String.format(URL_FORMAT, this.endpoint, this.port, this.database); props = new Properties(); } @@ -50,6 +53,10 @@ public void setEventProcessor(PostgresReplicationEventProcessor eventProcessor) this.eventProcessor = eventProcessor; } + public void setStartLsn(LogSequenceNumber startLsn) { + this.startLsn = startLsn; + } + public void connect() { PGProperty.USER.set(props, username); if (!password.isEmpty()) { @@ -62,42 +69,22 @@ public void connect() { LOG.info("Connect to server with JDBC URL: {}", jdbcUrl); try (Connection conn = DriverManager.getConnection(jdbcUrl, props)) { - // TODO: remove hard-coded tables - final String createPublicationStatement = "CREATE PUBLICATION my_publication FOR TABLE cars, houses;"; - try (Statement stmt = conn.createStatement()) { - stmt.executeUpdate(createPublicationStatement); - LOG.info("Publication created successfully."); - } catch (Exception e) { - LOG.info("Publication might already exist: {}", e.getMessage()); - } - PGConnection pgConnection = conn.unwrap(PGConnection.class); - // Get the replication connection - PGReplicationConnection replicationConnection = pgConnection.getReplicationAPI(); - try { - replicationConnection - .createReplicationSlot() - .logical() - .withSlotName("my_replication_slot") - .withOutputPlugin("pgoutput") - .make(); - LOG.info("Replication slot created successfully."); - } catch (Exception e) { - LOG.info("Replication slot might already exist: {}", e.getMessage()); - } - // Create a replication stream - PGReplicationStream stream = pgConnection.getReplicationAPI() + ChainedLogicalStreamBuilder logicalStreamBuilder = pgConnection.getReplicationAPI() .replicationStream() .logical() - .withSlotName("my_replication_slot") + .withSlotName(replicationSlotName) .withSlotOption("proto_version", "1") - .withSlotOption("publication_names", "my_publication") - .start(); + .withSlotOption("publication_names", "my_publication"); + if (startLsn != null) { + logicalStreamBuilder.withStartPosition(startLsn); + } + PGReplicationStream stream = logicalStreamBuilder.start(); if (eventProcessor != null) { - while (true) { + while (!disconnectRequested) { // Read changes ByteBuffer msg = stream.readPending(); @@ -115,14 +102,15 @@ public void connect() { stream.setAppliedLSN(lsn); } } + + stream.close(); + LOG.info("Replication stream closed successfully."); } catch (Exception e) { - e.printStackTrace(); + LOG.error("Exception while reading Postgres replication stream. ", e); } } public void disconnect() { - + disconnectRequested = true; } - - } diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/PostgresReplicationEventProcessor.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/PostgresReplicationEventProcessor.java index 9f06707dbf..f1f19eb91d 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/PostgresReplicationEventProcessor.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/PostgresReplicationEventProcessor.java @@ -74,6 +74,49 @@ public void process(ByteBuffer msg) { } } + private void processBeginMessage(ByteBuffer msg) { + currentLsn = msg.getLong(); + long epochMicro = msg.getLong(); + currentEventTimestamp = convertPostgresEventTimestamp(epochMicro); + int transaction_xid = msg.getInt(); + + LOG.debug("Processed BEGIN message with LSN: {}, Timestamp: {}, TransactionId: {}", currentLsn, currentEventTimestamp, transaction_xid); + } + + private void processRelationMessage(ByteBuffer msg) { + int tableId = msg.getInt(); + // null terminated string + String schemaName = getNullTerminatedString(msg); + String tableName = getNullTerminatedString(msg); + int replicaId = msg.get(); + short numberOfColumns = msg.getShort(); + + List columnNames = new ArrayList<>(); + for (int i = 0; i < numberOfColumns; i++) { + int flag = msg.get(); // 1 indicates this column is part of the replica identity + // null terminated string + String columnName = getNullTerminatedString(msg); + ColumnType columnType = ColumnType.getByTypeId(msg.getInt()); + String columnTypeName = columnType.getTypeName(); + int typeModifier = msg.getInt(); + if (columnType == ColumnType.VARCHAR) { + int varcharLength = typeModifier - 4; + } else if (columnType == ColumnType.NUMERIC) { + int precision = (typeModifier - 4) >> 16; + int scale = (typeModifier - 4) & 0xFFFF; + } + columnNames.add(columnName); + } + + // TODO: get primary keys in advance + final TableMetadata tableMetadata = new TableMetadata( + tableName, schemaName, columnNames, List.of("id")); + + tableMetadataMap.put((long) tableId, tableMetadata); + + LOG.debug("Processed an Relation message with RelationId: {} Namespace: {} RelationName: {} ReplicaId: {}", tableId, schemaName, tableName, replicaId); + } + private void processCommitMessage(ByteBuffer msg) { int flag = msg.get(); long commitLsn = msg.getLong(); @@ -88,85 +131,20 @@ private void processCommitMessage(ByteBuffer msg) { } writeToBuffer(bufferAccumulator); - LOG.info("Processed a COMMIT message with Flag: {} CommitLsn: {} EndLsn: {} Timestamp: {}", flag, commitLsn, endLsn, epochMicro); + LOG.debug("Processed a COMMIT message with Flag: {} CommitLsn: {} EndLsn: {} Timestamp: {}", flag, commitLsn, endLsn, epochMicro); } - private void writeToBuffer(BufferAccumulator> bufferAccumulator) { - for (Event pipelineEvent : pipelineEvents) { - addToBufferAccumulator(bufferAccumulator, new Record<>(pipelineEvent)); - } - - flushBufferAccumulator(bufferAccumulator, pipelineEvents.size()); - pipelineEvents.clear(); - } - - private void addToBufferAccumulator(final BufferAccumulator> bufferAccumulator, final Record record) { - try { - bufferAccumulator.add(record); - } catch (Exception e) { - LOG.error("Failed to add event to buffer", e); - } - } - - private void flushBufferAccumulator(BufferAccumulator> bufferAccumulator, int eventCount) { - try { - bufferAccumulator.flush(); - } catch (Exception e) { - // this will only happen if writing to buffer gets interrupted from shutdown, - // otherwise bufferAccumulator will keep retrying with backoff - LOG.error("Failed to flush buffer", e); - } - } - - private void processDeleteMessage(ByteBuffer msg) { + private void processInsertMessage(ByteBuffer msg) { int tableId = msg.getInt(); - char typeId = (char) msg.get(); + char n_char = (char) msg.get(); // Skip the 'N' character final TableMetadata tableMetadata = tableMetadataMap.get((long)tableId); final List columnNames = tableMetadata.getColumnNames(); final List primaryKeys = tableMetadata.getPrimaryKeys(); final long eventTimestampMillis = currentEventTimestamp; - short numberOfColumns = msg.getShort(); - System.out.println("Column length is: " + numberOfColumns); - Map rowDataMap = new HashMap<>(); - for (int i = 0; i < numberOfColumns; i++) { - char type = (char) msg.get(); - String columnName = columnNames.get(i); - if (type == 'n') { - rowDataMap.put(columnName, null); - } else if (type == 't') { - int length = msg.getInt(); - byte[] bytes = new byte[length]; - msg.get(bytes); - rowDataMap.put(columnName, new String(bytes)); - } else if (type == 'u') { - // TODO: put original value here; this should only happen if not all column data is present in - // the UPDATE message - rowDataMap.put(columnName, "UNCHANGED"); - } else { - LOG.warn("Unknown column type: {}", type); - } - } - - final Event dataPrepperEvent = JacksonEvent.builder() - .withEventType("event") - .withData(rowDataMap) - .build(); - - final Event pipelineEvent = recordConverter.convert( - dataPrepperEvent, - tableMetadata.getDatabaseName(), - tableMetadata.getTableName(), - OpenSearchBulkActions.INDEX, - primaryKeys, - eventTimestampMillis, - eventTimestampMillis, - null); - pipelineEvents.add(pipelineEvent); - - LOG.info("Processed a DELETE message with RelationId: " + tableId + " TypeId: " + typeId + - " DeletedColumnValues: " + rowDataMap); + doProcess(msg, columnNames, tableMetadata, primaryKeys, eventTimestampMillis, OpenSearchBulkActions.INDEX); + LOG.debug("Processed an INSERT message with table id: {}", tableId); } private void processUpdateMessage(ByteBuffer msg) { @@ -178,63 +156,34 @@ private void processUpdateMessage(ByteBuffer msg) { final List primaryKeys = tableMetadata.getPrimaryKeys(); final long eventTimestampMillis = currentEventTimestamp; - if (typeId == 'K') { + if (typeId == 'N') { + + doProcess(msg, columnNames, tableMetadata, primaryKeys, eventTimestampMillis, OpenSearchBulkActions.INDEX); + LOG.debug("Processed an UPDATE message with table id: {}", tableId); + } else if (typeId == 'K') { // TODO } else if (typeId == 'O') { // TODO - } else if (typeId == 'N') { - Map rowDataMap = new HashMap<>(); - short numberOfColumns = msg.getShort(); - for (int i = 0; i < numberOfColumns; i++) { - char type = (char) msg.get(); - String columnName = columnNames.get(i); - if (type == 'n') { - rowDataMap.put(columnName, null); - } else if (type == 't') { - int length = msg.getInt(); - byte[] bytes = new byte[length]; - msg.get(bytes); - rowDataMap.put(columnName, new String(bytes)); - } else if (type == 'u') { - // TODO: put original value here; this should only happen if not all column data is present in - // the UPDATE message - rowDataMap.put(columnName, "UNCHANGED"); - } else { - LOG.warn("Unknown column type: {}", type); - } - } - - final Event dataPrepperEvent = JacksonEvent.builder() - .withEventType("event") - .withData(rowDataMap) - .build(); - - final Event pipelineEvent = recordConverter.convert( - dataPrepperEvent, - tableMetadata.getDatabaseName(), - tableMetadata.getTableName(), - OpenSearchBulkActions.INDEX, - primaryKeys, - eventTimestampMillis, - eventTimestampMillis, - null); - pipelineEvents.add(pipelineEvent); - - LOG.info("Processed an UPDATE message with RelationId: {} UpdatedColumnValues: {}", tableId, rowDataMap); } } - private void processInsertMessage(ByteBuffer msg) { + private void processDeleteMessage(ByteBuffer msg) { int tableId = msg.getInt(); - char n_char = (char) msg.get(); // Skip the 'N' character - short numberOfColumns = msg.getShort(); + char typeId = (char) msg.get(); final TableMetadata tableMetadata = tableMetadataMap.get((long)tableId); final List columnNames = tableMetadata.getColumnNames(); final List primaryKeys = tableMetadata.getPrimaryKeys(); final long eventTimestampMillis = currentEventTimestamp; + doProcess(msg, columnNames, tableMetadata, primaryKeys, eventTimestampMillis, OpenSearchBulkActions.DELETE); + LOG.debug("Processed a DELETE message with table id: {}", tableId); + } + + private void doProcess(ByteBuffer msg, List columnNames, TableMetadata tableMetadata, + List primaryKeys, long eventTimestampMillis, OpenSearchBulkActions bulkAction) { Map rowDataMap = new HashMap<>(); + short numberOfColumns = msg.getShort(); for (int i = 0; i < numberOfColumns; i++) { char type = (char) msg.get(); if (type == 'n') { @@ -244,8 +193,6 @@ private void processInsertMessage(ByteBuffer msg) { byte[] bytes = new byte[length]; msg.get(bytes); rowDataMap.put(columnNames.get(i), new String(bytes)); - } else if (type == 'u') { - rowDataMap.put(columnNames.get(i), "UNCHANGED"); } else { LOG.warn("Unknown column type: {}", type); } @@ -260,64 +207,39 @@ private void processInsertMessage(ByteBuffer msg) { dataPrepperEvent, tableMetadata.getDatabaseName(), tableMetadata.getTableName(), - OpenSearchBulkActions.INDEX, + bulkAction, primaryKeys, eventTimestampMillis, eventTimestampMillis, null); pipelineEvents.add(pipelineEvent); - - LOG.info("Processed an INSERT message with RelationOid: {} ColumnValues: {}", tableId, rowDataMap); } - private void processRelationMessage(ByteBuffer msg) { - int tableId = msg.getInt(); - // null terminated string - String schemaName = getNullTerminatedString(msg); - String tableName = getNullTerminatedString(msg); - int replicaId = msg.get(); - short numberOfColumns = msg.getShort(); - - List columnNames = new ArrayList<>(); - List columnsInfo = new ArrayList<>(); - for (int i = 0; i < numberOfColumns; i++) { - String columnInfo; - int flag = msg.get(); // 1 indicates this column is part of the replica identity - // null terminated string - String columnName = getNullTerminatedString(msg); - ColumnType columnType = ColumnType.getByTypeId(msg.getInt()); - String columnTypeName = columnType.getTypeName(); - int typeModifier = msg.getInt(); - if (columnType == ColumnType.VARCHAR) { - int varcharLength = typeModifier - 4; - columnInfo = columnName + " (varchar with length " + varcharLength + ")"; - } else if (columnType == ColumnType.NUMERIC) { - int precision = (typeModifier - 4) >> 16; - int scale = (typeModifier - 4) & 0xFFFF; - columnInfo = columnName + " (numeric with precision " + precision + ", scale " + scale + ")"; - } else { - columnInfo = columnName + " (" + columnTypeName + ")"; - } - columnsInfo.add(columnInfo); - columnNames.add(columnName); + private void writeToBuffer(BufferAccumulator> bufferAccumulator) { + for (Event pipelineEvent : pipelineEvents) { + addToBufferAccumulator(bufferAccumulator, new Record<>(pipelineEvent)); } - // TODO: get primary key beforehand - final TableMetadata tableMetadata = new TableMetadata( - tableName, schemaName, columnNames, List.of("id")); - - tableMetadataMap.put((long) tableId, tableMetadata); - - LOG.info("Processed an Relation message with RelationId: {} Namespace: {} RelationName: {} ReplicaId: {} ColumnsInfo: {}", tableId, schemaName, tableName, replicaId, columnsInfo); + flushBufferAccumulator(bufferAccumulator, pipelineEvents.size()); + pipelineEvents.clear(); } - private void processBeginMessage(ByteBuffer msg) { - currentLsn = msg.getLong(); - long epochMicro = msg.getLong(); - currentEventTimestamp = convertPostgresEventTimestamp(epochMicro); - int transaction_xid = msg.getInt(); + private void addToBufferAccumulator(final BufferAccumulator> bufferAccumulator, final Record record) { + try { + bufferAccumulator.add(record); + } catch (Exception e) { + LOG.error("Failed to add event to buffer", e); + } + } - LOG.info("Processed BEGIN message with LSN: {}, Timestamp: {}, TransactionId: {}", currentLsn, currentEventTimestamp, transaction_xid); + private void flushBufferAccumulator(BufferAccumulator> bufferAccumulator, int eventCount) { + try { + bufferAccumulator.flush(); + } catch (Exception e) { + // this will only happen if writing to buffer gets interrupted from shutdown, + // otherwise bufferAccumulator will keep retrying with backoff + LOG.error("Failed to flush buffer", e); + } } private long convertPostgresEventTimestamp(long postgresMicro) { diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/ReplicationLogClientFactory.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/ReplicationLogClientFactory.java index a5baaa2b59..551b81982d 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/ReplicationLogClientFactory.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/ReplicationLogClientFactory.java @@ -10,9 +10,13 @@ import com.github.shyiko.mysql.binlog.network.SSLMode; import org.opensearch.dataprepper.plugins.source.rds.RdsSourceConfig; import org.opensearch.dataprepper.plugins.source.rds.configuration.EngineType; +import org.opensearch.dataprepper.plugins.source.rds.coordination.partition.StreamPartition; import org.opensearch.dataprepper.plugins.source.rds.model.DbMetadata; import software.amazon.awssdk.services.rds.RdsClient; +import java.util.List; +import java.util.NoSuchElementException; + public class ReplicationLogClientFactory { private final RdsClient rdsClient; @@ -31,35 +35,48 @@ public ReplicationLogClientFactory(final RdsSourceConfig sourceConfig, engineType = sourceConfig.getEngine(); username = sourceConfig.getAuthenticationConfig().getUsername(); password = sourceConfig.getAuthenticationConfig().getPassword(); - database = "my_db"; // TODO: extact database name from source config + if (sourceConfig.getEngine() == EngineType.POSTGRES) { + database = getDatabaseName(sourceConfig.getTableNames()); + } } - public ReplicationLogClient create() { + public ReplicationLogClient create(StreamPartition streamPartition) { if (engineType == EngineType.MYSQL) { - BinaryLogClient binaryLogClient = new BinaryLogClient( - dbMetadata.getEndpoint(), - dbMetadata.getPort(), - username, - password); - binaryLogClient.setSSLMode(sslMode); - final EventDeserializer eventDeserializer = new EventDeserializer(); - eventDeserializer.setCompatibilityMode( - EventDeserializer.CompatibilityMode.DATE_AND_TIME_AS_LONG - ); - binaryLogClient.setEventDeserializer(eventDeserializer); - return new BinlogClientWrapper(binaryLogClient); - } else if (engineType == EngineType.POSTGRES) { - return new LogicalReplicationClient( - dbMetadata.getEndpoint(), - dbMetadata.getPort(), - username, - password, - database); - } else { - throw new UnsupportedOperationException("Unsupported engine type: " + engineType); + return new BinlogClientWrapper(createBinaryLogClient()); + } else { // Postgres + return createLogicalReplicationClient(streamPartition); } } + private BinaryLogClient createBinaryLogClient() { + BinaryLogClient binaryLogClient = new BinaryLogClient( + dbMetadata.getEndpoint(), + dbMetadata.getPort(), + username, + password); + binaryLogClient.setSSLMode(sslMode); + final EventDeserializer eventDeserializer = new EventDeserializer(); + eventDeserializer.setCompatibilityMode( + EventDeserializer.CompatibilityMode.DATE_AND_TIME_AS_LONG + ); + binaryLogClient.setEventDeserializer(eventDeserializer); + return binaryLogClient; + } + + private LogicalReplicationClient createLogicalReplicationClient(StreamPartition streamPartition) { + final String replicationSlotName = streamPartition.getProgressState().get().getReplicationSlotName(); + if (replicationSlotName == null) { + throw new NoSuchElementException("Replication slot name is not found in progress state."); + } + return new LogicalReplicationClient( + dbMetadata.getEndpoint(), + dbMetadata.getPort(), + username, + password, + database, + replicationSlotName); + } + public void setSSLMode(SSLMode sslMode) { this.sslMode = sslMode; } @@ -68,4 +85,8 @@ public void setCredentials(String username, String password) { this.username = username; this.password = password; } + + private String getDatabaseName(List tableNames) { + return tableNames.get(0).split("\\.")[0]; + } } diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamWorker.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamWorker.java index 36e4d369b6..089cc57235 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamWorker.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamWorker.java @@ -11,6 +11,7 @@ import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourcePartition; import org.opensearch.dataprepper.plugins.source.rds.coordination.partition.StreamPartition; import org.opensearch.dataprepper.plugins.source.rds.model.BinlogCoordinate; +import org.postgresql.replication.LogSequenceNumber; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -51,8 +52,10 @@ public void processStream(final StreamPartition streamPartition) { } } - if (replicationLogClient instanceof BinaryLogClient) { + if (replicationLogClient instanceof BinlogClientWrapper) { setStartBinlogPosition(streamPartition); + } else { + setStartLsn(streamPartition); } try { @@ -92,9 +95,20 @@ private void setStartBinlogPosition(final StreamPartition streamPartition) { final String binlogFilename = startBinlogPosition.getBinlogFilename(); final long binlogPosition = startBinlogPosition.getBinlogPosition(); LOG.debug("Will start binlog stream from binlog file {} and position {}.", binlogFilename, binlogPosition); - BinaryLogClient binaryLogClient = (BinaryLogClient) replicationLogClient; + BinaryLogClient binaryLogClient = ((BinlogClientWrapper) replicationLogClient).getBinlogClient(); binaryLogClient.setBinlogFilename(binlogFilename); binaryLogClient.setBinlogPosition(binlogPosition); } } + + private void setStartLsn(final StreamPartition streamPartition) { + final String startLsn = streamPartition.getProgressState().get().getCurrentLsn(); + LOG.debug("Will start replication stream from LSN {}.", startLsn); + + if (startLsn != null) { + LOG.debug("Will start logical replication from LSN {}", startLsn); + LogicalReplicationClient logicalReplicationClient = (LogicalReplicationClient) replicationLogClient; + logicalReplicationClient.setStartLsn(LogSequenceNumber.valueOf(startLsn)); + } + } } diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamWorkerTaskRefresher.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamWorkerTaskRefresher.java index a7d6bf478e..b9adb0cee0 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamWorkerTaskRefresher.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamWorkerTaskRefresher.java @@ -121,9 +121,9 @@ private void refreshTask(RdsSourceConfig sourceConfig) { final DbTableMetadata dbTableMetadata = getDBTableMetadata(streamPartition); final CascadingActionDetector cascadeActionDetector = new CascadingActionDetector(sourceCoordinator); - final ReplicationLogClient replicationLogClient = replicationLogClientFactory.create(); + final ReplicationLogClient replicationLogClient = replicationLogClientFactory.create(streamPartition); if (sourceConfig.getEngine() == EngineType.MYSQL) { - final BinaryLogClient binaryLogClient = (BinaryLogClient) replicationLogClient; + final BinaryLogClient binaryLogClient = ((BinlogClientWrapper) replicationLogClient).getBinlogClient(); binaryLogClient.registerEventListener(BinlogEventListener.create( streamPartition, buffer, sourceConfig, s3Prefix, pluginMetrics, binaryLogClient, streamCheckpointer, acknowledgementSetManager, dbTableMetadata, cascadeActionDetector)); diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/RdsServiceTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/RdsServiceTest.java index 102b57f508..afde56fb63 100644 --- a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/RdsServiceTest.java +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/RdsServiceTest.java @@ -22,6 +22,7 @@ import org.opensearch.dataprepper.model.plugin.PluginConfigObservable; import org.opensearch.dataprepper.model.record.Record; import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourceCoordinator; +import org.opensearch.dataprepper.plugins.source.rds.configuration.EngineType; import org.opensearch.dataprepper.plugins.source.rds.configuration.TlsConfig; import org.opensearch.dataprepper.plugins.source.rds.export.DataFileScheduler; import org.opensearch.dataprepper.plugins.source.rds.export.ExportScheduler; @@ -92,6 +93,7 @@ class RdsServiceTest { @BeforeEach void setUp() { when(clientFactory.buildRdsClient()).thenReturn(rdsClient); + when(sourceConfig.getEngine()).thenReturn(EngineType.MYSQL); } @Test diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/leader/LeaderSchedulerTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/leader/LeaderSchedulerTest.java index dbd21cbe4a..0277258ac9 100644 --- a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/leader/LeaderSchedulerTest.java +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/leader/LeaderSchedulerTest.java @@ -20,7 +20,7 @@ import org.opensearch.dataprepper.plugins.source.rds.coordination.partition.LeaderPartition; import org.opensearch.dataprepper.plugins.source.rds.coordination.state.LeaderProgressState; import org.opensearch.dataprepper.plugins.source.rds.model.DbTableMetadata; -import org.opensearch.dataprepper.plugins.source.rds.schema.SchemaManager; +import org.opensearch.dataprepper.plugins.source.rds.schema.MySqlSchemaManager; import java.time.Duration; import java.util.Optional; @@ -49,7 +49,7 @@ class LeaderSchedulerTest { private RdsSourceConfig sourceConfig; @Mock - private SchemaManager schemaManager; + private MySqlSchemaManager schemaManager; @Mock private DbTableMetadata dbTableMetadata; diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/schema/SchemaManagerTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/schema/MySqlSchemaManagerTest.java similarity index 94% rename from data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/schema/SchemaManagerTest.java rename to data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/schema/MySqlSchemaManagerTest.java index ce6af88009..53531c10c0 100644 --- a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/schema/SchemaManagerTest.java +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/schema/MySqlSchemaManagerTest.java @@ -37,20 +37,20 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; import static org.opensearch.dataprepper.plugins.source.rds.model.TableMetadata.DOT_DELIMITER; -import static org.opensearch.dataprepper.plugins.source.rds.schema.SchemaManager.BINLOG_FILE; -import static org.opensearch.dataprepper.plugins.source.rds.schema.SchemaManager.BINLOG_POSITION; -import static org.opensearch.dataprepper.plugins.source.rds.schema.SchemaManager.BINLOG_STATUS_QUERY; -import static org.opensearch.dataprepper.plugins.source.rds.schema.SchemaManager.COLUMN_NAME; -import static org.opensearch.dataprepper.plugins.source.rds.schema.SchemaManager.TYPE_NAME; -import static org.opensearch.dataprepper.plugins.source.rds.schema.SchemaManager.DELETE_RULE; -import static org.opensearch.dataprepper.plugins.source.rds.schema.SchemaManager.FKCOLUMN_NAME; -import static org.opensearch.dataprepper.plugins.source.rds.schema.SchemaManager.FKTABLE_NAME; -import static org.opensearch.dataprepper.plugins.source.rds.schema.SchemaManager.PKCOLUMN_NAME; -import static org.opensearch.dataprepper.plugins.source.rds.schema.SchemaManager.PKTABLE_NAME; -import static org.opensearch.dataprepper.plugins.source.rds.schema.SchemaManager.UPDATE_RULE; +import static org.opensearch.dataprepper.plugins.source.rds.schema.MySqlSchemaManager.BINLOG_FILE; +import static org.opensearch.dataprepper.plugins.source.rds.schema.MySqlSchemaManager.BINLOG_POSITION; +import static org.opensearch.dataprepper.plugins.source.rds.schema.MySqlSchemaManager.BINLOG_STATUS_QUERY; +import static org.opensearch.dataprepper.plugins.source.rds.schema.MySqlSchemaManager.COLUMN_NAME; +import static org.opensearch.dataprepper.plugins.source.rds.schema.MySqlSchemaManager.TYPE_NAME; +import static org.opensearch.dataprepper.plugins.source.rds.schema.MySqlSchemaManager.DELETE_RULE; +import static org.opensearch.dataprepper.plugins.source.rds.schema.MySqlSchemaManager.FKCOLUMN_NAME; +import static org.opensearch.dataprepper.plugins.source.rds.schema.MySqlSchemaManager.FKTABLE_NAME; +import static org.opensearch.dataprepper.plugins.source.rds.schema.MySqlSchemaManager.PKCOLUMN_NAME; +import static org.opensearch.dataprepper.plugins.source.rds.schema.MySqlSchemaManager.PKTABLE_NAME; +import static org.opensearch.dataprepper.plugins.source.rds.schema.MySqlSchemaManager.UPDATE_RULE; @ExtendWith(MockitoExtension.class) -class SchemaManagerTest { +class MySqlSchemaManagerTest { @Mock private ConnectionManager connectionManager; @@ -64,7 +64,7 @@ class SchemaManagerTest { @Mock private ResultSet resultSet; - private SchemaManager schemaManager; + private MySqlSchemaManager schemaManager; @BeforeEach void setUp() { @@ -217,7 +217,7 @@ void test_getForeignKeyRelations_returns_foreign_key_relations() throws SQLExcep assertThat(foreignKeyRelation.getDeleteAction(), is(ForeignKeyAction.SET_NULL)); } - private SchemaManager createObjectUnderTest() { - return new SchemaManager(connectionManager); + private MySqlSchemaManager createObjectUnderTest() { + return new MySqlSchemaManager(connectionManager); } } \ No newline at end of file diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamSchedulerTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamSchedulerTest.java index fc0ac8268a..325be414d9 100644 --- a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamSchedulerTest.java +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamSchedulerTest.java @@ -51,7 +51,7 @@ class StreamSchedulerTest { private RdsSourceConfig sourceConfig; @Mock - private BinlogClientFactory binlogClientFactory; + private ReplicationLogClientFactory replicationLogClientFactory; @Mock private PluginMetrics pluginMetrics; @@ -88,7 +88,7 @@ void test_given_no_stream_partition_then_no_stream_actions() throws InterruptedE Thread.sleep(100); executorService.shutdownNow(); - verifyNoInteractions(binlogClientFactory, pluginConfigObservable); + verifyNoInteractions(replicationLogClientFactory, pluginConfigObservable); } @Test @@ -100,7 +100,7 @@ void test_given_stream_partition_then_start_stream() throws InterruptedException executorService.submit(() -> { try (MockedStatic streamWorkerTaskRefresherMockedStatic = mockStatic(StreamWorkerTaskRefresher.class)) { streamWorkerTaskRefresherMockedStatic.when(() -> StreamWorkerTaskRefresher.create(eq(sourceCoordinator), eq(streamPartition), any(StreamCheckpointer.class), - eq(s3Prefix), eq(binlogClientFactory), eq(buffer), any(Supplier.class), eq(acknowledgementSetManager), eq(pluginMetrics))) + eq(s3Prefix), eq(replicationLogClientFactory), eq(buffer), any(Supplier.class), eq(acknowledgementSetManager), eq(pluginMetrics))) .thenReturn(streamWorkerTaskRefresher); objectUnderTest.run(); } @@ -129,6 +129,6 @@ void test_shutdown() throws InterruptedException { private StreamScheduler createObjectUnderTest() { return new StreamScheduler( - sourceCoordinator, sourceConfig, s3Prefix, binlogClientFactory, buffer, pluginMetrics, acknowledgementSetManager, pluginConfigObservable); + sourceCoordinator, sourceConfig, s3Prefix, replicationLogClientFactory, buffer, pluginMetrics, acknowledgementSetManager, pluginConfigObservable); } } \ No newline at end of file diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamWorkerTaskRefresherTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamWorkerTaskRefresherTest.java index 7647a5c008..51c5c77636 100644 --- a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamWorkerTaskRefresherTest.java +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamWorkerTaskRefresherTest.java @@ -22,6 +22,7 @@ import org.opensearch.dataprepper.model.record.Record; import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourceCoordinator; import org.opensearch.dataprepper.plugins.source.rds.RdsSourceConfig; +import org.opensearch.dataprepper.plugins.source.rds.configuration.EngineType; import org.opensearch.dataprepper.plugins.source.rds.coordination.partition.GlobalState; import org.opensearch.dataprepper.plugins.source.rds.coordination.partition.StreamPartition; import org.opensearch.dataprepper.plugins.source.rds.model.DbMetadata; @@ -64,10 +65,16 @@ class StreamWorkerTaskRefresherTest { private StreamCheckpointer streamCheckpointer; @Mock - private BinlogClientFactory binlogClientFactory; + private ReplicationLogClientFactory replicationLogClientFactory; @Mock - private BinaryLogClient binlogClient; + private ReplicationLogClient replicationLogClient; + + @Mock + private BinlogClientWrapper binaryLogClientWrapper; + + @Mock + private BinaryLogClient binaryLogClient; @Mock private Buffer> buffer; @@ -112,28 +119,30 @@ void setUp() { when(pluginMetrics.counter(CREDENTIALS_CHANGED)).thenReturn(credentialsChangeCounter); when(pluginMetrics.counter(TASK_REFRESH_ERRORS)).thenReturn(taskRefreshErrorsCounter); when(executorServiceSupplier.get()).thenReturn(executorService).thenReturn(newExecutorService); + when(sourceConfig.getEngine()).thenReturn(EngineType.MYSQL); streamWorkerTaskRefresher = createObjectUnderTest(); } @Test void test_initialize_then_process_stream() { - when(binlogClientFactory.create()).thenReturn(binlogClient); + when(replicationLogClientFactory.create(streamPartition)).thenReturn(binaryLogClientWrapper); + when(binaryLogClientWrapper.getBinlogClient()).thenReturn(binaryLogClient); final Map progressState = mockGlobalStateAndProgressState(); try (MockedStatic streamWorkerMockedStatic = mockStatic(StreamWorker.class); MockedStatic binlogEventListenerMockedStatic = mockStatic(BinlogEventListener.class); MockedStatic dbTableMetadataMockedStatic = mockStatic(DbTableMetadata.class)) { dbTableMetadataMockedStatic.when(() -> DbTableMetadata.fromMap(progressState)).thenReturn(dbTableMetadata); - streamWorkerMockedStatic.when(() -> StreamWorker.create(eq(sourceCoordinator), any(BinaryLogClient.class), eq(pluginMetrics))) + streamWorkerMockedStatic.when(() -> StreamWorker.create(eq(sourceCoordinator), any(ReplicationLogClient.class), eq(pluginMetrics))) .thenReturn(streamWorker); binlogEventListenerMockedStatic.when(() -> BinlogEventListener.create(eq(streamPartition), eq(buffer), any(RdsSourceConfig.class), - any(String.class), eq(pluginMetrics), eq(binlogClient), eq(streamCheckpointer), + any(String.class), eq(pluginMetrics), eq(binaryLogClient), eq(streamCheckpointer), eq(acknowledgementSetManager), eq(dbTableMetadata), any(CascadingActionDetector.class))) .thenReturn(binlogEventListener); streamWorkerTaskRefresher.initialize(sourceConfig); } - verify(binlogClientFactory).create(); - verify(binlogClient).registerEventListener(binlogEventListener); + verify(replicationLogClientFactory).create(streamPartition); + verify(binaryLogClient).registerEventListener(binlogEventListener); ArgumentCaptor runnableArgumentCaptor = ArgumentCaptor.forClass(Runnable.class); verify(executorService).submit(runnableArgumentCaptor.capture()); @@ -154,17 +163,19 @@ void test_update_when_credentials_changed_then_refresh_task() { final String password2 = UUID.randomUUID().toString(); when(sourceConfig2.getAuthenticationConfig().getUsername()).thenReturn(username); when(sourceConfig2.getAuthenticationConfig().getPassword()).thenReturn(password2); + when(sourceConfig2.getEngine()).thenReturn(EngineType.MYSQL); - when(binlogClientFactory.create()).thenReturn(binlogClient).thenReturn(binlogClient); + when(replicationLogClientFactory.create(streamPartition)).thenReturn(binaryLogClientWrapper).thenReturn(binaryLogClientWrapper); + when(binaryLogClientWrapper.getBinlogClient()).thenReturn(binaryLogClient); final Map progressState = mockGlobalStateAndProgressState(); try (MockedStatic streamWorkerMockedStatic = mockStatic(StreamWorker.class); MockedStatic binlogEventListenerMockedStatic = mockStatic(BinlogEventListener.class); MockedStatic dbTableMetadataMockedStatic = mockStatic(DbTableMetadata.class)) { dbTableMetadataMockedStatic.when(() -> DbTableMetadata.fromMap(progressState)).thenReturn(dbTableMetadata); - streamWorkerMockedStatic.when(() -> StreamWorker.create(eq(sourceCoordinator), any(BinaryLogClient.class), eq(pluginMetrics))) + streamWorkerMockedStatic.when(() -> StreamWorker.create(eq(sourceCoordinator), any(ReplicationLogClient.class), eq(pluginMetrics))) .thenReturn(streamWorker); binlogEventListenerMockedStatic.when(() -> BinlogEventListener.create(eq(streamPartition), eq(buffer), any(RdsSourceConfig.class), - any(String.class), eq(pluginMetrics), eq(binlogClient), eq(streamCheckpointer), + any(String.class), eq(pluginMetrics), eq(binaryLogClient), eq(streamCheckpointer), eq(acknowledgementSetManager), eq(dbTableMetadata), any(CascadingActionDetector.class))) .thenReturn(binlogEventListener); streamWorkerTaskRefresher.initialize(sourceConfig); @@ -174,8 +185,8 @@ void test_update_when_credentials_changed_then_refresh_task() { verify(credentialsChangeCounter).increment(); verify(executorService).shutdownNow(); - verify(binlogClientFactory, times(2)).create(); - verify(binlogClient, times(2)).registerEventListener(binlogEventListener); + verify(replicationLogClientFactory, times(2)).create(streamPartition); + verify(binaryLogClient, times(2)).registerEventListener(binlogEventListener); ArgumentCaptor runnableArgumentCaptor = ArgumentCaptor.forClass(Runnable.class); verify(newExecutorService).submit(runnableArgumentCaptor.capture()); @@ -192,16 +203,17 @@ void test_update_when_credentials_unchanged_then_do_nothing() { when(sourceConfig.getAuthenticationConfig().getUsername()).thenReturn(username); when(sourceConfig.getAuthenticationConfig().getPassword()).thenReturn(password); - when(binlogClientFactory.create()).thenReturn(binlogClient); + when(replicationLogClientFactory.create(streamPartition)).thenReturn(binaryLogClientWrapper); + when(binaryLogClientWrapper.getBinlogClient()).thenReturn(binaryLogClient); final Map progressState = mockGlobalStateAndProgressState(); try (MockedStatic streamWorkerMockedStatic = mockStatic(StreamWorker.class); MockedStatic binlogEventListenerMockedStatic = mockStatic(BinlogEventListener.class); MockedStatic dbTableMetadataMockedStatic = mockStatic(DbTableMetadata.class)) { dbTableMetadataMockedStatic.when(() -> DbTableMetadata.fromMap(progressState)).thenReturn(dbTableMetadata); - streamWorkerMockedStatic.when(() -> StreamWorker.create(eq(sourceCoordinator), any(BinaryLogClient.class), eq(pluginMetrics))) + streamWorkerMockedStatic.when(() -> StreamWorker.create(eq(sourceCoordinator), any(ReplicationLogClient.class), eq(pluginMetrics))) .thenReturn(streamWorker); binlogEventListenerMockedStatic.when(() -> BinlogEventListener.create(eq(streamPartition), eq(buffer), any(RdsSourceConfig.class), - any(String.class), eq(pluginMetrics), eq(binlogClient), eq(streamCheckpointer), + any(String.class), eq(pluginMetrics), eq(binaryLogClient), eq(streamCheckpointer), eq(acknowledgementSetManager), eq(dbTableMetadata), any(CascadingActionDetector.class))) .thenReturn(binlogEventListener); streamWorkerTaskRefresher.initialize(sourceConfig); @@ -222,7 +234,7 @@ private StreamWorkerTaskRefresher createObjectUnderTest() { final String s3Prefix = UUID.randomUUID().toString(); return new StreamWorkerTaskRefresher( - sourceCoordinator, streamPartition, streamCheckpointer, s3Prefix, binlogClientFactory, buffer, executorServiceSupplier, acknowledgementSetManager, pluginMetrics); + sourceCoordinator, streamPartition, streamCheckpointer, s3Prefix, replicationLogClientFactory, buffer, executorServiceSupplier, acknowledgementSetManager, pluginMetrics); } private Map mockGlobalStateAndProgressState() { diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamWorkerTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamWorkerTest.java index ecc7d86d47..1eaf719cf5 100644 --- a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamWorkerTest.java +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamWorkerTest.java @@ -32,6 +32,9 @@ class StreamWorkerTest { @Mock private EnhancedSourceCoordinator sourceCoordinator; + @Mock + private BinlogClientWrapper binlogClientWrapper; + @Mock private BinaryLogClient binaryLogClient; @@ -56,12 +59,13 @@ void test_processStream_with_given_binlog_coordinates() throws IOException { when(streamPartition.getProgressState()).thenReturn(Optional.of(streamProgressState)); when(streamProgressState.getCurrentPosition()).thenReturn(new BinlogCoordinate(binlogFilename, binlogPosition)); when(streamProgressState.shouldWaitForExport()).thenReturn(false); + when(binlogClientWrapper.getBinlogClient()).thenReturn(binaryLogClient); streamWorker.processStream(streamPartition); verify(binaryLogClient).setBinlogFilename(binlogFilename); verify(binaryLogClient).setBinlogPosition(binlogPosition); - verify(binaryLogClient).connect(); + verify(binlogClientWrapper).connect(); } @Test @@ -69,7 +73,7 @@ void test_processStream_without_current_binlog_coordinates() throws IOException StreamProgressState streamProgressState = mock(StreamProgressState.class); when(streamPartition.getProgressState()).thenReturn(Optional.of(streamProgressState)); final String binlogFilename = "binlog-001"; - final Long binlogPosition = 100L; + final long binlogPosition = 100L; when(streamProgressState.getCurrentPosition()).thenReturn(null); when(streamProgressState.shouldWaitForExport()).thenReturn(false); @@ -77,10 +81,10 @@ void test_processStream_without_current_binlog_coordinates() throws IOException verify(binaryLogClient, never()).setBinlogFilename(binlogFilename); verify(binaryLogClient, never()).setBinlogPosition(binlogPosition); - verify(binaryLogClient).connect(); + verify(binlogClientWrapper).connect(); } private StreamWorker createObjectUnderTest() { - return new StreamWorker(sourceCoordinator, binaryLogClient, pluginMetrics); + return new StreamWorker(sourceCoordinator, binlogClientWrapper, pluginMetrics); } } \ No newline at end of file From 938148012021720b33710cb834960103022e0d41 Mon Sep 17 00:00:00 2001 From: Hai Yan Date: Mon, 6 Jan 2025 22:10:19 +0800 Subject: [PATCH 03/10] Add unit tests Signed-off-by: Hai Yan --- .../rds/schema/PostgresConnectionManager.java | 22 ++-- .../rds/schema/PostgresSchemaManager.java | 4 +- .../rds/stream/BinlogEventListener.java | 9 +- .../rds/stream/LogicalReplicationClient.java | 98 ++++++--------- ... => LogicalReplicationEventProcessor.java} | 26 ++-- .../stream/ReplicationLogClientFactory.java | 18 ++- .../source/rds/stream/StreamWorker.java | 1 - .../rds/stream/StreamWorkerTaskRefresher.java | 2 +- .../schema/PostgresConnectionManagerTest.java | 88 ++++++++++++++ .../rds/schema/PostgresSchemaManagerTest.java | 81 ++++++++++++ .../stream/LogicalReplicationClientTest.java | 81 ++++++++++++ .../LogicalReplicationEventProcessorTest.java | 115 ++++++++++++++++++ .../ReplicationLogClientFactoryTest.java | 86 +++++++++++++ 13 files changed, 534 insertions(+), 97 deletions(-) rename data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/{PostgresReplicationEventProcessor.java => LogicalReplicationEventProcessor.java} (91%) create mode 100644 data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/schema/PostgresConnectionManagerTest.java create mode 100644 data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/schema/PostgresSchemaManagerTest.java create mode 100644 data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationClientTest.java create mode 100644 data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationEventProcessorTest.java create mode 100644 data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/ReplicationLogClientFactoryTest.java diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/schema/PostgresConnectionManager.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/schema/PostgresConnectionManager.java index 2acb833b87..ce5c6ed8ba 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/schema/PostgresConnectionManager.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/schema/PostgresConnectionManager.java @@ -12,7 +12,13 @@ public class PostgresConnectionManager { private static final Logger LOG = LoggerFactory.getLogger(PostgresConnectionManager.class); - static final String URL_FORMAT = "jdbc:postgresql://%s:%d/%s"; + public static final String JDBC_URL_FORMAT = "jdbc:postgresql://%s:%d/%s"; + public static final String SERVER_VERSION_9_4 = "9.4"; + public static final String DATABASE_REPLICATION = "database"; + public static final String SIMPLE_QUERY = "simple"; + public static final String TRUE_VALUE = "true"; + public static final String FALSE_VALUE = "false"; + public static final String REQUIRE_SSL = "require"; private final String endpoint; private final int port; @@ -36,18 +42,18 @@ public Connection getConnection() throws SQLException { if (!password.isEmpty()) { PGProperty.PASSWORD.set(props, password); } - PGProperty.ASSUME_MIN_SERVER_VERSION.set(props, "9.4"); // This is required - PGProperty.REPLICATION.set(props, "database"); // This is also required - PGProperty.PREFER_QUERY_MODE.set(props, "simple"); + PGProperty.ASSUME_MIN_SERVER_VERSION.set(props, SERVER_VERSION_9_4); // This is required + PGProperty.REPLICATION.set(props, DATABASE_REPLICATION); // This is also required + PGProperty.PREFER_QUERY_MODE.set(props, SIMPLE_QUERY); if (requireSSL) { - PGProperty.SSL.set(props, "true"); - PGProperty.SSL_MODE.set(props, "require"); + PGProperty.SSL.set(props, TRUE_VALUE); + PGProperty.SSL_MODE.set(props, REQUIRE_SSL); } else { - PGProperty.SSL.set(props, "false"); + PGProperty.SSL.set(props, FALSE_VALUE); } - final String jdbcUrl = String.format(URL_FORMAT, this.endpoint, this.port, this.database); + final String jdbcUrl = String.format(JDBC_URL_FORMAT, this.endpoint, this.port, this.database); LOG.debug("Connecting to JDBC URL: {}", jdbcUrl); return doGetConnection(jdbcUrl, props); } diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/schema/PostgresSchemaManager.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/schema/PostgresSchemaManager.java index 366aa80a46..1bba798f65 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/schema/PostgresSchemaManager.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/schema/PostgresSchemaManager.java @@ -48,9 +48,9 @@ public void createLogicalReplicationSlot(final List tableNames, final St .withSlotName(slotName) .withOutputPlugin("pgoutput") .make(); - LOG.info("Replication slot created successfully."); + LOG.info("Replication slot {} created successfully. ", slotName); } catch (Exception e) { - LOG.info("Failed to create replication slot: {}", e.getMessage()); + LOG.info("Failed to create replication slot {}: {}", slotName, e.getMessage()); } } catch (Exception e) { LOG.error("Exception when creating replication slot. ", e); diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/BinlogEventListener.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/BinlogEventListener.java index 37debf481c..2bc21ca786 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/BinlogEventListener.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/BinlogEventListener.java @@ -6,7 +6,14 @@ package org.opensearch.dataprepper.plugins.source.rds.stream; import com.github.shyiko.mysql.binlog.BinaryLogClient; -import com.github.shyiko.mysql.binlog.event.*; +import com.github.shyiko.mysql.binlog.event.DeleteRowsEventData; +import com.github.shyiko.mysql.binlog.event.EventHeaderV4; +import com.github.shyiko.mysql.binlog.event.EventType; +import com.github.shyiko.mysql.binlog.event.RotateEventData; +import com.github.shyiko.mysql.binlog.event.TableMapEventData; +import com.github.shyiko.mysql.binlog.event.TableMapEventMetadata; +import com.github.shyiko.mysql.binlog.event.UpdateRowsEventData; +import com.github.shyiko.mysql.binlog.event.WriteRowsEventData; import io.micrometer.core.instrument.Counter; import io.micrometer.core.instrument.DistributionSummary; import io.micrometer.core.instrument.Timer; diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationClient.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationClient.java index 2319eb1368..ac77704704 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationClient.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationClient.java @@ -1,7 +1,7 @@ package org.opensearch.dataprepper.plugins.source.rds.stream; +import org.opensearch.dataprepper.plugins.source.rds.schema.PostgresConnectionManager; import org.postgresql.PGConnection; -import org.postgresql.PGProperty; import org.postgresql.replication.LogSequenceNumber; import org.postgresql.replication.PGReplicationStream; import org.postgresql.replication.fluent.logical.ChainedLogicalStreamBuilder; @@ -10,65 +10,27 @@ import java.nio.ByteBuffer; import java.sql.Connection; -import java.sql.DriverManager; -import java.util.Properties; -import java.util.concurrent.TimeUnit; public class LogicalReplicationClient implements ReplicationLogClient { private static final Logger LOG = LoggerFactory.getLogger(LogicalReplicationClient.class); - private static final String URL_FORMAT = "jdbc:postgresql://%s:%d/%s"; - private final String endpoint; - private final int port; - private final String username; - private final String password; - private final String database; - private final String jdbcUrl; + + private final PostgresConnectionManager connectionManager; private final String replicationSlotName; - private Properties props; private LogSequenceNumber startLsn; - private PostgresReplicationEventProcessor eventProcessor; + private LogicalReplicationEventProcessor eventProcessor; private volatile boolean disconnectRequested = false; - public LogicalReplicationClient(final String endpoint, - final int port, - final String username, - final String password, - final String database, + public LogicalReplicationClient(final PostgresConnectionManager connectionManager, final String replicationSlotName) { -// this.endpoint = endpoint; -// this.port = port; - this.endpoint = "127.0.0.1"; - this.port = 5432; - this.username = username; - this.password = password; - this.database = database; + this.connectionManager = connectionManager; this.replicationSlotName = replicationSlotName; - jdbcUrl = String.format(URL_FORMAT, this.endpoint, this.port, this.database); - props = new Properties(); - } - - public void setEventProcessor(PostgresReplicationEventProcessor eventProcessor) { - this.eventProcessor = eventProcessor; - } - - public void setStartLsn(LogSequenceNumber startLsn) { - this.startLsn = startLsn; } public void connect() { - PGProperty.USER.set(props, username); - if (!password.isEmpty()) { - PGProperty.PASSWORD.set(props, password); - } - PGProperty.ASSUME_MIN_SERVER_VERSION.set(props, "9.4"); // This is required - PGProperty.REPLICATION.set(props, "database"); // This is also required - PGProperty.PREFER_QUERY_MODE.set(props, "simple"); // This is also required - - LOG.info("Connect to server with JDBC URL: {}", jdbcUrl); - - try (Connection conn = DriverManager.getConnection(jdbcUrl, props)) { + PGReplicationStream stream; + try (Connection conn = connectionManager.getConnection()) { PGConnection pgConnection = conn.unwrap(PGConnection.class); // Create a replication stream @@ -81,36 +43,48 @@ public void connect() { if (startLsn != null) { logicalStreamBuilder.withStartPosition(startLsn); } - PGReplicationStream stream = logicalStreamBuilder.start(); + stream = logicalStreamBuilder.start(); if (eventProcessor != null) { while (!disconnectRequested) { - // Read changes - ByteBuffer msg = stream.readPending(); - - if (msg == null) { - TimeUnit.MILLISECONDS.sleep(10L); - continue; + try { + // Read changes + ByteBuffer msg = stream.readPending(); + + if (msg == null) { + Thread.sleep(10); + continue; + } + + // decode and convert events to Data Prepper events + eventProcessor.process(msg); + + // Acknowledge receiving the message + LogSequenceNumber lsn = stream.getLastReceiveLSN(); + stream.setFlushedLSN(lsn); + stream.setAppliedLSN(lsn); + } catch (Exception e) { + LOG.error("Exception while processing Postgres replication stream. ", e); } - - // decode and convert events to Data Prepper events - eventProcessor.process(msg); - - // Acknowledge receiving the message - LogSequenceNumber lsn = stream.getLastReceiveLSN(); - stream.setFlushedLSN(lsn); - stream.setAppliedLSN(lsn); } } stream.close(); LOG.info("Replication stream closed successfully."); } catch (Exception e) { - LOG.error("Exception while reading Postgres replication stream. ", e); + LOG.error("Exception while creating Postgres replication stream. ", e); } } public void disconnect() { disconnectRequested = true; } + + public void setEventProcessor(LogicalReplicationEventProcessor eventProcessor) { + this.eventProcessor = eventProcessor; + } + + public void setStartLsn(LogSequenceNumber startLsn) { + this.startLsn = startLsn; + } } diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/PostgresReplicationEventProcessor.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationEventProcessor.java similarity index 91% rename from data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/PostgresReplicationEventProcessor.java rename to data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationEventProcessor.java index f1f19eb91d..217e665f4e 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/PostgresReplicationEventProcessor.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationEventProcessor.java @@ -21,9 +21,9 @@ import java.util.List; import java.util.Map; -public class PostgresReplicationEventProcessor { +public class LogicalReplicationEventProcessor { - private static final Logger LOG = LoggerFactory.getLogger(PostgresReplicationEventProcessor.class); + private static final Logger LOG = LoggerFactory.getLogger(LogicalReplicationEventProcessor.class); static final Duration BUFFER_TIMEOUT = Duration.ofSeconds(60); static final int DEFAULT_BUFFER_BATCH_SIZE = 1_000; @@ -39,10 +39,10 @@ public class PostgresReplicationEventProcessor { private Map tableMetadataMap; - public PostgresReplicationEventProcessor(final StreamPartition streamPartition, - final RdsSourceConfig sourceConfig, - final Buffer> buffer, - final String s3Prefix) { + public LogicalReplicationEventProcessor(final StreamPartition streamPartition, + final RdsSourceConfig sourceConfig, + final Buffer> buffer, + final String s3Prefix) { this.streamPartition = streamPartition; recordConverter = new StreamRecordConverter(s3Prefix, sourceConfig.getPartitionCount()); this.buffer = buffer; @@ -71,10 +71,12 @@ public void process(ByteBuffer msg) { processDeleteMessage(msg); } else if (messageType == 'C') { processCommitMessage(msg); + } else { + throw new IllegalArgumentException("Replication message type [" + messageType + "] is not supported. "); } } - private void processBeginMessage(ByteBuffer msg) { + void processBeginMessage(ByteBuffer msg) { currentLsn = msg.getLong(); long epochMicro = msg.getLong(); currentEventTimestamp = convertPostgresEventTimestamp(epochMicro); @@ -83,7 +85,7 @@ private void processBeginMessage(ByteBuffer msg) { LOG.debug("Processed BEGIN message with LSN: {}, Timestamp: {}, TransactionId: {}", currentLsn, currentEventTimestamp, transaction_xid); } - private void processRelationMessage(ByteBuffer msg) { + void processRelationMessage(ByteBuffer msg) { int tableId = msg.getInt(); // null terminated string String schemaName = getNullTerminatedString(msg); @@ -117,7 +119,7 @@ private void processRelationMessage(ByteBuffer msg) { LOG.debug("Processed an Relation message with RelationId: {} Namespace: {} RelationName: {} ReplicaId: {}", tableId, schemaName, tableName, replicaId); } - private void processCommitMessage(ByteBuffer msg) { + void processCommitMessage(ByteBuffer msg) { int flag = msg.get(); long commitLsn = msg.getLong(); long endLsn = msg.getLong(); @@ -134,7 +136,7 @@ private void processCommitMessage(ByteBuffer msg) { LOG.debug("Processed a COMMIT message with Flag: {} CommitLsn: {} EndLsn: {} Timestamp: {}", flag, commitLsn, endLsn, epochMicro); } - private void processInsertMessage(ByteBuffer msg) { + void processInsertMessage(ByteBuffer msg) { int tableId = msg.getInt(); char n_char = (char) msg.get(); // Skip the 'N' character @@ -147,7 +149,7 @@ private void processInsertMessage(ByteBuffer msg) { LOG.debug("Processed an INSERT message with table id: {}", tableId); } - private void processUpdateMessage(ByteBuffer msg) { + void processUpdateMessage(ByteBuffer msg) { int tableId = msg.getInt(); char typeId = (char) msg.get(); @@ -167,7 +169,7 @@ private void processUpdateMessage(ByteBuffer msg) { } } - private void processDeleteMessage(ByteBuffer msg) { + void processDeleteMessage(ByteBuffer msg) { int tableId = msg.getInt(); char typeId = (char) msg.get(); diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/ReplicationLogClientFactory.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/ReplicationLogClientFactory.java index 551b81982d..d1441746fe 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/ReplicationLogClientFactory.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/ReplicationLogClientFactory.java @@ -12,6 +12,7 @@ import org.opensearch.dataprepper.plugins.source.rds.configuration.EngineType; import org.opensearch.dataprepper.plugins.source.rds.coordination.partition.StreamPartition; import org.opensearch.dataprepper.plugins.source.rds.model.DbMetadata; +import org.opensearch.dataprepper.plugins.source.rds.schema.PostgresConnectionManager; import software.amazon.awssdk.services.rds.RdsClient; import java.util.List; @@ -19,29 +20,25 @@ public class ReplicationLogClientFactory { + private final RdsSourceConfig sourceConfig; private final RdsClient rdsClient; private final DbMetadata dbMetadata; - private final EngineType engineType; private String username; private String password; - private String database; private SSLMode sslMode = SSLMode.REQUIRED; public ReplicationLogClientFactory(final RdsSourceConfig sourceConfig, final RdsClient rdsClient, final DbMetadata dbMetadata) { + this.sourceConfig = sourceConfig; this.rdsClient = rdsClient; this.dbMetadata = dbMetadata; - engineType = sourceConfig.getEngine(); username = sourceConfig.getAuthenticationConfig().getUsername(); password = sourceConfig.getAuthenticationConfig().getPassword(); - if (sourceConfig.getEngine() == EngineType.POSTGRES) { - database = getDatabaseName(sourceConfig.getTableNames()); - } } public ReplicationLogClient create(StreamPartition streamPartition) { - if (engineType == EngineType.MYSQL) { + if (sourceConfig.getEngine() == EngineType.MYSQL) { return new BinlogClientWrapper(createBinaryLogClient()); } else { // Postgres return createLogicalReplicationClient(streamPartition); @@ -68,13 +65,14 @@ private LogicalReplicationClient createLogicalReplicationClient(StreamPartition if (replicationSlotName == null) { throw new NoSuchElementException("Replication slot name is not found in progress state."); } - return new LogicalReplicationClient( + final PostgresConnectionManager connectionManager = new PostgresConnectionManager( dbMetadata.getEndpoint(), dbMetadata.getPort(), username, password, - database, - replicationSlotName); + !sourceConfig.getTlsConfig().isInsecure(), + getDatabaseName(sourceConfig.getTableNames())); + return new LogicalReplicationClient(connectionManager, replicationSlotName); } public void setSSLMode(SSLMode sslMode) { diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamWorker.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamWorker.java index 089cc57235..d6404a42ef 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamWorker.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamWorker.java @@ -103,7 +103,6 @@ private void setStartBinlogPosition(final StreamPartition streamPartition) { private void setStartLsn(final StreamPartition streamPartition) { final String startLsn = streamPartition.getProgressState().get().getCurrentLsn(); - LOG.debug("Will start replication stream from LSN {}.", startLsn); if (startLsn != null) { LOG.debug("Will start logical replication from LSN {}", startLsn); diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamWorkerTaskRefresher.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamWorkerTaskRefresher.java index b9adb0cee0..acd8d0535f 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamWorkerTaskRefresher.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamWorkerTaskRefresher.java @@ -129,7 +129,7 @@ private void refreshTask(RdsSourceConfig sourceConfig) { streamCheckpointer, acknowledgementSetManager, dbTableMetadata, cascadeActionDetector)); } else { final LogicalReplicationClient logicalReplicationClient = (LogicalReplicationClient) replicationLogClient; - logicalReplicationClient.setEventProcessor(new PostgresReplicationEventProcessor( + logicalReplicationClient.setEventProcessor(new LogicalReplicationEventProcessor( streamPartition, sourceConfig, buffer, s3Prefix )); } diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/schema/PostgresConnectionManagerTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/schema/PostgresConnectionManagerTest.java new file mode 100644 index 0000000000..e45f46ad58 --- /dev/null +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/schema/PostgresConnectionManagerTest.java @@ -0,0 +1,88 @@ +package org.opensearch.dataprepper.plugins.source.rds.schema; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.mockito.ArgumentCaptor; +import org.postgresql.PGProperty; + +import java.sql.Connection; +import java.sql.SQLException; +import java.util.Properties; +import java.util.Random; +import java.util.UUID; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.is; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; +import static org.opensearch.dataprepper.plugins.source.rds.schema.PostgresConnectionManager.DATABASE_REPLICATION; +import static org.opensearch.dataprepper.plugins.source.rds.schema.PostgresConnectionManager.FALSE_VALUE; +import static org.opensearch.dataprepper.plugins.source.rds.schema.PostgresConnectionManager.REQUIRE_SSL; +import static org.opensearch.dataprepper.plugins.source.rds.schema.PostgresConnectionManager.SERVER_VERSION_9_4; +import static org.opensearch.dataprepper.plugins.source.rds.schema.PostgresConnectionManager.SIMPLE_QUERY; +import static org.opensearch.dataprepper.plugins.source.rds.schema.PostgresConnectionManager.TRUE_VALUE; + +class PostgresConnectionManagerTest { + + private String endpoint; + private int port; + private String username; + private String password; + private boolean requireSSL; + private String database; + private final Random random = new Random(); + + @BeforeEach + void setUp() { + endpoint = UUID.randomUUID().toString(); + port = random.nextInt(65536); + username = UUID.randomUUID().toString(); + password = UUID.randomUUID().toString(); + } + + @Test + void test_getConnection_when_requireSSL_is_true() throws SQLException { + requireSSL = true; + final PostgresConnectionManager connectionManager = spy(createObjectUnderTest()); + final ArgumentCaptor jdbcUrlArgumentCaptor = ArgumentCaptor.forClass(String.class); + final ArgumentCaptor propertiesArgumentCaptor = ArgumentCaptor.forClass(Properties.class); + doReturn(mock(Connection.class)).when(connectionManager).doGetConnection(jdbcUrlArgumentCaptor.capture(), propertiesArgumentCaptor.capture()); + + connectionManager.getConnection(); + + assertThat(jdbcUrlArgumentCaptor.getValue(), is(String.format(PostgresConnectionManager.JDBC_URL_FORMAT, endpoint, port, database))); + final Properties properties = propertiesArgumentCaptor.getValue(); + assertThat(PGProperty.USER.getOrDefault(properties), is(username)); + assertThat(PGProperty.PASSWORD.getOrDefault(properties), is(password)); + assertThat(PGProperty.ASSUME_MIN_SERVER_VERSION.getOrDefault(properties), is(SERVER_VERSION_9_4)); + assertThat(PGProperty.REPLICATION.getOrDefault(properties), is(DATABASE_REPLICATION)); + assertThat(PGProperty.PREFER_QUERY_MODE.getOrDefault(properties), is(SIMPLE_QUERY)); + assertThat(PGProperty.SSL.getOrDefault(properties), is(TRUE_VALUE)); + assertThat(PGProperty.SSL_MODE.getOrDefault(properties), is(REQUIRE_SSL)); + } + + @Test + void test_getConnection_when_requireSSL_is_false() throws SQLException { + requireSSL = false; + final PostgresConnectionManager connectionManager = spy(createObjectUnderTest()); + final ArgumentCaptor jdbcUrlArgumentCaptor = ArgumentCaptor.forClass(String.class); + final ArgumentCaptor propertiesArgumentCaptor = ArgumentCaptor.forClass(Properties.class); + doReturn(mock(Connection.class)).when(connectionManager).doGetConnection(jdbcUrlArgumentCaptor.capture(), propertiesArgumentCaptor.capture()); + + connectionManager.getConnection(); + + assertThat(jdbcUrlArgumentCaptor.getValue(), is(String.format(PostgresConnectionManager.JDBC_URL_FORMAT, endpoint, port, database))); + final Properties properties = propertiesArgumentCaptor.getValue(); + assertThat(PGProperty.USER.getOrDefault(properties), is(username)); + assertThat(PGProperty.PASSWORD.getOrDefault(properties), is(password)); + assertThat(PGProperty.ASSUME_MIN_SERVER_VERSION.getOrDefault(properties), is(SERVER_VERSION_9_4)); + assertThat(PGProperty.REPLICATION.getOrDefault(properties), is(DATABASE_REPLICATION)); + assertThat(PGProperty.PREFER_QUERY_MODE.getOrDefault(properties), is(SIMPLE_QUERY)); + assertThat(PGProperty.SSL.getOrDefault(properties), is(FALSE_VALUE)); + } + + private PostgresConnectionManager createObjectUnderTest() { + return new PostgresConnectionManager(endpoint, port, username, password, requireSSL, database); + } +} \ No newline at end of file diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/schema/PostgresSchemaManagerTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/schema/PostgresSchemaManagerTest.java new file mode 100644 index 0000000000..dddeb3f3b9 --- /dev/null +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/schema/PostgresSchemaManagerTest.java @@ -0,0 +1,81 @@ +package org.opensearch.dataprepper.plugins.source.rds.schema; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Answers; +import org.mockito.ArgumentCaptor; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +import org.postgresql.PGConnection; +import org.postgresql.replication.PGReplicationConnection; +import org.postgresql.replication.fluent.ChainedCreateReplicationSlotBuilder; +import org.postgresql.replication.fluent.logical.ChainedLogicalCreateSlotBuilder; + +import java.sql.Connection; +import java.sql.PreparedStatement; +import java.sql.SQLException; +import java.util.List; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.is; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +@ExtendWith(MockitoExtension.class) +class PostgresSchemaManagerTest { + + @Mock + private PostgresConnectionManager connectionManager; + + @Mock(answer = Answers.RETURNS_DEEP_STUBS) + private Connection connection; + + private PostgresSchemaManager schemaManager; + + @BeforeEach + void setUp() { + schemaManager = createObjectUnderTest(); + } + + @Test + void test_createLogicalReplicationSlot() throws SQLException { + final List tableNames = List.of("table1", "table2"); + final String publicationName = "publication1"; + final String slotName = "slot1"; + final PreparedStatement preparedStatement = mock(PreparedStatement.class); + final PGConnection pgConnection = mock(PGConnection.class); + final PGReplicationConnection replicationConnection = mock(PGReplicationConnection.class); + final ChainedCreateReplicationSlotBuilder chainedCreateSlotBuilder = mock(ChainedCreateReplicationSlotBuilder.class); + final ChainedLogicalCreateSlotBuilder slotBuilder = mock(ChainedLogicalCreateSlotBuilder.class); + + ArgumentCaptor statementCaptor = ArgumentCaptor.forClass(String.class); + + when(connectionManager.getConnection()).thenReturn(connection); + when(connection.prepareStatement(statementCaptor.capture())).thenReturn(preparedStatement); + when(connection.unwrap(PGConnection.class)).thenReturn(pgConnection); + when(pgConnection.getReplicationAPI()).thenReturn(replicationConnection); + when(replicationConnection.createReplicationSlot()).thenReturn(chainedCreateSlotBuilder); + when(chainedCreateSlotBuilder.logical()).thenReturn(slotBuilder); + when(slotBuilder.withSlotName(anyString())).thenReturn(slotBuilder); + when(slotBuilder.withOutputPlugin(anyString())).thenReturn(slotBuilder); + + schemaManager.createLogicalReplicationSlot(tableNames, publicationName, slotName); + + String statement = statementCaptor.getValue(); + assertThat(statement, is("CREATE PUBLICATION " + publicationName + " FOR TABLE " + String.join(", ", tableNames) + ";")); + verify(preparedStatement).executeUpdate(); + verify(pgConnection).getReplicationAPI(); + verify(replicationConnection).createReplicationSlot(); + verify(chainedCreateSlotBuilder).logical(); + verify(slotBuilder).withSlotName(slotName); + verify(slotBuilder).withOutputPlugin("pgoutput"); + verify(slotBuilder).make(); + } + + private PostgresSchemaManager createObjectUnderTest() { + return new PostgresSchemaManager(connectionManager); + } +} diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationClientTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationClientTest.java new file mode 100644 index 0000000000..b3e0bf77f8 --- /dev/null +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationClientTest.java @@ -0,0 +1,81 @@ +package org.opensearch.dataprepper.plugins.source.rds.stream; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.dataprepper.plugins.source.rds.schema.PostgresConnectionManager; +import org.postgresql.PGConnection; +import org.postgresql.replication.LogSequenceNumber; +import org.postgresql.replication.PGReplicationStream; +import org.postgresql.replication.fluent.logical.ChainedLogicalStreamBuilder; + +import java.nio.ByteBuffer; +import java.sql.Connection; +import java.sql.SQLException; +import java.time.Duration; +import java.util.UUID; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; + +import static org.awaitility.Awaitility.await; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.Mockito.RETURNS_DEEP_STUBS; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +@ExtendWith(MockitoExtension.class) +class LogicalReplicationClientTest { + + @Mock + private PostgresConnectionManager connectionManager; + + @Mock + private LogicalReplicationEventProcessor eventProcessor; + + private String replicationSlotName; + private LogicalReplicationClient logicalReplicationClient; + + @BeforeEach + void setUp() { + replicationSlotName = UUID.randomUUID().toString(); + logicalReplicationClient = createObjectUnderTest(); + logicalReplicationClient.setEventProcessor(eventProcessor); + } + + @Test + void test_connect() throws SQLException, InterruptedException { + final Connection connection = mock(Connection.class); + final PGConnection pgConnection = mock(PGConnection.class, RETURNS_DEEP_STUBS); + final ChainedLogicalStreamBuilder logicalStreamBuilder = mock(ChainedLogicalStreamBuilder.class); + final PGReplicationStream stream = mock(PGReplicationStream.class); + final ByteBuffer message = mock(ByteBuffer.class); + final LogSequenceNumber lsn = mock(LogSequenceNumber.class); + + when(connectionManager.getConnection()).thenReturn(connection); + when(connection.unwrap(PGConnection.class)).thenReturn(pgConnection); + when(pgConnection.getReplicationAPI().replicationStream().logical()).thenReturn(logicalStreamBuilder); + when(logicalStreamBuilder.withSlotName(anyString())).thenReturn(logicalStreamBuilder); + when(logicalStreamBuilder.withSlotOption(anyString(), anyString())).thenReturn(logicalStreamBuilder); + when(logicalStreamBuilder.start()).thenReturn(stream); + when(stream.readPending()).thenReturn(message).thenReturn(null); + when(stream.getLastReceiveLSN()).thenReturn(lsn); + + final ExecutorService executorService = Executors.newSingleThreadExecutor(); + executorService.submit(() -> logicalReplicationClient.connect()); + + await().atMost(Duration.ofSeconds(1)) + .untilAsserted(() -> verify(eventProcessor).process(message)); + Thread.sleep(20); + executorService.shutdownNow(); + + verify(stream).setAppliedLSN(lsn); + verify(stream).setFlushedLSN(lsn); + } + + private LogicalReplicationClient createObjectUnderTest() { + return new LogicalReplicationClient(connectionManager, replicationSlotName); + } +} diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationEventProcessorTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationEventProcessorTest.java new file mode 100644 index 0000000000..a54062e2b1 --- /dev/null +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationEventProcessorTest.java @@ -0,0 +1,115 @@ +package org.opensearch.dataprepper.plugins.source.rds.stream; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.dataprepper.model.buffer.Buffer; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.plugins.source.rds.RdsSourceConfig; +import org.opensearch.dataprepper.plugins.source.rds.coordination.partition.StreamPartition; + +import java.nio.ByteBuffer; +import java.util.UUID; + +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.mockito.Mockito.doNothing; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +@ExtendWith(MockitoExtension.class) +class LogicalReplicationEventProcessorTest { + + @Mock + private StreamPartition streamPartition; + + @Mock + private RdsSourceConfig sourceConfig; + + @Mock + private Buffer> buffer; + + @Mock + private ByteBuffer message; + + private String s3Prefix; + + private LogicalReplicationEventProcessor objectUnderTest; + + @BeforeEach + void setUp() { + s3Prefix = UUID.randomUUID().toString(); + objectUnderTest = spy(createObjectUnderTest()); + } + + @Test + void test_correct_process_method_invoked_for_begin_message() { + when(message.get()).thenReturn((byte) 'B'); + + objectUnderTest.process(message); + + verify(objectUnderTest).processBeginMessage(message); + } + + @Test + void test_correct_process_method_invoked_for_relation_message() { + when(message.get()).thenReturn((byte) 'R'); + + objectUnderTest.process(message); + + verify(objectUnderTest).processRelationMessage(message); + } + + @Test + void test_correct_process_method_invoked_for_commit_message() { + when(message.get()).thenReturn((byte) 'C'); + + objectUnderTest.process(message); + + verify(objectUnderTest).processCommitMessage(message); + } + + @Test + void test_correct_process_method_invoked_for_insert_message() { + when(message.get()).thenReturn((byte) 'I'); + doNothing().when(objectUnderTest).processInsertMessage(message); + + objectUnderTest.process(message); + + verify(objectUnderTest).processInsertMessage(message); + } + + @Test + void test_correct_process_method_invoked_for_update_message() { + when(message.get()).thenReturn((byte) 'U'); + doNothing().when(objectUnderTest).processUpdateMessage(message); + + objectUnderTest.process(message); + + verify(objectUnderTest).processUpdateMessage(message); + } + + @Test + void test_correct_process_method_invoked_for_delete_message() { + when(message.get()).thenReturn((byte) 'D'); + doNothing().when(objectUnderTest).processDeleteMessage(message); + + objectUnderTest.process(message); + + verify(objectUnderTest).processDeleteMessage(message); + } + + @Test + void test_unsupported_message_type_throws_exception() { + when(message.get()).thenReturn((byte) 'A'); + + assertThrows(IllegalArgumentException.class, () -> objectUnderTest.process(message)); + } + + private LogicalReplicationEventProcessor createObjectUnderTest() { + return new LogicalReplicationEventProcessor(streamPartition, sourceConfig, buffer, s3Prefix); + } +} diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/ReplicationLogClientFactoryTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/ReplicationLogClientFactoryTest.java new file mode 100644 index 0000000000..3f7b41a20e --- /dev/null +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/ReplicationLogClientFactoryTest.java @@ -0,0 +1,86 @@ +package org.opensearch.dataprepper.plugins.source.rds.stream; + +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Answers; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.dataprepper.plugins.source.rds.RdsSourceConfig; +import org.opensearch.dataprepper.plugins.source.rds.configuration.EngineType; +import org.opensearch.dataprepper.plugins.source.rds.coordination.partition.StreamPartition; +import org.opensearch.dataprepper.plugins.source.rds.coordination.state.StreamProgressState; +import org.opensearch.dataprepper.plugins.source.rds.model.DbMetadata; +import software.amazon.awssdk.services.rds.RdsClient; + +import java.util.List; +import java.util.Optional; +import java.util.UUID; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.instanceOf; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +@ExtendWith(MockitoExtension.class) +class ReplicationLogClientFactoryTest { + + @Mock(answer = Answers.RETURNS_DEEP_STUBS) + private RdsSourceConfig sourceConfig; + + @Mock + private RdsClient rdsClient; + + @Mock + private DbMetadata dbMetadata; + + @Mock + private StreamPartition streamPartition; + + private ReplicationLogClientFactory replicationLogClientFactory; + + @Test + void test_create_binlog_client() { + final String username = UUID.randomUUID().toString(); + final String password = UUID.randomUUID().toString(); + + when(sourceConfig.getEngine()).thenReturn(EngineType.MYSQL); + when(sourceConfig.getAuthenticationConfig().getUsername()).thenReturn(username); + when(sourceConfig.getAuthenticationConfig().getPassword()).thenReturn(password); + + replicationLogClientFactory = createObjectUnderTest(); + ReplicationLogClient replicationLogClient = replicationLogClientFactory.create(streamPartition); + + verify(dbMetadata).getEndpoint(); + verify(dbMetadata).getPort(); + assertThat(replicationLogClient, instanceOf(BinlogClientWrapper.class)); + } + + @Test + void test_create_logical_replication_client() { + final String username = UUID.randomUUID().toString(); + final String password = UUID.randomUUID().toString(); + final StreamProgressState streamProgressState = mock(StreamProgressState.class); + final String slotName = UUID.randomUUID().toString(); + final List tableNames = List.of("table1", "table2"); + + when(sourceConfig.getEngine()).thenReturn(EngineType.POSTGRES); + when(sourceConfig.getTlsConfig().isInsecure()).thenReturn(false); + when(sourceConfig.getTableNames()).thenReturn(tableNames); + when(sourceConfig.getAuthenticationConfig().getUsername()).thenReturn(username); + when(sourceConfig.getAuthenticationConfig().getPassword()).thenReturn(password); + when(streamPartition.getProgressState()).thenReturn(Optional.of(streamProgressState)); + when(streamProgressState.getReplicationSlotName()).thenReturn(slotName); + + replicationLogClientFactory = createObjectUnderTest(); + ReplicationLogClient replicationLogClient = replicationLogClientFactory.create(streamPartition); + + verify(dbMetadata).getEndpoint(); + verify(dbMetadata).getPort(); + assertThat(replicationLogClient, instanceOf(LogicalReplicationClient.class)); + } + + private ReplicationLogClientFactory createObjectUnderTest() { + return new ReplicationLogClientFactory(sourceConfig, rdsClient, dbMetadata); + } +} From 4fbdbdb2ac7735f0e6142708715519a8890059e3 Mon Sep 17 00:00:00 2001 From: Hai Yan Date: Mon, 6 Jan 2025 22:25:19 +0800 Subject: [PATCH 04/10] Remove and rename classes Signed-off-by: Hai Yan --- .../plugins/source/rds/RdsService.java | 6 +- ...nager.java => MySqlConnectionManager.java} | 4 +- .../source/rds/schema/MySqlSchemaManager.java | 4 +- .../source/rds/schema/QueryManager.java | 4 +- .../rds/stream/BinlogClientFactory.java | 55 ------------------- ...t.java => MySqlConnectionManagerTest.java} | 28 +++++----- .../rds/schema/MySqlSchemaManagerTest.java | 2 +- .../source/rds/schema/QueryManagerTest.java | 2 +- .../rds/stream/BinlogClientFactoryTest.java | 53 ------------------ 9 files changed, 25 insertions(+), 133 deletions(-) rename data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/schema/{ConnectionManager.java => MySqlConnectionManager.java} (92%) delete mode 100644 data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/BinlogClientFactory.java rename data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/schema/{ConnectionManagerTest.java => MySqlConnectionManagerTest.java} (81%) delete mode 100644 data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/BinlogClientFactoryTest.java diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsService.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsService.java index e81158f5d2..c9b57fa267 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsService.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsService.java @@ -26,7 +26,7 @@ import org.opensearch.dataprepper.plugins.source.rds.model.DbMetadata; import org.opensearch.dataprepper.plugins.source.rds.model.DbTableMetadata; import org.opensearch.dataprepper.plugins.source.rds.resync.ResyncScheduler; -import org.opensearch.dataprepper.plugins.source.rds.schema.ConnectionManager; +import org.opensearch.dataprepper.plugins.source.rds.schema.MySqlConnectionManager; import org.opensearch.dataprepper.plugins.source.rds.schema.QueryManager; import org.opensearch.dataprepper.plugins.source.rds.schema.MySqlSchemaManager; import org.opensearch.dataprepper.plugins.source.rds.schema.SchemaManager; @@ -180,7 +180,7 @@ public void shutdown() { private SchemaManager getSchemaManager(final RdsSourceConfig sourceConfig, final DbMetadata dbMetadata) { // For MySQL if (sourceConfig.getEngine() == EngineType.MYSQL) { - final ConnectionManager connectionManager = new ConnectionManager( + final MySqlConnectionManager connectionManager = new MySqlConnectionManager( "127.0.0.1", 5432, sourceConfig.getAuthenticationConfig().getUsername(), @@ -206,7 +206,7 @@ private String getDatabaseName(List tableNames) { private QueryManager getQueryManager(final RdsSourceConfig sourceConfig, final DbMetadata dbMetadata) { final String readerEndpoint = dbMetadata.getReaderEndpoint() != null ? dbMetadata.getReaderEndpoint() : dbMetadata.getEndpoint(); final int readerPort = dbMetadata.getReaderPort() == 0 ? dbMetadata.getPort() : dbMetadata.getReaderPort(); - final ConnectionManager readerConnectionManager = new ConnectionManager( + final MySqlConnectionManager readerConnectionManager = new MySqlConnectionManager( "127.0.0.1", 5432, sourceConfig.getAuthenticationConfig().getUsername(), diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/schema/ConnectionManager.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/schema/MySqlConnectionManager.java similarity index 92% rename from data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/schema/ConnectionManager.java rename to data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/schema/MySqlConnectionManager.java index 542724d49d..a76d3fd3bf 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/schema/ConnectionManager.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/schema/MySqlConnectionManager.java @@ -10,7 +10,7 @@ import java.sql.SQLException; import java.util.Properties; -public class ConnectionManager { +public class MySqlConnectionManager { static final String JDBC_URL_FORMAT = "jdbc:mysql://%s:%d"; static final String USERNAME_KEY = "user"; static final String PASSWORD_KEY = "password"; @@ -25,7 +25,7 @@ public class ConnectionManager { private final String password; private final boolean requireSSL; - public ConnectionManager(String hostName, int port, String username, String password, boolean requireSSL) { + public MySqlConnectionManager(String hostName, int port, String username, String password, boolean requireSSL) { this.hostName = hostName; this.port = port; this.username = username; diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/schema/MySqlSchemaManager.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/schema/MySqlSchemaManager.java index 5421c7e7fb..d36b76a90d 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/schema/MySqlSchemaManager.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/schema/MySqlSchemaManager.java @@ -39,9 +39,9 @@ public class MySqlSchemaManager implements SchemaManager { static final String UPDATE_RULE = "UPDATE_RULE"; static final String DELETE_RULE = "DELETE_RULE"; static final String COLUMN_DEF = "COLUMN_DEF"; - private final ConnectionManager connectionManager; + private final MySqlConnectionManager connectionManager; - public MySqlSchemaManager(ConnectionManager connectionManager) { + public MySqlSchemaManager(MySqlConnectionManager connectionManager) { this.connectionManager = connectionManager; } diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/schema/QueryManager.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/schema/QueryManager.java index d89345fb71..95e61bc729 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/schema/QueryManager.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/schema/QueryManager.java @@ -26,9 +26,9 @@ public class QueryManager { static final int NUM_OF_RETRIES = 3; static final int BACKOFF_IN_MILLIS = 500; - private final ConnectionManager connectionManager; + private final MySqlConnectionManager connectionManager; - public QueryManager(ConnectionManager connectionManager) { + public QueryManager(MySqlConnectionManager connectionManager) { this.connectionManager = connectionManager; } diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/BinlogClientFactory.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/BinlogClientFactory.java deleted file mode 100644 index b63e588f01..0000000000 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/BinlogClientFactory.java +++ /dev/null @@ -1,55 +0,0 @@ -/* - * Copyright OpenSearch Contributors - * SPDX-License-Identifier: Apache-2.0 - */ - -package org.opensearch.dataprepper.plugins.source.rds.stream; - -import com.github.shyiko.mysql.binlog.BinaryLogClient; -import com.github.shyiko.mysql.binlog.event.deserialization.EventDeserializer; -import com.github.shyiko.mysql.binlog.network.SSLMode; -import org.opensearch.dataprepper.plugins.source.rds.RdsSourceConfig; -import org.opensearch.dataprepper.plugins.source.rds.model.DbMetadata; -import software.amazon.awssdk.services.rds.RdsClient; - -public class BinlogClientFactory { - - private final RdsClient rdsClient; - private final DbMetadata dbMetadata; - private String username; - private String password; - private SSLMode sslMode = SSLMode.REQUIRED; - - public BinlogClientFactory(final RdsSourceConfig sourceConfig, - final RdsClient rdsClient, - final DbMetadata dbMetadata) { - this.rdsClient = rdsClient; - this.dbMetadata = dbMetadata; - username = sourceConfig.getAuthenticationConfig().getUsername(); - password = sourceConfig.getAuthenticationConfig().getPassword(); - } - - public BinaryLogClient create() { - BinaryLogClient binaryLogClient = new BinaryLogClient( - dbMetadata.getEndpoint(), - dbMetadata.getPort(), - username, - password); - binaryLogClient.setSSLMode(sslMode); - final EventDeserializer eventDeserializer = new EventDeserializer(); - eventDeserializer.setCompatibilityMode( - EventDeserializer.CompatibilityMode.DATE_AND_TIME_AS_LONG - ); - binaryLogClient.setEventDeserializer(eventDeserializer); - return binaryLogClient; - } - - public void setSSLMode(SSLMode sslMode) { - this.sslMode = sslMode; - } - - public void setCredentials(String username, String password) { - this.username = username; - this.password = password; - } -} diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/schema/ConnectionManagerTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/schema/MySqlConnectionManagerTest.java similarity index 81% rename from data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/schema/ConnectionManagerTest.java rename to data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/schema/MySqlConnectionManagerTest.java index 83c93d91c3..3c18f5e1ff 100644 --- a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/schema/ConnectionManagerTest.java +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/schema/MySqlConnectionManagerTest.java @@ -20,16 +20,16 @@ import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; -import static org.opensearch.dataprepper.plugins.source.rds.schema.ConnectionManager.FALSE_VALUE; -import static org.opensearch.dataprepper.plugins.source.rds.schema.ConnectionManager.PASSWORD_KEY; -import static org.opensearch.dataprepper.plugins.source.rds.schema.ConnectionManager.REQUIRE_SSL_KEY; -import static org.opensearch.dataprepper.plugins.source.rds.schema.ConnectionManager.TINY_INT_ONE_IS_BIT_KEY; -import static org.opensearch.dataprepper.plugins.source.rds.schema.ConnectionManager.TRUE_VALUE; -import static org.opensearch.dataprepper.plugins.source.rds.schema.ConnectionManager.USERNAME_KEY; -import static org.opensearch.dataprepper.plugins.source.rds.schema.ConnectionManager.USE_SSL_KEY; +import static org.opensearch.dataprepper.plugins.source.rds.schema.MySqlConnectionManager.FALSE_VALUE; +import static org.opensearch.dataprepper.plugins.source.rds.schema.MySqlConnectionManager.PASSWORD_KEY; +import static org.opensearch.dataprepper.plugins.source.rds.schema.MySqlConnectionManager.REQUIRE_SSL_KEY; +import static org.opensearch.dataprepper.plugins.source.rds.schema.MySqlConnectionManager.TINY_INT_ONE_IS_BIT_KEY; +import static org.opensearch.dataprepper.plugins.source.rds.schema.MySqlConnectionManager.TRUE_VALUE; +import static org.opensearch.dataprepper.plugins.source.rds.schema.MySqlConnectionManager.USERNAME_KEY; +import static org.opensearch.dataprepper.plugins.source.rds.schema.MySqlConnectionManager.USE_SSL_KEY; -class ConnectionManagerTest { +class MySqlConnectionManagerTest { private String hostName; private int port; @@ -49,14 +49,14 @@ void setUp() { @Test void test_getConnection_when_requireSSL_is_true() throws SQLException { requireSSL = true; - final ConnectionManager connectionManager = spy(createObjectUnderTest()); + final MySqlConnectionManager connectionManager = spy(createObjectUnderTest()); final ArgumentCaptor jdbcUrlArgumentCaptor = ArgumentCaptor.forClass(String.class); final ArgumentCaptor propertiesArgumentCaptor = ArgumentCaptor.forClass(Properties.class); doReturn(mock(Connection.class)).when(connectionManager).doGetConnection(jdbcUrlArgumentCaptor.capture(), propertiesArgumentCaptor.capture()); connectionManager.getConnection(); - assertThat(jdbcUrlArgumentCaptor.getValue(), is(String.format(ConnectionManager.JDBC_URL_FORMAT, hostName, port))); + assertThat(jdbcUrlArgumentCaptor.getValue(), is(String.format(MySqlConnectionManager.JDBC_URL_FORMAT, hostName, port))); final Properties properties = propertiesArgumentCaptor.getValue(); assertThat(properties.getProperty(USERNAME_KEY), is(username)); assertThat(properties.getProperty(PASSWORD_KEY), is(password)); @@ -68,14 +68,14 @@ void test_getConnection_when_requireSSL_is_true() throws SQLException { @Test void test_getConnection_when_requireSSL_is_false() throws SQLException { requireSSL = false; - final ConnectionManager connectionManager = spy(createObjectUnderTest()); + final MySqlConnectionManager connectionManager = spy(createObjectUnderTest()); final ArgumentCaptor jdbcUrlArgumentCaptor = ArgumentCaptor.forClass(String.class); final ArgumentCaptor propertiesArgumentCaptor = ArgumentCaptor.forClass(Properties.class); doReturn(mock(Connection.class)).when(connectionManager).doGetConnection(jdbcUrlArgumentCaptor.capture(), propertiesArgumentCaptor.capture()); connectionManager.getConnection(); - assertThat(jdbcUrlArgumentCaptor.getValue(), is(String.format(ConnectionManager.JDBC_URL_FORMAT, hostName, port))); + assertThat(jdbcUrlArgumentCaptor.getValue(), is(String.format(MySqlConnectionManager.JDBC_URL_FORMAT, hostName, port))); final Properties properties = propertiesArgumentCaptor.getValue(); assertThat(properties.getProperty(USERNAME_KEY), is(username)); assertThat(properties.getProperty(PASSWORD_KEY), is(password)); @@ -83,7 +83,7 @@ void test_getConnection_when_requireSSL_is_false() throws SQLException { assertThat(properties.getProperty(TINY_INT_ONE_IS_BIT_KEY), is(FALSE_VALUE)); } - private ConnectionManager createObjectUnderTest() { - return new ConnectionManager(hostName, port, username, password, requireSSL); + private MySqlConnectionManager createObjectUnderTest() { + return new MySqlConnectionManager(hostName, port, username, password, requireSSL); } } \ No newline at end of file diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/schema/MySqlSchemaManagerTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/schema/MySqlSchemaManagerTest.java index 53531c10c0..c491dcfecb 100644 --- a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/schema/MySqlSchemaManagerTest.java +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/schema/MySqlSchemaManagerTest.java @@ -53,7 +53,7 @@ class MySqlSchemaManagerTest { @Mock - private ConnectionManager connectionManager; + private MySqlConnectionManager connectionManager; @Mock(answer = Answers.RETURNS_DEEP_STUBS) private Connection connection; diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/schema/QueryManagerTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/schema/QueryManagerTest.java index 9ed44e908c..86cc5d4431 100644 --- a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/schema/QueryManagerTest.java +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/schema/QueryManagerTest.java @@ -32,7 +32,7 @@ class QueryManagerTest { @Mock - private ConnectionManager connectionManager; + private MySqlConnectionManager connectionManager; @Mock(answer = Answers.RETURNS_DEEP_STUBS) private Connection connection; diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/BinlogClientFactoryTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/BinlogClientFactoryTest.java deleted file mode 100644 index c56ffd94d5..0000000000 --- a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/BinlogClientFactoryTest.java +++ /dev/null @@ -1,53 +0,0 @@ -/* - * Copyright OpenSearch Contributors - * SPDX-License-Identifier: Apache-2.0 - */ - -package org.opensearch.dataprepper.plugins.source.rds.stream; - -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.extension.ExtendWith; -import org.mockito.Answers; -import org.mockito.Mock; -import org.mockito.junit.jupiter.MockitoExtension; -import org.opensearch.dataprepper.plugins.source.rds.RdsSourceConfig; -import org.opensearch.dataprepper.plugins.source.rds.model.DbMetadata; -import software.amazon.awssdk.services.rds.RdsClient; - -import java.util.UUID; - -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; - -@ExtendWith(MockitoExtension.class) -class BinlogClientFactoryTest { - - @Mock(answer = Answers.RETURNS_DEEP_STUBS) - private RdsSourceConfig sourceConfig; - - @Mock - private RdsClient rdsClient; - - @Mock - private DbMetadata dbMetadata; - - private BinlogClientFactory binlogClientFactory; - - @Test - void test_create() { - final String username = UUID.randomUUID().toString(); - final String password = UUID.randomUUID().toString(); - when(sourceConfig.getAuthenticationConfig().getUsername()).thenReturn(username); - when(sourceConfig.getAuthenticationConfig().getPassword()).thenReturn(password); - - binlogClientFactory = createObjectUnderTest(); - binlogClientFactory.create(); - - verify(dbMetadata).getEndpoint(); - verify(dbMetadata).getPort(); - } - - private BinlogClientFactory createObjectUnderTest() { - return new BinlogClientFactory(sourceConfig, rdsClient, dbMetadata); - } -} \ No newline at end of file From 9e703726008f33efbef623a5e69974ed7621af40 Mon Sep 17 00:00:00 2001 From: Hai Yan Date: Mon, 6 Jan 2025 22:29:49 +0800 Subject: [PATCH 05/10] Remove test code Signed-off-by: Hai Yan --- .../dataprepper/plugins/source/rds/RdsService.java | 12 ++++++------ shared-config/log4j2.properties | 9 --------- 2 files changed, 6 insertions(+), 15 deletions(-) diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsService.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsService.java index c9b57fa267..cdbe1b04d8 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsService.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsService.java @@ -181,8 +181,8 @@ private SchemaManager getSchemaManager(final RdsSourceConfig sourceConfig, final // For MySQL if (sourceConfig.getEngine() == EngineType.MYSQL) { final MySqlConnectionManager connectionManager = new MySqlConnectionManager( - "127.0.0.1", - 5432, + dbMetadata.getEndpoint(), + dbMetadata.getPort(), sourceConfig.getAuthenticationConfig().getUsername(), sourceConfig.getAuthenticationConfig().getPassword(), sourceConfig.isTlsEnabled()); @@ -190,8 +190,8 @@ private SchemaManager getSchemaManager(final RdsSourceConfig sourceConfig, final } // For Postgres final PostgresConnectionManager connectionManager = new PostgresConnectionManager( - "127.0.0.1", - 5432, + dbMetadata.getEndpoint(), + dbMetadata.getPort(), sourceConfig.getAuthenticationConfig().getUsername(), sourceConfig.getAuthenticationConfig().getPassword(), sourceConfig.isTlsEnabled(), @@ -207,8 +207,8 @@ private QueryManager getQueryManager(final RdsSourceConfig sourceConfig, final D final String readerEndpoint = dbMetadata.getReaderEndpoint() != null ? dbMetadata.getReaderEndpoint() : dbMetadata.getEndpoint(); final int readerPort = dbMetadata.getReaderPort() == 0 ? dbMetadata.getPort() : dbMetadata.getReaderPort(); final MySqlConnectionManager readerConnectionManager = new MySqlConnectionManager( - "127.0.0.1", - 5432, + readerEndpoint, + readerPort, sourceConfig.getAuthenticationConfig().getUsername(), sourceConfig.getAuthenticationConfig().getPassword(), sourceConfig.isTlsEnabled()); diff --git a/shared-config/log4j2.properties b/shared-config/log4j2.properties index 0278fc095d..7fbab57e95 100644 --- a/shared-config/log4j2.properties +++ b/shared-config/log4j2.properties @@ -31,12 +31,3 @@ logger.plugins.level = info logger.springframework.name = org.springframework logger.springframework.level = info - -logger.rds.name = org.opensearch.dataprepper.plugins.source.rds -logger.rds.level = debug - -logger.s3source.name = org.opensearch.dataprepper.plugins.source.s3 -logger.s3source.level = warn - -logger.sourcecoord.name = org.opensearch.dataprepper.core.sourcecoordination.LeaseBasedSourceCoordinator -logger.sourcecoord.level = warn From 81c5883b2d80f04abfd0ef5b03c72f64cb64f31a Mon Sep 17 00:00:00 2001 From: Hai Yan Date: Thu, 16 Jan 2025 00:52:12 -0600 Subject: [PATCH 06/10] Address review comments Signed-off-by: Hai Yan --- .../plugins/source/rds/RdsService.java | 32 ++------ .../state/StreamProgressState.java | 38 ++++++++- .../rds/datatype/postgres/ColumnType.java | 10 +++ .../source/rds/leader/LeaderScheduler.java | 6 +- .../plugins/source/rds/model/MessageType.java | 30 +++++++ .../source/rds/schema/ConnectionManager.java | 22 ++++++ .../rds/schema/ConnectionManagerFactory.java | 50 ++++++++++++ .../rds/schema/MySqlConnectionManager.java | 3 +- .../source/rds/schema/MySqlSchemaManager.java | 14 +++- .../rds/schema/PostgresConnectionManager.java | 13 ++- .../rds/schema/PostgresSchemaManager.java | 52 +++++++++++- .../source/rds/schema/SchemaManager.java | 8 ++ .../rds/schema/SchemaManagerFactory.java | 27 +++++++ .../rds/stream/BinlogClientWrapper.java | 12 +++ .../rds/stream/LogicalReplicationClient.java | 12 +++ .../LogicalReplicationEventProcessor.java | 79 +++++++++++++++---- .../rds/stream/ReplicationLogClient.java | 10 +++ .../stream/ReplicationLogClientFactory.java | 5 ++ .../schema/ConnectionManagerFactoryTest.java | 56 +++++++++++++ .../rds/schema/MySqlSchemaManagerTest.java | 4 +- .../schema/PostgresConnectionManagerTest.java | 10 +++ .../rds/schema/PostgresSchemaManagerTest.java | 10 +++ .../rds/schema/SchemaManagerFactoryTest.java | 48 +++++++++++ .../stream/LogicalReplicationClientTest.java | 10 +++ .../LogicalReplicationEventProcessorTest.java | 22 +++++- .../ReplicationLogClientFactoryTest.java | 10 +++ 26 files changed, 535 insertions(+), 58 deletions(-) create mode 100644 data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/model/MessageType.java create mode 100644 data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/schema/ConnectionManager.java create mode 100644 data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/schema/ConnectionManagerFactory.java create mode 100644 data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/schema/SchemaManagerFactory.java create mode 100644 data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/schema/ConnectionManagerFactoryTest.java create mode 100644 data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/schema/SchemaManagerFactoryTest.java diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsService.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsService.java index cdbe1b04d8..106827cb69 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsService.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsService.java @@ -26,12 +26,13 @@ import org.opensearch.dataprepper.plugins.source.rds.model.DbMetadata; import org.opensearch.dataprepper.plugins.source.rds.model.DbTableMetadata; import org.opensearch.dataprepper.plugins.source.rds.resync.ResyncScheduler; +import org.opensearch.dataprepper.plugins.source.rds.schema.ConnectionManager; +import org.opensearch.dataprepper.plugins.source.rds.schema.ConnectionManagerFactory; import org.opensearch.dataprepper.plugins.source.rds.schema.MySqlConnectionManager; -import org.opensearch.dataprepper.plugins.source.rds.schema.QueryManager; import org.opensearch.dataprepper.plugins.source.rds.schema.MySqlSchemaManager; +import org.opensearch.dataprepper.plugins.source.rds.schema.QueryManager; import org.opensearch.dataprepper.plugins.source.rds.schema.SchemaManager; -import org.opensearch.dataprepper.plugins.source.rds.schema.PostgresConnectionManager; -import org.opensearch.dataprepper.plugins.source.rds.schema.PostgresSchemaManager; +import org.opensearch.dataprepper.plugins.source.rds.schema.SchemaManagerFactory; import org.opensearch.dataprepper.plugins.source.rds.stream.ReplicationLogClientFactory; import org.opensearch.dataprepper.plugins.source.rds.stream.StreamScheduler; import org.opensearch.dataprepper.plugins.source.rds.utils.IdentifierShortener; @@ -178,29 +179,8 @@ public void shutdown() { } private SchemaManager getSchemaManager(final RdsSourceConfig sourceConfig, final DbMetadata dbMetadata) { - // For MySQL - if (sourceConfig.getEngine() == EngineType.MYSQL) { - final MySqlConnectionManager connectionManager = new MySqlConnectionManager( - dbMetadata.getEndpoint(), - dbMetadata.getPort(), - sourceConfig.getAuthenticationConfig().getUsername(), - sourceConfig.getAuthenticationConfig().getPassword(), - sourceConfig.isTlsEnabled()); - return new MySqlSchemaManager(connectionManager); - } - // For Postgres - final PostgresConnectionManager connectionManager = new PostgresConnectionManager( - dbMetadata.getEndpoint(), - dbMetadata.getPort(), - sourceConfig.getAuthenticationConfig().getUsername(), - sourceConfig.getAuthenticationConfig().getPassword(), - sourceConfig.isTlsEnabled(), - getDatabaseName(sourceConfig.getTableNames())); - return new PostgresSchemaManager(connectionManager); - } - - private String getDatabaseName(List tableNames) { - return tableNames.get(0).split("\\.")[0]; + final ConnectionManager connectionManager = new ConnectionManagerFactory(sourceConfig, dbMetadata).getConnectionManager(); + return new SchemaManagerFactory(connectionManager).getSchemaManager(); } private QueryManager getQueryManager(final RdsSourceConfig sourceConfig, final DbMetadata dbMetadata) { diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/state/StreamProgressState.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/state/StreamProgressState.java index d9174ef538..80615bdebf 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/state/StreamProgressState.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/state/StreamProgressState.java @@ -10,23 +10,45 @@ import org.opensearch.dataprepper.plugins.source.rds.model.ForeignKeyRelation; import java.util.List; +import java.util.Map; public class StreamProgressState { + // TODO: separate MySQL and Postgres properties into different progress state classes + // Common + @JsonProperty("engineType") + private String engineType; + + @JsonProperty("waitForExport") + private boolean waitForExport = false; + + /** + * Map of table name to primary keys + */ + @JsonProperty("primaryKeyMap") + private Map> primaryKeyMap; + + // For MySQL @JsonProperty("currentPosition") private BinlogCoordinate currentPosition; + @JsonProperty("foreignKeyRelations") + private List foreignKeyRelations; + + // For Postgres @JsonProperty("currentLsn") private String currentLsn; @JsonProperty("replicationSlotName") private String replicationSlotName; - @JsonProperty("waitForExport") - private boolean waitForExport = false; + public String getEngineType() { + return engineType; + } - @JsonProperty("foreignKeyRelations") - private List foreignKeyRelations; + public void setEngineType(String engineType) { + this.engineType = engineType; + } public BinlogCoordinate getCurrentPosition() { return currentPosition; @@ -36,6 +58,14 @@ public String getCurrentLsn() { return currentLsn; } + public Map> getPrimaryKeyMap() { + return primaryKeyMap; + } + + public void setPrimaryKeyMap(Map> primaryKeyMap) { + this.primaryKeyMap = primaryKeyMap; + } + public String getReplicationSlotName() { return replicationSlotName; } diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/datatype/postgres/ColumnType.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/datatype/postgres/ColumnType.java index 0f959cc033..c03e4e67c4 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/datatype/postgres/ColumnType.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/datatype/postgres/ColumnType.java @@ -1,3 +1,13 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + package org.opensearch.dataprepper.plugins.source.rds.datatype.postgres; import java.util.HashMap; diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/leader/LeaderScheduler.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/leader/LeaderScheduler.java index 3f3bf19684..f3beb3e12c 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/leader/LeaderScheduler.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/leader/LeaderScheduler.java @@ -19,8 +19,8 @@ import org.opensearch.dataprepper.plugins.source.rds.model.BinlogCoordinate; import org.opensearch.dataprepper.plugins.source.rds.model.DbTableMetadata; import org.opensearch.dataprepper.plugins.source.rds.schema.MySqlSchemaManager; -import org.opensearch.dataprepper.plugins.source.rds.schema.SchemaManager; import org.opensearch.dataprepper.plugins.source.rds.schema.PostgresSchemaManager; +import org.opensearch.dataprepper.plugins.source.rds.schema.SchemaManager; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -156,13 +156,15 @@ private Map> getPrimaryKeyMap() { return sourceConfig.getTableNames().stream() .collect(Collectors.toMap( fullTableName -> fullTableName, - fullTableName -> ((MySqlSchemaManager)schemaManager).getPrimaryKeys(fullTableName.split("\\.")[0], fullTableName.split("\\.")[1]) + fullTableName -> schemaManager.getPrimaryKeys(fullTableName) )); } private void createStreamPartition(RdsSourceConfig sourceConfig) { final StreamProgressState progressState = new StreamProgressState(); + progressState.setEngineType(sourceConfig.getEngine().toString()); progressState.setWaitForExport(sourceConfig.isExportEnabled()); + progressState.setPrimaryKeyMap(getPrimaryKeyMap()); if (sourceConfig.getEngine() == EngineType.MYSQL) { getCurrentBinlogPosition().ifPresent(progressState::setCurrentPosition); progressState.setForeignKeyRelations(((MySqlSchemaManager)schemaManager).getForeignKeyRelations(sourceConfig.getTableNames())); diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/model/MessageType.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/model/MessageType.java new file mode 100644 index 0000000000..a537835099 --- /dev/null +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/model/MessageType.java @@ -0,0 +1,30 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + +package org.opensearch.dataprepper.plugins.source.rds.model; + +public enum MessageType { + BEGIN('B'), + RELATION('R'), + INSERT('I'), + UPDATE('U'), + DELETE('D'), + COMMIT('C'); + + private final char value; + + MessageType(char value) { + this.value = value; + } + + public char getValue() { + return value; + } +} diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/schema/ConnectionManager.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/schema/ConnectionManager.java new file mode 100644 index 0000000000..dc475d0173 --- /dev/null +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/schema/ConnectionManager.java @@ -0,0 +1,22 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + +package org.opensearch.dataprepper.plugins.source.rds.schema; + +import java.sql.Connection; +import java.sql.SQLException; + +/** + * Interface for managing connections to a database. + */ +public interface ConnectionManager { + + Connection getConnection() throws SQLException; +} diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/schema/ConnectionManagerFactory.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/schema/ConnectionManagerFactory.java new file mode 100644 index 0000000000..4a4dff966e --- /dev/null +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/schema/ConnectionManagerFactory.java @@ -0,0 +1,50 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + +package org.opensearch.dataprepper.plugins.source.rds.schema; + +import org.opensearch.dataprepper.plugins.source.rds.RdsSourceConfig; +import org.opensearch.dataprepper.plugins.source.rds.configuration.EngineType; +import org.opensearch.dataprepper.plugins.source.rds.model.DbMetadata; + +import java.util.List; + +public class ConnectionManagerFactory { + private final RdsSourceConfig sourceConfig; + private final DbMetadata dbMetadata; + + public ConnectionManagerFactory(final RdsSourceConfig sourceConfig, final DbMetadata dbMetadata) { + this.sourceConfig = sourceConfig; + this.dbMetadata = dbMetadata; + } + + public ConnectionManager getConnectionManager() { + if (sourceConfig.getEngine() == EngineType.MYSQL) { + return new MySqlConnectionManager( + dbMetadata.getEndpoint(), + dbMetadata.getPort(), + sourceConfig.getAuthenticationConfig().getUsername(), + sourceConfig.getAuthenticationConfig().getPassword(), + sourceConfig.isTlsEnabled()); + } + + return new PostgresConnectionManager( + dbMetadata.getEndpoint(), + dbMetadata.getPort(), + sourceConfig.getAuthenticationConfig().getUsername(), + sourceConfig.getAuthenticationConfig().getPassword(), + sourceConfig.isTlsEnabled(), + getDatabaseName(sourceConfig.getTableNames())); + } + + private String getDatabaseName(List tableNames) { + return tableNames.get(0).split("\\.")[0]; + } +} diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/schema/MySqlConnectionManager.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/schema/MySqlConnectionManager.java index a76d3fd3bf..6b0ff01ba8 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/schema/MySqlConnectionManager.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/schema/MySqlConnectionManager.java @@ -10,7 +10,7 @@ import java.sql.SQLException; import java.util.Properties; -public class MySqlConnectionManager { +public class MySqlConnectionManager implements ConnectionManager { static final String JDBC_URL_FORMAT = "jdbc:mysql://%s:%d"; static final String USERNAME_KEY = "user"; static final String PASSWORD_KEY = "password"; @@ -33,6 +33,7 @@ public MySqlConnectionManager(String hostName, int port, String username, String this.requireSSL = requireSSL; } + @Override public Connection getConnection() throws SQLException { final Properties props = new Properties(); props.setProperty(USERNAME_KEY, username); diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/schema/MySqlSchemaManager.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/schema/MySqlSchemaManager.java index d36b76a90d..1ca9182b40 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/schema/MySqlSchemaManager.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/schema/MySqlSchemaManager.java @@ -1,6 +1,11 @@ /* * Copyright OpenSearch Contributors * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * */ package org.opensearch.dataprepper.plugins.source.rds.schema; @@ -39,13 +44,16 @@ public class MySqlSchemaManager implements SchemaManager { static final String UPDATE_RULE = "UPDATE_RULE"; static final String DELETE_RULE = "DELETE_RULE"; static final String COLUMN_DEF = "COLUMN_DEF"; - private final MySqlConnectionManager connectionManager; + private final ConnectionManager connectionManager; - public MySqlSchemaManager(MySqlConnectionManager connectionManager) { + public MySqlSchemaManager(ConnectionManager connectionManager) { this.connectionManager = connectionManager; } - public List getPrimaryKeys(final String database, final String table) { + @Override + public List getPrimaryKeys(final String fullTableName) { + final String database = fullTableName.split("\\.")[0]; + final String table = fullTableName.split("\\.")[1]; int retry = 0; while (retry <= NUM_OF_RETRIES) { final List primaryKeys = new ArrayList<>(); diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/schema/PostgresConnectionManager.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/schema/PostgresConnectionManager.java index ce5c6ed8ba..c7b02a0c10 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/schema/PostgresConnectionManager.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/schema/PostgresConnectionManager.java @@ -1,3 +1,13 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + package org.opensearch.dataprepper.plugins.source.rds.schema; import org.postgresql.PGProperty; @@ -9,7 +19,7 @@ import java.sql.SQLException; import java.util.Properties; -public class PostgresConnectionManager { +public class PostgresConnectionManager implements ConnectionManager { private static final Logger LOG = LoggerFactory.getLogger(PostgresConnectionManager.class); public static final String JDBC_URL_FORMAT = "jdbc:postgresql://%s:%d/%s"; @@ -36,6 +46,7 @@ public PostgresConnectionManager(String endpoint, int port, String username, Str this.database = database; } + @Override public Connection getConnection() throws SQLException { final Properties props = new Properties(); PGProperty.USER.set(props, username); diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/schema/PostgresSchemaManager.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/schema/PostgresSchemaManager.java index 1bba798f65..9200240eee 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/schema/PostgresSchemaManager.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/schema/PostgresSchemaManager.java @@ -1,3 +1,13 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + package org.opensearch.dataprepper.plugins.source.rds.schema; import org.postgresql.PGConnection; @@ -7,13 +17,19 @@ import java.sql.Connection; import java.sql.PreparedStatement; +import java.sql.ResultSet; +import java.util.ArrayList; import java.util.List; public class PostgresSchemaManager implements SchemaManager { private static final Logger LOG = LoggerFactory.getLogger(PostgresSchemaManager.class); - private final PostgresConnectionManager connectionManager; + private final ConnectionManager connectionManager; + + static final int NUM_OF_RETRIES = 3; + static final int BACKOFF_IN_MILLIS = 500; + static final String COLUMN_NAME = "COLUMN_NAME"; - public PostgresSchemaManager(PostgresConnectionManager connectionManager) { + public PostgresSchemaManager(ConnectionManager connectionManager) { this.connectionManager = connectionManager; } @@ -56,4 +72,36 @@ public void createLogicalReplicationSlot(final List tableNames, final St LOG.error("Exception when creating replication slot. ", e); } } + + @Override + public List getPrimaryKeys(final String fullTableName) { + final String schema = fullTableName.split("\\.")[0]; + final String table = fullTableName.split("\\.")[1]; + int retry = 0; + while (retry <= NUM_OF_RETRIES) { + final List primaryKeys = new ArrayList<>(); + try (final Connection connection = connectionManager.getConnection()) { + try (final ResultSet rs = connection.getMetaData().getPrimaryKeys(null, schema, table)) { + while (rs.next()) { + primaryKeys.add(rs.getString(COLUMN_NAME)); + } + return primaryKeys; + } + } catch (Exception e) { + LOG.error("Failed to get primary keys for table {}, retrying", table, e); + } + applyBackoff(); + retry++; + } + LOG.warn("Failed to get primary keys for table {}", table); + return List.of(); + } + + private void applyBackoff() { + try { + Thread.sleep(BACKOFF_IN_MILLIS); + } catch (final InterruptedException e){ + Thread.currentThread().interrupt(); + } + } } diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/schema/SchemaManager.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/schema/SchemaManager.java index 9f6b53979d..000a1eea05 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/schema/SchemaManager.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/schema/SchemaManager.java @@ -1,7 +1,15 @@ package org.opensearch.dataprepper.plugins.source.rds.schema; +import java.util.List; + /** * Interface for manager classes that are used to get metadata of a database, such as table schemas */ public interface SchemaManager { + /** + * Get the primary keys for a table + * @param fullTableName The full table name + * @return List of primary keys + */ + List getPrimaryKeys(final String fullTableName); } diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/schema/SchemaManagerFactory.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/schema/SchemaManagerFactory.java new file mode 100644 index 0000000000..c11d427e12 --- /dev/null +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/schema/SchemaManagerFactory.java @@ -0,0 +1,27 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + +package org.opensearch.dataprepper.plugins.source.rds.schema; + +public class SchemaManagerFactory { + private final ConnectionManager connectionManager; + + public SchemaManagerFactory(final ConnectionManager connectionManager) { + this.connectionManager = connectionManager; + } + + public SchemaManager getSchemaManager() { + if (connectionManager instanceof MySqlConnectionManager) { + return new MySqlSchemaManager(connectionManager); + } + + return new PostgresSchemaManager(connectionManager); + } +} diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/BinlogClientWrapper.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/BinlogClientWrapper.java index 20e8613ddf..36d8195106 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/BinlogClientWrapper.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/BinlogClientWrapper.java @@ -1,3 +1,13 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + package org.opensearch.dataprepper.plugins.source.rds.stream; import com.github.shyiko.mysql.binlog.BinaryLogClient; @@ -12,10 +22,12 @@ public BinlogClientWrapper(final BinaryLogClient binlogClient) { this.binlogClient = binlogClient; } + @Override public void connect() throws IOException { binlogClient.connect(); } + @Override public void disconnect() throws IOException { binlogClient.disconnect(); } diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationClient.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationClient.java index ac77704704..230f05c5f6 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationClient.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationClient.java @@ -1,3 +1,13 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + package org.opensearch.dataprepper.plugins.source.rds.stream; import org.opensearch.dataprepper.plugins.source.rds.schema.PostgresConnectionManager; @@ -28,6 +38,7 @@ public LogicalReplicationClient(final PostgresConnectionManager connectionManage this.replicationSlotName = replicationSlotName; } + @Override public void connect() { PGReplicationStream stream; try (Connection conn = connectionManager.getConnection()) { @@ -76,6 +87,7 @@ public void connect() { } } + @Override public void disconnect() { disconnectRequested = true; } diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationEventProcessor.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationEventProcessor.java index 217e665f4e..11c30237d7 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationEventProcessor.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationEventProcessor.java @@ -1,3 +1,13 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + package org.opensearch.dataprepper.plugins.source.rds.stream; import org.opensearch.dataprepper.buffer.common.BufferAccumulator; @@ -9,7 +19,9 @@ import org.opensearch.dataprepper.plugins.source.rds.RdsSourceConfig; import org.opensearch.dataprepper.plugins.source.rds.converter.StreamRecordConverter; import org.opensearch.dataprepper.plugins.source.rds.coordination.partition.StreamPartition; +import org.opensearch.dataprepper.plugins.source.rds.coordination.state.StreamProgressState; import org.opensearch.dataprepper.plugins.source.rds.datatype.postgres.ColumnType; +import org.opensearch.dataprepper.plugins.source.rds.model.MessageType; import org.opensearch.dataprepper.plugins.source.rds.model.TableMetadata; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -59,17 +71,17 @@ public void process(ByteBuffer msg) { // If it's INSERT/UPDATE/DELETE, prepare events // If it's a COMMIT, convert all prepared events and send to buffer char messageType = (char) msg.get(); - if (messageType == 'B') { + if (messageType == MessageType.BEGIN.getValue()) { processBeginMessage(msg); - } else if (messageType == 'R') { + } else if (messageType == MessageType.RELATION.getValue()) { processRelationMessage(msg); - } else if (messageType == 'I') { + } else if (messageType == MessageType.INSERT.getValue()) { processInsertMessage(msg); - } else if (messageType == 'U') { + } else if (messageType == MessageType.UPDATE.getValue()) { processUpdateMessage(msg); - } else if (messageType == 'D') { + } else if (messageType == MessageType.DELETE.getValue()) { processDeleteMessage(msg); - } else if (messageType == 'C') { + } else if (messageType == MessageType.COMMIT.getValue()) { processCommitMessage(msg); } else { throw new IllegalArgumentException("Replication message type [" + messageType + "] is not supported. "); @@ -110,9 +122,9 @@ void processRelationMessage(ByteBuffer msg) { columnNames.add(columnName); } - // TODO: get primary keys in advance + final List primaryKeys = getPrimaryKeys(schemaName + "." + tableName); final TableMetadata tableMetadata = new TableMetadata( - tableName, schemaName, columnNames, List.of("id")); + tableName, schemaName, columnNames, primaryKeys); tableMetadataMap.put((long) tableId, tableMetadata); @@ -127,9 +139,8 @@ void processCommitMessage(ByteBuffer msg) { if (currentLsn != commitLsn) { // This shouldn't happen - LOG.warn("Commit LSN does not match current LSN, skipping"); pipelineEvents.clear(); - return; + throw new RuntimeException("Commit LSN does not match current LSN, skipping"); } writeToBuffer(bufferAccumulator); @@ -150,28 +161,49 @@ void processInsertMessage(ByteBuffer msg) { } void processUpdateMessage(ByteBuffer msg) { - int tableId = msg.getInt(); - char typeId = (char) msg.get(); + final int tableId = msg.getInt(); final TableMetadata tableMetadata = tableMetadataMap.get((long)tableId); final List columnNames = tableMetadata.getColumnNames(); final List primaryKeys = tableMetadata.getPrimaryKeys(); final long eventTimestampMillis = currentEventTimestamp; + char typeId = (char) msg.get(); if (typeId == 'N') { - doProcess(msg, columnNames, tableMetadata, primaryKeys, eventTimestampMillis, OpenSearchBulkActions.INDEX); LOG.debug("Processed an UPDATE message with table id: {}", tableId); } else if (typeId == 'K') { - // TODO + // Primary keys were changed + doProcess(msg, columnNames, tableMetadata, primaryKeys, eventTimestampMillis, OpenSearchBulkActions.DELETE); + msg.get(); // should be a char 'N' + doProcess(msg, columnNames, tableMetadata, primaryKeys, eventTimestampMillis, OpenSearchBulkActions.INDEX); + LOG.debug("Processed an UPDATE message with table id: {} and primary key(s) were changed", tableId); + } else if (typeId == 'O') { - // TODO + // Replica Identity is set to full, containing both old and new row data + Map oldRowDataMap = getRowDataMap(msg, columnNames); + msg.get(); // should be a char 'N' + Map newRowDataMap = getRowDataMap(msg, columnNames); + + if (isPrimaryKeyChanged(oldRowDataMap, newRowDataMap, primaryKeys)) { + createPipelineEvent(oldRowDataMap, tableMetadata, primaryKeys, eventTimestampMillis, OpenSearchBulkActions.DELETE); + } + createPipelineEvent(newRowDataMap, tableMetadata, primaryKeys, eventTimestampMillis, OpenSearchBulkActions.INDEX); + } + } + + private boolean isPrimaryKeyChanged(Map oldRowDataMap, Map newRowDataMap, List primaryKeys) { + for (String primaryKey : primaryKeys) { + if (!oldRowDataMap.get(primaryKey).equals(newRowDataMap.get(primaryKey))) { + return true; + } } + return false; } void processDeleteMessage(ByteBuffer msg) { int tableId = msg.getInt(); - char typeId = (char) msg.get(); + char n_char = (char) msg.get(); // Skip the 'N' character final TableMetadata tableMetadata = tableMetadataMap.get((long)tableId); final List columnNames = tableMetadata.getColumnNames(); @@ -184,6 +216,12 @@ void processDeleteMessage(ByteBuffer msg) { private void doProcess(ByteBuffer msg, List columnNames, TableMetadata tableMetadata, List primaryKeys, long eventTimestampMillis, OpenSearchBulkActions bulkAction) { + Map rowDataMap = getRowDataMap(msg, columnNames); + + createPipelineEvent(rowDataMap, tableMetadata, primaryKeys, eventTimestampMillis, bulkAction); + } + + private Map getRowDataMap(ByteBuffer msg, List columnNames) { Map rowDataMap = new HashMap<>(); short numberOfColumns = msg.getShort(); for (int i = 0; i < numberOfColumns; i++) { @@ -199,7 +237,10 @@ private void doProcess(ByteBuffer msg, List columnNames, TableMetadata t LOG.warn("Unknown column type: {}", type); } } + return rowDataMap; + } + private void createPipelineEvent(Map rowDataMap, TableMetadata tableMetadata, List primaryKeys, long eventTimestampMillis, OpenSearchBulkActions bulkAction) { final Event dataPrepperEvent = JacksonEvent.builder() .withEventType("event") .withData(rowDataMap) @@ -259,4 +300,10 @@ private String getNullTerminatedString(ByteBuffer msg) { } return sb.toString(); } + + private List getPrimaryKeys(String fullTableName) { + StreamProgressState progressState = streamPartition.getProgressState().get(); + + return progressState.getPrimaryKeyMap().get(fullTableName); + } } diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/ReplicationLogClient.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/ReplicationLogClient.java index 29d20f5913..bef9064b9c 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/ReplicationLogClient.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/ReplicationLogClient.java @@ -1,3 +1,13 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + package org.opensearch.dataprepper.plugins.source.rds.stream; import java.io.IOException; diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/ReplicationLogClientFactory.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/ReplicationLogClientFactory.java index d1441746fe..4b0a2c209a 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/ReplicationLogClientFactory.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/ReplicationLogClientFactory.java @@ -1,6 +1,11 @@ /* * Copyright OpenSearch Contributors * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * */ package org.opensearch.dataprepper.plugins.source.rds.stream; diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/schema/ConnectionManagerFactoryTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/schema/ConnectionManagerFactoryTest.java new file mode 100644 index 0000000000..91d76fef04 --- /dev/null +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/schema/ConnectionManagerFactoryTest.java @@ -0,0 +1,56 @@ +package org.opensearch.dataprepper.plugins.source.rds.schema; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Answers; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.dataprepper.plugins.source.rds.RdsSourceConfig; +import org.opensearch.dataprepper.plugins.source.rds.configuration.EngineType; +import org.opensearch.dataprepper.plugins.source.rds.model.DbMetadata; + +import java.util.List; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.instanceOf; +import static org.hamcrest.Matchers.notNullValue; +import static org.mockito.Mockito.when; + +@ExtendWith(MockitoExtension.class) +class ConnectionManagerFactoryTest { + + @Mock(answer = Answers.RETURNS_DEEP_STUBS) + private RdsSourceConfig sourceConfig; + + @Mock + private DbMetadata dbMetadata; + + private ConnectionManagerFactory connectionManagerFactory; + + @BeforeEach + void setUp() { + connectionManagerFactory = createObjectUnderTest(); + } + + @Test + void test_getConnectionManager_for_mysql() { + when(sourceConfig.getEngine()).thenReturn(EngineType.MYSQL); + final ConnectionManager connectionManager = connectionManagerFactory.getConnectionManager(); + assertThat(connectionManager, notNullValue()); + assertThat(connectionManager, instanceOf(MySqlConnectionManager.class)); + } + + @Test + void test_getConnectionManager_for_postgres() { + when(sourceConfig.getEngine()).thenReturn(EngineType.POSTGRES); + when(sourceConfig.getTableNames()).thenReturn(List.of("schema1.table1", "schema1.table2")); + final ConnectionManager connectionManager = connectionManagerFactory.getConnectionManager(); + assertThat(connectionManager, notNullValue()); + assertThat(connectionManager, instanceOf(PostgresConnectionManager.class)); + } + + private ConnectionManagerFactory createObjectUnderTest() { + return new ConnectionManagerFactory(sourceConfig, dbMetadata); + } +} diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/schema/MySqlSchemaManagerTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/schema/MySqlSchemaManagerTest.java index c491dcfecb..3856cb3f00 100644 --- a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/schema/MySqlSchemaManagerTest.java +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/schema/MySqlSchemaManagerTest.java @@ -81,7 +81,7 @@ void test_getPrimaryKeys_returns_primary_keys() throws SQLException { when(resultSet.next()).thenReturn(true, false); when(resultSet.getString(COLUMN_NAME)).thenReturn(primaryKey); - final List primaryKeys = schemaManager.getPrimaryKeys(databaseName, tableName); + final List primaryKeys = schemaManager.getPrimaryKeys(databaseName + "." + tableName); assertThat(primaryKeys, contains(primaryKey)); } @@ -92,7 +92,7 @@ void test_getPrimaryKeys_throws_exception_then_returns_empty_list() throws SQLEx final String tableName = UUID.randomUUID().toString(); when(connectionManager.getConnection()).thenThrow(SQLException.class); - final List primaryKeys = schemaManager.getPrimaryKeys(databaseName, tableName); + final List primaryKeys = schemaManager.getPrimaryKeys(databaseName + "." + tableName); assertThat(primaryKeys, empty()); } diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/schema/PostgresConnectionManagerTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/schema/PostgresConnectionManagerTest.java index e45f46ad58..e66e830684 100644 --- a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/schema/PostgresConnectionManagerTest.java +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/schema/PostgresConnectionManagerTest.java @@ -1,3 +1,13 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + package org.opensearch.dataprepper.plugins.source.rds.schema; import org.junit.jupiter.api.BeforeEach; diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/schema/PostgresSchemaManagerTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/schema/PostgresSchemaManagerTest.java index dddeb3f3b9..0602b2c6d5 100644 --- a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/schema/PostgresSchemaManagerTest.java +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/schema/PostgresSchemaManagerTest.java @@ -1,3 +1,13 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + package org.opensearch.dataprepper.plugins.source.rds.schema; import org.junit.jupiter.api.BeforeEach; diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/schema/SchemaManagerFactoryTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/schema/SchemaManagerFactoryTest.java new file mode 100644 index 0000000000..e98752ceeb --- /dev/null +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/schema/SchemaManagerFactoryTest.java @@ -0,0 +1,48 @@ +package org.opensearch.dataprepper.plugins.source.rds.schema; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.instanceOf; + + +@ExtendWith(MockitoExtension.class) +class SchemaManagerFactoryTest { + + @Mock + private MySqlConnectionManager mySqlConnectionManager; + + @Mock + private PostgresConnectionManager postgresConnectionManager; + + private SchemaManagerFactory schemaManagerFactory; + private ConnectionManager connectionManager; + + @BeforeEach + void setUp() { + } + + @Test + void test_getSchemaManager_for_mysql() { + connectionManager = mySqlConnectionManager; + schemaManagerFactory = createObjectUnderTest(); + + assertThat(schemaManagerFactory.getSchemaManager(), instanceOf(MySqlSchemaManager.class)); + } + + @Test + void test_getSchemaManager_for_postgres() { + connectionManager = postgresConnectionManager; + schemaManagerFactory = createObjectUnderTest(); + + assertThat(schemaManagerFactory.getSchemaManager(), instanceOf(PostgresSchemaManager.class)); + } + + private SchemaManagerFactory createObjectUnderTest() { + return new SchemaManagerFactory(connectionManager); + } +} diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationClientTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationClientTest.java index b3e0bf77f8..5089f0cf67 100644 --- a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationClientTest.java +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationClientTest.java @@ -1,3 +1,13 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + package org.opensearch.dataprepper.plugins.source.rds.stream; import org.junit.jupiter.api.BeforeEach; diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationEventProcessorTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationEventProcessorTest.java index a54062e2b1..6c5c025926 100644 --- a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationEventProcessorTest.java +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationEventProcessorTest.java @@ -1,8 +1,19 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + package org.opensearch.dataprepper.plugins.source.rds.stream; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Answers; import org.mockito.Mock; import org.mockito.junit.jupiter.MockitoExtension; import org.opensearch.dataprepper.model.buffer.Buffer; @@ -10,12 +21,17 @@ import org.opensearch.dataprepper.model.record.Record; import org.opensearch.dataprepper.plugins.source.rds.RdsSourceConfig; import org.opensearch.dataprepper.plugins.source.rds.coordination.partition.StreamPartition; +import org.opensearch.dataprepper.plugins.source.rds.coordination.state.StreamProgressState; import java.nio.ByteBuffer; +import java.util.List; +import java.util.Map; +import java.util.Optional; import java.util.UUID; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.mockito.Mockito.doNothing; +import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; @@ -23,7 +39,7 @@ @ExtendWith(MockitoExtension.class) class LogicalReplicationEventProcessorTest { - @Mock + @Mock(answer = Answers.RETURNS_DEEP_STUBS) private StreamPartition streamPartition; @Mock @@ -42,6 +58,7 @@ class LogicalReplicationEventProcessorTest { @BeforeEach void setUp() { s3Prefix = UUID.randomUUID().toString(); + objectUnderTest = spy(createObjectUnderTest()); } @@ -57,6 +74,9 @@ void test_correct_process_method_invoked_for_begin_message() { @Test void test_correct_process_method_invoked_for_relation_message() { when(message.get()).thenReturn((byte) 'R'); + final StreamProgressState progressState = mock(StreamProgressState.class); + when(streamPartition.getProgressState()).thenReturn(Optional.of(progressState)); + when(progressState.getPrimaryKeyMap()).thenReturn(Map.of(".", List.of("key1", "key2"))); objectUnderTest.process(message); diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/ReplicationLogClientFactoryTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/ReplicationLogClientFactoryTest.java index 3f7b41a20e..84a94ff1a5 100644 --- a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/ReplicationLogClientFactoryTest.java +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/ReplicationLogClientFactoryTest.java @@ -1,3 +1,13 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + package org.opensearch.dataprepper.plugins.source.rds.stream; import org.junit.jupiter.api.Test; From a3bec6f7a6cb63b4862f04ea99eb385ce97be515 Mon Sep 17 00:00:00 2001 From: Hai Yan Date: Fri, 17 Jan 2025 00:19:29 -0600 Subject: [PATCH 07/10] Address minor issues Signed-off-by: Hai Yan --- .../source/rds/schema/PostgresSchemaManager.java | 8 +++++--- .../source/rds/stream/LogicalReplicationClient.java | 6 +++--- .../rds/stream/LogicalReplicationEventProcessor.java | 9 ++++++--- .../rds/stream/ReplicationLogClientFactory.java | 12 ++++-------- .../stream/LogicalReplicationEventProcessorTest.java | 3 ++- .../rds/stream/ReplicationLogClientFactoryTest.java | 2 +- 6 files changed, 21 insertions(+), 19 deletions(-) diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/schema/PostgresSchemaManager.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/schema/PostgresSchemaManager.java index 9200240eee..dcd604f8a8 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/schema/PostgresSchemaManager.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/schema/PostgresSchemaManager.java @@ -75,13 +75,15 @@ public void createLogicalReplicationSlot(final List tableNames, final St @Override public List getPrimaryKeys(final String fullTableName) { - final String schema = fullTableName.split("\\.")[0]; - final String table = fullTableName.split("\\.")[1]; + final String[] splits = fullTableName.split("\\."); + final String database = splits[0]; + final String schema = splits[1]; + final String table = splits[2]; int retry = 0; while (retry <= NUM_OF_RETRIES) { final List primaryKeys = new ArrayList<>(); try (final Connection connection = connectionManager.getConnection()) { - try (final ResultSet rs = connection.getMetaData().getPrimaryKeys(null, schema, table)) { + try (final ResultSet rs = connection.getMetaData().getPrimaryKeys(database, schema, table)) { while (rs.next()) { primaryKeys.add(rs.getString(COLUMN_NAME)); } diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationClient.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationClient.java index 230f05c5f6..cc83fdd232 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationClient.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationClient.java @@ -10,7 +10,7 @@ package org.opensearch.dataprepper.plugins.source.rds.stream; -import org.opensearch.dataprepper.plugins.source.rds.schema.PostgresConnectionManager; +import org.opensearch.dataprepper.plugins.source.rds.schema.ConnectionManager; import org.postgresql.PGConnection; import org.postgresql.replication.LogSequenceNumber; import org.postgresql.replication.PGReplicationStream; @@ -25,14 +25,14 @@ public class LogicalReplicationClient implements ReplicationLogClient { private static final Logger LOG = LoggerFactory.getLogger(LogicalReplicationClient.class); - private final PostgresConnectionManager connectionManager; + private final ConnectionManager connectionManager; private final String replicationSlotName; private LogSequenceNumber startLsn; private LogicalReplicationEventProcessor eventProcessor; private volatile boolean disconnectRequested = false; - public LogicalReplicationClient(final PostgresConnectionManager connectionManager, + public LogicalReplicationClient(final ConnectionManager connectionManager, final String replicationSlotName) { this.connectionManager = connectionManager; this.replicationSlotName = replicationSlotName; diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationEventProcessor.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationEventProcessor.java index 11c30237d7..a48e468586 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationEventProcessor.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationEventProcessor.java @@ -41,6 +41,7 @@ public class LogicalReplicationEventProcessor { static final int DEFAULT_BUFFER_BATCH_SIZE = 1_000; private final StreamPartition streamPartition; + private final RdsSourceConfig sourceConfig; private final StreamRecordConverter recordConverter; private final Buffer> buffer; private final BufferAccumulator> bufferAccumulator; @@ -56,6 +57,7 @@ public LogicalReplicationEventProcessor(final StreamPartition streamPartition, final Buffer> buffer, final String s3Prefix) { this.streamPartition = streamPartition; + this.sourceConfig = sourceConfig; recordConverter = new StreamRecordConverter(s3Prefix, sourceConfig.getPartitionCount()); this.buffer = buffer; bufferAccumulator = BufferAccumulator.create(buffer, DEFAULT_BUFFER_BATCH_SIZE, BUFFER_TIMEOUT); @@ -122,7 +124,7 @@ void processRelationMessage(ByteBuffer msg) { columnNames.add(columnName); } - final List primaryKeys = getPrimaryKeys(schemaName + "." + tableName); + final List primaryKeys = getPrimaryKeys(schemaName, tableName); final TableMetadata tableMetadata = new TableMetadata( tableName, schemaName, columnNames, primaryKeys); @@ -301,9 +303,10 @@ private String getNullTerminatedString(ByteBuffer msg) { return sb.toString(); } - private List getPrimaryKeys(String fullTableName) { + private List getPrimaryKeys(String schemaName, String tableName) { + final String databaseName = sourceConfig.getTableNames().get(0).split("\\.")[0]; StreamProgressState progressState = streamPartition.getProgressState().get(); - return progressState.getPrimaryKeyMap().get(fullTableName); + return progressState.getPrimaryKeyMap().get(databaseName + "." + schemaName + "." + tableName); } } diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/ReplicationLogClientFactory.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/ReplicationLogClientFactory.java index 4b0a2c209a..d9bc54570a 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/ReplicationLogClientFactory.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/ReplicationLogClientFactory.java @@ -17,7 +17,8 @@ import org.opensearch.dataprepper.plugins.source.rds.configuration.EngineType; import org.opensearch.dataprepper.plugins.source.rds.coordination.partition.StreamPartition; import org.opensearch.dataprepper.plugins.source.rds.model.DbMetadata; -import org.opensearch.dataprepper.plugins.source.rds.schema.PostgresConnectionManager; +import org.opensearch.dataprepper.plugins.source.rds.schema.ConnectionManager; +import org.opensearch.dataprepper.plugins.source.rds.schema.ConnectionManagerFactory; import software.amazon.awssdk.services.rds.RdsClient; import java.util.List; @@ -70,13 +71,8 @@ private LogicalReplicationClient createLogicalReplicationClient(StreamPartition if (replicationSlotName == null) { throw new NoSuchElementException("Replication slot name is not found in progress state."); } - final PostgresConnectionManager connectionManager = new PostgresConnectionManager( - dbMetadata.getEndpoint(), - dbMetadata.getPort(), - username, - password, - !sourceConfig.getTlsConfig().isInsecure(), - getDatabaseName(sourceConfig.getTableNames())); + final ConnectionManagerFactory connectionManagerFactory = new ConnectionManagerFactory(sourceConfig, dbMetadata); + final ConnectionManager connectionManager = connectionManagerFactory.getConnectionManager(); return new LogicalReplicationClient(connectionManager, replicationSlotName); } diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationEventProcessorTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationEventProcessorTest.java index 6c5c025926..22614e4f02 100644 --- a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationEventProcessorTest.java +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationEventProcessorTest.java @@ -76,7 +76,8 @@ void test_correct_process_method_invoked_for_relation_message() { when(message.get()).thenReturn((byte) 'R'); final StreamProgressState progressState = mock(StreamProgressState.class); when(streamPartition.getProgressState()).thenReturn(Optional.of(progressState)); - when(progressState.getPrimaryKeyMap()).thenReturn(Map.of(".", List.of("key1", "key2"))); + when(sourceConfig.getTableNames()).thenReturn(List.of("database.schema.table1")); + when(progressState.getPrimaryKeyMap()).thenReturn(Map.of("database.schema.table1", List.of("key1", "key2"))); objectUnderTest.process(message); diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/ReplicationLogClientFactoryTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/ReplicationLogClientFactoryTest.java index 84a94ff1a5..43978eaea4 100644 --- a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/ReplicationLogClientFactoryTest.java +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/ReplicationLogClientFactoryTest.java @@ -75,7 +75,7 @@ void test_create_logical_replication_client() { final List tableNames = List.of("table1", "table2"); when(sourceConfig.getEngine()).thenReturn(EngineType.POSTGRES); - when(sourceConfig.getTlsConfig().isInsecure()).thenReturn(false); + when(sourceConfig.isTlsEnabled()).thenReturn(true); when(sourceConfig.getTableNames()).thenReturn(tableNames); when(sourceConfig.getAuthenticationConfig().getUsername()).thenReturn(username); when(sourceConfig.getAuthenticationConfig().getPassword()).thenReturn(password); From 00abf6868b0466d66942229c782982224601d251 Mon Sep 17 00:00:00 2001 From: Hai Yan Date: Tue, 21 Jan 2025 23:56:00 -0600 Subject: [PATCH 08/10] Group MySQL and Postgres stream states Signed-off-by: Hai Yan --- .../coordination/state/MySqlStreamState.java | 41 ++++++++++++++ .../state/PostgresStreamState.java | 37 ++++++++++++ .../state/StreamProgressState.java | 56 ++++++------------- .../source/rds/leader/LeaderScheduler.java | 8 ++- .../rds/resync/CascadingActionDetector.java | 2 +- .../stream/ReplicationLogClientFactory.java | 2 +- .../source/rds/stream/StreamCheckpointer.java | 2 +- .../source/rds/stream/StreamWorker.java | 4 +- .../resync/CascadingActionDetectorTest.java | 5 +- .../ReplicationLogClientFactoryTest.java | 5 +- .../rds/stream/StreamCheckpointerTest.java | 7 ++- .../source/rds/stream/StreamWorkerTest.java | 11 +++- 12 files changed, 126 insertions(+), 54 deletions(-) create mode 100644 data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/state/MySqlStreamState.java create mode 100644 data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/state/PostgresStreamState.java diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/state/MySqlStreamState.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/state/MySqlStreamState.java new file mode 100644 index 0000000000..a42d45f073 --- /dev/null +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/state/MySqlStreamState.java @@ -0,0 +1,41 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.dataprepper.plugins.source.rds.coordination.state; + +import com.fasterxml.jackson.annotation.JsonProperty; +import org.opensearch.dataprepper.plugins.source.rds.model.BinlogCoordinate; +import org.opensearch.dataprepper.plugins.source.rds.model.ForeignKeyRelation; + +import java.util.List; + +public class MySqlStreamState { + + @JsonProperty("currentPosition") + private BinlogCoordinate currentPosition; + + @JsonProperty("foreignKeyRelations") + private List foreignKeyRelations; + + public BinlogCoordinate getCurrentPosition() { + return currentPosition; + } + + public void setCurrentPosition(BinlogCoordinate currentPosition) { + this.currentPosition = currentPosition; + } + + public List getForeignKeyRelations() { + return foreignKeyRelations; + } + + public void setForeignKeyRelations(List foreignKeyRelations) { + this.foreignKeyRelations = foreignKeyRelations; + } +} diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/state/PostgresStreamState.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/state/PostgresStreamState.java new file mode 100644 index 0000000000..011b1d37de --- /dev/null +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/state/PostgresStreamState.java @@ -0,0 +1,37 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.dataprepper.plugins.source.rds.coordination.state; + +import com.fasterxml.jackson.annotation.JsonProperty; + +public class PostgresStreamState { + + @JsonProperty("currentLsn") + private String currentLsn; + + @JsonProperty("replicationSlotName") + private String replicationSlotName; + + public String getCurrentLsn() { + return currentLsn; + } + + public void setCurrentLsn(String currentLsn) { + this.currentLsn = currentLsn; + } + + public String getReplicationSlotName() { + return replicationSlotName; + } + + public void setReplicationSlotName(String replicationSlotName) { + this.replicationSlotName = replicationSlotName; + } +} diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/state/StreamProgressState.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/state/StreamProgressState.java index 80615bdebf..10423cb15b 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/state/StreamProgressState.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/state/StreamProgressState.java @@ -6,16 +6,12 @@ package org.opensearch.dataprepper.plugins.source.rds.coordination.state; import com.fasterxml.jackson.annotation.JsonProperty; -import org.opensearch.dataprepper.plugins.source.rds.model.BinlogCoordinate; -import org.opensearch.dataprepper.plugins.source.rds.model.ForeignKeyRelation; import java.util.List; import java.util.Map; public class StreamProgressState { - // TODO: separate MySQL and Postgres properties into different progress state classes - // Common @JsonProperty("engineType") private String engineType; @@ -28,19 +24,11 @@ public class StreamProgressState { @JsonProperty("primaryKeyMap") private Map> primaryKeyMap; - // For MySQL - @JsonProperty("currentPosition") - private BinlogCoordinate currentPosition; + @JsonProperty("mySqlStreamState") + private MySqlStreamState mySqlStreamState; - @JsonProperty("foreignKeyRelations") - private List foreignKeyRelations; - - // For Postgres - @JsonProperty("currentLsn") - private String currentLsn; - - @JsonProperty("replicationSlotName") - private String replicationSlotName; + @JsonProperty("postgresStreamState") + private PostgresStreamState postgresStreamState; public String getEngineType() { return engineType; @@ -50,14 +38,6 @@ public void setEngineType(String engineType) { this.engineType = engineType; } - public BinlogCoordinate getCurrentPosition() { - return currentPosition; - } - - public String getCurrentLsn() { - return currentLsn; - } - public Map> getPrimaryKeyMap() { return primaryKeyMap; } @@ -66,18 +46,6 @@ public void setPrimaryKeyMap(Map> primaryKeyMap) { this.primaryKeyMap = primaryKeyMap; } - public String getReplicationSlotName() { - return replicationSlotName; - } - - public void setCurrentPosition(BinlogCoordinate currentPosition) { - this.currentPosition = currentPosition; - } - - public void setReplicationSlotName(String replicationSlotName) { - this.replicationSlotName = replicationSlotName; - } - public boolean shouldWaitForExport() { return waitForExport; } @@ -86,11 +54,19 @@ public void setWaitForExport(boolean waitForExport) { this.waitForExport = waitForExport; } - public List getForeignKeyRelations() { - return foreignKeyRelations; + public MySqlStreamState getMySqlStreamState() { + return mySqlStreamState; + } + + public void setMySqlStreamState(MySqlStreamState mySqlStreamState) { + this.mySqlStreamState = mySqlStreamState; + } + + public PostgresStreamState getPostgresStreamState() { + return postgresStreamState; } - public void setForeignKeyRelations(List foreignKeyRelations) { - this.foreignKeyRelations = foreignKeyRelations; + public void setPostgresStreamState(PostgresStreamState postgresStreamState) { + this.postgresStreamState = postgresStreamState; } } diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/leader/LeaderScheduler.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/leader/LeaderScheduler.java index f3beb3e12c..31ac120ce7 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/leader/LeaderScheduler.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/leader/LeaderScheduler.java @@ -15,6 +15,7 @@ import org.opensearch.dataprepper.plugins.source.rds.coordination.partition.StreamPartition; import org.opensearch.dataprepper.plugins.source.rds.coordination.state.ExportProgressState; import org.opensearch.dataprepper.plugins.source.rds.coordination.state.LeaderProgressState; +import org.opensearch.dataprepper.plugins.source.rds.coordination.state.MySqlStreamState; import org.opensearch.dataprepper.plugins.source.rds.coordination.state.StreamProgressState; import org.opensearch.dataprepper.plugins.source.rds.model.BinlogCoordinate; import org.opensearch.dataprepper.plugins.source.rds.model.DbTableMetadata; @@ -166,15 +167,16 @@ private void createStreamPartition(RdsSourceConfig sourceConfig) { progressState.setWaitForExport(sourceConfig.isExportEnabled()); progressState.setPrimaryKeyMap(getPrimaryKeyMap()); if (sourceConfig.getEngine() == EngineType.MYSQL) { - getCurrentBinlogPosition().ifPresent(progressState::setCurrentPosition); - progressState.setForeignKeyRelations(((MySqlSchemaManager)schemaManager).getForeignKeyRelations(sourceConfig.getTableNames())); + final MySqlStreamState mySqlStreamState = progressState.getMySqlStreamState(); + getCurrentBinlogPosition().ifPresent(mySqlStreamState::setCurrentPosition); + mySqlStreamState.setForeignKeyRelations(((MySqlSchemaManager)schemaManager).getForeignKeyRelations(sourceConfig.getTableNames())); } else { // Postgres // Create replication slot, which will mark the starting point for stream final String publicationName = generatePublicationName(); final String slotName = generateReplicationSlotName(); ((PostgresSchemaManager)schemaManager).createLogicalReplicationSlot(sourceConfig.getTableNames(), publicationName, slotName); - progressState.setReplicationSlotName(slotName); + progressState.getPostgresStreamState().setReplicationSlotName(slotName); } StreamPartition streamPartition = new StreamPartition(sourceConfig.getDbIdentifier(), progressState); sourceCoordinator.createPartition(streamPartition); diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/resync/CascadingActionDetector.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/resync/CascadingActionDetector.java index 98df985ccb..b8e5a21662 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/resync/CascadingActionDetector.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/resync/CascadingActionDetector.java @@ -50,7 +50,7 @@ public Map getParentTableMap(StreamPartition streamPartitio return parentTableMap; } - List foreignKeyRelations = streamPartition.getProgressState().get().getForeignKeyRelations();; + List foreignKeyRelations = streamPartition.getProgressState().get().getMySqlStreamState().getForeignKeyRelations();; for (ForeignKeyRelation foreignKeyRelation : foreignKeyRelations) { if (!ForeignKeyRelation.containsCascadingAction(foreignKeyRelation)) { diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/ReplicationLogClientFactory.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/ReplicationLogClientFactory.java index d9bc54570a..183ce05299 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/ReplicationLogClientFactory.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/ReplicationLogClientFactory.java @@ -67,7 +67,7 @@ private BinaryLogClient createBinaryLogClient() { } private LogicalReplicationClient createLogicalReplicationClient(StreamPartition streamPartition) { - final String replicationSlotName = streamPartition.getProgressState().get().getReplicationSlotName(); + final String replicationSlotName = streamPartition.getProgressState().get().getPostgresStreamState().getReplicationSlotName(); if (replicationSlotName == null) { throw new NoSuchElementException("Replication slot name is not found in progress state."); } diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamCheckpointer.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamCheckpointer.java index b76dbab7c9..1f60f9715f 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamCheckpointer.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamCheckpointer.java @@ -41,7 +41,7 @@ public StreamCheckpointer(final EnhancedSourceCoordinator sourceCoordinator, public void checkpoint(final BinlogCoordinate binlogCoordinate) { LOG.debug("Checkpointing stream partition {} with binlog coordinate {}", streamPartition.getPartitionKey(), binlogCoordinate); Optional progressState = streamPartition.getProgressState(); - progressState.get().setCurrentPosition(binlogCoordinate); + progressState.get().getMySqlStreamState().setCurrentPosition(binlogCoordinate); sourceCoordinator.saveProgressStateForPartition(streamPartition, CHECKPOINT_OWNERSHIP_TIMEOUT_INCREASE); checkpointCounter.increment(); } diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamWorker.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamWorker.java index d6404a42ef..4da8798c90 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamWorker.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamWorker.java @@ -88,7 +88,7 @@ private boolean isExportDone(StreamPartition streamPartition) { } private void setStartBinlogPosition(final StreamPartition streamPartition) { - final BinlogCoordinate startBinlogPosition = streamPartition.getProgressState().get().getCurrentPosition(); + final BinlogCoordinate startBinlogPosition = streamPartition.getProgressState().get().getMySqlStreamState().getCurrentPosition(); // set start of binlog stream to current position if exists if (startBinlogPosition != null) { @@ -102,7 +102,7 @@ private void setStartBinlogPosition(final StreamPartition streamPartition) { } private void setStartLsn(final StreamPartition streamPartition) { - final String startLsn = streamPartition.getProgressState().get().getCurrentLsn(); + final String startLsn = streamPartition.getProgressState().get().getPostgresStreamState().getCurrentLsn(); if (startLsn != null) { LOG.debug("Will start logical replication from LSN {}", startLsn); diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/resync/CascadingActionDetectorTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/resync/CascadingActionDetectorTest.java index c17e003782..1bac04249d 100644 --- a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/resync/CascadingActionDetectorTest.java +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/resync/CascadingActionDetectorTest.java @@ -17,6 +17,7 @@ import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourceCoordinator; import org.opensearch.dataprepper.plugins.source.rds.coordination.partition.ResyncPartition; import org.opensearch.dataprepper.plugins.source.rds.coordination.partition.StreamPartition; +import org.opensearch.dataprepper.plugins.source.rds.coordination.state.MySqlStreamState; import org.opensearch.dataprepper.plugins.source.rds.coordination.state.ResyncProgressState; import org.opensearch.dataprepper.plugins.source.rds.coordination.state.StreamProgressState; import org.opensearch.dataprepper.plugins.source.rds.model.ForeignKeyAction; @@ -75,8 +76,10 @@ void testGetParentTableMap_returns_empty_list_if_stream_progress_state_is_empty( @Test void testGetParentTableMap_returns_only_foreign_relations_with_cascading_actions() { final StreamProgressState progressState = mock(StreamProgressState.class); + final MySqlStreamState mySqlStreamState = mock(MySqlStreamState.class); when(streamPartition.getProgressState()).thenReturn(Optional.of(progressState)); - when(progressState.getForeignKeyRelations()).thenReturn(List.of(foreignKeyRelationWithCascading, foreignKeyRelationWithoutCascading)); + when(progressState.getMySqlStreamState()).thenReturn(mySqlStreamState); + when(mySqlStreamState.getForeignKeyRelations()).thenReturn(List.of(foreignKeyRelationWithCascading, foreignKeyRelationWithoutCascading)); Map actualParentTableMap = objectUnderTest.getParentTableMap(streamPartition); diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/ReplicationLogClientFactoryTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/ReplicationLogClientFactoryTest.java index 43978eaea4..83dafe9887 100644 --- a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/ReplicationLogClientFactoryTest.java +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/ReplicationLogClientFactoryTest.java @@ -18,6 +18,7 @@ import org.opensearch.dataprepper.plugins.source.rds.RdsSourceConfig; import org.opensearch.dataprepper.plugins.source.rds.configuration.EngineType; import org.opensearch.dataprepper.plugins.source.rds.coordination.partition.StreamPartition; +import org.opensearch.dataprepper.plugins.source.rds.coordination.state.PostgresStreamState; import org.opensearch.dataprepper.plugins.source.rds.coordination.state.StreamProgressState; import org.opensearch.dataprepper.plugins.source.rds.model.DbMetadata; import software.amazon.awssdk.services.rds.RdsClient; @@ -71,6 +72,7 @@ void test_create_logical_replication_client() { final String username = UUID.randomUUID().toString(); final String password = UUID.randomUUID().toString(); final StreamProgressState streamProgressState = mock(StreamProgressState.class); + final PostgresStreamState postgresStreamState = mock(PostgresStreamState.class); final String slotName = UUID.randomUUID().toString(); final List tableNames = List.of("table1", "table2"); @@ -80,7 +82,8 @@ void test_create_logical_replication_client() { when(sourceConfig.getAuthenticationConfig().getUsername()).thenReturn(username); when(sourceConfig.getAuthenticationConfig().getPassword()).thenReturn(password); when(streamPartition.getProgressState()).thenReturn(Optional.of(streamProgressState)); - when(streamProgressState.getReplicationSlotName()).thenReturn(slotName); + when(streamProgressState.getPostgresStreamState()).thenReturn(postgresStreamState); + when(postgresStreamState.getReplicationSlotName()).thenReturn(slotName); replicationLogClientFactory = createObjectUnderTest(); ReplicationLogClient replicationLogClient = replicationLogClientFactory.create(streamPartition); diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamCheckpointerTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamCheckpointerTest.java index 2fdac1065f..3327e847f5 100644 --- a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamCheckpointerTest.java +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamCheckpointerTest.java @@ -14,6 +14,7 @@ import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourceCoordinator; import org.opensearch.dataprepper.plugins.source.rds.coordination.partition.StreamPartition; +import org.opensearch.dataprepper.plugins.source.rds.coordination.state.MySqlStreamState; import org.opensearch.dataprepper.plugins.source.rds.coordination.state.StreamProgressState; import org.opensearch.dataprepper.plugins.source.rds.model.BinlogCoordinate; @@ -35,6 +36,9 @@ class StreamCheckpointerTest { @Mock private StreamPartition streamPartition; + @Mock + private MySqlStreamState mySqlStreamState; + @Mock private PluginMetrics pluginMetrics; @@ -55,10 +59,11 @@ void test_checkpoint() { final BinlogCoordinate binlogCoordinate = mock(BinlogCoordinate.class); final StreamProgressState streamProgressState = mock(StreamProgressState.class); when(streamPartition.getProgressState()).thenReturn(Optional.of(streamProgressState)); + when(streamProgressState.getMySqlStreamState()).thenReturn(mySqlStreamState); streamCheckpointer.checkpoint(binlogCoordinate); - verify(streamProgressState).setCurrentPosition(binlogCoordinate); + verify(mySqlStreamState).setCurrentPosition(binlogCoordinate); verify(sourceCoordinator).saveProgressStateForPartition(streamPartition, CHECKPOINT_OWNERSHIP_TIMEOUT_INCREASE); verify(checkpointCounter).increment(); } diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamWorkerTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamWorkerTest.java index 1eaf719cf5..ede13a40ec 100644 --- a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamWorkerTest.java +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamWorkerTest.java @@ -14,6 +14,7 @@ import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourceCoordinator; import org.opensearch.dataprepper.plugins.source.rds.coordination.partition.StreamPartition; +import org.opensearch.dataprepper.plugins.source.rds.coordination.state.MySqlStreamState; import org.opensearch.dataprepper.plugins.source.rds.coordination.state.StreamProgressState; import org.opensearch.dataprepper.plugins.source.rds.model.BinlogCoordinate; @@ -54,10 +55,12 @@ void setUp() { @Test void test_processStream_with_given_binlog_coordinates() throws IOException { final StreamProgressState streamProgressState = mock(StreamProgressState.class); + final MySqlStreamState mySqlStreamState = mock(MySqlStreamState.class); final String binlogFilename = UUID.randomUUID().toString(); final long binlogPosition = 100L; when(streamPartition.getProgressState()).thenReturn(Optional.of(streamProgressState)); - when(streamProgressState.getCurrentPosition()).thenReturn(new BinlogCoordinate(binlogFilename, binlogPosition)); + when(streamProgressState.getMySqlStreamState()).thenReturn(mySqlStreamState); + when(mySqlStreamState.getCurrentPosition()).thenReturn(new BinlogCoordinate(binlogFilename, binlogPosition)); when(streamProgressState.shouldWaitForExport()).thenReturn(false); when(binlogClientWrapper.getBinlogClient()).thenReturn(binaryLogClient); @@ -70,11 +73,13 @@ void test_processStream_with_given_binlog_coordinates() throws IOException { @Test void test_processStream_without_current_binlog_coordinates() throws IOException { - StreamProgressState streamProgressState = mock(StreamProgressState.class); + final StreamProgressState streamProgressState = mock(StreamProgressState.class); + final MySqlStreamState mySqlStreamState = mock(MySqlStreamState.class); when(streamPartition.getProgressState()).thenReturn(Optional.of(streamProgressState)); final String binlogFilename = "binlog-001"; final long binlogPosition = 100L; - when(streamProgressState.getCurrentPosition()).thenReturn(null); + when(streamProgressState.getMySqlStreamState()).thenReturn(mySqlStreamState); + when(mySqlStreamState.getCurrentPosition()).thenReturn(null); when(streamProgressState.shouldWaitForExport()).thenReturn(false); streamWorker.processStream(streamPartition); From 561d93801db1ef7231246024a3109f300177880c Mon Sep 17 00:00:00 2001 From: Hai Yan Date: Fri, 24 Jan 2025 00:24:50 -0600 Subject: [PATCH 09/10] Address more comments Signed-off-by: Hai Yan --- .../state/PostgresStreamState.java | 11 +++ .../rds/datatype/postgres/ColumnType.java | 3 + .../source/rds/leader/LeaderScheduler.java | 1 + .../rds/schema/PostgresSchemaManager.java | 28 +++++-- .../rds/stream/LogicalReplicationClient.java | 10 ++- .../LogicalReplicationEventProcessor.java | 32 +++++++- .../stream/ReplicationLogClientFactory.java | 7 +- .../rds/schema/PostgresSchemaManagerTest.java | 81 ++++++++++++++++++- .../stream/LogicalReplicationClientTest.java | 4 +- 9 files changed, 159 insertions(+), 18 deletions(-) diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/state/PostgresStreamState.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/state/PostgresStreamState.java index 011b1d37de..e916e04c9d 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/state/PostgresStreamState.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/state/PostgresStreamState.java @@ -16,6 +16,9 @@ public class PostgresStreamState { @JsonProperty("currentLsn") private String currentLsn; + @JsonProperty("publicationName") + private String publicationName; + @JsonProperty("replicationSlotName") private String replicationSlotName; @@ -27,6 +30,14 @@ public void setCurrentLsn(String currentLsn) { this.currentLsn = currentLsn; } + public String getPublicationName() { + return publicationName; + } + + public void setPublicationName(String publicationName) { + this.publicationName = publicationName; + } + public String getReplicationSlotName() { return replicationSlotName; } diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/datatype/postgres/ColumnType.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/datatype/postgres/ColumnType.java index c03e4e67c4..7d935ea6f8 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/datatype/postgres/ColumnType.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/datatype/postgres/ColumnType.java @@ -56,6 +56,9 @@ public String getTypeName() { } public static ColumnType getByTypeId(int typeId) { + if (!TYPE_ID_MAP.containsKey(typeId)) { + throw new IllegalArgumentException("Unsupported column type id: " + typeId); + } return TYPE_ID_MAP.get(typeId); } diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/leader/LeaderScheduler.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/leader/LeaderScheduler.java index 31ac120ce7..d7ab374684 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/leader/LeaderScheduler.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/leader/LeaderScheduler.java @@ -176,6 +176,7 @@ private void createStreamPartition(RdsSourceConfig sourceConfig) { final String publicationName = generatePublicationName(); final String slotName = generateReplicationSlotName(); ((PostgresSchemaManager)schemaManager).createLogicalReplicationSlot(sourceConfig.getTableNames(), publicationName, slotName); + progressState.getPostgresStreamState().setPublicationName(publicationName); progressState.getPostgresStreamState().setReplicationSlotName(slotName); } StreamPartition streamPartition = new StreamPartition(sourceConfig.getDbIdentifier(), progressState); diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/schema/PostgresSchemaManager.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/schema/PostgresSchemaManager.java index dcd604f8a8..430b258e16 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/schema/PostgresSchemaManager.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/schema/PostgresSchemaManager.java @@ -20,6 +20,7 @@ import java.sql.ResultSet; import java.util.ArrayList; import java.util.List; +import java.util.NoSuchElementException; public class PostgresSchemaManager implements SchemaManager { private static final Logger LOG = LoggerFactory.getLogger(PostgresSchemaManager.class); @@ -28,6 +29,7 @@ public class PostgresSchemaManager implements SchemaManager { static final int NUM_OF_RETRIES = 3; static final int BACKOFF_IN_MILLIS = 500; static final String COLUMN_NAME = "COLUMN_NAME"; + static final String PGOUTPUT = "pgoutput"; public PostgresSchemaManager(ConnectionManager connectionManager) { this.connectionManager = connectionManager; @@ -51,7 +53,7 @@ public void createLogicalReplicationSlot(final List tableNames, final St PreparedStatement statement = conn.prepareStatement(createPublicationStatement); statement.executeUpdate(); } catch (Exception e) { - LOG.info("Failed to create publication: {}", e.getMessage()); + LOG.warn("Failed to create publication: {}", e.getMessage()); } PGConnection pgConnection = conn.unwrap(PGConnection.class); @@ -59,14 +61,26 @@ public void createLogicalReplicationSlot(final List tableNames, final St // Create replication slot PGReplicationConnection replicationConnection = pgConnection.getReplicationAPI(); try { + // Check if replication slot exists + String checkSlotQuery = "SELECT EXISTS (SELECT 1 FROM pg_replication_slots WHERE slot_name = ?);"; + PreparedStatement checkSlotStatement = conn.prepareStatement(checkSlotQuery); + checkSlotStatement.setString(1, slotName); + try (ResultSet resultSet = checkSlotStatement.executeQuery()) { + if (resultSet.next() && resultSet.getBoolean(1)) { + LOG.info("Replication slot {} already exists. ", slotName); + return; + } + } + + LOG.info("Creating replication slot {}...", slotName); replicationConnection.createReplicationSlot() .logical() .withSlotName(slotName) - .withOutputPlugin("pgoutput") + .withOutputPlugin(PGOUTPUT) .make(); LOG.info("Replication slot {} created successfully. ", slotName); } catch (Exception e) { - LOG.info("Failed to create replication slot {}: {}", slotName, e.getMessage()); + LOG.warn("Failed to create replication slot {}: {}", slotName, e.getMessage()); } } catch (Exception e) { LOG.error("Exception when creating replication slot. ", e); @@ -87,16 +101,20 @@ public List getPrimaryKeys(final String fullTableName) { while (rs.next()) { primaryKeys.add(rs.getString(COLUMN_NAME)); } + if (primaryKeys.isEmpty()) { + throw new NoSuchElementException("No primary keys found for table " + table); + } return primaryKeys; } + } catch (NoSuchElementException e) { + throw e; } catch (Exception e) { LOG.error("Failed to get primary keys for table {}, retrying", table, e); } applyBackoff(); retry++; } - LOG.warn("Failed to get primary keys for table {}", table); - return List.of(); + throw new RuntimeException("Failed to get primary keys for table " + table); } private void applyBackoff() { diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationClient.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationClient.java index cc83fdd232..130f004960 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationClient.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationClient.java @@ -25,7 +25,10 @@ public class LogicalReplicationClient implements ReplicationLogClient { private static final Logger LOG = LoggerFactory.getLogger(LogicalReplicationClient.class); + static final String PUBLICATION_NAMES_KEY = "publication_names"; + private final ConnectionManager connectionManager; + private final String publicationName; private final String replicationSlotName; private LogSequenceNumber startLsn; private LogicalReplicationEventProcessor eventProcessor; @@ -33,7 +36,9 @@ public class LogicalReplicationClient implements ReplicationLogClient { private volatile boolean disconnectRequested = false; public LogicalReplicationClient(final ConnectionManager connectionManager, - final String replicationSlotName) { + final String replicationSlotName, + final String publicationName) { + this.publicationName = publicationName; this.connectionManager = connectionManager; this.replicationSlotName = replicationSlotName; } @@ -49,8 +54,7 @@ public void connect() { .replicationStream() .logical() .withSlotName(replicationSlotName) - .withSlotOption("proto_version", "1") - .withSlotOption("publication_names", "my_publication"); + .withSlotOption(PUBLICATION_NAMES_KEY, publicationName); if (startLsn != null) { logicalStreamBuilder.withStartPosition(startLsn); } diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationEventProcessor.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationEventProcessor.java index a48e468586..142eb43c31 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationEventProcessor.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationEventProcessor.java @@ -34,6 +34,30 @@ import java.util.Map; public class LogicalReplicationEventProcessor { + enum TupleDataType { + NEW('N'), + KEY('K'), + OLD('O'); + + private final char value; + + TupleDataType(char value) { + this.value = value; + } + + public char getValue() { + return value; + } + + public static TupleDataType fromValue(char value) { + for (TupleDataType type : TupleDataType.values()) { + if (type.getValue() == value) { + return type; + } + } + throw new IllegalArgumentException("Invalid TupleDataType value: " + value); + } + } private static final Logger LOG = LoggerFactory.getLogger(LogicalReplicationEventProcessor.class); @@ -170,18 +194,18 @@ void processUpdateMessage(ByteBuffer msg) { final List primaryKeys = tableMetadata.getPrimaryKeys(); final long eventTimestampMillis = currentEventTimestamp; - char typeId = (char) msg.get(); - if (typeId == 'N') { + TupleDataType tupleDataType = TupleDataType.fromValue((char) msg.get()); + if (tupleDataType == TupleDataType.NEW) { doProcess(msg, columnNames, tableMetadata, primaryKeys, eventTimestampMillis, OpenSearchBulkActions.INDEX); LOG.debug("Processed an UPDATE message with table id: {}", tableId); - } else if (typeId == 'K') { + } else if (tupleDataType == TupleDataType.KEY) { // Primary keys were changed doProcess(msg, columnNames, tableMetadata, primaryKeys, eventTimestampMillis, OpenSearchBulkActions.DELETE); msg.get(); // should be a char 'N' doProcess(msg, columnNames, tableMetadata, primaryKeys, eventTimestampMillis, OpenSearchBulkActions.INDEX); LOG.debug("Processed an UPDATE message with table id: {} and primary key(s) were changed", tableId); - } else if (typeId == 'O') { + } else if (tupleDataType == TupleDataType.OLD) { // Replica Identity is set to full, containing both old and new row data Map oldRowDataMap = getRowDataMap(msg, columnNames); msg.get(); // should be a char 'N' diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/ReplicationLogClientFactory.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/ReplicationLogClientFactory.java index 183ce05299..a1608f56b0 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/ReplicationLogClientFactory.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/ReplicationLogClientFactory.java @@ -16,6 +16,7 @@ import org.opensearch.dataprepper.plugins.source.rds.RdsSourceConfig; import org.opensearch.dataprepper.plugins.source.rds.configuration.EngineType; import org.opensearch.dataprepper.plugins.source.rds.coordination.partition.StreamPartition; +import org.opensearch.dataprepper.plugins.source.rds.coordination.state.PostgresStreamState; import org.opensearch.dataprepper.plugins.source.rds.model.DbMetadata; import org.opensearch.dataprepper.plugins.source.rds.schema.ConnectionManager; import org.opensearch.dataprepper.plugins.source.rds.schema.ConnectionManagerFactory; @@ -67,13 +68,15 @@ private BinaryLogClient createBinaryLogClient() { } private LogicalReplicationClient createLogicalReplicationClient(StreamPartition streamPartition) { - final String replicationSlotName = streamPartition.getProgressState().get().getPostgresStreamState().getReplicationSlotName(); + final PostgresStreamState postgresStreamState = streamPartition.getProgressState().get().getPostgresStreamState(); + final String publicationName = postgresStreamState.getPublicationName(); + final String replicationSlotName = postgresStreamState.getReplicationSlotName(); if (replicationSlotName == null) { throw new NoSuchElementException("Replication slot name is not found in progress state."); } final ConnectionManagerFactory connectionManagerFactory = new ConnectionManagerFactory(sourceConfig, dbMetadata); final ConnectionManager connectionManager = connectionManagerFactory.getConnectionManager(); - return new LogicalReplicationClient(connectionManager, replicationSlotName); + return new LogicalReplicationClient(connectionManager, publicationName, replicationSlotName); } public void setSSLMode(SSLMode sslMode) { diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/schema/PostgresSchemaManagerTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/schema/PostgresSchemaManagerTest.java index 0602b2c6d5..9eebee0348 100644 --- a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/schema/PostgresSchemaManagerTest.java +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/schema/PostgresSchemaManagerTest.java @@ -24,13 +24,17 @@ import java.sql.Connection; import java.sql.PreparedStatement; +import java.sql.ResultSet; import java.sql.SQLException; import java.util.List; +import java.util.UUID; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.is; +import static org.junit.jupiter.api.Assertions.assertThrows; import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; @@ -51,13 +55,14 @@ void setUp() { } @Test - void test_createLogicalReplicationSlot() throws SQLException { + void test_createLogicalReplicationSlot_creates_slot_if_not_exists() throws SQLException { final List tableNames = List.of("table1", "table2"); final String publicationName = "publication1"; final String slotName = "slot1"; final PreparedStatement preparedStatement = mock(PreparedStatement.class); final PGConnection pgConnection = mock(PGConnection.class); final PGReplicationConnection replicationConnection = mock(PGReplicationConnection.class); + final ResultSet resultSet = mock(ResultSet.class); final ChainedCreateReplicationSlotBuilder chainedCreateSlotBuilder = mock(ChainedCreateReplicationSlotBuilder.class); final ChainedLogicalCreateSlotBuilder slotBuilder = mock(ChainedLogicalCreateSlotBuilder.class); @@ -66,6 +71,8 @@ void test_createLogicalReplicationSlot() throws SQLException { when(connectionManager.getConnection()).thenReturn(connection); when(connection.prepareStatement(statementCaptor.capture())).thenReturn(preparedStatement); when(connection.unwrap(PGConnection.class)).thenReturn(pgConnection); + when(preparedStatement.executeQuery()).thenReturn(resultSet); + when(resultSet.next()).thenReturn(false); // Replication slot doesn't exist when(pgConnection.getReplicationAPI()).thenReturn(replicationConnection); when(replicationConnection.createReplicationSlot()).thenReturn(chainedCreateSlotBuilder); when(chainedCreateSlotBuilder.logical()).thenReturn(slotBuilder); @@ -74,9 +81,11 @@ void test_createLogicalReplicationSlot() throws SQLException { schemaManager.createLogicalReplicationSlot(tableNames, publicationName, slotName); - String statement = statementCaptor.getValue(); - assertThat(statement, is("CREATE PUBLICATION " + publicationName + " FOR TABLE " + String.join(", ", tableNames) + ";")); + List statements = statementCaptor.getAllValues(); + assertThat(statements.get(0), is("CREATE PUBLICATION " + publicationName + " FOR TABLE " + String.join(", ", tableNames) + ";")); + assertThat(statements.get(1), is("SELECT EXISTS (SELECT 1 FROM pg_replication_slots WHERE slot_name = ?);")); verify(preparedStatement).executeUpdate(); + verify(preparedStatement).executeQuery(); verify(pgConnection).getReplicationAPI(); verify(replicationConnection).createReplicationSlot(); verify(chainedCreateSlotBuilder).logical(); @@ -85,6 +94,72 @@ void test_createLogicalReplicationSlot() throws SQLException { verify(slotBuilder).make(); } + @Test + void test_createLogicalReplicationSlot_skip_creation_if_slot_exists() throws SQLException { + final List tableNames = List.of("table1", "table2"); + final String publicationName = "publication1"; + final String slotName = "slot1"; + final PreparedStatement preparedStatement = mock(PreparedStatement.class); + final PGConnection pgConnection = mock(PGConnection.class); + final PGReplicationConnection replicationConnection = mock(PGReplicationConnection.class); + final ResultSet resultSet = mock(ResultSet.class); + + ArgumentCaptor statementCaptor = ArgumentCaptor.forClass(String.class); + + when(connectionManager.getConnection()).thenReturn(connection); + when(connection.prepareStatement(statementCaptor.capture())).thenReturn(preparedStatement); + when(connection.unwrap(PGConnection.class)).thenReturn(pgConnection); + when(preparedStatement.executeQuery()).thenReturn(resultSet); + when(resultSet.next()).thenReturn(true); // Replication slot exists + when(resultSet.getBoolean(1)).thenReturn(true); + when(pgConnection.getReplicationAPI()).thenReturn(replicationConnection); + + schemaManager.createLogicalReplicationSlot(tableNames, publicationName, slotName); + + List statements = statementCaptor.getAllValues(); + assertThat(statements.get(0), is("CREATE PUBLICATION " + publicationName + " FOR TABLE " + String.join(", ", tableNames) + ";")); + assertThat(statements.get(1), is("SELECT EXISTS (SELECT 1 FROM pg_replication_slots WHERE slot_name = ?);")); + verify(preparedStatement).executeUpdate(); + verify(preparedStatement).executeQuery(); + verify(pgConnection).getReplicationAPI(); + verify(replicationConnection, never()).createReplicationSlot(); + } + + @Test + void test_getPrimaryKeys_returns_primary_keys() throws SQLException { + final String database = UUID.randomUUID().toString(); + final String schema = UUID.randomUUID().toString(); + final String table = UUID.randomUUID().toString(); + final String fullTableName = database + "." + schema + "." + table; + final ResultSet resultSet = mock(ResultSet.class); + final String primaryKeyName = UUID.randomUUID().toString(); + + when(connectionManager.getConnection()).thenReturn(connection); + when(connection.getMetaData().getPrimaryKeys(database, schema, table)).thenReturn(resultSet); + when(resultSet.next()).thenReturn(true, false); + when(resultSet.getString("COLUMN_NAME")).thenReturn(primaryKeyName); + + final List primaryKeys = schemaManager.getPrimaryKeys(fullTableName); + + assertThat(primaryKeys.size(), is(1)); + assertThat(primaryKeys.get(0), is(primaryKeyName)); + } + + @Test + void test_getPrimaryKeys_throws_exception_if_failed() throws SQLException { + final String database = UUID.randomUUID().toString(); + final String schema = UUID.randomUUID().toString(); + final String table = UUID.randomUUID().toString(); + final String fullTableName = database + "." + schema + "." + table; + final ResultSet resultSet = mock(ResultSet.class); + + when(connectionManager.getConnection()).thenReturn(connection); + when(connection.getMetaData().getPrimaryKeys(database, schema, table)).thenReturn(resultSet); + when(resultSet.next()).thenThrow(RuntimeException.class); + + assertThrows(RuntimeException.class, () -> schemaManager.getPrimaryKeys(fullTableName)); + } + private PostgresSchemaManager createObjectUnderTest() { return new PostgresSchemaManager(connectionManager); } diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationClientTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationClientTest.java index 5089f0cf67..e157d1832a 100644 --- a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationClientTest.java +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationClientTest.java @@ -45,11 +45,13 @@ class LogicalReplicationClientTest { @Mock private LogicalReplicationEventProcessor eventProcessor; + private String publicationName; private String replicationSlotName; private LogicalReplicationClient logicalReplicationClient; @BeforeEach void setUp() { + publicationName = UUID.randomUUID().toString(); replicationSlotName = UUID.randomUUID().toString(); logicalReplicationClient = createObjectUnderTest(); logicalReplicationClient.setEventProcessor(eventProcessor); @@ -86,6 +88,6 @@ void test_connect() throws SQLException, InterruptedException { } private LogicalReplicationClient createObjectUnderTest() { - return new LogicalReplicationClient(connectionManager, replicationSlotName); + return new LogicalReplicationClient(connectionManager, replicationSlotName, publicationName); } } From 5108fa75f92d869a70724f54635c88f5c5f6d29f Mon Sep 17 00:00:00 2001 From: Hai Yan Date: Mon, 27 Jan 2025 15:29:03 -0600 Subject: [PATCH 10/10] Fix Java21 build Signed-off-by: Hai Yan --- .../plugins/source/rds/model/MessageType.java | 18 ++++++++ .../LogicalReplicationEventProcessor.java | 16 ++++---- .../stream/LogicalReplicationClientTest.java | 2 +- .../LogicalReplicationEventProcessorTest.java | 41 +++++++++++-------- 4 files changed, 50 insertions(+), 27 deletions(-) diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/model/MessageType.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/model/MessageType.java index a537835099..4bb4bcb288 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/model/MessageType.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/model/MessageType.java @@ -10,6 +10,8 @@ package org.opensearch.dataprepper.plugins.source.rds.model; +import java.util.Map; + public enum MessageType { BEGIN('B'), RELATION('R'), @@ -20,6 +22,15 @@ public enum MessageType { private final char value; + private static final Map MESSAGE_TYPE_MAP = Map.of( + BEGIN.getValue(), BEGIN, + RELATION.getValue(), RELATION, + INSERT.getValue(), INSERT, + UPDATE.getValue(), UPDATE, + DELETE.getValue(), DELETE, + COMMIT.getValue(), COMMIT + ); + MessageType(char value) { this.value = value; } @@ -27,4 +38,11 @@ public enum MessageType { public char getValue() { return value; } + + public static MessageType from(char value) { + if (!MESSAGE_TYPE_MAP.containsKey(value)) { + throw new IllegalArgumentException("Invalid MessageType value: " + value); + } + return MESSAGE_TYPE_MAP.get(value); + } } diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationEventProcessor.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationEventProcessor.java index 142eb43c31..f9881d0063 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationEventProcessor.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationEventProcessor.java @@ -96,18 +96,18 @@ public void process(ByteBuffer msg) { // If it's a RELATION, update table metadata map // If it's INSERT/UPDATE/DELETE, prepare events // If it's a COMMIT, convert all prepared events and send to buffer - char messageType = (char) msg.get(); - if (messageType == MessageType.BEGIN.getValue()) { + MessageType messageType = MessageType.from((char) msg.get()); + if (messageType == MessageType.BEGIN) { processBeginMessage(msg); - } else if (messageType == MessageType.RELATION.getValue()) { + } else if (messageType == MessageType.RELATION) { processRelationMessage(msg); - } else if (messageType == MessageType.INSERT.getValue()) { + } else if (messageType == MessageType.INSERT) { processInsertMessage(msg); - } else if (messageType == MessageType.UPDATE.getValue()) { + } else if (messageType == MessageType.UPDATE) { processUpdateMessage(msg); - } else if (messageType == MessageType.DELETE.getValue()) { + } else if (messageType == MessageType.DELETE) { processDeleteMessage(msg); - } else if (messageType == MessageType.COMMIT.getValue()) { + } else if (messageType == MessageType.COMMIT) { processCommitMessage(msg); } else { throw new IllegalArgumentException("Replication message type [" + messageType + "] is not supported. "); @@ -158,7 +158,7 @@ void processRelationMessage(ByteBuffer msg) { } void processCommitMessage(ByteBuffer msg) { - int flag = msg.get(); + int flag = msg.getInt(); long commitLsn = msg.getLong(); long endLsn = msg.getLong(); long epochMicro = msg.getLong(); diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationClientTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationClientTest.java index e157d1832a..9cd410ee44 100644 --- a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationClientTest.java +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationClientTest.java @@ -63,7 +63,7 @@ void test_connect() throws SQLException, InterruptedException { final PGConnection pgConnection = mock(PGConnection.class, RETURNS_DEEP_STUBS); final ChainedLogicalStreamBuilder logicalStreamBuilder = mock(ChainedLogicalStreamBuilder.class); final PGReplicationStream stream = mock(PGReplicationStream.class); - final ByteBuffer message = mock(ByteBuffer.class); + final ByteBuffer message = ByteBuffer.allocate(0); final LogSequenceNumber lsn = mock(LogSequenceNumber.class); when(connectionManager.getConnection()).thenReturn(connection); diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationEventProcessorTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationEventProcessorTest.java index 22614e4f02..31ec9618a2 100644 --- a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationEventProcessorTest.java +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationEventProcessorTest.java @@ -21,20 +21,16 @@ import org.opensearch.dataprepper.model.record.Record; import org.opensearch.dataprepper.plugins.source.rds.RdsSourceConfig; import org.opensearch.dataprepper.plugins.source.rds.coordination.partition.StreamPartition; -import org.opensearch.dataprepper.plugins.source.rds.coordination.state.StreamProgressState; +import org.opensearch.dataprepper.plugins.source.rds.model.MessageType; import java.nio.ByteBuffer; -import java.util.List; -import java.util.Map; -import java.util.Optional; +import java.util.Random; import java.util.UUID; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.mockito.Mockito.doNothing; -import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; @ExtendWith(MockitoExtension.class) class LogicalReplicationEventProcessorTest { @@ -48,23 +44,26 @@ class LogicalReplicationEventProcessorTest { @Mock private Buffer> buffer; - @Mock private ByteBuffer message; private String s3Prefix; private LogicalReplicationEventProcessor objectUnderTest; + private Random random; + @BeforeEach void setUp() { s3Prefix = UUID.randomUUID().toString(); + random = new Random(); objectUnderTest = spy(createObjectUnderTest()); } @Test void test_correct_process_method_invoked_for_begin_message() { - when(message.get()).thenReturn((byte) 'B'); + setMessageType(MessageType.BEGIN); + doNothing().when(objectUnderTest).processBeginMessage(message); objectUnderTest.process(message); @@ -73,11 +72,8 @@ void test_correct_process_method_invoked_for_begin_message() { @Test void test_correct_process_method_invoked_for_relation_message() { - when(message.get()).thenReturn((byte) 'R'); - final StreamProgressState progressState = mock(StreamProgressState.class); - when(streamPartition.getProgressState()).thenReturn(Optional.of(progressState)); - when(sourceConfig.getTableNames()).thenReturn(List.of("database.schema.table1")); - when(progressState.getPrimaryKeyMap()).thenReturn(Map.of("database.schema.table1", List.of("key1", "key2"))); + setMessageType(MessageType.RELATION); + doNothing().when(objectUnderTest).processRelationMessage(message); objectUnderTest.process(message); @@ -86,7 +82,8 @@ void test_correct_process_method_invoked_for_relation_message() { @Test void test_correct_process_method_invoked_for_commit_message() { - when(message.get()).thenReturn((byte) 'C'); + setMessageType(MessageType.COMMIT); + doNothing().when(objectUnderTest).processCommitMessage(message); objectUnderTest.process(message); @@ -95,7 +92,7 @@ void test_correct_process_method_invoked_for_commit_message() { @Test void test_correct_process_method_invoked_for_insert_message() { - when(message.get()).thenReturn((byte) 'I'); + setMessageType(MessageType.INSERT); doNothing().when(objectUnderTest).processInsertMessage(message); objectUnderTest.process(message); @@ -105,7 +102,7 @@ void test_correct_process_method_invoked_for_insert_message() { @Test void test_correct_process_method_invoked_for_update_message() { - when(message.get()).thenReturn((byte) 'U'); + setMessageType(MessageType.UPDATE); doNothing().when(objectUnderTest).processUpdateMessage(message); objectUnderTest.process(message); @@ -115,7 +112,7 @@ void test_correct_process_method_invoked_for_update_message() { @Test void test_correct_process_method_invoked_for_delete_message() { - when(message.get()).thenReturn((byte) 'D'); + setMessageType(MessageType.DELETE); doNothing().when(objectUnderTest).processDeleteMessage(message); objectUnderTest.process(message); @@ -125,7 +122,9 @@ void test_correct_process_method_invoked_for_delete_message() { @Test void test_unsupported_message_type_throws_exception() { - when(message.get()).thenReturn((byte) 'A'); + message = ByteBuffer.allocate(1); + message.put((byte) 'A'); + message.flip(); assertThrows(IllegalArgumentException.class, () -> objectUnderTest.process(message)); } @@ -133,4 +132,10 @@ void test_unsupported_message_type_throws_exception() { private LogicalReplicationEventProcessor createObjectUnderTest() { return new LogicalReplicationEventProcessor(streamPartition, sourceConfig, buffer, s3Prefix); } + + private void setMessageType(MessageType messageType) { + message = ByteBuffer.allocate(1); + message.put((byte) messageType.getValue()); + message.flip(); + } }