From 7bec86540dbaefc59925988c9dfc41d94207a070 Mon Sep 17 00:00:00 2001 From: Hai Yan <8153134+oeyh@users.noreply.github.com> Date: Fri, 26 Jul 2024 13:31:54 -0500 Subject: [PATCH 01/25] A few improvements to rds source (#4765) * Add error logging to event handlers Signed-off-by: Hai Yan * Add tls config and enable tls by default Signed-off-by: Hai Yan * Add original event name to metadata Signed-off-by: Hai Yan * Update metadata for export and stream events Signed-off-by: Hai Yan * Add some fixes Signed-off-by: Hai Yan * Remove config alias ssl Signed-off-by: Hai Yan --------- Signed-off-by: Hai Yan --- .../plugins/source/rds/RdsService.java | 6 +++ .../plugins/source/rds/RdsSourceConfig.java | 12 +++-- .../source/rds/configuration/TlsConfig.java | 20 ++++++++ .../rds/converter/ExportRecordConverter.java | 24 ++++++++- .../rds/converter/MetadataKeyAttributes.java | 4 +- .../rds/converter/StreamRecordConverter.java | 27 +++++++--- .../state/DataFileProgressState.java | 22 ++++++++ .../state/ExportProgressState.java | 12 ++--- .../source/rds/export/DataFileLoader.java | 20 +++++++- .../source/rds/export/ExportScheduler.java | 28 ++++++---- .../source/rds/leader/LeaderScheduler.java | 7 +-- .../rds/stream/BinlogClientFactory.java | 32 +++++++++--- .../rds/stream/BinlogEventListener.java | 51 ++++++++++++++++--- .../source/rds/stream/StreamWorker.java | 2 +- .../plugins/source/rds/RdsServiceTest.java | 49 +++++++++++++++++- .../converter/ExportRecordConverterTest.java | 34 ++++++++++++- .../converter/StreamRecordConverterTest.java | 22 ++++++-- 17 files changed, 312 insertions(+), 60 deletions(-) create mode 100644 data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/configuration/TlsConfig.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 c7f7274763..982751a3db 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 @@ -6,6 +6,7 @@ package org.opensearch.dataprepper.plugins.source.rds; import com.github.shyiko.mysql.binlog.BinaryLogClient; +import com.github.shyiko.mysql.binlog.network.SSLMode; import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.buffer.Buffer; import org.opensearch.dataprepper.model.event.Event; @@ -94,6 +95,11 @@ public void start(Buffer> buffer) { if (sourceConfig.isStreamEnabled()) { BinaryLogClient binaryLogClient = new BinlogClientFactory(sourceConfig, rdsClient).create(); + if (sourceConfig.getTlsConfig() == null || !sourceConfig.getTlsConfig().isInsecure()) { + binaryLogClient.setSSLMode(SSLMode.REQUIRED); + } else { + binaryLogClient.setSSLMode(SSLMode.DISABLED); + } streamScheduler = new StreamScheduler(sourceCoordinator, sourceConfig, binaryLogClient, buffer, pluginMetrics); runnableList.add(streamScheduler); } diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsSourceConfig.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsSourceConfig.java index c58c9f5974..548dc4a2fb 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsSourceConfig.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsSourceConfig.java @@ -12,6 +12,7 @@ import org.opensearch.dataprepper.plugins.source.rds.configuration.EngineType; import org.opensearch.dataprepper.plugins.source.rds.configuration.ExportConfig; import org.opensearch.dataprepper.plugins.source.rds.configuration.StreamConfig; +import org.opensearch.dataprepper.plugins.source.rds.configuration.TlsConfig; import java.util.List; @@ -55,9 +56,6 @@ public class RdsSourceConfig { @JsonProperty("acknowledgments") private boolean acknowledgments = false; - /** - * S3 bucket for holding both export and stream data - */ @JsonProperty("s3_bucket") private String s3Bucket; @@ -77,6 +75,9 @@ public class RdsSourceConfig { @JsonProperty("authentication") private AuthenticationConfig authenticationConfig; + @JsonProperty("tls") + private TlsConfig tlsConfig; + public String getDbIdentifier() { return dbIdentifier; } @@ -133,6 +134,10 @@ public boolean isStreamEnabled() { return streamConfig != null; } + public TlsConfig getTlsConfig() { + return tlsConfig; + } + public AuthenticationConfig getAuthenticationConfig() { return this.authenticationConfig; } @@ -151,6 +156,5 @@ public String getUsername() { public String getPassword() { return password; } - } } diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/configuration/TlsConfig.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/configuration/TlsConfig.java new file mode 100644 index 0000000000..8a8d1c1929 --- /dev/null +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/configuration/TlsConfig.java @@ -0,0 +1,20 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.rds.configuration; + +import com.fasterxml.jackson.annotation.JsonProperty; + +public class TlsConfig { + + @JsonProperty("insecure") + private boolean insecure = false; + + // TODO: add options for server identity verification + + public boolean isInsecure() { + return insecure; + } +} diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/converter/ExportRecordConverter.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/converter/ExportRecordConverter.java index 11932cd512..428f121788 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/converter/ExportRecordConverter.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/converter/ExportRecordConverter.java @@ -7,11 +7,19 @@ import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.model.event.EventMetadata; +import org.opensearch.dataprepper.model.opensearch.OpenSearchBulkActions; import org.opensearch.dataprepper.model.record.Record; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.List; +import java.util.stream.Collectors; + +import static org.opensearch.dataprepper.plugins.source.rds.converter.MetadataKeyAttributes.BULK_ACTION_METADATA_ATTRIBUTE; +import static org.opensearch.dataprepper.plugins.source.rds.converter.MetadataKeyAttributes.EVENT_DATABASE_NAME_METADATA_ATTRIBUTE; import static org.opensearch.dataprepper.plugins.source.rds.converter.MetadataKeyAttributes.EVENT_TABLE_NAME_METADATA_ATTRIBUTE; +import static org.opensearch.dataprepper.plugins.source.rds.converter.MetadataKeyAttributes.EVENT_TIMESTAMP_METADATA_ATTRIBUTE; +import static org.opensearch.dataprepper.plugins.source.rds.converter.MetadataKeyAttributes.EVENT_VERSION_FROM_TIMESTAMP; import static org.opensearch.dataprepper.plugins.source.rds.converter.MetadataKeyAttributes.INGESTION_EVENT_TYPE_ATTRIBUTE; import static org.opensearch.dataprepper.plugins.source.rds.converter.MetadataKeyAttributes.PRIMARY_KEY_DOCUMENT_ID_METADATA_ATTRIBUTE; @@ -21,16 +29,28 @@ public class ExportRecordConverter { static final String EXPORT_EVENT_TYPE = "EXPORT"; - public Event convert(Record record, String tableName, String primaryKeyName) { + public Event convert(final Record record, + final String databaseName, + final String tableName, + final List primaryKeys, + final long eventCreateTimeEpochMillis, + final long eventVersionNumber) { Event event = record.getData(); EventMetadata eventMetadata = event.getMetadata(); + eventMetadata.setAttribute(EVENT_DATABASE_NAME_METADATA_ATTRIBUTE, databaseName); eventMetadata.setAttribute(EVENT_TABLE_NAME_METADATA_ATTRIBUTE, tableName); + eventMetadata.setAttribute(BULK_ACTION_METADATA_ATTRIBUTE, OpenSearchBulkActions.INDEX.toString()); eventMetadata.setAttribute(INGESTION_EVENT_TYPE_ATTRIBUTE, EXPORT_EVENT_TYPE); - final Object primaryKeyValue = record.getData().get(primaryKeyName, Object.class); + final String primaryKeyValue = primaryKeys.stream() + .map(key -> event.get(key, String.class)) + .collect(Collectors.joining("|")); eventMetadata.setAttribute(PRIMARY_KEY_DOCUMENT_ID_METADATA_ATTRIBUTE, primaryKeyValue); + eventMetadata.setAttribute(EVENT_TIMESTAMP_METADATA_ATTRIBUTE, eventCreateTimeEpochMillis); + eventMetadata.setAttribute(EVENT_VERSION_FROM_TIMESTAMP, eventVersionNumber); + return event; } } diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/converter/MetadataKeyAttributes.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/converter/MetadataKeyAttributes.java index 60e1ba2bd3..0b7d808deb 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/converter/MetadataKeyAttributes.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/converter/MetadataKeyAttributes.java @@ -12,7 +12,9 @@ public class MetadataKeyAttributes { static final String EVENT_TIMESTAMP_METADATA_ATTRIBUTE = "event_timestamp"; - static final String EVENT_NAME_BULK_ACTION_METADATA_ATTRIBUTE = "opensearch_action"; + static final String BULK_ACTION_METADATA_ATTRIBUTE = "opensearch_action"; + + static final String CHANGE_EVENT_TYPE_METADATA_ATTRIBUTE = "change_event_type"; static final String EVENT_DATABASE_NAME_METADATA_ATTRIBUTE = "database_name"; diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/converter/StreamRecordConverter.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/converter/StreamRecordConverter.java index 511876d668..51cce6541e 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/converter/StreamRecordConverter.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/converter/StreamRecordConverter.java @@ -5,6 +5,7 @@ package org.opensearch.dataprepper.plugins.source.rds.converter; +import com.github.shyiko.mysql.binlog.event.EventType; import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.model.event.EventMetadata; import org.opensearch.dataprepper.model.event.JacksonEvent; @@ -19,9 +20,12 @@ import java.util.Map; import java.util.stream.Collectors; +import static org.opensearch.dataprepper.plugins.source.rds.converter.MetadataKeyAttributes.BULK_ACTION_METADATA_ATTRIBUTE; +import static org.opensearch.dataprepper.plugins.source.rds.converter.MetadataKeyAttributes.CHANGE_EVENT_TYPE_METADATA_ATTRIBUTE; import static org.opensearch.dataprepper.plugins.source.rds.converter.MetadataKeyAttributes.EVENT_DATABASE_NAME_METADATA_ATTRIBUTE; -import static org.opensearch.dataprepper.plugins.source.rds.converter.MetadataKeyAttributes.EVENT_NAME_BULK_ACTION_METADATA_ATTRIBUTE; import static org.opensearch.dataprepper.plugins.source.rds.converter.MetadataKeyAttributes.EVENT_TABLE_NAME_METADATA_ATTRIBUTE; +import static org.opensearch.dataprepper.plugins.source.rds.converter.MetadataKeyAttributes.EVENT_TIMESTAMP_METADATA_ATTRIBUTE; +import static org.opensearch.dataprepper.plugins.source.rds.converter.MetadataKeyAttributes.EVENT_VERSION_FROM_TIMESTAMP; import static org.opensearch.dataprepper.plugins.source.rds.converter.MetadataKeyAttributes.INGESTION_EVENT_TYPE_ATTRIBUTE; import static org.opensearch.dataprepper.plugins.source.rds.converter.MetadataKeyAttributes.PRIMARY_KEY_DOCUMENT_ID_METADATA_ATTRIBUTE; @@ -43,12 +47,15 @@ public StreamRecordConverter(final int partitionCount) { folderNames = s3PartitionCreator.createPartitions(); } - public Event convert(Map rowData, - String databaseName, - String tableName, - OpenSearchBulkActions bulkAction, - List primaryKeys, - String s3Prefix) { + public Event convert(final Map rowData, + final String databaseName, + final String tableName, + final EventType eventType, + final OpenSearchBulkActions bulkAction, + final List primaryKeys, + final String s3Prefix, + final long eventCreateTimeEpochMillis, + final long eventVersionNumber) { final Event event = JacksonEvent.builder() .withEventType("event") .withData(rowData) @@ -58,8 +65,9 @@ public Event convert(Map rowData, eventMetadata.setAttribute(EVENT_DATABASE_NAME_METADATA_ATTRIBUTE, databaseName); eventMetadata.setAttribute(EVENT_TABLE_NAME_METADATA_ATTRIBUTE, tableName); - eventMetadata.setAttribute(EVENT_NAME_BULK_ACTION_METADATA_ATTRIBUTE, bulkAction.toString()); + eventMetadata.setAttribute(BULK_ACTION_METADATA_ATTRIBUTE, bulkAction.toString()); eventMetadata.setAttribute(INGESTION_EVENT_TYPE_ATTRIBUTE, STREAM_EVENT_TYPE); + eventMetadata.setAttribute(CHANGE_EVENT_TYPE_METADATA_ATTRIBUTE, eventType.toString()); final String primaryKeyValue = primaryKeys.stream() .map(rowData::get) @@ -68,6 +76,9 @@ public Event convert(Map rowData, eventMetadata.setAttribute(PRIMARY_KEY_DOCUMENT_ID_METADATA_ATTRIBUTE, primaryKeyValue); eventMetadata.setAttribute(MetadataKeyAttributes.EVENT_S3_PARTITION_KEY, s3Prefix + S3_PATH_DELIMITER + hashKeyToPartition(primaryKeyValue)); + eventMetadata.setAttribute(EVENT_TIMESTAMP_METADATA_ATTRIBUTE, eventCreateTimeEpochMillis); + eventMetadata.setAttribute(EVENT_VERSION_FROM_TIMESTAMP, eventVersionNumber); + return event; } diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/state/DataFileProgressState.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/state/DataFileProgressState.java index c65c0bbe01..4f94f57d15 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/state/DataFileProgressState.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/state/DataFileProgressState.java @@ -15,9 +15,15 @@ public class DataFileProgressState { @JsonProperty("totalRecords") private int totalRecords; + @JsonProperty("sourceDatabase") + private String sourceDatabase; + @JsonProperty("sourceTable") private String sourceTable; + @JsonProperty("snapshotTime") + private long snapshotTime; + public int getTotalRecords() { return totalRecords; } @@ -34,6 +40,14 @@ public void setLoaded(boolean loaded) { this.isLoaded = loaded; } + public String getSourceDatabase() { + return sourceDatabase; + } + + public void setSourceDatabase(String sourceDatabase) { + this.sourceDatabase = sourceDatabase; + } + public String getSourceTable() { return sourceTable; } @@ -41,4 +55,12 @@ public String getSourceTable() { public void setSourceTable(String sourceTable) { this.sourceTable = sourceTable; } + + public long getSnapshotTime() { + return snapshotTime; + } + + public void setSnapshotTime(long snapshotTime) { + this.snapshotTime = snapshotTime; + } } diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/state/ExportProgressState.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/state/ExportProgressState.java index cde2be6dd8..b224f63a97 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/state/ExportProgressState.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/state/ExportProgressState.java @@ -35,8 +35,8 @@ public class ExportProgressState { @JsonProperty("kmsKeyId") private String kmsKeyId; - @JsonProperty("exportTime") - private String exportTime; + @JsonProperty("snapshotTime") + private long snapshotTime; @JsonProperty("status") private String status; @@ -97,12 +97,12 @@ public void setKmsKeyId(String kmsKeyId) { this.kmsKeyId = kmsKeyId; } - public String getExportTime() { - return exportTime; + public long getSnapshotTime() { + return snapshotTime; } - public void setExportTime(String exportTime) { - this.exportTime = exportTime; + public void setSnapshotTime(long snapshotTime) { + this.snapshotTime = snapshotTime; } public String getStatus() { diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileLoader.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileLoader.java index e76a04e99d..f12d44a75f 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileLoader.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileLoader.java @@ -11,15 +11,20 @@ import org.opensearch.dataprepper.model.record.Record; import org.opensearch.dataprepper.plugins.source.rds.converter.ExportRecordConverter; import org.opensearch.dataprepper.plugins.source.rds.coordination.partition.DataFilePartition; +import org.opensearch.dataprepper.plugins.source.rds.coordination.state.DataFileProgressState; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.InputStream; +import java.time.Duration; +import java.util.List; public class DataFileLoader implements Runnable { private static final Logger LOG = LoggerFactory.getLogger(DataFileLoader.class); + static final Duration VERSION_OVERLAP_TIME_FOR_EXPORT = Duration.ofMinutes(5); + private final DataFilePartition dataFilePartition; private final String bucket; private final String objectKey; @@ -58,10 +63,21 @@ public void run() { codec.parse(inputStream, record -> { try { - final String tableName = dataFilePartition.getProgressState().get().getSourceTable(); + DataFileProgressState progressState = dataFilePartition.getProgressState().get(); + // TODO: primary key to be obtained by querying database schema final String primaryKeyName = "id"; - Record transformedRecord = new Record<>(recordConverter.convert(record, tableName, primaryKeyName)); + + final long snapshotTime = progressState.getSnapshotTime(); + final long eventVersionNumber = snapshotTime - VERSION_OVERLAP_TIME_FOR_EXPORT.toMillis(); + Record transformedRecord = new Record<>( + recordConverter.convert( + record, + progressState.getSourceDatabase(), + progressState.getSourceTable(), + List.of(primaryKeyName), + snapshotTime, + eventVersionNumber)); bufferAccumulator.add(transformedRecord); } catch (Exception e) { throw new RuntimeException(e); diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/ExportScheduler.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/ExportScheduler.java index c56a70482c..79ef3d5a61 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/ExportScheduler.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/ExportScheduler.java @@ -140,12 +140,13 @@ private String getOrCreateExportTaskId(ExportPartition exportPartition) { final String snapshotId = snapshotInfo.getSnapshotId(); try { - checkSnapshotStatus(snapshotId, DEFAULT_SNAPSHOT_STATUS_CHECK_TIMEOUT); + snapshotInfo = checkSnapshotStatus(snapshotId, DEFAULT_SNAPSHOT_STATUS_CHECK_TIMEOUT); } catch (Exception e) { LOG.warn("Check snapshot status for {} failed", snapshotId, e); sourceCoordinator.giveUpPartition(exportPartition); return null; } + progressState.setSnapshotTime(snapshotInfo.getCreateTime().toEpochMilli()); LOG.info("Creating an export task for db {} from snapshot {}", exportPartition.getDbIdentifier(), snapshotId); String exportTaskId = exportTaskManager.startExportTask( @@ -172,7 +173,7 @@ private void closeExportPartitionWithError(ExportPartition exportPartition) { sourceCoordinator.closePartition(exportPartition, DEFAULT_CLOSE_DURATION, DEFAULT_MAX_CLOSE_COUNT); } - private String checkSnapshotStatus(String snapshotId, Duration timeout) { + private SnapshotInfo checkSnapshotStatus(String snapshotId, Duration timeout) { final Instant endTime = Instant.now().plus(timeout); LOG.debug("Start checking status of snapshot {}", snapshotId); @@ -183,7 +184,7 @@ private String checkSnapshotStatus(String snapshotId, Duration timeout) { // The status should never be "copying" here if (SnapshotStatus.AVAILABLE.getStatusName().equals(status)) { LOG.info("Snapshot {} is available.", snapshotId); - return status; + return snapshotInfo; } LOG.debug("Snapshot {} is still creating. Wait and check later", snapshotId); @@ -254,13 +255,14 @@ private BiConsumer completeExport(ExportPartition exportParti LOG.info("Export for {} completed successfully", exportPartition.getPartitionKey()); ExportProgressState state = exportPartition.getProgressState().get(); - String bucket = state.getBucket(); - String prefix = state.getPrefix(); - String exportTaskId = state.getExportTaskId(); + final String bucket = state.getBucket(); + final String prefix = state.getPrefix(); + final String exportTaskId = state.getExportTaskId(); + final long snapshotTime = state.getSnapshotTime(); // Create data file partitions for processing S3 files List dataFileObjectKeys = getDataFileObjectKeys(bucket, prefix, exportTaskId); - createDataFilePartitions(bucket, exportTaskId, dataFileObjectKeys); + createDataFilePartitions(bucket, exportTaskId, dataFileObjectKeys, snapshotTime); completeExportPartition(exportPartition); } @@ -289,14 +291,18 @@ private List getDataFileObjectKeys(String bucket, String prefix, String return objectKeys; } - private void createDataFilePartitions(String bucket, String exportTaskId, List dataFileObjectKeys) { + private void createDataFilePartitions(String bucket, String exportTaskId, List dataFileObjectKeys, long snapshotTime) { LOG.info("Total of {} data files generated for export {}", dataFileObjectKeys.size(), exportTaskId); AtomicInteger totalFiles = new AtomicInteger(); for (final String objectKey : dataFileObjectKeys) { - DataFileProgressState progressState = new DataFileProgressState(); - ExportObjectKey exportObjectKey = ExportObjectKey.fromString(objectKey); - String table = exportObjectKey.getTableName(); + final DataFileProgressState progressState = new DataFileProgressState(); + final ExportObjectKey exportObjectKey = ExportObjectKey.fromString(objectKey); + final String database = exportObjectKey.getDatabaseName(); + final String table = exportObjectKey.getTableName(); + + progressState.setSourceDatabase(database); progressState.setSourceTable(table); + progressState.setSnapshotTime(snapshotTime); DataFilePartition dataFilePartition = new DataFilePartition(exportTaskId, bucket, objectKey, Optional.of(progressState)); sourceCoordinator.createPartition(dataFilePartition); 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 f35975f5b2..83b86c096d 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,7 +19,6 @@ import org.slf4j.LoggerFactory; import java.time.Duration; -import java.time.Instant; import java.util.Optional; public class LeaderScheduler implements Runnable { @@ -98,9 +97,8 @@ private void init() { sourceCoordinator.createPartition(new GlobalState(sourceConfig.getDbIdentifier(), null)); if (sourceConfig.isExportEnabled()) { - Instant startTime = Instant.now(); LOG.debug("Export is enabled. Creating export partition in the source coordination store."); - createExportPartition(sourceConfig, startTime); + createExportPartition(sourceConfig); } if (sourceConfig.isStreamEnabled()) { @@ -113,14 +111,13 @@ private void init() { leaderProgressState.setInitialized(true); } - private void createExportPartition(RdsSourceConfig sourceConfig, Instant exportTime) { + private void createExportPartition(RdsSourceConfig sourceConfig) { ExportProgressState progressState = new ExportProgressState(); progressState.setIamRoleArn(sourceConfig.getAwsAuthenticationConfig().getAwsStsRoleArn()); progressState.setBucket(sourceConfig.getS3Bucket()); progressState.setPrefix(sourceConfig.getS3Prefix()); progressState.setTables(sourceConfig.getTableNames()); progressState.setKmsKeyId(sourceConfig.getExport().getKmsKeyId()); - progressState.setExportTime(exportTime.toString()); ExportPartition exportPartition = new ExportPartition(sourceConfig.getDbIdentifier(), sourceConfig.isCluster(), progressState); sourceCoordinator.createPartition(exportPartition); } 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 index 8802d7de46..7c28eea8db 100644 --- 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 @@ -8,7 +8,10 @@ import com.github.shyiko.mysql.binlog.BinaryLogClient; import org.opensearch.dataprepper.plugins.source.rds.RdsSourceConfig; import software.amazon.awssdk.services.rds.RdsClient; +import software.amazon.awssdk.services.rds.model.DBCluster; import software.amazon.awssdk.services.rds.model.DBInstance; +import software.amazon.awssdk.services.rds.model.DescribeDbClustersRequest; +import software.amazon.awssdk.services.rds.model.DescribeDbClustersResponse; import software.amazon.awssdk.services.rds.model.DescribeDbInstancesRequest; import software.amazon.awssdk.services.rds.model.DescribeDbInstancesResponse; @@ -24,13 +27,21 @@ public BinlogClientFactory(final RdsSourceConfig sourceConfig, final RdsClient r } public BinaryLogClient create() { - DBInstance dbInstance = describeDbInstance(sourceConfig.getDbIdentifier()); + // TODO: refactor SnapshotStrategy to RdsApiStrategy to accommodate more APIs for clusters and instances + String hostName; + int port; + if (sourceConfig.isCluster()) { + DBCluster dbCluster = describeDbCluster(sourceConfig.getDbIdentifier()); + hostName = dbCluster.endpoint(); + port = dbCluster.port(); + } else { + DBInstance dbInstance = describeDbInstance(sourceConfig.getDbIdentifier()); + hostName = dbInstance.endpoint().address(); + port = dbInstance.endpoint().port(); + } return new BinaryLogClient( - dbInstance.endpoint().address(), - dbInstance.endpoint().port(), - // For test - // "127.0.0.1", - // 3306, + hostName, + port, sourceConfig.getAuthenticationConfig().getUsername(), sourceConfig.getAuthenticationConfig().getPassword()); } @@ -43,4 +54,13 @@ private DBInstance describeDbInstance(final String dbInstanceIdentifier) { DescribeDbInstancesResponse response = rdsClient.describeDBInstances(request); return response.dbInstances().get(0); } + + private DBCluster describeDbCluster(final String dbClusterIdentifier) { + DescribeDbClustersRequest request = DescribeDbClustersRequest.builder() + .dbClusterIdentifier(dbClusterIdentifier) + .build(); + + DescribeDbClustersResponse response = rdsClient.describeDBClusters(request); + return response.dbClusters().get(0); + } } 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 7ac0dcbe2b..39beff3f32 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 @@ -29,6 +29,7 @@ import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.function.Consumer; import java.util.stream.Collectors; public class BinlogEventListener implements BinaryLogClient.EventListener { @@ -62,19 +63,19 @@ public void onEvent(com.github.shyiko.mysql.binlog.event.Event event) { switch (eventType) { case TABLE_MAP: - handleTableMapEvent(event); + handleEventAndErrors(event, this::handleTableMapEvent); break; case WRITE_ROWS: case EXT_WRITE_ROWS: - handleInsertEvent(event); + handleEventAndErrors(event, this::handleInsertEvent); break; case UPDATE_ROWS: case EXT_UPDATE_ROWS: - handleUpdateEvent(event); + handleEventAndErrors(event, this::handleUpdateEvent); break; case DELETE_ROWS: case EXT_DELETE_ROWS: - handleDeleteEvent(event); + handleEventAndErrors(event, this::handleDeleteEvent); break; } } @@ -109,6 +110,7 @@ void handleInsertEvent(com.github.shyiko.mysql.binlog.event.Event event) { } final List columnNames = tableMetadata.getColumnNames(); final List primaryKeys = tableMetadata.getPrimaryKeys(); + final long eventTimestampMillis = event.getHeader().getTimestamp(); // Construct data prepper JacksonEvent for (final Object[] rowDataArray : data.getRows()) { @@ -118,7 +120,15 @@ void handleInsertEvent(com.github.shyiko.mysql.binlog.event.Event event) { } Event pipelineEvent = recordConverter.convert( - rowDataMap, tableMetadata.getDatabaseName(), tableMetadata.getTableName(), OpenSearchBulkActions.INDEX, primaryKeys, s3Prefix); + rowDataMap, + tableMetadata.getDatabaseName(), + tableMetadata.getTableName(), + event.getHeader().getEventType(), + OpenSearchBulkActions.INDEX, + primaryKeys, + s3Prefix, + eventTimestampMillis, + eventTimestampMillis); addToBuffer(new Record<>(pipelineEvent)); } @@ -139,6 +149,7 @@ void handleUpdateEvent(com.github.shyiko.mysql.binlog.event.Event event) { } final List columnNames = tableMetadata.getColumnNames(); final List primaryKeys = tableMetadata.getPrimaryKeys(); + final long eventTimestampMillis = event.getHeader().getTimestamp(); for (Map.Entry updatedRow : data.getRows()) { // updatedRow contains data before update as key and data after update as value @@ -150,7 +161,15 @@ void handleUpdateEvent(com.github.shyiko.mysql.binlog.event.Event event) { } final Event pipelineEvent = recordConverter.convert( - dataMap, tableMetadata.getDatabaseName(), tableMetadata.getTableName(), OpenSearchBulkActions.INDEX, primaryKeys, s3Prefix); + dataMap, + tableMetadata.getDatabaseName(), + tableMetadata.getTableName(), + event.getHeader().getEventType(), + OpenSearchBulkActions.INDEX, + primaryKeys, + s3Prefix, + eventTimestampMillis, + eventTimestampMillis); addToBuffer(new Record<>(pipelineEvent)); } @@ -172,6 +191,7 @@ void handleDeleteEvent(com.github.shyiko.mysql.binlog.event.Event event) { } final List columnNames = tableMetadata.getColumnNames(); final List primaryKeys = tableMetadata.getPrimaryKeys(); + final long eventTimestampMillis = event.getHeader().getTimestamp(); for (Object[] rowDataArray : data.getRows()) { final Map rowDataMap = new HashMap<>(); @@ -180,7 +200,15 @@ void handleDeleteEvent(com.github.shyiko.mysql.binlog.event.Event event) { } final Event pipelineEvent = recordConverter.convert( - rowDataMap, tableMetadata.getDatabaseName(), tableMetadata.getTableName(), OpenSearchBulkActions.DELETE, primaryKeys, s3Prefix); + rowDataMap, + tableMetadata.getDatabaseName(), + tableMetadata.getTableName(), + event.getHeader().getEventType(), + OpenSearchBulkActions.DELETE, + primaryKeys, + s3Prefix, + eventTimestampMillis, + eventTimestampMillis); addToBuffer(new Record<>(pipelineEvent)); } @@ -206,4 +234,13 @@ private void flushBuffer() { LOG.error("Failed to flush buffer", e); } } + + private void handleEventAndErrors(com.github.shyiko.mysql.binlog.event.Event event, + Consumer function) { + try { + function.accept(event); + } catch (Exception e) { + LOG.error("Failed to process change event", e); + } + } } 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 ce130d0d12..e1e169dea5 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 @@ -65,7 +65,7 @@ public void processStream(final StreamPartition streamPartition) { } try { - LOG.info("Connecting to binary log stream."); + LOG.info("Connect to database to read change events."); binaryLogClient.connect(); } catch (IOException e) { throw new RuntimeException(e); 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 7a18dd6159..275d5c0c38 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 @@ -8,6 +8,7 @@ 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.MockedStatic; import org.mockito.junit.jupiter.MockitoExtension; @@ -17,17 +18,27 @@ import org.opensearch.dataprepper.model.event.EventFactory; import org.opensearch.dataprepper.model.record.Record; import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourceCoordinator; +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; import org.opensearch.dataprepper.plugins.source.rds.leader.LeaderScheduler; +import org.opensearch.dataprepper.plugins.source.rds.stream.StreamScheduler; import software.amazon.awssdk.services.rds.RdsClient; +import software.amazon.awssdk.services.rds.model.DBInstance; +import software.amazon.awssdk.services.rds.model.DescribeDbInstancesRequest; +import software.amazon.awssdk.services.rds.model.DescribeDbInstancesResponse; +import software.amazon.awssdk.services.rds.model.Endpoint; +import java.util.UUID; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyInt; +import static org.mockito.Mockito.RETURNS_DEEP_STUBS; +import static org.mockito.Mockito.mock; import static org.mockito.Mockito.mockStatic; +import static org.mockito.Mockito.never; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; @@ -43,7 +54,7 @@ class RdsServiceTest { @Mock private PluginMetrics pluginMetrics; - @Mock + @Mock(answer = Answers.RETURNS_DEEP_STUBS) private RdsSourceConfig sourceConfig; @Mock @@ -67,6 +78,7 @@ void setUp() { void test_normal_service_start_when_export_is_enabled() { RdsService rdsService = createObjectUnderTest(); when(sourceConfig.isExportEnabled()).thenReturn(true); + when(sourceConfig.isStreamEnabled()).thenReturn(false); try (final MockedStatic executorsMockedStatic = mockStatic(Executors.class)) { executorsMockedStatic.when(() -> Executors.newFixedThreadPool(anyInt())).thenReturn(executor); rdsService.start(buffer); @@ -75,6 +87,41 @@ void test_normal_service_start_when_export_is_enabled() { verify(executor).submit(any(LeaderScheduler.class)); verify(executor).submit(any(ExportScheduler.class)); verify(executor).submit(any(DataFileScheduler.class)); + verify(executor, never()).submit(any(StreamScheduler.class)); + } + + @Test + void test_normal_service_start_when_stream_is_enabled() { + RdsService rdsService = createObjectUnderTest(); + when(sourceConfig.isStreamEnabled()).thenReturn(true); + when(sourceConfig.isExportEnabled()).thenReturn(false); + final String dbIdentifier = UUID.randomUUID().toString(); + when(sourceConfig.getDbIdentifier()).thenReturn(dbIdentifier); + final DescribeDbInstancesResponse describeDbInstancesResponse = mock(DescribeDbInstancesResponse.class, RETURNS_DEEP_STUBS); + final Endpoint hostEndpoint = Endpoint.builder() + .address(UUID.randomUUID().toString()) + .port(3306) + .build(); + when(describeDbInstancesResponse.dbInstances().get(0)).thenReturn( + DBInstance.builder() + .endpoint(hostEndpoint) + .build()); + when(rdsClient.describeDBInstances(any(DescribeDbInstancesRequest.class))).thenReturn(describeDbInstancesResponse); + final RdsSourceConfig.AuthenticationConfig authConfig = mock(RdsSourceConfig.AuthenticationConfig.class); + when(authConfig.getUsername()).thenReturn(UUID.randomUUID().toString()); + when(authConfig.getPassword()).thenReturn(UUID.randomUUID().toString()); + when(sourceConfig.getAuthenticationConfig()).thenReturn(authConfig); + when(sourceConfig.getTlsConfig()).thenReturn(mock(TlsConfig.class)); + + try (final MockedStatic executorsMockedStatic = mockStatic(Executors.class)) { + executorsMockedStatic.when(() -> Executors.newFixedThreadPool(anyInt())).thenReturn(executor); + rdsService.start(buffer); + } + + verify(executor).submit(any(LeaderScheduler.class)); + verify(executor).submit(any(StreamScheduler.class)); + verify(executor, never()).submit(any(ExportScheduler.class)); + verify(executor, never()).submit(any(DataFileScheduler.class)); } @Test diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/converter/ExportRecordConverterTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/converter/ExportRecordConverterTest.java index 79c5597c3b..564fde166b 100644 --- a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/converter/ExportRecordConverterTest.java +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/converter/ExportRecordConverterTest.java @@ -5,47 +5,77 @@ package org.opensearch.dataprepper.plugins.source.rds.converter; +import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; import org.mockito.junit.jupiter.MockitoExtension; import org.opensearch.dataprepper.event.TestEventFactory; import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.model.event.EventBuilder; +import org.opensearch.dataprepper.model.opensearch.OpenSearchBulkActions; import org.opensearch.dataprepper.model.record.Record; +import java.util.List; import java.util.Map; +import java.util.Random; import java.util.UUID; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.sameInstance; import static org.opensearch.dataprepper.plugins.source.rds.converter.ExportRecordConverter.EXPORT_EVENT_TYPE; +import static org.opensearch.dataprepper.plugins.source.rds.converter.MetadataKeyAttributes.BULK_ACTION_METADATA_ATTRIBUTE; +import static org.opensearch.dataprepper.plugins.source.rds.converter.MetadataKeyAttributes.EVENT_DATABASE_NAME_METADATA_ATTRIBUTE; import static org.opensearch.dataprepper.plugins.source.rds.converter.MetadataKeyAttributes.EVENT_TABLE_NAME_METADATA_ATTRIBUTE; +import static org.opensearch.dataprepper.plugins.source.rds.converter.MetadataKeyAttributes.EVENT_TIMESTAMP_METADATA_ATTRIBUTE; +import static org.opensearch.dataprepper.plugins.source.rds.converter.MetadataKeyAttributes.EVENT_VERSION_FROM_TIMESTAMP; import static org.opensearch.dataprepper.plugins.source.rds.converter.MetadataKeyAttributes.INGESTION_EVENT_TYPE_ATTRIBUTE; import static org.opensearch.dataprepper.plugins.source.rds.converter.MetadataKeyAttributes.PRIMARY_KEY_DOCUMENT_ID_METADATA_ATTRIBUTE; @ExtendWith(MockitoExtension.class) class ExportRecordConverterTest { + private Random random; + private ExportRecordConverter exportRecordConverter; + + @BeforeEach + void setUp() { + random = new Random(); + exportRecordConverter = createObjectUnderTest(); + } + @Test void test_convert() { + final String databaseName = UUID.randomUUID().toString(); final String tableName = UUID.randomUUID().toString(); final String primaryKeyName = UUID.randomUUID().toString(); final String primaryKeyValue = UUID.randomUUID().toString(); + final long eventCreateTimeEpochMillis = random.nextLong(); + final long eventVersionNumber = random.nextLong(); + final Event testEvent = TestEventFactory.getTestEventFactory().eventBuilder(EventBuilder.class) - .withEventType("EVENT") + .withEventType("event") .withData(Map.of(primaryKeyName, primaryKeyValue)) .build(); Record testRecord = new Record<>(testEvent); ExportRecordConverter exportRecordConverter = new ExportRecordConverter(); - Event actualEvent = exportRecordConverter.convert(testRecord, tableName, primaryKeyName); + Event actualEvent = exportRecordConverter.convert( + testRecord, databaseName, tableName, List.of(primaryKeyName), eventCreateTimeEpochMillis, eventVersionNumber); // Assert + assertThat(actualEvent.getMetadata().getAttribute(EVENT_DATABASE_NAME_METADATA_ATTRIBUTE), equalTo(databaseName)); assertThat(actualEvent.getMetadata().getAttribute(EVENT_TABLE_NAME_METADATA_ATTRIBUTE), equalTo(tableName)); + assertThat(actualEvent.getMetadata().getAttribute(BULK_ACTION_METADATA_ATTRIBUTE), equalTo(OpenSearchBulkActions.INDEX.toString())); assertThat(actualEvent.getMetadata().getAttribute(PRIMARY_KEY_DOCUMENT_ID_METADATA_ATTRIBUTE), equalTo(primaryKeyValue)); assertThat(actualEvent.getMetadata().getAttribute(INGESTION_EVENT_TYPE_ATTRIBUTE), equalTo(EXPORT_EVENT_TYPE)); + assertThat(actualEvent.getMetadata().getAttribute(EVENT_TIMESTAMP_METADATA_ATTRIBUTE), equalTo(eventCreateTimeEpochMillis)); + assertThat(actualEvent.getMetadata().getAttribute(EVENT_VERSION_FROM_TIMESTAMP), equalTo(eventVersionNumber)); assertThat(actualEvent, sameInstance(testRecord.getData())); } + + private ExportRecordConverter createObjectUnderTest() { + return new ExportRecordConverter(); + } } \ No newline at end of file diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/converter/StreamRecordConverterTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/converter/StreamRecordConverterTest.java index 8ccecc3ff1..8b857bcf4f 100644 --- a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/converter/StreamRecordConverterTest.java +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/converter/StreamRecordConverterTest.java @@ -5,6 +5,7 @@ package org.opensearch.dataprepper.plugins.source.rds.converter; +import com.github.shyiko.mysql.binlog.event.EventType; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.opensearch.dataprepper.model.event.Event; @@ -18,10 +19,13 @@ import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.startsWith; +import static org.opensearch.dataprepper.plugins.source.rds.converter.MetadataKeyAttributes.BULK_ACTION_METADATA_ATTRIBUTE; +import static org.opensearch.dataprepper.plugins.source.rds.converter.MetadataKeyAttributes.CHANGE_EVENT_TYPE_METADATA_ATTRIBUTE; import static org.opensearch.dataprepper.plugins.source.rds.converter.MetadataKeyAttributes.EVENT_DATABASE_NAME_METADATA_ATTRIBUTE; -import static org.opensearch.dataprepper.plugins.source.rds.converter.MetadataKeyAttributes.EVENT_NAME_BULK_ACTION_METADATA_ATTRIBUTE; import static org.opensearch.dataprepper.plugins.source.rds.converter.MetadataKeyAttributes.EVENT_S3_PARTITION_KEY; import static org.opensearch.dataprepper.plugins.source.rds.converter.MetadataKeyAttributes.EVENT_TABLE_NAME_METADATA_ATTRIBUTE; +import static org.opensearch.dataprepper.plugins.source.rds.converter.MetadataKeyAttributes.EVENT_TIMESTAMP_METADATA_ATTRIBUTE; +import static org.opensearch.dataprepper.plugins.source.rds.converter.MetadataKeyAttributes.EVENT_VERSION_FROM_TIMESTAMP; import static org.opensearch.dataprepper.plugins.source.rds.converter.MetadataKeyAttributes.PRIMARY_KEY_DOCUMENT_ID_METADATA_ATTRIBUTE; import static org.opensearch.dataprepper.plugins.source.rds.converter.StreamRecordConverter.S3_PATH_DELIMITER; @@ -29,29 +33,39 @@ class StreamRecordConverterTest { private StreamRecordConverter streamRecordConverter; + private Random random; @BeforeEach void setUp() { streamRecordConverter = createObjectUnderTest(); + random = new Random(); } @Test void test_convert_returns_expected_event() { - Map rowData = Map.of("key1", "value1", "key2", "value2"); + final Map rowData = Map.of("key1", "value1", "key2", "value2"); final String databaseName = UUID.randomUUID().toString(); final String tableName = UUID.randomUUID().toString(); + final EventType eventType = EventType.EXT_WRITE_ROWS; final OpenSearchBulkActions bulkAction = OpenSearchBulkActions.INDEX; final List primaryKeys = List.of("key1"); final String s3Prefix = UUID.randomUUID().toString(); + final long eventCreateTimeEpochMillis = random.nextLong(); + final long eventVersionNumber = random.nextLong(); - Event event = streamRecordConverter.convert(rowData, databaseName, tableName, bulkAction, primaryKeys, s3Prefix); + Event event = streamRecordConverter.convert( + rowData, databaseName, tableName, eventType, bulkAction, + primaryKeys, s3Prefix, eventCreateTimeEpochMillis, eventVersionNumber); assertThat(event.toMap(), is(rowData)); assertThat(event.getMetadata().getAttribute(EVENT_DATABASE_NAME_METADATA_ATTRIBUTE), is(databaseName)); assertThat(event.getMetadata().getAttribute(EVENT_TABLE_NAME_METADATA_ATTRIBUTE), is(tableName)); - assertThat(event.getMetadata().getAttribute(EVENT_NAME_BULK_ACTION_METADATA_ATTRIBUTE), is(bulkAction.toString())); + assertThat(event.getMetadata().getAttribute(CHANGE_EVENT_TYPE_METADATA_ATTRIBUTE), is(eventType.toString())); + assertThat(event.getMetadata().getAttribute(BULK_ACTION_METADATA_ATTRIBUTE), is(bulkAction.toString())); assertThat(event.getMetadata().getAttribute(PRIMARY_KEY_DOCUMENT_ID_METADATA_ATTRIBUTE), is("value1")); assertThat(event.getMetadata().getAttribute(EVENT_S3_PARTITION_KEY).toString(), startsWith(s3Prefix + S3_PATH_DELIMITER)); + assertThat(event.getMetadata().getAttribute(EVENT_TIMESTAMP_METADATA_ATTRIBUTE), is(eventCreateTimeEpochMillis)); + assertThat(event.getMetadata().getAttribute(EVENT_VERSION_FROM_TIMESTAMP), is(eventVersionNumber)); } private StreamRecordConverter createObjectUnderTest() { From b90380ab57c73aeda0f0bb7d321382dc7d0b2e5d Mon Sep 17 00:00:00 2001 From: Srikanth Govindarajan Date: Fri, 26 Jul 2024 20:56:36 -0700 Subject: [PATCH 02/25] Lambda sink refactor (#4766) * Lambda sink refactor Signed-off-by: Srikanth Govindarajan * Address comments Signed-off-by: Srikanth Govindarajan --------- Signed-off-by: Srikanth Govindarajan --- .../{lambda => aws-lambda}/README.md | 5 +- .../{lambda => aws-lambda}/build.gradle | 10 ++-- .../lambda/sink}/LambdaSinkServiceIT.java | 4 +- .../lambda/common/accumlator/Buffer.java | 14 +++++ .../common/accumlator/BufferFactory.java | 0 .../common/accumlator/InMemoryBuffer.java | 60 +++++++++++++++++-- .../accumlator/InMemoryBufferFactory.java | 0 .../lambda/common/codec/LambdaJsonCodec.java | 0 .../config/AwsAuthenticationOptions.java | 0 .../lambda/common/config/BatchOptions.java | 0 .../common/config/ThresholdOptions.java | 2 +- .../lambda/common/util/ThresholdCheck.java | 0 .../lambda/sink/LambdaClientFactory.java | 0 .../plugins/lambda/sink/LambdaSink.java | 2 +- .../plugins/lambda/sink/LambdaSinkConfig.java | 0 .../lambda/sink/LambdaSinkService.java | 0 .../lambda/sink/dlq/DlqPushHandler.java | 0 .../sink/dlq/LambdaSinkFailedDlqData.java | 0 .../lambda/common/ThresholdCheckTest.java | 0 .../InMemoryBufferFactoryTest.java | 0 .../accumulator/InMemoryBufferTest.java | 0 .../common/codec/LambdaJsonCodecTest.java | 0 .../common/config/ThresholdOptionsTest.java | 2 +- .../lambda/sink/LambdaClientFactoryTest.java | 0 .../lambda/sink/LambdaSinkConfigTest.java | 0 .../lambda/sink/LambdaSinkServiceTest.java | 0 .../plugins/lambda/sink/LambdaSinkTest.java | 2 +- .../lambda/sink/dlq/DlqPushHandlerTest.java | 0 .../org.mockito.plugins.MockMaker | 3 + .../test/resources/simplelogger.properties | 0 settings.gradle | 2 +- 31 files changed, 87 insertions(+), 19 deletions(-) rename data-prepper-plugins/{lambda => aws-lambda}/README.md (75%) rename data-prepper-plugins/{lambda => aws-lambda}/build.gradle (82%) rename data-prepper-plugins/{lambda/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/lambda => aws-lambda/src/integrationTest/java/org/opensearch/dataprepper/plugins/lambda/sink}/LambdaSinkServiceIT.java (97%) rename data-prepper-plugins/{lambda => aws-lambda}/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/accumlator/Buffer.java (67%) rename data-prepper-plugins/{lambda => aws-lambda}/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/accumlator/BufferFactory.java (100%) rename data-prepper-plugins/{lambda => aws-lambda}/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/accumlator/InMemoryBuffer.java (64%) rename data-prepper-plugins/{lambda => aws-lambda}/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/accumlator/InMemoryBufferFactory.java (100%) rename data-prepper-plugins/{lambda => aws-lambda}/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/codec/LambdaJsonCodec.java (100%) rename data-prepper-plugins/{lambda => aws-lambda}/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/config/AwsAuthenticationOptions.java (100%) rename data-prepper-plugins/{lambda => aws-lambda}/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/config/BatchOptions.java (100%) rename data-prepper-plugins/{lambda => aws-lambda}/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/config/ThresholdOptions.java (95%) rename data-prepper-plugins/{lambda => aws-lambda}/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/util/ThresholdCheck.java (100%) rename data-prepper-plugins/{lambda => aws-lambda}/src/main/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaClientFactory.java (100%) rename data-prepper-plugins/{lambda => aws-lambda}/src/main/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSink.java (97%) rename data-prepper-plugins/{lambda => aws-lambda}/src/main/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSinkConfig.java (100%) rename data-prepper-plugins/{lambda => aws-lambda}/src/main/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSinkService.java (100%) rename data-prepper-plugins/{lambda => aws-lambda}/src/main/java/org/opensearch/dataprepper/plugins/lambda/sink/dlq/DlqPushHandler.java (100%) rename data-prepper-plugins/{lambda => aws-lambda}/src/main/java/org/opensearch/dataprepper/plugins/lambda/sink/dlq/LambdaSinkFailedDlqData.java (100%) rename data-prepper-plugins/{lambda => aws-lambda}/src/test/java/org/opensearch/dataprepper/plugins/lambda/common/ThresholdCheckTest.java (100%) rename data-prepper-plugins/{lambda => aws-lambda}/src/test/java/org/opensearch/dataprepper/plugins/lambda/common/accumulator/InMemoryBufferFactoryTest.java (100%) rename data-prepper-plugins/{lambda => aws-lambda}/src/test/java/org/opensearch/dataprepper/plugins/lambda/common/accumulator/InMemoryBufferTest.java (100%) rename data-prepper-plugins/{lambda => aws-lambda}/src/test/java/org/opensearch/dataprepper/plugins/lambda/common/codec/LambdaJsonCodecTest.java (100%) rename data-prepper-plugins/{lambda => aws-lambda}/src/test/java/org/opensearch/dataprepper/plugins/lambda/common/config/ThresholdOptionsTest.java (93%) rename data-prepper-plugins/{lambda => aws-lambda}/src/test/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaClientFactoryTest.java (100%) rename data-prepper-plugins/{lambda => aws-lambda}/src/test/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSinkConfigTest.java (100%) rename data-prepper-plugins/{lambda => aws-lambda}/src/test/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSinkServiceTest.java (100%) rename data-prepper-plugins/{lambda => aws-lambda}/src/test/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSinkTest.java (98%) rename data-prepper-plugins/{lambda => aws-lambda}/src/test/java/org/opensearch/dataprepper/plugins/lambda/sink/dlq/DlqPushHandlerTest.java (100%) create mode 100644 data-prepper-plugins/aws-lambda/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker rename data-prepper-plugins/{lambda => aws-lambda}/src/test/resources/simplelogger.properties (100%) diff --git a/data-prepper-plugins/lambda/README.md b/data-prepper-plugins/aws-lambda/README.md similarity index 75% rename from data-prepper-plugins/lambda/README.md rename to data-prepper-plugins/aws-lambda/README.md index c0b2c29211..4c49873350 100644 --- a/data-prepper-plugins/lambda/README.md +++ b/data-prepper-plugins/aws-lambda/README.md @@ -1,3 +1,4 @@ + # Lambda Sink This plugin enables you to send data from your Data Prepper pipeline directly to AWS Lambda functions for further processing. @@ -7,7 +8,7 @@ This plugin enables you to send data from your Data Prepper pipeline directly to lambda-pipeline: ... sink: - - lambda: + - aws_lambda: aws: region: "us-east-1" sts_role_arn: "" @@ -31,6 +32,6 @@ The integration tests for this plugin do not run as part of the Data Prepper bui The following command runs the integration tests: ``` -./gradlew :data-prepper-plugins:lambda-sink:integrationTest -Dtests.sink.lambda.region="us-east-1" -Dtests.sink.lambda.functionName="lambda_test_function" -Dtests.sink.lambda.sts_role_arn="arn:aws:iam::123456789012:role/dataprepper-role +./gradlew :data-prepper-plugins:aws-lambda:integrationTest -Dtests.sink.lambda.region="us-east-1" -Dtests.sink.lambda.functionName="lambda_test_function" -Dtests.sink.lambda.sts_role_arn="arn:aws:iam::123456789012:role/dataprepper-role ``` diff --git a/data-prepper-plugins/lambda/build.gradle b/data-prepper-plugins/aws-lambda/build.gradle similarity index 82% rename from data-prepper-plugins/lambda/build.gradle rename to data-prepper-plugins/aws-lambda/build.gradle index 8447c3abdf..be9280e8c8 100644 --- a/data-prepper-plugins/lambda/build.gradle +++ b/data-prepper-plugins/aws-lambda/build.gradle @@ -26,8 +26,10 @@ dependencies { testAnnotationProcessor 'org.projectlombok:lombok:1.18.20' testImplementation 'com.fasterxml.jackson.datatype:jackson-datatype-jsr310' testImplementation project(':data-prepper-test-common') - testImplementation project(':data-prepper-plugins:parse-json-processor') testImplementation testLibs.slf4j.simple + testImplementation 'org.mockito:mockito-core:4.6.1' + testImplementation 'org.junit.jupiter:junit-jupiter-api:5.8.2' + testRuntimeOnly 'org.junit.jupiter:junit-jupiter-engine:5.8.2' } test { @@ -59,9 +61,9 @@ task integrationTest(type: Test) { classpath = sourceSets.integrationTest.runtimeClasspath systemProperty 'log4j.configurationFile', 'src/test/resources/log4j2.properties' - systemProperty 'tests.sink.lambda.region', System.getProperty('tests.sink.lambda.region') - systemProperty 'tests.sink.lambda.functionName', System.getProperty('tests.sink.lambda.functionName') - systemProperty 'tests.sink.lambda.sts_role_arn', System.getProperty('tests.sink.lambda.sts_role_arn') + systemProperty 'tests.lambda.sink.region', System.getProperty('tests.lambda.sink.region') + systemProperty 'tests.lambda.sink.functionName', System.getProperty('tests.lambda.sink.functionName') + systemProperty 'tests.lambda.sink.sts_role_arn', System.getProperty('tests.lambda.sink.sts_role_arn') filter { includeTestsMatching '*IT' diff --git a/data-prepper-plugins/lambda/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/lambda/LambdaSinkServiceIT.java b/data-prepper-plugins/aws-lambda/src/integrationTest/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSinkServiceIT.java similarity index 97% rename from data-prepper-plugins/lambda/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/lambda/LambdaSinkServiceIT.java rename to data-prepper-plugins/aws-lambda/src/integrationTest/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSinkServiceIT.java index 76fb4831ce..1a7e169a47 100644 --- a/data-prepper-plugins/lambda/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/lambda/LambdaSinkServiceIT.java +++ b/data-prepper-plugins/aws-lambda/src/integrationTest/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSinkServiceIT.java @@ -3,7 +3,7 @@ * SPDX-License-Identifier: Apache-2.0 */ -package org.opensearch.dataprepper.plugins.sink.lambda; +package org.opensearch.dataprepper.plugins.lambda.sink; import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; @@ -35,8 +35,6 @@ import org.opensearch.dataprepper.plugins.lambda.common.config.AwsAuthenticationOptions; import org.opensearch.dataprepper.plugins.lambda.common.config.BatchOptions; import org.opensearch.dataprepper.plugins.lambda.common.config.ThresholdOptions; -import org.opensearch.dataprepper.plugins.lambda.sink.LambdaSinkConfig; -import org.opensearch.dataprepper.plugins.lambda.sink.LambdaSinkService; import org.opensearch.dataprepper.plugins.lambda.sink.dlq.DlqPushHandler; import software.amazon.awssdk.regions.Region; import software.amazon.awssdk.services.lambda.LambdaClient; diff --git a/data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/accumlator/Buffer.java b/data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/accumlator/Buffer.java similarity index 67% rename from data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/accumlator/Buffer.java rename to data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/accumlator/Buffer.java index f52a8e5de0..a2c5dde4a9 100644 --- a/data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/accumlator/Buffer.java +++ b/data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/accumlator/Buffer.java @@ -31,4 +31,18 @@ public interface Buffer { SdkBytes getPayload(); void setEventCount(int eventCount); + + //Metrics + public Duration getFlushLambdaSyncLatencyMetric(); + + public Long getPayloadRequestSyncSize(); + + public Duration getFlushLambdaAsyncLatencyMetric(); + + public Long getPayloadResponseSyncSize(); + + public Long getPayloadRequestAsyncSize(); + + public Long getPayloadResponseAsyncSize(); + } diff --git a/data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/accumlator/BufferFactory.java b/data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/accumlator/BufferFactory.java similarity index 100% rename from data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/accumlator/BufferFactory.java rename to data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/accumlator/BufferFactory.java diff --git a/data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/accumlator/InMemoryBuffer.java b/data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/accumlator/InMemoryBuffer.java similarity index 64% rename from data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/accumlator/InMemoryBuffer.java rename to data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/accumlator/InMemoryBuffer.java index 5d9d5a5134..095e6f47b2 100644 --- a/data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/accumlator/InMemoryBuffer.java +++ b/data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/accumlator/InMemoryBuffer.java @@ -12,6 +12,7 @@ import software.amazon.awssdk.services.lambda.LambdaClient; import software.amazon.awssdk.services.lambda.model.InvokeRequest; import software.amazon.awssdk.services.lambda.model.InvokeResponse; +import software.amazon.awssdk.services.lambda.model.LambdaException; import java.io.ByteArrayOutputStream; import java.io.IOException; @@ -31,7 +32,13 @@ public class InMemoryBuffer implements Buffer { private final String invocationType; private int eventCount; private final StopWatch watch; + private final StopWatch lambdaSyncLatencyWatch; + private final StopWatch lambdaAsyncLatencyWatch; private boolean isCodecStarted; + private long payloadRequestSyncSize; + private long payloadResponseSyncSize; + private long payloadRequestAsyncSize; + private long payloadResponseAsyncSize; public InMemoryBuffer(LambdaClient lambdaClient, String functionName, String invocationType) { @@ -44,6 +51,12 @@ public InMemoryBuffer(LambdaClient lambdaClient, String functionName, String inv watch = new StopWatch(); watch.start(); isCodecStarted = false; + lambdaSyncLatencyWatch = new StopWatch(); + lambdaAsyncLatencyWatch = new StopWatch(); + payloadRequestSyncSize = 0; + payloadResponseSyncSize = 0; + payloadRequestAsyncSize = 0; + payloadResponseAsyncSize =0; } @Override @@ -65,6 +78,7 @@ public Duration getDuration() { public void flushToLambdaAsync() { InvokeResponse resp; SdkBytes payload = getPayload(); + payloadRequestAsyncSize = payload.asByteArray().length; // Setup an InvokeRequest. InvokeRequest request = InvokeRequest.builder() @@ -73,13 +87,17 @@ public void flushToLambdaAsync() { .invocationType(invocationType) .build(); - lambdaClient.invoke(request); + lambdaAsyncLatencyWatch.start(); + resp = lambdaClient.invoke(request); + lambdaAsyncLatencyWatch.stop(); + payloadResponseAsyncSize = resp.payload().asByteArray().length; } @Override public InvokeResponse flushToLambdaSync() { - InvokeResponse resp; + InvokeResponse resp = null; SdkBytes payload = getPayload(); + payloadRequestSyncSize = payload.asByteArray().length; // Setup an InvokeRequest. InvokeRequest request = InvokeRequest.builder() @@ -88,8 +106,16 @@ public InvokeResponse flushToLambdaSync() { .invocationType(invocationType) .build(); - resp = lambdaClient.invoke(request); - return resp; + lambdaSyncLatencyWatch.start(); + try { + resp = lambdaClient.invoke(request); + payloadResponseSyncSize = resp.payload().asByteArray().length; + lambdaSyncLatencyWatch.stop(); + return resp; + } catch (LambdaException e){ + lambdaSyncLatencyWatch.stop(); + throw new RuntimeException(e); + } } private SdkBytes validatePayload(String payload_string) { @@ -121,6 +147,30 @@ public SdkBytes getPayload() { byte[] bytes = byteArrayOutputStream.toByteArray(); SdkBytes sdkBytes = SdkBytes.fromByteArray(bytes); return sdkBytes; - } + } + + public Duration getFlushLambdaSyncLatencyMetric (){ + return Duration.ofMillis(lambdaSyncLatencyWatch.getTime(TimeUnit.MILLISECONDS)); + } + + public Duration getFlushLambdaAsyncLatencyMetric (){ + return Duration.ofMillis(lambdaAsyncLatencyWatch.getTime(TimeUnit.MILLISECONDS)); + } + + public Long getPayloadRequestSyncSize() { + return payloadRequestSyncSize; + } + + public Long getPayloadResponseSyncSize() { + return payloadResponseSyncSize; + } + + public Long getPayloadRequestAsyncSize() { + return payloadRequestAsyncSize; + } + + public Long getPayloadResponseAsyncSize() { + return payloadResponseAsyncSize; + } } diff --git a/data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/accumlator/InMemoryBufferFactory.java b/data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/accumlator/InMemoryBufferFactory.java similarity index 100% rename from data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/accumlator/InMemoryBufferFactory.java rename to data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/accumlator/InMemoryBufferFactory.java diff --git a/data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/codec/LambdaJsonCodec.java b/data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/codec/LambdaJsonCodec.java similarity index 100% rename from data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/codec/LambdaJsonCodec.java rename to data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/codec/LambdaJsonCodec.java diff --git a/data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/config/AwsAuthenticationOptions.java b/data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/config/AwsAuthenticationOptions.java similarity index 100% rename from data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/config/AwsAuthenticationOptions.java rename to data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/config/AwsAuthenticationOptions.java diff --git a/data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/config/BatchOptions.java b/data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/config/BatchOptions.java similarity index 100% rename from data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/config/BatchOptions.java rename to data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/config/BatchOptions.java diff --git a/data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/config/ThresholdOptions.java b/data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/config/ThresholdOptions.java similarity index 95% rename from data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/config/ThresholdOptions.java rename to data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/config/ThresholdOptions.java index 1f92b90b48..ca8ed6e574 100644 --- a/data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/config/ThresholdOptions.java +++ b/data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/config/ThresholdOptions.java @@ -17,7 +17,7 @@ public class ThresholdOptions { - private static final String DEFAULT_BYTE_CAPACITY = "6mb"; + private static final String DEFAULT_BYTE_CAPACITY = "3mb"; @JsonProperty("event_count") @Size(min = 0, max = 10000000, message = "event_count size should be between 0 and 10000000") diff --git a/data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/util/ThresholdCheck.java b/data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/util/ThresholdCheck.java similarity index 100% rename from data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/util/ThresholdCheck.java rename to data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/util/ThresholdCheck.java diff --git a/data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaClientFactory.java b/data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaClientFactory.java similarity index 100% rename from data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaClientFactory.java rename to data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaClientFactory.java diff --git a/data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSink.java b/data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSink.java similarity index 97% rename from data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSink.java rename to data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSink.java index 54e484fd13..715ef3295d 100644 --- a/data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSink.java +++ b/data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSink.java @@ -26,7 +26,7 @@ import java.util.Collection; -@DataPrepperPlugin(name = "lambda", pluginType = Sink.class, pluginConfigurationType = LambdaSinkConfig.class) +@DataPrepperPlugin(name = "aws_lambda", pluginType = Sink.class, pluginConfigurationType = LambdaSinkConfig.class) public class LambdaSink extends AbstractSink> { private static final Logger LOG = LoggerFactory.getLogger(LambdaSink.class); diff --git a/data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSinkConfig.java b/data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSinkConfig.java similarity index 100% rename from data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSinkConfig.java rename to data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSinkConfig.java diff --git a/data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSinkService.java b/data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSinkService.java similarity index 100% rename from data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSinkService.java rename to data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSinkService.java diff --git a/data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/sink/dlq/DlqPushHandler.java b/data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/sink/dlq/DlqPushHandler.java similarity index 100% rename from data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/sink/dlq/DlqPushHandler.java rename to data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/sink/dlq/DlqPushHandler.java diff --git a/data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/sink/dlq/LambdaSinkFailedDlqData.java b/data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/sink/dlq/LambdaSinkFailedDlqData.java similarity index 100% rename from data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/sink/dlq/LambdaSinkFailedDlqData.java rename to data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/sink/dlq/LambdaSinkFailedDlqData.java diff --git a/data-prepper-plugins/lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/common/ThresholdCheckTest.java b/data-prepper-plugins/aws-lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/common/ThresholdCheckTest.java similarity index 100% rename from data-prepper-plugins/lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/common/ThresholdCheckTest.java rename to data-prepper-plugins/aws-lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/common/ThresholdCheckTest.java diff --git a/data-prepper-plugins/lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/common/accumulator/InMemoryBufferFactoryTest.java b/data-prepper-plugins/aws-lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/common/accumulator/InMemoryBufferFactoryTest.java similarity index 100% rename from data-prepper-plugins/lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/common/accumulator/InMemoryBufferFactoryTest.java rename to data-prepper-plugins/aws-lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/common/accumulator/InMemoryBufferFactoryTest.java diff --git a/data-prepper-plugins/lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/common/accumulator/InMemoryBufferTest.java b/data-prepper-plugins/aws-lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/common/accumulator/InMemoryBufferTest.java similarity index 100% rename from data-prepper-plugins/lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/common/accumulator/InMemoryBufferTest.java rename to data-prepper-plugins/aws-lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/common/accumulator/InMemoryBufferTest.java diff --git a/data-prepper-plugins/lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/common/codec/LambdaJsonCodecTest.java b/data-prepper-plugins/aws-lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/common/codec/LambdaJsonCodecTest.java similarity index 100% rename from data-prepper-plugins/lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/common/codec/LambdaJsonCodecTest.java rename to data-prepper-plugins/aws-lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/common/codec/LambdaJsonCodecTest.java diff --git a/data-prepper-plugins/lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/common/config/ThresholdOptionsTest.java b/data-prepper-plugins/aws-lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/common/config/ThresholdOptionsTest.java similarity index 93% rename from data-prepper-plugins/lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/common/config/ThresholdOptionsTest.java rename to data-prepper-plugins/aws-lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/common/config/ThresholdOptionsTest.java index 5d12aca3da..98437b49fe 100644 --- a/data-prepper-plugins/lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/common/config/ThresholdOptionsTest.java +++ b/data-prepper-plugins/aws-lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/common/config/ThresholdOptionsTest.java @@ -11,7 +11,7 @@ import org.opensearch.dataprepper.model.types.ByteCount; class ThresholdOptionsTest { - private static final String DEFAULT_BYTE_CAPACITY = "6mb"; + private static final String DEFAULT_BYTE_CAPACITY = "3mb"; private static final int DEFAULT_EVENT_COUNT = 0; @Test diff --git a/data-prepper-plugins/lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaClientFactoryTest.java b/data-prepper-plugins/aws-lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaClientFactoryTest.java similarity index 100% rename from data-prepper-plugins/lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaClientFactoryTest.java rename to data-prepper-plugins/aws-lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaClientFactoryTest.java diff --git a/data-prepper-plugins/lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSinkConfigTest.java b/data-prepper-plugins/aws-lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSinkConfigTest.java similarity index 100% rename from data-prepper-plugins/lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSinkConfigTest.java rename to data-prepper-plugins/aws-lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSinkConfigTest.java diff --git a/data-prepper-plugins/lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSinkServiceTest.java b/data-prepper-plugins/aws-lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSinkServiceTest.java similarity index 100% rename from data-prepper-plugins/lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSinkServiceTest.java rename to data-prepper-plugins/aws-lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSinkServiceTest.java diff --git a/data-prepper-plugins/lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSinkTest.java b/data-prepper-plugins/aws-lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSinkTest.java similarity index 98% rename from data-prepper-plugins/lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSinkTest.java rename to data-prepper-plugins/aws-lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSinkTest.java index 9a042014f0..1842795e7c 100644 --- a/data-prepper-plugins/lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSinkTest.java +++ b/data-prepper-plugins/aws-lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSinkTest.java @@ -28,7 +28,7 @@ class LambdaSinkTest { public static final String S3_REGION = "us-east-1"; public static final String CODEC_PLUGIN_NAME = "json"; - public static final String SINK_PLUGIN_NAME = "lambda"; + public static final String SINK_PLUGIN_NAME = "aws_lambda"; public static final String SINK_PIPELINE_NAME = "lambda-sink-pipeline"; private LambdaSinkConfig lambdaSinkConfig; private LambdaSink lambdaSink; diff --git a/data-prepper-plugins/lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/sink/dlq/DlqPushHandlerTest.java b/data-prepper-plugins/aws-lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/sink/dlq/DlqPushHandlerTest.java similarity index 100% rename from data-prepper-plugins/lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/sink/dlq/DlqPushHandlerTest.java rename to data-prepper-plugins/aws-lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/sink/dlq/DlqPushHandlerTest.java diff --git a/data-prepper-plugins/aws-lambda/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker b/data-prepper-plugins/aws-lambda/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker new file mode 100644 index 0000000000..23c33feb6d --- /dev/null +++ b/data-prepper-plugins/aws-lambda/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker @@ -0,0 +1,3 @@ +# To enable mocking of final classes with vanilla Mockito +# https://github.com/mockito/mockito/wiki/What%27s-new-in-Mockito-2#mock-the-unmockable-opt-in-mocking-of-final-classesmethods +mock-maker-inline diff --git a/data-prepper-plugins/lambda/src/test/resources/simplelogger.properties b/data-prepper-plugins/aws-lambda/src/test/resources/simplelogger.properties similarity index 100% rename from data-prepper-plugins/lambda/src/test/resources/simplelogger.properties rename to data-prepper-plugins/aws-lambda/src/test/resources/simplelogger.properties diff --git a/settings.gradle b/settings.gradle index 9d84b2ccf0..cb7e888c53 100644 --- a/settings.gradle +++ b/settings.gradle @@ -178,4 +178,4 @@ include 'data-prepper-plugins:mongodb' include 'data-prepper-plugins:rds-source' include 'data-prepper-plugins:http-source-common' include 'data-prepper-plugins:http-common' -include 'data-prepper-plugins:lambda' \ No newline at end of file +include 'data-prepper-plugins:aws-lambda' \ No newline at end of file From 101be0400db89e24e35dfaf16c732d8663620895 Mon Sep 17 00:00:00 2001 From: Hai Yan <8153134+oeyh@users.noreply.github.com> Date: Mon, 29 Jul 2024 11:32:25 -0500 Subject: [PATCH 03/25] Add json property descriptions to dissect, flatten, copy_value and translate processor (#4760) Signed-off-by: Hai Yan --- .../dissect/DissectProcessorConfig.java | 8 ++++++++ .../flatten/FlattenProcessorConfig.java | 16 ++++++++++++++++ .../mutateevent/CopyValueProcessorConfig.java | 11 +++++++++++ .../processor/translate/FileParameterConfig.java | 3 +++ .../translate/MappingsParameterConfig.java | 3 +++ .../processor/translate/S3ObjectConfig.java | 4 ++++ .../translate/TargetsParameterConfig.java | 11 +++++++++++ .../translate/TranslateProcessorConfig.java | 3 +++ 8 files changed, 59 insertions(+) diff --git a/data-prepper-plugins/dissect-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/dissect/DissectProcessorConfig.java b/data-prepper-plugins/dissect-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/dissect/DissectProcessorConfig.java index bd1ca0d910..bc8ef4705a 100644 --- a/data-prepper-plugins/dissect-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/dissect/DissectProcessorConfig.java +++ b/data-prepper-plugins/dissect-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/dissect/DissectProcessorConfig.java @@ -1,6 +1,7 @@ package org.opensearch.dataprepper.plugins.processor.dissect; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonPropertyDescription; import jakarta.validation.constraints.NotNull; import org.opensearch.dataprepper.plugins.processor.mutateevent.TargetType; @@ -9,10 +10,17 @@ public class DissectProcessorConfig { @NotNull @JsonProperty("map") + @JsonPropertyDescription("Defines the `dissect` patterns for specific keys. For details on how to define fields " + + "in the `dissect` pattern, see [Field notations](#field-notations).") private Map map; @JsonProperty("target_types") + @JsonPropertyDescription("Specifies the data types for extract fields. Valid options are `integer`, " + + "`double`, `string`, and `boolean`. By default, all fields are of the `string` type.") private Map targetTypes; @JsonProperty("dissect_when") + @JsonPropertyDescription("Specifies a condition for performing the `dissect` operation using a " + + "[Data Prepper expression]({{site.url}}{{site.baseurl}}/data-prepper/pipelines/expression-syntax/). " + + "If specified, the `dissect` operation will only run when the expression evaluates to true.") private String dissectWhen; public String getDissectWhen(){ diff --git a/data-prepper-plugins/flatten-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/flatten/FlattenProcessorConfig.java b/data-prepper-plugins/flatten-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/flatten/FlattenProcessorConfig.java index c1208f5f40..783f73a9da 100644 --- a/data-prepper-plugins/flatten-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/flatten/FlattenProcessorConfig.java +++ b/data-prepper-plugins/flatten-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/flatten/FlattenProcessorConfig.java @@ -7,6 +7,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonPropertyDescription; import jakarta.validation.constraints.AssertTrue; import jakarta.validation.constraints.NotNull; @@ -19,28 +20,43 @@ public class FlattenProcessorConfig { @NotNull @JsonProperty("source") + @JsonPropertyDescription("The source key on which to perform the operation. If set to an empty string (`\"\"`), " + + "then the processor uses the root of the event as the source.") private String source; @NotNull @JsonProperty("target") + @JsonPropertyDescription("The target key to put into the flattened fields. If set to an empty string (`\"\"`), " + + "then the processor uses the root of the event as the target.") private String target; @JsonProperty("remove_processed_fields") + @JsonPropertyDescription("When `true`, the processor removes all processed fields from the source. Default is `false`.") private boolean removeProcessedFields = false; @JsonProperty("remove_list_indices") + @JsonPropertyDescription("When `true`, the processor converts the fields from the source map into lists and " + + "puts the lists into the target field. Default is `false`.") private boolean removeListIndices = false; @JsonProperty("remove_brackets") + @JsonPropertyDescription("When `true`, the processor also removes brackets around the indices. Can only be " + + "set to `true` when `remove_list_indices` is `true`.") private boolean removeBrackets = false; @JsonProperty("exclude_keys") + @JsonPropertyDescription("The keys from the source field that should be excluded from processing. " + + "Default is an empty list (`[]`).") private List excludeKeys = DEFAULT_EXCLUDE_KEYS; @JsonProperty("flatten_when") + @JsonPropertyDescription("A [conditional expression](https://opensearch.org/docs/latest/data-prepper/pipelines/expression-syntax/), " + + "such as `/some-key == \"test\"'`, that determines whether the `flatten` processor will be run on the " + + "event. Default is `null`, which means that all events will be processed unless otherwise stated.") private String flattenWhen; @JsonProperty("tags_on_failure") + @JsonPropertyDescription("A list of tags to add to the event metadata when the event fails to process.") private List tagsOnFailure; public String getSource() { diff --git a/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/CopyValueProcessorConfig.java b/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/CopyValueProcessorConfig.java index b6aee5c7ba..1d24198d84 100644 --- a/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/CopyValueProcessorConfig.java +++ b/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/CopyValueProcessorConfig.java @@ -6,6 +6,7 @@ package org.opensearch.dataprepper.plugins.processor.mutateevent; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonPropertyDescription; import jakarta.validation.Valid; import jakarta.validation.constraints.AssertTrue; import jakarta.validation.constraints.NotEmpty; @@ -18,17 +19,23 @@ public static class Entry { @NotEmpty @NotNull @JsonProperty("from_key") + @JsonPropertyDescription("The key of the entry to be copied.") private String fromKey; @NotEmpty @NotNull @JsonProperty("to_key") + @JsonPropertyDescription("The key of the new entry to be added.") private String toKey; @JsonProperty("copy_when") + @JsonPropertyDescription("A [conditional expression](https://opensearch.org/docs/latest/data-prepper/pipelines/expression-syntax/), " + + "such as `/some-key == \"test\"'`, that will be evaluated to determine whether the processor will be run on the event.") private String copyWhen; @JsonProperty("overwrite_if_to_key_exists") + @JsonPropertyDescription("When set to `true`, the existing value is overwritten if `key` already exists in " + + "the event. The default value is `false`.") private boolean overwriteIfToKeyExists = false; public String getFromKey() { @@ -60,15 +67,19 @@ public Entry() { @NotEmpty @NotNull @Valid + @JsonPropertyDescription("A list of entries to be copied in an event.") private List entries; @JsonProperty("from_list") + @JsonPropertyDescription("The source list to copy values from.") private String fromList; @JsonProperty("to_list") + @JsonPropertyDescription("The target list to copy values to.") private String toList; @JsonProperty("overwrite_if_to_list_exists") + @JsonPropertyDescription("When set to `true`, the existing value is overwritten if `key` already exists in the event. The default value is `false`.") private boolean overwriteIfToListExists = false; @AssertTrue(message = "Both from_list and to_list should be specified when copying entries between lists.") diff --git a/data-prepper-plugins/translate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/translate/FileParameterConfig.java b/data-prepper-plugins/translate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/translate/FileParameterConfig.java index 8d5cc674f1..83dc30952e 100644 --- a/data-prepper-plugins/translate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/translate/FileParameterConfig.java +++ b/data-prepper-plugins/translate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/translate/FileParameterConfig.java @@ -6,6 +6,7 @@ package org.opensearch.dataprepper.plugins.processor.translate; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonPropertyDescription; import jakarta.validation.Valid; import jakarta.validation.constraints.NotNull; import java.util.List; @@ -13,10 +14,12 @@ public class FileParameterConfig { @JsonProperty("name") + @JsonPropertyDescription("The full path to a local file or key name for an S3 object.") @NotNull private String fileName; @JsonProperty("aws") + @JsonPropertyDescription("The AWS configuration when the file is an S3 object. ") @Valid private S3ObjectConfig awsConfig; diff --git a/data-prepper-plugins/translate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/translate/MappingsParameterConfig.java b/data-prepper-plugins/translate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/translate/MappingsParameterConfig.java index 7146f62a4c..1bcb14c7a8 100644 --- a/data-prepper-plugins/translate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/translate/MappingsParameterConfig.java +++ b/data-prepper-plugins/translate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/translate/MappingsParameterConfig.java @@ -1,6 +1,7 @@ package org.opensearch.dataprepper.plugins.processor.translate; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonPropertyDescription; import jakarta.validation.Valid; import jakarta.validation.constraints.AssertTrue; import jakarta.validation.constraints.NotNull; @@ -12,10 +13,12 @@ public class MappingsParameterConfig { @JsonProperty("source") + @JsonPropertyDescription("The source field to translate. Can be a string or a list of strings.") @NotNull private Object source; @JsonProperty("targets") + @JsonPropertyDescription("A list of target field configurations, such as the target field key or translation maps.") @Valid private List targetsParameterConfigs = new ArrayList<>(); diff --git a/data-prepper-plugins/translate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/translate/S3ObjectConfig.java b/data-prepper-plugins/translate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/translate/S3ObjectConfig.java index 56e7aacf27..34dd421994 100644 --- a/data-prepper-plugins/translate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/translate/S3ObjectConfig.java +++ b/data-prepper-plugins/translate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/translate/S3ObjectConfig.java @@ -6,19 +6,23 @@ package org.opensearch.dataprepper.plugins.processor.translate; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonPropertyDescription; import jakarta.validation.constraints.NotNull; public class S3ObjectConfig { @JsonProperty("bucket") + @JsonPropertyDescription("The Amazon S3 bucket name.") @NotNull private String bucket; @JsonProperty("region") + @JsonPropertyDescription("The AWS Region to use for credentials.") @NotNull private String region; @JsonProperty("sts_role_arn") + @JsonPropertyDescription("The AWS Security Token Service (AWS STS) role to assume for requests to Amazon S3.") @NotNull private String stsRoleArn; diff --git a/data-prepper-plugins/translate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/translate/TargetsParameterConfig.java b/data-prepper-plugins/translate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/translate/TargetsParameterConfig.java index 53a250ee37..18e796b593 100644 --- a/data-prepper-plugins/translate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/translate/TargetsParameterConfig.java +++ b/data-prepper-plugins/translate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/translate/TargetsParameterConfig.java @@ -1,6 +1,7 @@ package org.opensearch.dataprepper.plugins.processor.translate; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonPropertyDescription; import jakarta.validation.constraints.AssertTrue; import jakarta.validation.constraints.NotEmpty; import jakarta.validation.constraints.NotNull; @@ -22,18 +23,28 @@ public class TargetsParameterConfig { private final Map individualMappings = new HashMap<>(); private final Map compiledPatterns = new HashMap<>(); @JsonProperty("target") + @JsonPropertyDescription("The key that specifies the field in the output in which the translated value will be placed.") @NotNull @NotEmpty private String target; @JsonProperty("map") + @JsonPropertyDescription("A list of key-value pairs that define the translations. Each key represents a possible " + + "value in the source field, and the corresponding value represents what it should be translated to. " + + "For examples, see [map option](#map-option). At least one of `map` and `regex` should be configured.") private Map map; @JsonProperty("translate_when") + @JsonPropertyDescription("Uses a [Data Prepper expression]({{site.url}}{{site.baseurl}}/data-prepper/pipelines/expression-syntax/) " + + "to specify a condition for performing the translation. When specified, the expression will only translate when the condition is met.") private String translateWhen; @JsonProperty("regex") + @JsonPropertyDescription("A map of keys that defines the translation map. For more options, see [regex option](#regex-option). " + + "At least one of `map` and `regex` should be configured.") private RegexParameterConfiguration regexParameterConfig; @JsonProperty("default") + @JsonPropertyDescription("The default value to use when no match is found during translation.") private String defaultValue; @JsonProperty("type") + @JsonPropertyDescription("Specifies the data type for the target value.") private TargetType targetType = TargetType.STRING; public TargetsParameterConfig(){ diff --git a/data-prepper-plugins/translate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/translate/TranslateProcessorConfig.java b/data-prepper-plugins/translate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/translate/TranslateProcessorConfig.java index 70744f683a..de0949a31c 100644 --- a/data-prepper-plugins/translate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/translate/TranslateProcessorConfig.java +++ b/data-prepper-plugins/translate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/translate/TranslateProcessorConfig.java @@ -8,6 +8,7 @@ import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonPropertyDescription; import jakarta.validation.Valid; import jakarta.validation.constraints.AssertTrue; @@ -19,10 +20,12 @@ public class TranslateProcessorConfig { @JsonProperty("file") + @JsonPropertyDescription("Points to the file that contains mapping configurations. For more information, see [file](#file).") @Valid private FileParameterConfig fileParameterConfig; @JsonProperty("mappings") + @JsonPropertyDescription("Defines inline mappings. For more information, see [mappings](#mappings).") @Valid private List mappingsParameterConfigs = new ArrayList<>(); From 8f33225a32bc4ba226f6f83eb88d3ffff5212ad6 Mon Sep 17 00:00:00 2001 From: Hai Yan <8153134+oeyh@users.noreply.github.com> Date: Mon, 29 Jul 2024 11:32:31 -0500 Subject: [PATCH 04/25] Add json property description for list-to-map, map-to-list and user-agent processor (#4759) Signed-off-by: Hai Yan --- .../mutateevent/ListToMapProcessorConfig.java | 21 +++++++++++++++++++ .../mutateevent/MapToListProcessorConfig.java | 16 ++++++++++++++ .../useragent/UserAgentProcessorConfig.java | 6 ++++++ 3 files changed, 43 insertions(+) diff --git a/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/ListToMapProcessorConfig.java b/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/ListToMapProcessorConfig.java index 228f07f08e..b63deb727c 100644 --- a/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/ListToMapProcessorConfig.java +++ b/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/ListToMapProcessorConfig.java @@ -7,6 +7,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonPropertyDescription; import jakarta.validation.constraints.NotEmpty; import jakarta.validation.constraints.NotNull; @@ -41,35 +42,55 @@ static FlattenedElement fromOptionValue(final String option) { @NotEmpty @NotNull @JsonProperty("source") + @JsonPropertyDescription("The list of objects with `key` fields to be converted into keys for the generated map.") private String source; @JsonProperty("target") + @JsonPropertyDescription("The target for the generated map. When not specified, the generated map will be " + + "placed in the root node.") private String target = null; @JsonProperty("key") + @JsonPropertyDescription("The key of the fields to be extracted as keys in the generated mappings. Must be " + + "specified if `use_source_key` is `false`.") private String key; @JsonProperty("value_key") + @JsonPropertyDescription("When specified, values given a `value_key` in objects contained in the source list " + + "will be extracted and converted into the value specified by this option based on the generated map. " + + "When not specified, objects contained in the source list retain their original value when mapped.") private String valueKey = null; @JsonProperty("use_source_key") + @JsonPropertyDescription("When `true`, keys in the generated map will use original keys from the source. " + + "Default is `false`.") private boolean useSourceKey = false; @JsonProperty("extract_value") + @JsonPropertyDescription("When `true`, object values from the source list will be extracted and added to " + + "the generated map. When `false`, object values from the source list are added to the generated map " + + "as they appear in the source list. Default is `false`") private boolean extractValue = false; @NotNull @JsonProperty("flatten") + @JsonPropertyDescription("When `true`, values in the generated map output flatten into single items based on " + + "the `flattened_element`. Otherwise, objects mapped to values from the generated map appear as lists.") private boolean flatten = false; @NotNull @JsonProperty("flattened_element") + @JsonPropertyDescription("The element to keep, either `first` or `last`, when `flatten` is set to `true`.") private FlattenedElement flattenedElement = FlattenedElement.FIRST; @JsonProperty("list_to_map_when") + @JsonPropertyDescription("A [conditional expression](https://opensearch.org/docs/latest/data-prepper/pipelines/expression-syntax/), " + + "such as `/some-key == \"test\"'`, that will be evaluated to determine whether the processor will be " + + "run on the event. Default is `null`. All events will be processed unless otherwise stated.") private String listToMapWhen; @JsonProperty("tags_on_failure") + @JsonPropertyDescription("A list of tags to add to the event metadata when the event fails to process.") private List tagsOnFailure; public String getSource() { diff --git a/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/MapToListProcessorConfig.java b/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/MapToListProcessorConfig.java index 46a2ec79f0..ce317eca49 100644 --- a/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/MapToListProcessorConfig.java +++ b/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/MapToListProcessorConfig.java @@ -6,6 +6,7 @@ package org.opensearch.dataprepper.plugins.processor.mutateevent; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonPropertyDescription; import jakarta.validation.constraints.NotEmpty; import jakarta.validation.constraints.NotNull; @@ -20,32 +21,47 @@ public class MapToListProcessorConfig { @NotNull @JsonProperty("source") + @JsonPropertyDescription("The source map used to perform the mapping operation. When set to an empty " + + "string (`\"\"`), it will use the root of the event as the `source`.") private String source; @NotEmpty @NotNull @JsonProperty("target") + @JsonPropertyDescription("The target for the generated list.") private String target; @JsonProperty("key_name") + @JsonPropertyDescription("The name of the field in which to store the original key. Default is `key`.") private String keyName = DEFAULT_KEY_NAME; @JsonProperty("value_name") + @JsonPropertyDescription("The name of the field in which to store the original value. Default is `value`.") private String valueName = DEFAULT_VALUE_NAME; @JsonProperty("map_to_list_when") + @JsonPropertyDescription("A [conditional expression](https://opensearch.org/docs/latest/data-prepper/pipelines/expression-syntax/), " + + "such as `/some-key == \"test\"'`, that will be evaluated to determine whether the processor will " + + "be run on the event. Default is `null`. All events will be processed unless otherwise stated.") private String mapToListWhen; @JsonProperty("exclude_keys") + @JsonPropertyDescription("The keys in the source map that will be excluded from processing. Default is an " + + "empty list (`[]`).") private List excludeKeys = DEFAULT_EXCLUDE_KEYS; @JsonProperty("remove_processed_fields") + @JsonPropertyDescription("When `true`, the processor will remove the processed fields from the source map. " + + "Default is `false`.") private boolean removeProcessedFields = DEFAULT_REMOVE_PROCESSED_FIELDS; @JsonProperty("convert_field_to_list") + @JsonPropertyDescription("If `true`, the processor will convert the fields from the source map into lists and " + + "place them in fields in the target list. Default is `false`.") private boolean convertFieldToList = false; @JsonProperty("tags_on_failure") + @JsonPropertyDescription("A list of tags to add to the event metadata when the event fails to process.") private List tagsOnFailure; public String getSource() { diff --git a/data-prepper-plugins/user-agent-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/useragent/UserAgentProcessorConfig.java b/data-prepper-plugins/user-agent-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/useragent/UserAgentProcessorConfig.java index 0dcf46e2a1..b1660b37d0 100644 --- a/data-prepper-plugins/user-agent-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/useragent/UserAgentProcessorConfig.java +++ b/data-prepper-plugins/user-agent-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/useragent/UserAgentProcessorConfig.java @@ -6,6 +6,7 @@ package org.opensearch.dataprepper.plugins.processor.useragent; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonPropertyDescription; import jakarta.validation.constraints.NotEmpty; import jakarta.validation.constraints.NotNull; import org.opensearch.dataprepper.model.event.EventKey; @@ -21,21 +22,26 @@ public class UserAgentProcessorConfig { @NotEmpty @NotNull @JsonProperty("source") + @JsonPropertyDescription("The field in the event that will be parsed.") @EventKeyConfiguration(EventKeyFactory.EventAction.GET) private EventKey source; @NotNull @JsonProperty("target") + @JsonPropertyDescription("The field to which the parsed event will write. Default is `user_agent`.") private String target = "user_agent"; @NotNull @JsonProperty("exclude_original") + @JsonPropertyDescription("Determines whether to exclude the original UA string from the parsing result. Defaults to `false`. ") private boolean excludeOriginal = false; @JsonProperty("cache_size") + @JsonPropertyDescription("The cache size of the parser in megabytes. Defaults to `1000`.") private int cacheSize = DEFAULT_CACHE_SIZE; @JsonProperty("tags_on_parse_failure") + @JsonPropertyDescription("The tag to add to an event if the `user_agent` processor fails to parse the UA string.") private List tagsOnParseFailure; public EventKey getSource() { From 1ca5c693a7c996f8521254dcc3fb240a7ee29354 Mon Sep 17 00:00:00 2001 From: David Venable Date: Mon, 29 Jul 2024 18:34:34 -0500 Subject: [PATCH 05/25] Corrects the TRIAGING.md with a video meeting since we currently use Chime. (#4743) Signed-off-by: David Venable --- TRIAGING.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/TRIAGING.md b/TRIAGING.md index a4a25e1932..ba20857061 100644 --- a/TRIAGING.md +++ b/TRIAGING.md @@ -19,7 +19,7 @@ However, should we run out of time before your issue is discussed, you are alway Meetings are hosted regularly Tuesdays at 2:30 PM US Central Time (12:30 PM Pacific Time) and can be joined via the links posted on the [OpenSearch Meetup Group](https://www.meetup.com/opensearch/events/) list of events. The event will be titled `Data Prepper Triage Meeting`. -After joining the Zoom meeting, you can enable your video / voice to join the discussion. +After joining the video meeting, you can enable your video / voice to join the discussion. If you do not have a webcam or microphone available, you can still join in via the text chat. If you have an issue you'd like to bring forth please consider getting a link to the issue so it can be presented to everyone in the meeting. From 9e084a0356388dac4326c54c993343c20eb873e0 Mon Sep 17 00:00:00 2001 From: mishavay-aws <140549901+mishavay-aws@users.noreply.github.com> Date: Mon, 29 Jul 2024 21:41:20 -0400 Subject: [PATCH 06/25] 4602 one way hash (#4750) added capabilities for working with OneWay Hash Signed-off-by: mishavay-aws <140549901+mishavay-aws@users.noreply.github.com> --- .../obfuscate-processor/README.md | 53 +++++- .../obfuscate-processor/build.gradle | 1 + .../obfuscation/ObfuscationProcessor.java | 2 +- .../obfuscation/action/MaskAction.java | 4 +- .../obfuscation/action/ObfuscationAction.java | 6 +- .../obfuscation/action/OneWayHashAction.java | 142 ++++++++++++++ .../action/OneWayHashActionConfig.java | 58 ++++++ .../obfuscation/ObfuscationProcessorTest.java | 3 +- .../obfuscation/action/MaskActionTest.java | 6 +- .../action/ObfuscationActionTest.java | 24 +++ .../action/OneWayHashActionTest.java | 174 ++++++++++++++++++ 11 files changed, 465 insertions(+), 8 deletions(-) create mode 100644 data-prepper-plugins/obfuscate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/obfuscation/action/OneWayHashAction.java create mode 100644 data-prepper-plugins/obfuscate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/obfuscation/action/OneWayHashActionConfig.java create mode 100644 data-prepper-plugins/obfuscate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/obfuscation/action/ObfuscationActionTest.java create mode 100644 data-prepper-plugins/obfuscate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/obfuscation/action/OneWayHashActionTest.java diff --git a/data-prepper-plugins/obfuscate-processor/README.md b/data-prepper-plugins/obfuscate-processor/README.md index 8e48582cf1..842d08106d 100644 --- a/data-prepper-plugins/obfuscate-processor/README.md +++ b/data-prepper-plugins/obfuscate-processor/README.md @@ -64,7 +64,7 @@ Below are the list of configuration options. * `patterns` - (optional) - A list of Regex patterns. You can define multiple patterns for the same field. Only the parts that matched the Regex patterns to be obfuscated. If not provided, the full field will be obfuscated. * `single_word_only` - (optional) - When set to `true`, a word boundary `\b` is added to the pattern, due to which obfuscation would be applied only to words that are standalone in the input text. By default, it is `false`, meaning obfuscation patterns are applied to all occurrences. -* `action` - (optional) - Obfuscation action, default to `mask`. Currently, `mask` is the only supported action. +* `action` - (optional) - Obfuscation action, `mask` or `hash` to use one way hashing. Default to `mask` ### Configuration - Mask Action @@ -75,6 +75,57 @@ There are some additional configuration options for Mask action. * `mask_character_length` - (optional) - Default to 3. The value must be between 1 and 10. There will be n numbers of obfuscation characters, e.g. '***' +### Configuration - One Way Hash Action + +There are some additional configuration options for One Way Hash action. + +* `format` - (optional) - Default to SHA-512. Format of One Way Hash to use. +* `salt` - (optional) - Default to generate random salt. +* `salt_key` - (optional) - Instructs to generate salt for each record based on a value of a specified field in the message + +```yaml +pipeline: + source: + http: + processor: + - obfuscate: + source: "log" + target: "new_log" + patterns: + - "[A-Za-z0-9+_.-]+@([\\w-]+\\.)+[\\w-]{2,4}" + action: + hash: + salt_key: "/" + salt: "" + - obfuscate: + source: "phone" + action: + hash: + salt: "" + sink: + - stdout: +``` + +Take below input + +```json +{ + "id": 1, + "phone": "(555) 555 5555", + "log": "My name is Bob and my email address is abc@example.com" +} +``` + +When run, the processor will parse the message into the following output: + +```json +{ + "id": 1, + "phone": "***", + "log": "My name is Bob and my email address is ", + "newLog": "My name is Bob and my email address is " +} +``` --- ## FAQ: diff --git a/data-prepper-plugins/obfuscate-processor/build.gradle b/data-prepper-plugins/obfuscate-processor/build.gradle index 22909eecd9..83e21a5889 100644 --- a/data-prepper-plugins/obfuscate-processor/build.gradle +++ b/data-prepper-plugins/obfuscate-processor/build.gradle @@ -4,6 +4,7 @@ dependencies { implementation 'com.fasterxml.jackson.core:jackson-core' implementation 'com.fasterxml.jackson.core:jackson-databind' testImplementation project(':data-prepper-test-common') + testImplementation project(':data-prepper-test-event') } test { diff --git a/data-prepper-plugins/obfuscate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/obfuscation/ObfuscationProcessor.java b/data-prepper-plugins/obfuscate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/obfuscation/ObfuscationProcessor.java index bbb1a1600a..472ffec940 100644 --- a/data-prepper-plugins/obfuscate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/obfuscation/ObfuscationProcessor.java +++ b/data-prepper-plugins/obfuscate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/obfuscation/ObfuscationProcessor.java @@ -126,7 +126,7 @@ public Collection> doExecute(Collection> records) { String rawValue = recordEvent.get(source, String.class); // Call obfuscation action - String newValue = this.action.obfuscate(rawValue, patterns); + String newValue = this.action.obfuscate(rawValue, patterns, record); // No changes means it does not match any patterns if (rawValue.equals(newValue)) { diff --git a/data-prepper-plugins/obfuscate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/obfuscation/action/MaskAction.java b/data-prepper-plugins/obfuscate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/obfuscation/action/MaskAction.java index 45fc27fe27..2435156b5f 100644 --- a/data-prepper-plugins/obfuscate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/obfuscation/action/MaskAction.java +++ b/data-prepper-plugins/obfuscate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/obfuscation/action/MaskAction.java @@ -7,6 +7,8 @@ import org.opensearch.dataprepper.model.annotations.DataPrepperPlugin; import org.opensearch.dataprepper.model.annotations.DataPrepperPluginConstructor; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.record.Record; import java.util.List; import java.util.regex.Pattern; @@ -21,7 +23,7 @@ public MaskAction(final MaskActionConfig config) { } @Override - public String obfuscate(String source, List patterns) { + public String obfuscate(String source, List patterns, Record record) { if (patterns == null || patterns.size() == 0) { // This is to replace the whole field. diff --git a/data-prepper-plugins/obfuscate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/obfuscation/action/ObfuscationAction.java b/data-prepper-plugins/obfuscate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/obfuscation/action/ObfuscationAction.java index 1a0376cb89..0e6b71e2aa 100644 --- a/data-prepper-plugins/obfuscate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/obfuscation/action/ObfuscationAction.java +++ b/data-prepper-plugins/obfuscate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/obfuscation/action/ObfuscationAction.java @@ -8,6 +8,9 @@ import java.util.List; import java.util.regex.Pattern; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.record.Record; + /** * Interface represents a specific action to be taken for obfuscation. @@ -20,7 +23,8 @@ public interface ObfuscationAction { * * @param source source string * @param patterns a list of patterns to match + * @param record raw record * @return obfuscated string */ - String obfuscate(String source, List patterns); + String obfuscate(String source, List patterns, Record record); } diff --git a/data-prepper-plugins/obfuscate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/obfuscation/action/OneWayHashAction.java b/data-prepper-plugins/obfuscate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/obfuscation/action/OneWayHashAction.java new file mode 100644 index 0000000000..28e47eae08 --- /dev/null +++ b/data-prepper-plugins/obfuscate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/obfuscation/action/OneWayHashAction.java @@ -0,0 +1,142 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.processor.obfuscation.action; + +import java.nio.charset.StandardCharsets; +import java.security.MessageDigest; +import java.security.NoSuchAlgorithmException; +import java.security.SecureRandom; +import java.util.Arrays; +import java.util.Base64; +import java.util.List; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +import org.opensearch.dataprepper.model.annotations.DataPrepperPlugin; +import org.opensearch.dataprepper.model.annotations.DataPrepperPluginConstructor; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.plugins.processor.obfuscation.ObfuscationProcessor; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.opensearch.dataprepper.model.event.EventKey; + +@DataPrepperPlugin(name = "hash", pluginType = ObfuscationAction.class, pluginConfigurationType = OneWayHashActionConfig.class) +public class OneWayHashAction implements ObfuscationAction { + + + private final MessageDigest messageDigest; + private final byte[] salt; + private EventKey saltKey; + private static final Logger LOG = LoggerFactory.getLogger(ObfuscationProcessor.class); + + @DataPrepperPluginConstructor + public OneWayHashAction(final OneWayHashActionConfig config) { + + this.saltKey = config.getSaltKey(); + + if (config.getSalt() == null || config.getSalt().isEmpty() ) { + this.salt = generateSalt(); + } else { + this.salt = config.getSalt().getBytes(StandardCharsets.UTF_8); + } + + try { + messageDigest = MessageDigest.getInstance(config.getFormat()); + } catch (NoSuchAlgorithmException noSuchAlgorithmException){ + LOG.error("The hash format provided ({}) is not a known algorithm [{}]", config.getFormat(), noSuchAlgorithmException); + throw new RuntimeException(noSuchAlgorithmException); + } + } + + @Override + public String obfuscate(String source, List patterns, Record record) { + + byte [] saltToApply = this.salt; + + // Resolve salt to compute based on a path provided in the configuration. + // For records where path was not found, the salt value defined in the pipeline configuration will be used, if salt value was not configured, one will be generated. + + if(saltKey != null && saltKey.equals("") == false) { + + final Event recordEvent = record.getData(); + + if (recordEvent.containsKey(saltKey)) { + + saltToApply = computeSaltBasedOnKeyValue(recordEvent.get(saltKey, String.class)); + } else { + LOG.info("Unable to find a key '{}' for using as salt, using default salt pipeline configuration for the record instead", saltKey); + } + } + + if (patterns == null || patterns.size() == 0) { + // no pattern to match, replace the whole string + return oneWayHashString(source,saltToApply); + } + + String replacementString = source; + + for (Pattern pattern : patterns) { + + Matcher matcher = Pattern.compile(pattern.pattern()).matcher(replacementString); + StringBuffer stringBuffer = new StringBuffer(); + + while (matcher.find()) { + + String stringToHash = replacementString.substring(matcher.start(),matcher.end()); + matcher.appendReplacement(stringBuffer, oneWayHashString(stringToHash,saltToApply)); + } + + matcher.appendTail(stringBuffer); + replacementString = stringBuffer.toString(); + } + return replacementString; + + + } + + private String oneWayHashString(String source, byte[] salt) { + + String oneWayHashedSource = ""; + + try { + MessageDigest messageDigestClone = (MessageDigest) messageDigest.clone(); + + messageDigestClone.update(salt); + byte[] bytes = messageDigestClone.digest(source.getBytes(StandardCharsets.UTF_8)); + + oneWayHashedSource = Base64.getEncoder().encodeToString(bytes); + + } catch (CloneNotSupportedException cloneNotSupportedException) { + LOG.error("There was an exception while processing Event [{}]", cloneNotSupportedException); + throw new RuntimeException(cloneNotSupportedException); + } + + return oneWayHashedSource; + } + + private byte [] computeSaltBasedOnKeyValue(String saltValue) { + + byte [] value = saltValue.getBytes(StandardCharsets.UTF_8); + byte [] result = new byte [64]; + + Arrays.fill(result, Byte.MIN_VALUE); + + System.arraycopy(value, 0, result, 0, + (value.length >= result.length) ? result.length : value.length); + + return result; + } + + private byte[] generateSalt() { + + byte [] saltBytes = new byte[64]; + SecureRandom secureRandom = new SecureRandom(); + secureRandom.nextBytes(saltBytes); + return saltBytes; + } +} diff --git a/data-prepper-plugins/obfuscate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/obfuscation/action/OneWayHashActionConfig.java b/data-prepper-plugins/obfuscate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/obfuscation/action/OneWayHashActionConfig.java new file mode 100644 index 0000000000..3e3ab622c5 --- /dev/null +++ b/data-prepper-plugins/obfuscate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/obfuscation/action/OneWayHashActionConfig.java @@ -0,0 +1,58 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.processor.obfuscation.action; + +import org.opensearch.dataprepper.model.event.EventKeyConfiguration; +import org.opensearch.dataprepper.model.event.EventKeyFactory; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonPropertyDescription; + +import jakarta.validation.constraints.Pattern; +import jakarta.validation.constraints.Size; + +import org.opensearch.dataprepper.model.event.EventKey; +import org.opensearch.dataprepper.model.event.EventKeyConfiguration; +import org.opensearch.dataprepper.model.event.EventKeyFactory; + + +public class OneWayHashActionConfig { + + @JsonProperty("salt") + @JsonPropertyDescription("Salt value to use when generating hash. If not specified, salt will be randomly generated by the processor.") + @Size(min = 16, message = "Minimum size of salt string is 16.") + @Size(max = 64, message = "Maximum size of salt string is 64") + private String salt; + + @JsonProperty("format") + @Pattern(regexp = "SHA-512", message = "Valid values: SHA-512") + @JsonPropertyDescription("Format of one way hash to generate. Default to SHA-512.") + private String format = "SHA-512"; + + @JsonProperty("salt_key") + @JsonPropertyDescription("A key to compute salt based on a value provided as part of a record." + + "If key or value was not found in the record(s), a salt defined in the pipeline configuration will be used instead.") + @EventKeyConfiguration(EventKeyFactory.EventAction.GET) + private EventKey saltKey; + + public OneWayHashActionConfig(){ + + } + + public String getSalt () { + return salt; + } + + public String getFormat() { + return format; + } + + public EventKey getSaltKey() { + return saltKey; + } + +} + diff --git a/data-prepper-plugins/obfuscate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/obfuscation/ObfuscationProcessorTest.java b/data-prepper-plugins/obfuscate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/obfuscation/ObfuscationProcessorTest.java index be35b2cf01..8e1f556110 100644 --- a/data-prepper-plugins/obfuscate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/obfuscation/ObfuscationProcessorTest.java +++ b/data-prepper-plugins/obfuscate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/obfuscation/ObfuscationProcessorTest.java @@ -140,11 +140,12 @@ void testBasicProcessor(String message) { void testProcessorWithDifferentAction() { final PluginModel mockModel = mock(PluginModel.class); final ObfuscationAction mockAction = mock(ObfuscationAction.class); + when(mockModel.getPluginName()).thenReturn("mock"); when(mockModel.getPluginSettings()).thenReturn(new HashMap<>()); when(mockConfig.getAction()).thenReturn(mockModel); when(mockConfig.getTarget()).thenReturn(""); - when(mockAction.obfuscate(anyString(), anyList())).thenReturn("abc"); + when(mockAction.obfuscate(anyString(), anyList(),any())).thenReturn("abc"); when(mockFactory.loadPlugin(eq(ObfuscationAction.class), any(PluginSetting.class))) .thenReturn(mockAction); diff --git a/data-prepper-plugins/obfuscate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/obfuscation/action/MaskActionTest.java b/data-prepper-plugins/obfuscate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/obfuscation/action/MaskActionTest.java index 3abedf5a61..ffcb336a01 100644 --- a/data-prepper-plugins/obfuscate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/obfuscation/action/MaskActionTest.java +++ b/data-prepper-plugins/obfuscate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/obfuscation/action/MaskActionTest.java @@ -16,7 +16,7 @@ import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.equalTo; -class MaskActionTest { +class MaskActionTest implements ObfuscationActionTest { private MaskAction maskAction; @@ -25,7 +25,7 @@ class MaskActionTest { void testObfuscateWithPatternAsNull() { String message = "Hello"; maskAction = createMaskAction("*", 3); - String result = maskAction.obfuscate(message, null); + String result = maskAction.obfuscate(message, null, createRecord(message)); assertThat(result, equalTo("***")); } @@ -39,7 +39,7 @@ void testObfuscateWithPatternAsNull() { void testObfuscateWithDifferentConfig(String message, String maskCharacter, int maskCharacterLength, String expected) { maskAction = createMaskAction(maskCharacter, maskCharacterLength); List patterns = new ArrayList<>(); - String result = maskAction.obfuscate(message, patterns); + String result = maskAction.obfuscate(message, patterns,createRecord(message)); assertThat(result, equalTo(expected)); } diff --git a/data-prepper-plugins/obfuscate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/obfuscation/action/ObfuscationActionTest.java b/data-prepper-plugins/obfuscate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/obfuscation/action/ObfuscationActionTest.java new file mode 100644 index 0000000000..f43f3f6f99 --- /dev/null +++ b/data-prepper-plugins/obfuscate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/obfuscation/action/ObfuscationActionTest.java @@ -0,0 +1,24 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.processor.obfuscation.action; + +import java.util.HashMap; +import java.util.Map; + +import org.opensearch.dataprepper.event.TestEventFactory; +import org.opensearch.dataprepper.model.event.EventBuilder; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.record.Record; + +interface ObfuscationActionTest { + + default Record createRecord(String message) { + final Map testData = new HashMap<>(); + testData.put("message", message); + + return new Record<>(TestEventFactory.getTestEventFactory().eventBuilder(EventBuilder.class).withEventType("event").withData(testData).build()); + } +} diff --git a/data-prepper-plugins/obfuscate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/obfuscation/action/OneWayHashActionTest.java b/data-prepper-plugins/obfuscate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/obfuscation/action/OneWayHashActionTest.java new file mode 100644 index 0000000000..8b974bed30 --- /dev/null +++ b/data-prepper-plugins/obfuscate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/obfuscation/action/OneWayHashActionTest.java @@ -0,0 +1,174 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.processor.obfuscation.action; + +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.params.provider.CsvSource; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; + +import org.opensearch.dataprepper.model.event.EventKeyFactory; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.not; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.mockito.Mockito.when; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.regex.Pattern; + +import org.opensearch.dataprepper.event.TestEventKeyFactory; +import org.opensearch.dataprepper.model.event.EventKeyFactory; + +import static org.mockito.Mockito.when; + +@ExtendWith(MockitoExtension.class) +public class OneWayHashActionTest implements ObfuscationActionTest { + + @Mock + OneWayHashActionConfig mockConfig; + + private final EventKeyFactory eventKeyFactory = TestEventKeyFactory.getTestEventFactory(); + + @Test + void testObfuscateWithPatternAsNull() { + String message = "Hello"; + when(mockConfig.getSaltKey()).thenReturn(null); + when(mockConfig.getSalt()).thenReturn(""); + when(mockConfig.getFormat()).thenReturn("SHA-512"); + + String result = new OneWayHashAction(mockConfig).obfuscate(message, null,createRecord(message)); + assertNotNull(result); + + assertThat(result, not(containsString(message))); + assertThat(result.length(), equalTo(88)); + } + + + @ParameterizedTest + @CsvSource({ + "Hello,AAAAAAAAAAAAAAAA,2NYZBaQ9nySumhHENpiKatKJhU3jqHC8jJ4DZC612RPGvkzPK1K12DskOI8Cn3qeOMSCTNIWErcGZr8JV4i9HQ==", + "Hi,BBBBBBBBBBBBBBBB,s3S4lyurJvJpQJ6EHN3gi/kexv79Ox+nIqXuVdbvgZP0b718AAxX0bOCPLeOZCnq3p3+DS+a0q0xLSJoMqjsNQ==", + "Hello,CCCCCCCCCCCCCCCC,SsUUpl/+GtU7cRg3ffuRKAtPU7cftdN440sNKR+gABy6JV6crwn5VTNSIqGKaTgBcZeYICy2ZmxP1DiHcW31rA==", + "H,DDDDDDDDDDDDDDDD,XR6utNkOp9te4+0vaRE0+ky/Zyw/gok1sI8qR/stZqFPoU733KwFcur36FCTUZd+i/UpyyJ9L/W6ObwPIf7iuw==", + }) + void testObfuscateWithDifferentConfig(String message, String salt, String expected) { + + when(mockConfig.getSalt()).thenReturn(salt); + when(mockConfig.getSaltKey()).thenReturn(null); + when(mockConfig.getFormat()).thenReturn("SHA-512"); + + OneWayHashAction oneWayHashAction = new OneWayHashAction(mockConfig); + + List patterns = new ArrayList<>(); + String result = oneWayHashAction.obfuscate(message, patterns,createRecord(message)); + assertThat(result, equalTo(expected)); + } + + + @ParameterizedTest + @CsvSource({ + "testing this functionality, test, AAAAAAAAAAAAAAAA, ILsULwmg32tiEQGqeX1rpWI9PGZXSX2Q9tRzXCD0cD/OKMMEBEXKYZhnXj1Xr9q+Dxa11iOmuXd+hx4ZTUaBCg==ing this functionality", + "test this functionality, test, BBBBBBBBBBBBBBBB, QT4wuvJSvgrxa/27gf4cZ1jzeNyiOnDxsY0oS7SsC/eVpBNyhj2I8Rh6/wCsvqRyzAvVoksTKOuRzSFUm6vAQw== this functionality", + "another test of this functionality, test, CCCCCCCCCCCCCCCC, another H9YrqOIlLtaoSCkNR2M0go3npf118KbsHFemyvJUX4+zt8FvjoiReq/0pk5va5i+7eX6XTOMwNokUUl4r+PTHw== of this functionality", + "My name is Bob and my email address is abc@example.com as of now and xyz@example.org in the future, [A-Za-z0-9+_.-]+@([\\w-]+\\.)+[\\w-] ,DDDDDDDDDDDDDDDD, My name is Bob and my email address is DdijIn6L3Cs4+PCYwCy+3bzLZ7w228quoodeI+VDlyMeFe+uZ/Ec1x/DK7MHSmZm8N5SZrINhvGgyig7aEBflg==om as of now and XQGlFjysVX1lkTFoRVCY+QEOfOf6nCoaRy5lxGAHyaFRgMGDpq93PwgZd18DZ3ZfWFRCwgPDGaExJDuRa0kkEQ==rg in the future", + }) + void testObfuscateWithPatterns(String message, String pattern, String salt, String expected) { + + when(mockConfig.getSalt()).thenReturn(salt); + when(mockConfig.getFormat()).thenReturn("SHA-512"); + + OneWayHashAction oneWayHashAction = new OneWayHashAction(mockConfig); + + + Pattern compiledPattern = Pattern.compile(pattern); + List patterns = new ArrayList<>(); + patterns.add(compiledPattern); + String result = oneWayHashAction.obfuscate(message, patterns,createRecord(message)); + assertThat(result, equalTo(expected)); + } + + @ParameterizedTest + @CsvSource({ + "testing this functionality and this test, test, this, AAAAAAAAAAAAAAAA, ILsULwmg32tiEQGqeX1rpWI9PGZXSX2Q9tRzXCD0cD/OKMMEBEXKYZhnXj1Xr9q+Dxa11iOmuXd+hx4ZTUaBCg==ing VsljIdInUvEk2ShjqBF94jgwWDk1lqcE/Fmb/LACPRlwIKsdmlk2PPX2o0XHObp4kRDqd+gUU5iUa/4HXhaA8g== functionality and VsljIdInUvEk2ShjqBF94jgwWDk1lqcE/Fmb/LACPRlwIKsdmlk2PPX2o0XHObp4kRDqd+gUU5iUa/4HXhaA8g== ILsULwmg32tiEQGqeX1rpWI9PGZXSX2Q9tRzXCD0cD/OKMMEBEXKYZhnXj1Xr9q+Dxa11iOmuXd+hx4ZTUaBCg==", + "test this functionality, test, this, BBBBBBBBBBBBBBBB, QT4wuvJSvgrxa/27gf4cZ1jzeNyiOnDxsY0oS7SsC/eVpBNyhj2I8Rh6/wCsvqRyzAvVoksTKOuRzSFUm6vAQw== LAD8UPdf/1cMoKY7Py17uRFNA+OEpVpa9lulTW8wEhsfQsDf/FvBIYxt/YO04sBI8CA1WY+i4elM5nY0xh13Lw== functionality", + "another test of this functionality, test, this, CCCCCCCCCCCCCCCC, another H9YrqOIlLtaoSCkNR2M0go3npf118KbsHFemyvJUX4+zt8FvjoiReq/0pk5va5i+7eX6XTOMwNokUUl4r+PTHw== of oAY9W4VW35Z14mrUisMks9mTILHsswbjjrJt96swt20/lnkMyf0izXV8OhQIh2N7Ml88uXU1fUfk0jTq41udfw== functionality", + "My name is Bob and my email address is abc@example.com as of now and xyz@example.org in the future, [A-Za-z0-9+_.-]+@([\\w-]+\\.)+[\\w-], Bob ,DDDDDDDDDDDDDDDD, My name is aDNCnlEqYbJO9KKnHEhhJSSyy2BB10CUSJxRMCSGLD1gdRNFVTo+Pz7xFepWfVOhuUGulvbnitdPoc8JIlEIFg== and my email address is DdijIn6L3Cs4+PCYwCy+3bzLZ7w228quoodeI+VDlyMeFe+uZ/Ec1x/DK7MHSmZm8N5SZrINhvGgyig7aEBflg==om as of now and XQGlFjysVX1lkTFoRVCY+QEOfOf6nCoaRy5lxGAHyaFRgMGDpq93PwgZd18DZ3ZfWFRCwgPDGaExJDuRa0kkEQ==rg in the future", + }) + void testObfuscateWithTwoPatterns(String message, String pattern1, String pattern2, String salt, String expected) { + + when(mockConfig.getSalt()).thenReturn(salt); + when(mockConfig.getFormat()).thenReturn("SHA-512"); + + OneWayHashAction oneWayHashAction = new OneWayHashAction(mockConfig); + + Pattern compiledPattern1 = Pattern.compile(pattern1); + Pattern compiledPattern2 = Pattern.compile(pattern2); + + List patterns = new ArrayList<>(); + patterns.add(compiledPattern1); + patterns.add(compiledPattern2); + String result = oneWayHashAction.obfuscate(message, patterns,createRecord(message)); + assertThat(result, equalTo(expected)); + } + + @ParameterizedTest + @CsvSource({ + "testing this functionality, test, AAAAAAAAAAAAAAAA, ILsULwmg32tiEQGqeX1rpWI9PGZXSX2Q9tRzXCD0cD/OKMMEBEXKYZhnXj1Xr9q+Dxa11iOmuXd+hx4ZTUaBCg==ing this functionality", + "test this functionality, test, BBBBBBBBBBBBBBBB, QT4wuvJSvgrxa/27gf4cZ1jzeNyiOnDxsY0oS7SsC/eVpBNyhj2I8Rh6/wCsvqRyzAvVoksTKOuRzSFUm6vAQw== this functionality", + "another test of this functionality, test, CCCCCCCCCCCCCCCC, another H9YrqOIlLtaoSCkNR2M0go3npf118KbsHFemyvJUX4+zt8FvjoiReq/0pk5va5i+7eX6XTOMwNokUUl4r+PTHw== of this functionality", + "My name is Bob and my email address is abc@example.com as of now and xyz@example.org in the future, [A-Za-z0-9+_.-]+@([\\w-]+\\.)+[\\w-] ,DDDDDDDDDDDDDDDD, My name is Bob and my email address is DdijIn6L3Cs4+PCYwCy+3bzLZ7w228quoodeI+VDlyMeFe+uZ/Ec1x/DK7MHSmZm8N5SZrINhvGgyig7aEBflg==om as of now and XQGlFjysVX1lkTFoRVCY+QEOfOf6nCoaRy5lxGAHyaFRgMGDpq93PwgZd18DZ3ZfWFRCwgPDGaExJDuRa0kkEQ==rg in the future", + }) + void testObfuscateWithPatternsAndInvalidSaltKey(String message, String pattern, String salt, String expected) { + + //adding SaltKey that cannot be found, to ensure that logic is defaulted back to the configured salt value. + when(mockConfig.getSaltKey()).thenReturn(eventKeyFactory.createEventKey("id")); + when(mockConfig.getSalt()).thenReturn(salt); + when(mockConfig.getFormat()).thenReturn("SHA-512"); + + OneWayHashAction oneWayHashAction = new OneWayHashAction(mockConfig); + + Pattern compiledPattern = Pattern.compile(pattern); + List patterns = new ArrayList<>(); + patterns.add(compiledPattern); + String result = oneWayHashAction.obfuscate(message, patterns,createRecord(message)); + assertThat(result, equalTo(expected)); + } + + @ParameterizedTest + @CsvSource({ + "testing this functionality, test, AAAAAAAAAAAAAAAA, 8g+p3Td+ClA+PttgNrZ8Qsg+tIc9/46TwNDtLeM6lQILI8jcQzPz0bOUM4IrbTlqgHYuOD8r6j6EElj4E6dZLw==ing this functionality", + "test this functionality, test, BBBBBBBBBBBBBBBB, 8g+p3Td+ClA+PttgNrZ8Qsg+tIc9/46TwNDtLeM6lQILI8jcQzPz0bOUM4IrbTlqgHYuOD8r6j6EElj4E6dZLw== this functionality", + "another test of this functionality, test, CCCCCCCCCCCCCCCC, another 8g+p3Td+ClA+PttgNrZ8Qsg+tIc9/46TwNDtLeM6lQILI8jcQzPz0bOUM4IrbTlqgHYuOD8r6j6EElj4E6dZLw== of this functionality", + "My name is Bob and my email address is abc@example.com as of now and xyz@example.org in the future, [A-Za-z0-9+_.-]+@([\\w-]+\\.)+[\\w-] ,DDDDDDDDDDDDDDDD, My name is Bob and my email address is 9zuqdjZfSkx7Xh6rO7bxRpREOmEA8EdtlNXOSviW6C41+sAK2QE/z9PGtRTf+T4bvTuzWBVv7SKVov6jII5+gw==om as of now and KAn0LtIRQYzoPtJqHczu21+gWcXl1OUUwbT9nY+2s+6164/PG4OuW/CZJIUZvOfrUICiL6BUJE32JCEaOfrwjA==rg in the future", + }) + void testObfuscateWithPatternsAndValidSaltKey(String message, String pattern, String salt, String expected) { + + //adding SaltKey that cannot be found, to ensure that logic is defaulted back to the configured salt value. + when(mockConfig.getSaltKey()).thenReturn(eventKeyFactory.createEventKey("message")); + when(mockConfig.getSalt()).thenReturn(salt); + when(mockConfig.getFormat()).thenReturn("SHA-512"); + + OneWayHashAction oneWayHashAction = new OneWayHashAction(mockConfig); + + final Map testData = new HashMap<>(); + testData.put("message", message); + Pattern compiledPattern = Pattern.compile(pattern); + List patterns = new ArrayList<>(); + patterns.add(compiledPattern); + + String result = oneWayHashAction.obfuscate(message, patterns,createRecord("12345")); + assertThat(result, equalTo(expected)); + } + +} From 00f5864f0f6a92ab31080ff80f84c113d8cfee79 Mon Sep 17 00:00:00 2001 From: David Venable Date: Tue, 30 Jul 2024 11:35:12 -0500 Subject: [PATCH 07/25] Reapply "Run tests on the current JVM for Java 17 & 21 / Gradle 8.8 (#4730)" (#4762) (#4771) This reverts commit 5c7d58c03059c7a753d882f5b74fa6ed32f45641. Signed-off-by: David Venable --- build.gradle | 3 ++ .../dataprepper/model/sink/AbstractSink.java | 6 ++-- .../dataprepper/model/sink/SinkThread.java | 8 ++++- .../model/sink/AbstractSinkTest.java | 22 +++++++----- data-prepper-core/build.gradle | 3 -- .../avro/AvroAutoSchemaGeneratorTest.java | 4 +-- .../blockingbuffer/BlockingBufferTests.java | 2 +- .../event_json/EventJsonInputCodecTest.java | 34 +++++++++++-------- .../EventJsonInputOutputCodecTest.java | 26 ++++++++------ .../event_json/EventJsonOutputCodecTest.java | 10 +++--- gradle/wrapper/gradle-wrapper.properties | 2 +- gradlew | 2 +- 12 files changed, 72 insertions(+), 50 deletions(-) diff --git a/build.gradle b/build.gradle index f77ecc442b..3dccd497cf 100644 --- a/build.gradle +++ b/build.gradle @@ -226,6 +226,9 @@ subprojects { test { useJUnitPlatform() + javaLauncher = javaToolchains.launcherFor { + languageVersion = JavaLanguageVersion.current() + } reports { junitXml.required html.required diff --git a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/sink/AbstractSink.java b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/sink/AbstractSink.java index 1c3e596265..26dd7e98a6 100644 --- a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/sink/AbstractSink.java +++ b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/sink/AbstractSink.java @@ -28,6 +28,7 @@ public abstract class AbstractSink> implements Sink { private Thread retryThread; private int maxRetries; private int waitTimeMs; + private SinkThread sinkThread; public AbstractSink(final PluginSetting pluginSetting, int numRetries, int waitTimeMs) { this.pluginMetrics = PluginMetrics.fromPluginSetting(pluginSetting); @@ -51,7 +52,8 @@ public void initialize() { // the exceptions which are not retryable. doInitialize(); if (!isReady() && retryThread == null) { - retryThread = new Thread(new SinkThread(this, maxRetries, waitTimeMs)); + sinkThread = new SinkThread(this, maxRetries, waitTimeMs); + retryThread = new Thread(sinkThread); retryThread.start(); } } @@ -76,7 +78,7 @@ public void output(Collection records) { @Override public void shutdown() { if (retryThread != null) { - retryThread.stop(); + sinkThread.stop(); } } diff --git a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/sink/SinkThread.java b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/sink/SinkThread.java index c304de37af..451cef7dff 100644 --- a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/sink/SinkThread.java +++ b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/sink/SinkThread.java @@ -10,6 +10,8 @@ class SinkThread implements Runnable { private int maxRetries; private int waitTimeMs; + private volatile boolean isStopped = false; + public SinkThread(AbstractSink sink, int maxRetries, int waitTimeMs) { this.sink = sink; this.maxRetries = maxRetries; @@ -19,11 +21,15 @@ public SinkThread(AbstractSink sink, int maxRetries, int waitTimeMs) { @Override public void run() { int numRetries = 0; - while (!sink.isReady() && numRetries++ < maxRetries) { + while (!sink.isReady() && numRetries++ < maxRetries && !isStopped) { try { Thread.sleep(waitTimeMs); sink.doInitialize(); } catch (InterruptedException e){} } } + + public void stop() { + isStopped = true; + } } diff --git a/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/sink/AbstractSinkTest.java b/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/sink/AbstractSinkTest.java index 3b9fe7c007..8d1af7ea44 100644 --- a/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/sink/AbstractSinkTest.java +++ b/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/sink/AbstractSinkTest.java @@ -11,15 +11,10 @@ import org.opensearch.dataprepper.metrics.MetricNames; import org.opensearch.dataprepper.metrics.MetricsTestUtil; import org.opensearch.dataprepper.model.configuration.PluginSetting; -import org.opensearch.dataprepper.model.record.Record; import org.opensearch.dataprepper.model.event.Event; -import org.opensearch.dataprepper.model.event.JacksonEvent; import org.opensearch.dataprepper.model.event.EventHandle; - -import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertTrue; -import static org.mockito.Mockito.when; -import static org.mockito.Mockito.mock; +import org.opensearch.dataprepper.model.event.JacksonEvent; +import org.opensearch.dataprepper.model.record.Record; import java.time.Duration; import java.util.Arrays; @@ -30,6 +25,12 @@ import java.util.UUID; import static org.awaitility.Awaitility.await; +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; class AbstractSinkTest { private int count; @@ -71,13 +72,13 @@ void testMetrics() { } @Test - void testSinkNotReady() { + void testSinkNotReady() throws InterruptedException { final String sinkName = "testSink"; final String pipelineName = "pipelineName"; MetricsTestUtil.initMetrics(); PluginSetting pluginSetting = new PluginSetting(sinkName, Collections.emptyMap()); pluginSetting.setPipelineName(pipelineName); - AbstractSink> abstractSink = new AbstractSinkNotReadyImpl(pluginSetting); + AbstractSinkNotReadyImpl abstractSink = new AbstractSinkNotReadyImpl(pluginSetting); abstractSink.initialize(); assertEquals(abstractSink.isReady(), false); assertEquals(abstractSink.getRetryThreadState(), Thread.State.RUNNABLE); @@ -87,7 +88,10 @@ void testSinkNotReady() { await().atMost(Duration.ofSeconds(5)) .until(abstractSink::isReady); assertEquals(abstractSink.getRetryThreadState(), Thread.State.TERMINATED); + int initCountBeforeShutdown = abstractSink.initCount; abstractSink.shutdown(); + Thread.sleep(200); + assertThat(abstractSink.initCount, equalTo(initCountBeforeShutdown)); } @Test diff --git a/data-prepper-core/build.gradle b/data-prepper-core/build.gradle index 080538c5e4..c939129a1c 100644 --- a/data-prepper-core/build.gradle +++ b/data-prepper-core/build.gradle @@ -48,7 +48,6 @@ dependencies { exclude group: 'commons-logging', module: 'commons-logging' } implementation 'software.amazon.cloudwatchlogs:aws-embedded-metrics:2.0.0-beta-1' - testImplementation 'org.apache.logging.log4j:log4j-jpl:2.23.0' testImplementation testLibs.spring.test implementation libs.armeria.core implementation libs.armeria.grpc @@ -89,8 +88,6 @@ task integrationTest(type: Test) { classpath = sourceSets.integrationTest.runtimeClasspath - systemProperty 'log4j.configurationFile', 'src/test/resources/log4j2.properties' - filter { includeTestsMatching '*IT' } diff --git a/data-prepper-plugins/avro-codecs/src/test/java/org/opensearch/dataprepper/avro/AvroAutoSchemaGeneratorTest.java b/data-prepper-plugins/avro-codecs/src/test/java/org/opensearch/dataprepper/avro/AvroAutoSchemaGeneratorTest.java index 622eb56a1b..1b66b62c37 100644 --- a/data-prepper-plugins/avro-codecs/src/test/java/org/opensearch/dataprepper/avro/AvroAutoSchemaGeneratorTest.java +++ b/data-prepper-plugins/avro-codecs/src/test/java/org/opensearch/dataprepper/avro/AvroAutoSchemaGeneratorTest.java @@ -17,7 +17,7 @@ import java.util.Collections; import java.util.List; import java.util.Map; -import java.util.Random; +import java.util.Timer; import java.util.UUID; import java.util.stream.Stream; @@ -218,7 +218,7 @@ static class SomeUnknownTypesArgumentsProvider implements ArgumentsProvider { @Override public Stream provideArguments(ExtensionContext context) { return Stream.of( - arguments(Random.class), + arguments(Timer.class), arguments(InputStream.class), arguments(File.class) ); diff --git a/data-prepper-plugins/blocking-buffer/src/test/java/org/opensearch/dataprepper/plugins/buffer/blockingbuffer/BlockingBufferTests.java b/data-prepper-plugins/blocking-buffer/src/test/java/org/opensearch/dataprepper/plugins/buffer/blockingbuffer/BlockingBufferTests.java index 194c810ec4..f3f28db174 100644 --- a/data-prepper-plugins/blocking-buffer/src/test/java/org/opensearch/dataprepper/plugins/buffer/blockingbuffer/BlockingBufferTests.java +++ b/data-prepper-plugins/blocking-buffer/src/test/java/org/opensearch/dataprepper/plugins/buffer/blockingbuffer/BlockingBufferTests.java @@ -328,7 +328,7 @@ public Stream provideArguments(final ExtensionContext conte return Stream.of( Arguments.of(0, randomInt + 1, 0.0), Arguments.of(1, 100, 1.0), - Arguments.of(randomInt, randomInt, 100.0), + Arguments.of(randomInt + 1, randomInt + 1, 100.0), Arguments.of(randomInt, randomInt + 250, ((double) randomInt / (randomInt + 250)) * 100), Arguments.of(6, 9, 66.66666666666666), Arguments.of(531, 1000, 53.1), diff --git a/data-prepper-plugins/event-json-codecs/src/test/java/org/opensearch/dataprepper/plugins/codec/event_json/EventJsonInputCodecTest.java b/data-prepper-plugins/event-json-codecs/src/test/java/org/opensearch/dataprepper/plugins/codec/event_json/EventJsonInputCodecTest.java index f85d1c6605..a4b0377963 100644 --- a/data-prepper-plugins/event-json-codecs/src/test/java/org/opensearch/dataprepper/plugins/codec/event_json/EventJsonInputCodecTest.java +++ b/data-prepper-plugins/event-json-codecs/src/test/java/org/opensearch/dataprepper/plugins/codec/event_json/EventJsonInputCodecTest.java @@ -11,9 +11,12 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ValueSource; + import static org.mockito.Mockito.when; import static org.mockito.Mockito.mock; + import org.mockito.Mock; + import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.CoreMatchers.not; @@ -28,6 +31,7 @@ import java.io.ByteArrayInputStream; import java.time.Instant; +import java.time.temporal.ChronoUnit; import java.util.List; import java.util.LinkedList; import java.util.Map; @@ -56,7 +60,7 @@ public EventJsonInputCodec createInputCodec() { @ParameterizedTest @ValueSource(strings = {"", "{}"}) public void emptyTest(String input) throws Exception { - input = "{\""+EventJsonDefines.VERSION+"\":\""+DataPrepperVersion.getCurrentVersion().toString()+"\", \""+EventJsonDefines.EVENTS+"\":["+input+"]}"; + input = "{\"" + EventJsonDefines.VERSION + "\":\"" + DataPrepperVersion.getCurrentVersion().toString() + "\", \"" + EventJsonDefines.EVENTS + "\":[" + input + "]}"; ByteArrayInputStream inputStream = new ByteArrayInputStream(input.getBytes()); inputCodec = createInputCodec(); Consumer> consumer = mock(Consumer.class); @@ -70,15 +74,15 @@ public void inCompatibleVersionTest() throws Exception { final String key = UUID.randomUUID().toString(); final String value = UUID.randomUUID().toString(); Map data = Map.of(key, value); - Instant startTime = Instant.now(); + Instant startTime = Instant.now().truncatedTo(ChronoUnit.MICROS); Event event = createEvent(data, startTime); Map dataMap = event.toMap(); Map metadataMap = objectMapper.convertValue(event.getMetadata(), Map.class); - String input = "{\""+EventJsonDefines.VERSION+"\":\"3.0\", \""+EventJsonDefines.EVENTS+"\":["; + String input = "{\"" + EventJsonDefines.VERSION + "\":\"3.0\", \"" + EventJsonDefines.EVENTS + "\":["; String comma = ""; for (int i = 0; i < 2; i++) { - input += comma+"{\"data\":"+objectMapper.writeValueAsString(dataMap)+","+"\"metadata\":"+objectMapper.writeValueAsString(metadataMap)+"}"; + input += comma + "{\"data\":" + objectMapper.writeValueAsString(dataMap) + "," + "\"metadata\":" + objectMapper.writeValueAsString(metadataMap) + "}"; comma = ","; } input += "]}"; @@ -95,15 +99,15 @@ public void basicTest() throws Exception { final String key = UUID.randomUUID().toString(); final String value = UUID.randomUUID().toString(); Map data = Map.of(key, value); - Instant startTime = Instant.now(); + Instant startTime = Instant.now().truncatedTo(ChronoUnit.MICROS); Event event = createEvent(data, startTime); Map dataMap = event.toMap(); Map metadataMap = objectMapper.convertValue(event.getMetadata(), Map.class); - String input = "{\""+EventJsonDefines.VERSION+"\":\""+DataPrepperVersion.getCurrentVersion().toString()+"\", \""+EventJsonDefines.EVENTS+"\":["; + String input = "{\"" + EventJsonDefines.VERSION + "\":\"" + DataPrepperVersion.getCurrentVersion().toString() + "\", \"" + EventJsonDefines.EVENTS + "\":["; String comma = ""; for (int i = 0; i < 2; i++) { - input += comma+"{\"data\":"+objectMapper.writeValueAsString(dataMap)+","+"\"metadata\":"+objectMapper.writeValueAsString(metadataMap)+"}"; + input += comma + "{\"data\":" + objectMapper.writeValueAsString(dataMap) + "," + "\"metadata\":" + objectMapper.writeValueAsString(metadataMap) + "}"; comma = ","; } input += "]}"; @@ -111,8 +115,8 @@ public void basicTest() throws Exception { List> records = new LinkedList<>(); inputCodec.parse(inputStream, records::add); assertThat(records.size(), equalTo(2)); - for(Record record : records) { - Event e = (Event)record.getData(); + for (Record record : records) { + Event e = (Event) record.getData(); assertThat(e.get(key, String.class), equalTo(value)); assertThat(e.getMetadata().getTimeReceived(), equalTo(startTime)); assertThat(e.getMetadata().getTags().size(), equalTo(0)); @@ -126,15 +130,15 @@ public void test_with_timeReceivedOverridden() throws Exception { final String key = UUID.randomUUID().toString(); final String value = UUID.randomUUID().toString(); Map data = Map.of(key, value); - Instant startTime = Instant.now().minusSeconds(5); + Instant startTime = Instant.now().truncatedTo(ChronoUnit.MICROS).minusSeconds(5); Event event = createEvent(data, startTime); Map dataMap = event.toMap(); Map metadataMap = objectMapper.convertValue(event.getMetadata(), Map.class); - String input = "{\""+EventJsonDefines.VERSION+"\":\""+DataPrepperVersion.getCurrentVersion().toString()+"\", \""+EventJsonDefines.EVENTS+"\":["; + String input = "{\"" + EventJsonDefines.VERSION + "\":\"" + DataPrepperVersion.getCurrentVersion().toString() + "\", \"" + EventJsonDefines.EVENTS + "\":["; String comma = ""; for (int i = 0; i < 2; i++) { - input += comma+"{\"data\":"+objectMapper.writeValueAsString(dataMap)+","+"\"metadata\":"+objectMapper.writeValueAsString(metadataMap)+"}"; + input += comma + "{\"data\":" + objectMapper.writeValueAsString(dataMap) + "," + "\"metadata\":" + objectMapper.writeValueAsString(metadataMap) + "}"; comma = ","; } input += "]}"; @@ -142,8 +146,8 @@ public void test_with_timeReceivedOverridden() throws Exception { List> records = new LinkedList<>(); inputCodec.parse(inputStream, records::add); assertThat(records.size(), equalTo(2)); - for(Record record : records) { - Event e = (Event)record.getData(); + for (Record record : records) { + Event e = (Event) record.getData(); assertThat(e.get(key, String.class), equalTo(value)); assertThat(e.getMetadata().getTimeReceived(), not(equalTo(startTime))); assertThat(e.getMetadata().getTags().size(), equalTo(0)); @@ -159,7 +163,7 @@ private Event createEvent(final Map json, final Instant timeRece if (timeReceived != null) { logBuilder.withTimeReceived(timeReceived); } - final JacksonEvent event = (JacksonEvent)logBuilder.build(); + final JacksonEvent event = (JacksonEvent) logBuilder.build(); return event; } diff --git a/data-prepper-plugins/event-json-codecs/src/test/java/org/opensearch/dataprepper/plugins/codec/event_json/EventJsonInputOutputCodecTest.java b/data-prepper-plugins/event-json-codecs/src/test/java/org/opensearch/dataprepper/plugins/codec/event_json/EventJsonInputOutputCodecTest.java index 85e91e5a55..7ea8c49cd0 100644 --- a/data-prepper-plugins/event-json-codecs/src/test/java/org/opensearch/dataprepper/plugins/codec/event_json/EventJsonInputOutputCodecTest.java +++ b/data-prepper-plugins/event-json-codecs/src/test/java/org/opensearch/dataprepper/plugins/codec/event_json/EventJsonInputOutputCodecTest.java @@ -6,9 +6,12 @@ import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; + import static org.mockito.Mockito.when; import static org.mockito.Mockito.mock; + import org.mockito.Mock; + import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.equalTo; @@ -22,6 +25,7 @@ import org.opensearch.dataprepper.model.log.JacksonLog; import java.time.Instant; +import java.time.temporal.ChronoUnit; import java.util.List; import java.util.LinkedList; import java.util.Map; @@ -64,7 +68,7 @@ public void basicTest() throws Exception { final String value = UUID.randomUUID().toString(); Map data = Map.of(key, value); - Instant startTime = Instant.now(); + Instant startTime = Instant.now().truncatedTo(ChronoUnit.MICROS); Event event = createEvent(data, startTime); outputCodec = createOutputCodec(); inputCodec = createInputCodec(); @@ -75,8 +79,8 @@ public void basicTest() throws Exception { inputCodec.parse(new ByteArrayInputStream(outputStream.toByteArray()), records::add); assertThat(records.size(), equalTo(1)); - for(Record record : records) { - Event e = (Event)record.getData(); + for (Record record : records) { + Event e = (Event) record.getData(); assertThat(e.get(key, String.class), equalTo(value)); assertThat(e.getMetadata().getTimeReceived(), equalTo(startTime)); assertThat(e.getMetadata().getTags().size(), equalTo(0)); @@ -90,7 +94,7 @@ public void multipleEventsTest() throws Exception { final String value = UUID.randomUUID().toString(); Map data = Map.of(key, value); - Instant startTime = Instant.now(); + Instant startTime = Instant.now().truncatedTo(ChronoUnit.MICROS); Event event = createEvent(data, startTime); outputCodec = createOutputCodec(); inputCodec = createInputCodec(); @@ -103,8 +107,8 @@ public void multipleEventsTest() throws Exception { inputCodec.parse(new ByteArrayInputStream(outputStream.toByteArray()), records::add); assertThat(records.size(), equalTo(3)); - for(Record record : records) { - Event e = (Event)record.getData(); + for (Record record : records) { + Event e = (Event) record.getData(); assertThat(e.get(key, String.class), equalTo(value)); assertThat(e.getMetadata().getTimeReceived(), equalTo(startTime)); assertThat(e.getMetadata().getTags().size(), equalTo(0)); @@ -122,7 +126,7 @@ public void extendedTest() throws Exception { Set tags = Set.of(UUID.randomUUID().toString(), UUID.randomUUID().toString()); List tagsList = tags.stream().collect(Collectors.toList()); - Instant startTime = Instant.now(); + Instant startTime = Instant.now().truncatedTo(ChronoUnit.MICROS); Event event = createEvent(data, startTime); Instant origTime = startTime.minusSeconds(5); event.getMetadata().setExternalOriginationTime(origTime); @@ -135,11 +139,11 @@ public void extendedTest() throws Exception { outputCodec.complete(outputStream); assertThat(outputCodec.getExtension(), equalTo(EventJsonOutputCodec.EVENT_JSON)); List> records = new LinkedList<>(); -inputCodec.parse(new ByteArrayInputStream(outputStream.toByteArray()), records::add); + inputCodec.parse(new ByteArrayInputStream(outputStream.toByteArray()), records::add); assertThat(records.size(), equalTo(1)); - for(Record record : records) { - Event e = (Event)record.getData(); + for (Record record : records) { + Event e = (Event) record.getData(); assertThat(e.get(key, String.class), equalTo(value)); assertThat(e.getMetadata().getTimeReceived(), equalTo(startTime)); assertThat(e.getMetadata().getTags(), equalTo(tags)); @@ -157,7 +161,7 @@ private Event createEvent(final Map json, final Instant timeRece if (timeReceived != null) { logBuilder.withTimeReceived(timeReceived); } - final JacksonEvent event = (JacksonEvent)logBuilder.build(); + final JacksonEvent event = (JacksonEvent) logBuilder.build(); return event; } diff --git a/data-prepper-plugins/event-json-codecs/src/test/java/org/opensearch/dataprepper/plugins/codec/event_json/EventJsonOutputCodecTest.java b/data-prepper-plugins/event-json-codecs/src/test/java/org/opensearch/dataprepper/plugins/codec/event_json/EventJsonOutputCodecTest.java index 51dda545cb..b32d2b62e9 100644 --- a/data-prepper-plugins/event-json-codecs/src/test/java/org/opensearch/dataprepper/plugins/codec/event_json/EventJsonOutputCodecTest.java +++ b/data-prepper-plugins/event-json-codecs/src/test/java/org/opensearch/dataprepper/plugins/codec/event_json/EventJsonOutputCodecTest.java @@ -11,6 +11,7 @@ import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.mockito.Mock; + import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.equalTo; @@ -22,6 +23,7 @@ import org.opensearch.dataprepper.model.log.JacksonLog; import java.time.Instant; +import java.time.temporal.ChronoUnit; import java.util.Map; import java.util.UUID; @@ -49,7 +51,7 @@ public void basicTest() throws Exception { final String value = UUID.randomUUID().toString(); Map data = Map.of(key, value); - Instant startTime = Instant.now(); + Instant startTime = Instant.now().truncatedTo(ChronoUnit.MICROS); Event event = createEvent(data, startTime); outputCodec = createOutputCodec(); outputCodec.start(outputStream, null, null); @@ -59,10 +61,10 @@ public void basicTest() throws Exception { Map dataMap = event.toMap(); Map metadataMap = objectMapper.convertValue(event.getMetadata(), Map.class); //String expectedOutput = "{\"version\":\""+DataPrepperVersion.getCurrentVersion().toString()+"\",\""+EventJsonDefines.EVENTS+"\":["; - String expectedOutput = "{\""+EventJsonDefines.VERSION+"\":\""+DataPrepperVersion.getCurrentVersion().toString()+"\",\""+EventJsonDefines.EVENTS+"\":["; + String expectedOutput = "{\"" + EventJsonDefines.VERSION + "\":\"" + DataPrepperVersion.getCurrentVersion().toString() + "\",\"" + EventJsonDefines.EVENTS + "\":["; String comma = ""; for (int i = 0; i < 2; i++) { - expectedOutput += comma+"{\""+EventJsonDefines.DATA+"\":"+objectMapper.writeValueAsString(dataMap)+","+"\""+EventJsonDefines.METADATA+"\":"+objectMapper.writeValueAsString(metadataMap)+"}"; + expectedOutput += comma + "{\"" + EventJsonDefines.DATA + "\":" + objectMapper.writeValueAsString(dataMap) + "," + "\"" + EventJsonDefines.METADATA + "\":" + objectMapper.writeValueAsString(metadataMap) + "}"; comma = ","; } expectedOutput += "]}"; @@ -78,7 +80,7 @@ private Event createEvent(final Map json, final Instant timeRece if (timeReceived != null) { logBuilder.withTimeReceived(timeReceived); } - final JacksonEvent event = (JacksonEvent)logBuilder.build(); + final JacksonEvent event = (JacksonEvent) logBuilder.build(); return event; } diff --git a/gradle/wrapper/gradle-wrapper.properties b/gradle/wrapper/gradle-wrapper.properties index b82aa23a4f..a4413138c9 100644 --- a/gradle/wrapper/gradle-wrapper.properties +++ b/gradle/wrapper/gradle-wrapper.properties @@ -1,6 +1,6 @@ distributionBase=GRADLE_USER_HOME distributionPath=wrapper/dists -distributionUrl=https\://services.gradle.org/distributions/gradle-8.7-bin.zip +distributionUrl=https\://services.gradle.org/distributions/gradle-8.8-bin.zip networkTimeout=10000 validateDistributionUrl=true zipStoreBase=GRADLE_USER_HOME diff --git a/gradlew b/gradlew index 1aa94a4269..b740cf1339 100755 --- a/gradlew +++ b/gradlew @@ -55,7 +55,7 @@ # Darwin, MinGW, and NonStop. # # (3) This script is generated from the Groovy template -# https://github.com/gradle/gradle/blob/HEAD/subprojects/plugins/src/main/resources/org/gradle/api/internal/plugins/unixStartScript.txt +# https://github.com/gradle/gradle/blob/HEAD/platforms/jvm/plugins-application/src/main/resources/org/gradle/api/internal/plugins/unixStartScript.txt # within the Gradle project. # # You can find Gradle at https://github.com/gradle/gradle/. From 944681f438ba6c84c1d9c01f72594d22a93a49f5 Mon Sep 17 00:00:00 2001 From: Hai Yan <8153134+oeyh@users.noreply.github.com> Date: Tue, 30 Jul 2024 12:09:20 -0500 Subject: [PATCH 08/25] Add rds source metrics (#4769) * Add rds source metrics Signed-off-by: Hai Yan * Remove unused imports Signed-off-by: Hai Yan * Add exportS3ObjectsErrors metric Signed-off-by: Hai Yan --------- Signed-off-by: Hai Yan --- data-prepper-plugins/rds-source/build.gradle | 3 + .../plugins/source/rds/RdsService.java | 2 +- .../source/rds/export/DataFileLoader.java | 46 +++++- .../source/rds/export/DataFileScheduler.java | 24 ++- .../source/rds/export/ExportScheduler.java | 27 +++- .../rds/stream/BinlogEventListener.java | 52 ++++++- .../source/rds/stream/StreamScheduler.java | 3 +- .../source/rds/export/DataFileLoaderTest.java | 137 ++++++++++++++++-- .../rds/export/DataFileSchedulerTest.java | 63 +++++++- .../rds/export/ExportSchedulerTest.java | 82 ++++++++++- .../rds/stream/BinlogEventListenerTest.java | 6 +- 11 files changed, 404 insertions(+), 41 deletions(-) diff --git a/data-prepper-plugins/rds-source/build.gradle b/data-prepper-plugins/rds-source/build.gradle index 6d6a681646..14c851f645 100644 --- a/data-prepper-plugins/rds-source/build.gradle +++ b/data-prepper-plugins/rds-source/build.gradle @@ -26,4 +26,7 @@ dependencies { testImplementation project(path: ':data-prepper-test-common') testImplementation 'com.fasterxml.jackson.dataformat:jackson-dataformat-yaml' testImplementation project(path: ':data-prepper-test-event') + testImplementation libs.avro.core + testImplementation libs.parquet.hadoop + testImplementation libs.parquet.avro } 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 982751a3db..73b71e0085 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 @@ -88,7 +88,7 @@ public void start(Buffer> buffer) { exportScheduler = new ExportScheduler( sourceCoordinator, snapshotManager, exportTaskManager, s3Client, pluginMetrics); dataFileScheduler = new DataFileScheduler( - sourceCoordinator, sourceConfig, s3Client, eventFactory, buffer); + sourceCoordinator, sourceConfig, s3Client, eventFactory, buffer, pluginMetrics); runnableList.add(exportScheduler); runnableList.add(dataFileScheduler); } diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileLoader.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileLoader.java index f12d44a75f..42a5b3a0d5 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileLoader.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileLoader.java @@ -5,7 +5,10 @@ package org.opensearch.dataprepper.plugins.source.rds.export; +import io.micrometer.core.instrument.Counter; +import io.micrometer.core.instrument.DistributionSummary; import org.opensearch.dataprepper.buffer.common.BufferAccumulator; +import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.codec.InputCodec; import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.model.record.Record; @@ -18,12 +21,18 @@ import java.io.InputStream; import java.time.Duration; import java.util.List; +import java.util.concurrent.atomic.AtomicLong; public class DataFileLoader implements Runnable { private static final Logger LOG = LoggerFactory.getLogger(DataFileLoader.class); static final Duration VERSION_OVERLAP_TIME_FOR_EXPORT = Duration.ofMinutes(5); + static final String EXPORT_RECORDS_TOTAL_COUNT = "exportRecordsTotal"; + static final String EXPORT_RECORDS_PROCESSED_COUNT = "exportRecordsProcessed"; + static final String EXPORT_RECORDS_PROCESSING_ERROR_COUNT = "exportRecordsProcessingErrors"; + static final String BYTES_RECEIVED = "bytesReceived"; + static final String BYTES_PROCESSED = "bytesProcessed"; private final DataFilePartition dataFilePartition; private final String bucket; @@ -32,12 +41,18 @@ public class DataFileLoader implements Runnable { private final InputCodec codec; private final BufferAccumulator> bufferAccumulator; private final ExportRecordConverter recordConverter; + private final Counter exportRecordsTotalCounter; + private final Counter exportRecordSuccessCounter; + private final Counter exportRecordErrorCounter; + private final DistributionSummary bytesReceivedSummary; + private final DistributionSummary bytesProcessedSummary; private DataFileLoader(final DataFilePartition dataFilePartition, - final InputCodec codec, - final BufferAccumulator> bufferAccumulator, - final S3ObjectReader objectReader, - final ExportRecordConverter recordConverter) { + final InputCodec codec, + final BufferAccumulator> bufferAccumulator, + final S3ObjectReader objectReader, + final ExportRecordConverter recordConverter, + final PluginMetrics pluginMetrics) { this.dataFilePartition = dataFilePartition; bucket = dataFilePartition.getBucket(); objectKey = dataFilePartition.getKey(); @@ -45,24 +60,37 @@ private DataFileLoader(final DataFilePartition dataFilePartition, this.codec = codec; this.bufferAccumulator = bufferAccumulator; this.recordConverter = recordConverter; + + exportRecordsTotalCounter = pluginMetrics.counter(EXPORT_RECORDS_TOTAL_COUNT); + exportRecordSuccessCounter = pluginMetrics.counter(EXPORT_RECORDS_PROCESSED_COUNT); + exportRecordErrorCounter = pluginMetrics.counter(EXPORT_RECORDS_PROCESSING_ERROR_COUNT); + bytesReceivedSummary = pluginMetrics.summary(BYTES_RECEIVED); + bytesProcessedSummary = pluginMetrics.summary(BYTES_PROCESSED); } public static DataFileLoader create(final DataFilePartition dataFilePartition, final InputCodec codec, final BufferAccumulator> bufferAccumulator, final S3ObjectReader objectReader, - final ExportRecordConverter recordConverter) { - return new DataFileLoader(dataFilePartition, codec, bufferAccumulator, objectReader, recordConverter); + final ExportRecordConverter recordConverter, + final PluginMetrics pluginMetrics) { + return new DataFileLoader(dataFilePartition, codec, bufferAccumulator, objectReader, recordConverter, pluginMetrics); } @Override public void run() { LOG.info("Start loading s3://{}/{}", bucket, objectKey); + AtomicLong eventCount = new AtomicLong(); try (InputStream inputStream = objectReader.readFile(bucket, objectKey)) { - codec.parse(inputStream, record -> { try { + exportRecordsTotalCounter.increment(); + final Event event = record.getData(); + final String string = event.toJsonString(); + final long bytes = string.getBytes().length; + bytesReceivedSummary.record(bytes); + DataFileProgressState progressState = dataFilePartition.getProgressState().get(); // TODO: primary key to be obtained by querying database schema @@ -79,6 +107,8 @@ public void run() { snapshotTime, eventVersionNumber)); bufferAccumulator.add(transformedRecord); + eventCount.getAndIncrement(); + bytesProcessedSummary.record(bytes); } catch (Exception e) { throw new RuntimeException(e); } @@ -92,8 +122,10 @@ public void run() { try { bufferAccumulator.flush(); + exportRecordSuccessCounter.increment(eventCount.get()); } catch (Exception e) { LOG.error("Failed to write events to buffer", e); + exportRecordErrorCounter.increment(eventCount.get()); } } } diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileScheduler.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileScheduler.java index 0a2b2fb638..f766aec3d2 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileScheduler.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileScheduler.java @@ -5,7 +5,9 @@ package org.opensearch.dataprepper.plugins.source.rds.export; +import io.micrometer.core.instrument.Counter; import org.opensearch.dataprepper.buffer.common.BufferAccumulator; +import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.buffer.Buffer; import org.opensearch.dataprepper.model.codec.InputCodec; import org.opensearch.dataprepper.model.event.Event; @@ -48,6 +50,9 @@ public class DataFileScheduler implements Runnable { static final Duration BUFFER_TIMEOUT = Duration.ofSeconds(60); static final int DEFAULT_BUFFER_BATCH_SIZE = 1_000; + static final String EXPORT_S3_OBJECTS_PROCESSED_COUNT = "exportS3ObjectsProcessed"; + static final String EXPORT_S3_OBJECTS_ERROR_COUNT = "exportS3ObjectsErrors"; + static final String ACTIVE_EXPORT_S3_OBJECT_CONSUMERS_GAUGE = "activeExportS3ObjectConsumers"; private final EnhancedSourceCoordinator sourceCoordinator; @@ -57,6 +62,11 @@ public class DataFileScheduler implements Runnable { private final InputCodec codec; private final BufferAccumulator> bufferAccumulator; private final ExportRecordConverter recordConverter; + private final PluginMetrics pluginMetrics; + + private final Counter exportFileSuccessCounter; + private final Counter exportFileErrorCounter; + private final AtomicInteger activeExportS3ObjectConsumersGauge; private volatile boolean shutdownRequested = false; @@ -64,7 +74,8 @@ public DataFileScheduler(final EnhancedSourceCoordinator sourceCoordinator, final RdsSourceConfig sourceConfig, final S3Client s3Client, final EventFactory eventFactory, - final Buffer> buffer) { + final Buffer> buffer, + final PluginMetrics pluginMetrics) { this.sourceCoordinator = sourceCoordinator; this.sourceConfig = sourceConfig; codec = new ParquetInputCodec(eventFactory); @@ -72,6 +83,12 @@ public DataFileScheduler(final EnhancedSourceCoordinator sourceCoordinator, objectReader = new S3ObjectReader(s3Client); recordConverter = new ExportRecordConverter(); executor = Executors.newFixedThreadPool(DATA_LOADER_MAX_JOB_COUNT); + this.pluginMetrics = pluginMetrics; + + this.exportFileSuccessCounter = pluginMetrics.counter(EXPORT_S3_OBJECTS_PROCESSED_COUNT); + this.exportFileErrorCounter = pluginMetrics.counter(EXPORT_S3_OBJECTS_ERROR_COUNT); + this.activeExportS3ObjectConsumersGauge = pluginMetrics.gauge( + ACTIVE_EXPORT_S3_OBJECT_CONSUMERS_GAUGE, numOfWorkers, AtomicInteger::get); } @Override @@ -116,15 +133,18 @@ public void shutdown() { } private void processDataFilePartition(DataFilePartition dataFilePartition) { - Runnable loader = DataFileLoader.create(dataFilePartition, codec, bufferAccumulator, objectReader, recordConverter); + Runnable loader = DataFileLoader.create( + dataFilePartition, codec, bufferAccumulator, objectReader, recordConverter, pluginMetrics); CompletableFuture runLoader = CompletableFuture.runAsync(loader, executor); runLoader.whenComplete((v, ex) -> { if (ex == null) { + exportFileSuccessCounter.increment(); // Update global state so we know if all s3 files have been loaded updateLoadStatus(dataFilePartition.getExportTaskId(), DEFAULT_UPDATE_LOAD_STATUS_TIMEOUT); sourceCoordinator.completePartition(dataFilePartition); } else { + exportFileErrorCounter.increment(); LOG.error("There was an exception while processing an S3 data file", ex); sourceCoordinator.giveUpPartition(dataFilePartition); } diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/ExportScheduler.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/ExportScheduler.java index 79ef3d5a61..343ade8b85 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/ExportScheduler.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/ExportScheduler.java @@ -5,6 +5,7 @@ package org.opensearch.dataprepper.plugins.source.rds.export; +import io.micrometer.core.instrument.Counter; import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourceCoordinator; import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourcePartition; @@ -42,12 +43,15 @@ public class ExportScheduler implements Runnable { private static final Logger LOG = LoggerFactory.getLogger(ExportScheduler.class); private static final int DEFAULT_TAKE_LEASE_INTERVAL_MILLIS = 60_000; - private static final Duration DEFAULT_CLOSE_DURATION = Duration.ofMinutes(10); - private static final int DEFAULT_MAX_CLOSE_COUNT = 36; + static final Duration DEFAULT_CLOSE_DURATION = Duration.ofMinutes(10); + static final int DEFAULT_MAX_CLOSE_COUNT = 36; private static final int DEFAULT_CHECKPOINT_INTERVAL_MILLS = 5 * 60_000; private static final int DEFAULT_CHECK_STATUS_INTERVAL_MILLS = 30 * 1000; private static final Duration DEFAULT_SNAPSHOT_STATUS_CHECK_TIMEOUT = Duration.ofMinutes(60); static final String PARQUET_SUFFIX = ".parquet"; + static final String EXPORT_JOB_SUCCESS_COUNT = "exportJobSuccess"; + static final String EXPORT_JOB_FAILURE_COUNT = "exportJobFailure"; + static final String EXPORT_S3_OBJECTS_TOTAL_COUNT = "exportS3ObjectsTotal"; private final S3Client s3Client; private final PluginMetrics pluginMetrics; @@ -56,6 +60,10 @@ public class ExportScheduler implements Runnable { private final ExportTaskManager exportTaskManager; private final SnapshotManager snapshotManager; + private final Counter exportJobSuccessCounter; + private final Counter exportJobFailureCounter; + private final Counter exportS3ObjectsTotalCounter; + private volatile boolean shutdownRequested = false; public ExportScheduler(final EnhancedSourceCoordinator sourceCoordinator, @@ -69,6 +77,10 @@ public ExportScheduler(final EnhancedSourceCoordinator sourceCoordinator, this.executor = Executors.newCachedThreadPool(); this.snapshotManager = snapshotManager; this.exportTaskManager = exportTaskManager; + + exportJobSuccessCounter = pluginMetrics.counter(EXPORT_JOB_SUCCESS_COUNT); + exportJobFailureCounter = pluginMetrics.counter(EXPORT_JOB_FAILURE_COUNT); + exportS3ObjectsTotalCounter = pluginMetrics.counter(EXPORT_S3_OBJECTS_TOTAL_COUNT); } @Override @@ -133,8 +145,7 @@ private String getOrCreateExportTaskId(ExportPartition exportPartition) { progressState.setSnapshotId(snapshotInfo.getSnapshotId()); sourceCoordinator.saveProgressStateForPartition(exportPartition, null); } else { - LOG.error("The snapshot failed to create, it will be retried"); - closeExportPartitionWithError(exportPartition); + LOG.error("The snapshot failed to create. The export will be retried"); return null; } @@ -142,8 +153,7 @@ private String getOrCreateExportTaskId(ExportPartition exportPartition) { try { snapshotInfo = checkSnapshotStatus(snapshotId, DEFAULT_SNAPSHOT_STATUS_CHECK_TIMEOUT); } catch (Exception e) { - LOG.warn("Check snapshot status for {} failed", snapshotId, e); - sourceCoordinator.giveUpPartition(exportPartition); + LOG.warn("Check snapshot status for {} failed. The export will be retried", snapshotId, e); return null; } progressState.setSnapshotTime(snapshotInfo.getCreateTime().toEpochMilli()); @@ -159,7 +169,6 @@ private String getOrCreateExportTaskId(ExportPartition exportPartition) { sourceCoordinator.saveProgressStateForPartition(exportPartition, null); } else { LOG.error("The export task failed to create, it will be retried"); - closeExportPartitionWithError(exportPartition); return null; } @@ -167,6 +176,7 @@ private String getOrCreateExportTaskId(ExportPartition exportPartition) { } private void closeExportPartitionWithError(ExportPartition exportPartition) { + exportJobFailureCounter.increment(); ExportProgressState exportProgressState = exportPartition.getProgressState().get(); // Clear current task id, so that a new export can be submitted. exportProgressState.setExportTaskId(null); @@ -309,12 +319,15 @@ private void createDataFilePartitions(String bucket, String exportTaskId, List> bufferAccumulator; private final List tableNames; private final String s3Prefix; + private final PluginMetrics pluginMetrics; - public BinlogEventListener(final Buffer> buffer, final RdsSourceConfig sourceConfig) { + private final Counter changeEventSuccessCounter; + private final Counter changeEventErrorCounter; + private final DistributionSummary bytesReceivedSummary; + private final DistributionSummary bytesProcessedSummary; + + public BinlogEventListener(final Buffer> buffer, + final RdsSourceConfig sourceConfig, + final PluginMetrics pluginMetrics) { tableMetadataMap = new HashMap<>(); recordConverter = new StreamRecordConverter(sourceConfig.getStream().getPartitionCount()); bufferAccumulator = BufferAccumulator.create(buffer, DEFAULT_BUFFER_BATCH_SIZE, BUFFER_TIMEOUT); s3Prefix = sourceConfig.getS3Prefix(); tableNames = sourceConfig.getTableNames(); + this.pluginMetrics = pluginMetrics; + + changeEventSuccessCounter = pluginMetrics.counter(CHANGE_EVENTS_PROCESSED_COUNT); + changeEventErrorCounter = pluginMetrics.counter(CHANGE_EVENTS_PROCESSING_ERROR_COUNT); + bytesReceivedSummary = pluginMetrics.summary(BYTES_RECEIVED); + bytesProcessedSummary = pluginMetrics.summary(BYTES_PROCESSED); } @Override @@ -95,7 +116,9 @@ void handleTableMapEvent(com.github.shyiko.mysql.binlog.event.Event event) { } void handleInsertEvent(com.github.shyiko.mysql.binlog.event.Event event) { - // get new row data from the event + final long bytes = event.toString().getBytes().length; + bytesReceivedSummary.record(bytes); + LOG.debug("Handling insert event"); final WriteRowsEventData data = event.getData(); if (!tableMetadataMap.containsKey(data.getTableId())) { @@ -113,6 +136,7 @@ void handleInsertEvent(com.github.shyiko.mysql.binlog.event.Event event) { final long eventTimestampMillis = event.getHeader().getTimestamp(); // Construct data prepper JacksonEvent + int eventCount = 0; for (final Object[] rowDataArray : data.getRows()) { final Map rowDataMap = new HashMap<>(); for (int i = 0; i < rowDataArray.length; i++) { @@ -130,12 +154,17 @@ void handleInsertEvent(com.github.shyiko.mysql.binlog.event.Event event) { eventTimestampMillis, eventTimestampMillis); addToBuffer(new Record<>(pipelineEvent)); + eventCount++; } + bytesProcessedSummary.record(bytes); - flushBuffer(); + flushBuffer(eventCount); } void handleUpdateEvent(com.github.shyiko.mysql.binlog.event.Event event) { + final long bytes = event.toString().getBytes().length; + bytesReceivedSummary.record(bytes); + LOG.debug("Handling update event"); final UpdateRowsEventData data = event.getData(); if (!tableMetadataMap.containsKey(data.getTableId())) { @@ -151,6 +180,7 @@ void handleUpdateEvent(com.github.shyiko.mysql.binlog.event.Event event) { final List primaryKeys = tableMetadata.getPrimaryKeys(); final long eventTimestampMillis = event.getHeader().getTimestamp(); + int eventCount = 0; for (Map.Entry updatedRow : data.getRows()) { // updatedRow contains data before update as key and data after update as value final Object[] rowData = updatedRow.getValue(); @@ -171,12 +201,17 @@ void handleUpdateEvent(com.github.shyiko.mysql.binlog.event.Event event) { eventTimestampMillis, eventTimestampMillis); addToBuffer(new Record<>(pipelineEvent)); + eventCount++; } + bytesProcessedSummary.record(bytes); - flushBuffer(); + flushBuffer(eventCount); } void handleDeleteEvent(com.github.shyiko.mysql.binlog.event.Event event) { + final long bytes = event.toString().getBytes().length; + bytesReceivedSummary.record(bytes); + LOG.debug("Handling delete event"); final DeleteRowsEventData data = event.getData(); if (!tableMetadataMap.containsKey(data.getTableId())) { @@ -193,6 +228,7 @@ void handleDeleteEvent(com.github.shyiko.mysql.binlog.event.Event event) { final List primaryKeys = tableMetadata.getPrimaryKeys(); final long eventTimestampMillis = event.getHeader().getTimestamp(); + int eventCount = 0; for (Object[] rowDataArray : data.getRows()) { final Map rowDataMap = new HashMap<>(); for (int i = 0; i < rowDataArray.length; i++) { @@ -210,9 +246,11 @@ void handleDeleteEvent(com.github.shyiko.mysql.binlog.event.Event event) { eventTimestampMillis, eventTimestampMillis); addToBuffer(new Record<>(pipelineEvent)); + eventCount++; } + bytesProcessedSummary.record(bytes); - flushBuffer(); + flushBuffer(eventCount); } private boolean isTableOfInterest(String tableName) { @@ -227,11 +265,13 @@ private void addToBuffer(final Record record) { } } - private void flushBuffer() { + private void flushBuffer(int eventCount) { try { bufferAccumulator.flush(); + changeEventSuccessCounter.increment(eventCount); } catch (Exception e) { LOG.error("Failed to flush buffer", e); + changeEventErrorCounter.increment(eventCount); } } 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 0b42c95c38..acb4ea3f85 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 @@ -41,8 +41,9 @@ public StreamScheduler(final EnhancedSourceCoordinator sourceCoordinator, this.sourceCoordinator = sourceCoordinator; this.sourceConfig = sourceConfig; this.binaryLogClient = binaryLogClient; - this.binaryLogClient.registerEventListener(new BinlogEventListener(buffer, sourceConfig)); + this.binaryLogClient.registerEventListener(new BinlogEventListener(buffer, sourceConfig, pluginMetrics)); this.pluginMetrics = pluginMetrics; + } @Override diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileLoaderTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileLoaderTest.java index 1ed91bc031..ccb36347fa 100644 --- a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileLoaderTest.java +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileLoaderTest.java @@ -5,38 +5,61 @@ package org.opensearch.dataprepper.plugins.source.rds.export; +import io.micrometer.core.instrument.Counter; +import io.micrometer.core.instrument.DistributionSummary; +import org.apache.avro.generic.GenericRecord; +import org.apache.parquet.avro.AvroParquetReader; +import org.apache.parquet.hadoop.ParquetReader; +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.MockedStatic; import org.mockito.junit.jupiter.MockitoExtension; import org.opensearch.dataprepper.buffer.common.BufferAccumulator; +import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.codec.InputCodec; +import org.opensearch.dataprepper.model.event.BaseEventBuilder; import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.event.EventBuilder; +import org.opensearch.dataprepper.model.event.EventFactory; +import org.opensearch.dataprepper.model.io.InputFile; import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.plugins.codec.parquet.ParquetInputCodec; import org.opensearch.dataprepper.plugins.source.rds.converter.ExportRecordConverter; import org.opensearch.dataprepper.plugins.source.rds.coordination.partition.DataFilePartition; +import org.opensearch.dataprepper.plugins.source.rds.coordination.state.DataFileProgressState; import java.io.InputStream; +import java.util.Optional; import java.util.UUID; -import java.util.function.Consumer; import static org.mockito.ArgumentMatchers.any; -import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.RETURNS_DEEP_STUBS; +import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.mockStatic; +import static org.mockito.Mockito.never; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; +import static org.opensearch.dataprepper.plugins.source.rds.export.DataFileLoader.BYTES_PROCESSED; +import static org.opensearch.dataprepper.plugins.source.rds.export.DataFileLoader.BYTES_RECEIVED; +import static org.opensearch.dataprepper.plugins.source.rds.export.DataFileLoader.EXPORT_RECORDS_PROCESSED_COUNT; +import static org.opensearch.dataprepper.plugins.source.rds.export.DataFileLoader.EXPORT_RECORDS_PROCESSING_ERROR_COUNT; +import static org.opensearch.dataprepper.plugins.source.rds.export.DataFileLoader.EXPORT_RECORDS_TOTAL_COUNT; @ExtendWith(MockitoExtension.class) class DataFileLoaderTest { - @Mock + @Mock(answer = Answers.RETURNS_DEEP_STUBS) private DataFilePartition dataFilePartition; @Mock private BufferAccumulator> bufferAccumulator; @Mock - private InputCodec codec; + private EventFactory eventFactory; @Mock private S3ObjectReader s3ObjectReader; @@ -44,24 +67,120 @@ class DataFileLoaderTest { @Mock private ExportRecordConverter recordConverter; + @Mock + private PluginMetrics pluginMetrics; + + @Mock + private Counter exportRecordsTotalCounter; + + @Mock + private Counter exportRecordSuccessCounter; + + @Mock + private Counter exportRecordErrorCounter; + + @Mock + private DistributionSummary bytesReceivedSummary; + + @Mock + private DistributionSummary bytesProcessedSummary; + + @BeforeEach + void setUp() { + when(pluginMetrics.counter(EXPORT_RECORDS_TOTAL_COUNT)).thenReturn(exportRecordsTotalCounter); + when(pluginMetrics.counter(EXPORT_RECORDS_PROCESSED_COUNT)).thenReturn(exportRecordSuccessCounter); + when(pluginMetrics.counter(EXPORT_RECORDS_PROCESSING_ERROR_COUNT)).thenReturn(exportRecordErrorCounter); + when(pluginMetrics.summary(BYTES_RECEIVED)).thenReturn(bytesReceivedSummary); + when(pluginMetrics.summary(BYTES_PROCESSED)).thenReturn(bytesProcessedSummary); + } + + @Test + void test_run_success() throws Exception { + final String bucket = UUID.randomUUID().toString(); + final String key = UUID.randomUUID().toString(); + when(dataFilePartition.getBucket()).thenReturn(bucket); + when(dataFilePartition.getKey()).thenReturn(key); + final DataFileProgressState progressState = mock(DataFileProgressState.class, RETURNS_DEEP_STUBS); + when(dataFilePartition.getProgressState()).thenReturn(Optional.of(progressState)); + + InputStream inputStream = mock(InputStream.class); + when(s3ObjectReader.readFile(bucket, key)).thenReturn(inputStream); + + DataFileLoader dataFileLoader = createObjectUnderTest(); + + final String randomString = UUID.randomUUID().toString(); + final long sizeBytes = randomString.getBytes().length; + final BaseEventBuilder eventBuilder = mock(EventBuilder.class, RETURNS_DEEP_STUBS); + final Event event = mock(Event.class); + when(eventFactory.eventBuilder(any())).thenReturn(eventBuilder); + when(eventBuilder.withEventType(any()).withData(any()).build()).thenReturn(event); + when(event.toJsonString()).thenReturn(randomString); + + try (MockedStatic readerMockedStatic = mockStatic(AvroParquetReader.class)) { + ParquetReader parquetReader = mock(ParquetReader.class); + AvroParquetReader.Builder builder = mock(AvroParquetReader.Builder.class); + readerMockedStatic.when(() -> AvroParquetReader.builder(any(InputFile.class), any())).thenReturn(builder); + when(builder.build()).thenReturn(parquetReader); + when(parquetReader.read()).thenReturn(mock(GenericRecord.class, RETURNS_DEEP_STUBS), null); + + dataFileLoader.run(); + } + + verify(bufferAccumulator).add(any(Record.class)); + verify(bufferAccumulator).flush(); + + verify(exportRecordsTotalCounter).increment(); + verify(bytesReceivedSummary).record(sizeBytes); + verify(bytesProcessedSummary).record(sizeBytes); + verify(exportRecordSuccessCounter).increment(1); + verify(exportRecordErrorCounter, never()).increment(1); + } + @Test - void test_run() throws Exception { + void test_flush_failure_then_error_metric_updated() throws Exception { final String bucket = UUID.randomUUID().toString(); final String key = UUID.randomUUID().toString(); when(dataFilePartition.getBucket()).thenReturn(bucket); when(dataFilePartition.getKey()).thenReturn(key); + final DataFileProgressState progressState = mock(DataFileProgressState.class, RETURNS_DEEP_STUBS); + when(dataFilePartition.getProgressState()).thenReturn(Optional.of(progressState)); InputStream inputStream = mock(InputStream.class); when(s3ObjectReader.readFile(bucket, key)).thenReturn(inputStream); - DataFileLoader objectUnderTest = createObjectUnderTest(); - objectUnderTest.run(); + DataFileLoader dataFileLoader = createObjectUnderTest(); - verify(codec).parse(eq(inputStream), any(Consumer.class)); + final String randomString = UUID.randomUUID().toString(); + final long sizeBytes = randomString.getBytes().length; + final BaseEventBuilder eventBuilder = mock(EventBuilder.class, RETURNS_DEEP_STUBS); + final Event event = mock(Event.class); + when(eventFactory.eventBuilder(any())).thenReturn(eventBuilder); + when(eventBuilder.withEventType(any()).withData(any()).build()).thenReturn(event); + when(event.toJsonString()).thenReturn(randomString); + doThrow(new RuntimeException("testing")).when(bufferAccumulator).flush(); + + try (MockedStatic readerMockedStatic = mockStatic(AvroParquetReader.class)) { + ParquetReader parquetReader = mock(ParquetReader.class); + AvroParquetReader.Builder builder = mock(AvroParquetReader.Builder.class); + readerMockedStatic.when(() -> AvroParquetReader.builder(any(InputFile.class), any())).thenReturn(builder); + when(builder.build()).thenReturn(parquetReader); + when(parquetReader.read()).thenReturn(mock(GenericRecord.class, RETURNS_DEEP_STUBS), null); + + dataFileLoader.run(); + } + + verify(bufferAccumulator).add(any(Record.class)); verify(bufferAccumulator).flush(); + + verify(exportRecordsTotalCounter).increment(); + verify(bytesReceivedSummary).record(sizeBytes); + verify(bytesProcessedSummary).record(sizeBytes); + verify(exportRecordSuccessCounter, never()).increment(1); + verify(exportRecordErrorCounter).increment(1); } private DataFileLoader createObjectUnderTest() { - return DataFileLoader.create(dataFilePartition, codec, bufferAccumulator, s3ObjectReader, recordConverter); + final InputCodec codec = new ParquetInputCodec(eventFactory); + return DataFileLoader.create(dataFilePartition, codec, bufferAccumulator, s3ObjectReader, recordConverter, pluginMetrics); } } \ No newline at end of file diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileSchedulerTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileSchedulerTest.java index ee0d0e2852..5a5a56c6fd 100644 --- a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileSchedulerTest.java +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileSchedulerTest.java @@ -5,6 +5,7 @@ package org.opensearch.dataprepper.plugins.source.rds.export; +import io.micrometer.core.instrument.Counter; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; @@ -12,6 +13,7 @@ import org.mockito.MockedStatic; import org.mockito.junit.jupiter.MockitoExtension; import org.opensearch.dataprepper.buffer.common.BufferAccumulator; +import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.buffer.Buffer; import org.opensearch.dataprepper.model.codec.InputCodec; import org.opensearch.dataprepper.model.event.Event; @@ -32,17 +34,22 @@ import java.util.UUID; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; +import java.util.concurrent.atomic.AtomicInteger; import static org.awaitility.Awaitility.await; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.doNothing; +import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.mockStatic; +import static org.mockito.Mockito.never; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.verifyNoInteractions; import static org.mockito.Mockito.verifyNoMoreInteractions; import static org.mockito.Mockito.when; +import static org.opensearch.dataprepper.plugins.source.rds.export.DataFileScheduler.ACTIVE_EXPORT_S3_OBJECT_CONSUMERS_GAUGE; +import static org.opensearch.dataprepper.plugins.source.rds.export.DataFileScheduler.EXPORT_S3_OBJECTS_PROCESSED_COUNT; @ExtendWith(MockitoExtension.class) class DataFileSchedulerTest { @@ -62,20 +69,37 @@ class DataFileSchedulerTest { @Mock private Buffer> buffer; + @Mock + private PluginMetrics pluginMetrics; + @Mock private DataFilePartition dataFilePartition; + @Mock + private Counter exportFileSuccessCounter; + + @Mock + private Counter exportFileErrorCounter; + + @Mock + private AtomicInteger activeExportS3ObjectConsumersGauge; + private Random random; @BeforeEach void setUp() { random = new Random(); + when(pluginMetrics.counter(EXPORT_S3_OBJECTS_PROCESSED_COUNT)).thenReturn(exportFileSuccessCounter); + when(pluginMetrics.counter(eq(DataFileScheduler.EXPORT_S3_OBJECTS_ERROR_COUNT))).thenReturn(exportFileErrorCounter); + when(pluginMetrics.gauge(eq(ACTIVE_EXPORT_S3_OBJECT_CONSUMERS_GAUGE), any(AtomicInteger.class), any())) + .thenReturn(activeExportS3ObjectConsumersGauge); } @Test void test_given_no_datafile_partition_then_no_export() throws InterruptedException { when(sourceCoordinator.acquireAvailablePartition(DataFilePartition.PARTITION_TYPE)).thenReturn(Optional.empty()); + final DataFileScheduler objectUnderTest = createObjectUnderTest(); final ExecutorService executorService = Executors.newSingleThreadExecutor(); executorService.submit(objectUnderTest); @@ -84,12 +108,11 @@ void test_given_no_datafile_partition_then_no_export() throws InterruptedExcepti Thread.sleep(100); executorService.shutdownNow(); - verifyNoInteractions(s3Client, buffer); + verifyNoInteractions(s3Client, buffer, exportFileSuccessCounter, activeExportS3ObjectConsumersGauge); } @Test void test_given_available_datafile_partition_then_load_datafile() { - DataFileScheduler objectUnderTest = createObjectUnderTest(); final String exportTaskId = UUID.randomUUID().toString(); when(dataFilePartition.getExportTaskId()).thenReturn(exportTaskId); @@ -100,13 +123,15 @@ void test_given_available_datafile_partition_then_load_datafile() { when(globalStatePartition.getProgressState()).thenReturn(Optional.of(loadStatusMap)); when(sourceCoordinator.getPartition(exportTaskId)).thenReturn(Optional.of(globalStatePartition)); + DataFileScheduler objectUnderTest = createObjectUnderTest(); final ExecutorService executorService = Executors.newSingleThreadExecutor(); executorService.submit(() -> { // MockedStatic needs to be created on the same thread it's used try (MockedStatic dataFileLoaderMockedStatic = mockStatic(DataFileLoader.class)) { DataFileLoader dataFileLoader = mock(DataFileLoader.class); dataFileLoaderMockedStatic.when(() -> DataFileLoader.create( - eq(dataFilePartition), any(InputCodec.class), any(BufferAccumulator.class), any(S3ObjectReader.class), any(ExportRecordConverter.class))) + eq(dataFilePartition), any(InputCodec.class), any(BufferAccumulator.class), any(S3ObjectReader.class), + any(ExportRecordConverter.class), any(PluginMetrics.class))) .thenReturn(dataFileLoader); doNothing().when(dataFileLoader).run(); objectUnderTest.run(); @@ -116,9 +141,39 @@ void test_given_available_datafile_partition_then_load_datafile() { .untilAsserted(() -> verify(sourceCoordinator).completePartition(dataFilePartition)); executorService.shutdownNow(); + verify(exportFileSuccessCounter).increment(); + verify(exportFileErrorCounter, never()).increment(); verify(sourceCoordinator).completePartition(dataFilePartition); } + @Test + void test_data_file_loader_throws_exception_then_give_up_partition() { + + when(sourceCoordinator.acquireAvailablePartition(DataFilePartition.PARTITION_TYPE)).thenReturn(Optional.of(dataFilePartition)); + + DataFileScheduler objectUnderTest = createObjectUnderTest(); + final ExecutorService executorService = Executors.newSingleThreadExecutor(); + executorService.submit(() -> { + // MockedStatic needs to be created on the same thread it's used + try (MockedStatic dataFileLoaderMockedStatic = mockStatic(DataFileLoader.class)) { + DataFileLoader dataFileLoader = mock(DataFileLoader.class); + dataFileLoaderMockedStatic.when(() -> DataFileLoader.create( + eq(dataFilePartition), any(InputCodec.class), any(BufferAccumulator.class), any(S3ObjectReader.class), + any(ExportRecordConverter.class), any(PluginMetrics.class))) + .thenReturn(dataFileLoader); + doThrow(new RuntimeException()).when(dataFileLoader).run(); + objectUnderTest.run(); + } + }); + await().atMost(Duration.ofSeconds(1)) + .untilAsserted(() -> verify(sourceCoordinator).giveUpPartition(dataFilePartition)); + executorService.shutdownNow(); + + verify(exportFileSuccessCounter, never()).increment(); + verify(exportFileErrorCounter).increment(); + verify(sourceCoordinator).giveUpPartition(dataFilePartition); + } + @Test void test_shutdown() { DataFileScheduler objectUnderTest = createObjectUnderTest(); @@ -132,6 +187,6 @@ void test_shutdown() { } private DataFileScheduler createObjectUnderTest() { - return new DataFileScheduler(sourceCoordinator, sourceConfig, s3Client, eventFactory, buffer); + return new DataFileScheduler(sourceCoordinator, sourceConfig, s3Client, eventFactory, buffer, pluginMetrics); } } \ No newline at end of file diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/ExportSchedulerTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/ExportSchedulerTest.java index a1a520a47a..f5036e8890 100644 --- a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/ExportSchedulerTest.java +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/ExportSchedulerTest.java @@ -6,6 +6,7 @@ package org.opensearch.dataprepper.plugins.source.rds.export; +import io.micrometer.core.instrument.Counter; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; @@ -42,6 +43,11 @@ import static org.mockito.Mockito.verifyNoInteractions; import static org.mockito.Mockito.verifyNoMoreInteractions; import static org.mockito.Mockito.when; +import static org.opensearch.dataprepper.plugins.source.rds.export.ExportScheduler.DEFAULT_CLOSE_DURATION; +import static org.opensearch.dataprepper.plugins.source.rds.export.ExportScheduler.DEFAULT_MAX_CLOSE_COUNT; +import static org.opensearch.dataprepper.plugins.source.rds.export.ExportScheduler.EXPORT_JOB_FAILURE_COUNT; +import static org.opensearch.dataprepper.plugins.source.rds.export.ExportScheduler.EXPORT_JOB_SUCCESS_COUNT; +import static org.opensearch.dataprepper.plugins.source.rds.export.ExportScheduler.EXPORT_S3_OBJECTS_TOTAL_COUNT; import static org.opensearch.dataprepper.plugins.source.rds.export.ExportScheduler.PARQUET_SUFFIX; @@ -63,6 +69,15 @@ class ExportSchedulerTest { @Mock private PluginMetrics pluginMetrics; + @Mock + private Counter exportJobSuccessCounter; + + @Mock + private Counter exportJobFailureCounter; + + @Mock + private Counter exportS3ObjectsTotalCounter; + @Mock private ExportPartition exportPartition; @@ -73,6 +88,10 @@ class ExportSchedulerTest { @BeforeEach void setUp() { + when(pluginMetrics.counter(EXPORT_JOB_SUCCESS_COUNT)).thenReturn(exportJobSuccessCounter); + when(pluginMetrics.counter(EXPORT_JOB_FAILURE_COUNT)).thenReturn(exportJobFailureCounter); + when(pluginMetrics.counter(EXPORT_S3_OBJECTS_TOTAL_COUNT)).thenReturn(exportS3ObjectsTotalCounter); + exportScheduler = createObjectUnderTest(); } @@ -87,7 +106,8 @@ void test_given_no_export_partition_then_not_export() throws InterruptedExceptio Thread.sleep(100); executorService.shutdownNow(); - verifyNoInteractions(snapshotManager, exportTaskManager, s3Client); + verifyNoInteractions(snapshotManager, exportTaskManager, s3Client, exportJobSuccessCounter, + exportJobFailureCounter, exportS3ObjectsTotalCounter); } @Test @@ -123,9 +143,11 @@ void test_given_export_partition_and_export_task_id_then_complete_export() throw any(String.class), any(String.class), any(List.class)); verify(sourceCoordinator).createPartition(any(DataFilePartition.class)); verify(sourceCoordinator).completePartition(exportPartition); + verify(exportJobSuccessCounter).increment(); + verify(exportS3ObjectsTotalCounter).increment(1); + verify(exportJobFailureCounter, never()).increment(); } - @Test void test_given_export_partition_without_export_task_id_then_start_and_complete_export() throws InterruptedException { when(sourceCoordinator.acquireAvailablePartition(ExportPartition.PARTITION_TYPE)).thenReturn(Optional.of(exportPartition)); @@ -184,6 +206,59 @@ void test_given_export_partition_without_export_task_id_then_start_and_complete_ any(String.class), any(String.class), any(List.class)); verify(sourceCoordinator).createPartition(any(DataFilePartition.class)); verify(sourceCoordinator).completePartition(exportPartition); + verify(exportJobSuccessCounter).increment(); + verify(exportS3ObjectsTotalCounter).increment(1); + verify(exportJobFailureCounter, never()).increment(); + } + + @Test + void test_given_export_partition_and_null_export_task_id_then_close_partition_with_error() throws InterruptedException { + when(sourceCoordinator.acquireAvailablePartition(ExportPartition.PARTITION_TYPE)).thenReturn(Optional.of(exportPartition)); + when(exportPartition.getPartitionKey()).thenReturn(UUID.randomUUID().toString()); + when(exportProgressState.getExportTaskId()).thenReturn(null); + when(exportPartition.getProgressState()).thenReturn(Optional.of(exportProgressState)); + final String dbIdentifier = UUID.randomUUID().toString(); + when(exportPartition.getDbIdentifier()).thenReturn(dbIdentifier); + + // Mock snapshot response + final String snapshotId = UUID.randomUUID().toString(); + final String snapshotArn = "arn:aws:rds:us-east-1:123456789012:snapshot:" + snapshotId; + final Instant createTime = Instant.now(); + final SnapshotInfo snapshotInfoWhenCreate = new SnapshotInfo( + snapshotId, snapshotArn, createTime, SnapshotStatus.CREATING.getStatusName()); + final SnapshotInfo snapshotInfoWhenComplete = new SnapshotInfo( + snapshotId, snapshotArn, createTime, SnapshotStatus.AVAILABLE.getStatusName()); + when(snapshotManager.createSnapshot(dbIdentifier)).thenReturn(snapshotInfoWhenCreate); + when(snapshotManager.checkSnapshotStatus(snapshotId)).thenReturn(snapshotInfoWhenComplete); + + // Mock export response + when(exportProgressState.getIamRoleArn()).thenReturn(UUID.randomUUID().toString()); + when(exportProgressState.getBucket()).thenReturn(UUID.randomUUID().toString()); + when(exportProgressState.getPrefix()).thenReturn(UUID.randomUUID().toString()); + when(exportProgressState.getKmsKeyId()).thenReturn(UUID.randomUUID().toString()); + when(exportTaskManager.startExportTask(any(String.class), any(String.class), any(String.class), + any(String.class), any(String.class), any(List.class))).thenReturn(null); + + // Act + final ExecutorService executorService = Executors.newSingleThreadExecutor(); + executorService.submit(exportScheduler); + await().atMost(Duration.ofSeconds(1)) + .untilAsserted(() -> verify(sourceCoordinator).acquireAvailablePartition(ExportPartition.PARTITION_TYPE)); + Thread.sleep(200); + executorService.shutdownNow(); + + // Assert + verify(snapshotManager).createSnapshot(dbIdentifier); + verify(exportTaskManager).startExportTask( + any(String.class), any(String.class), any(String.class), + any(String.class), any(String.class), any(List.class)); + verify(sourceCoordinator).closePartition(exportPartition, DEFAULT_CLOSE_DURATION, DEFAULT_MAX_CLOSE_COUNT); + verify(sourceCoordinator, never()).createPartition(any(DataFilePartition.class)); + verify(sourceCoordinator, never()).completePartition(exportPartition); + + verify(exportJobFailureCounter).increment(); + verify(exportJobSuccessCounter, never()).increment(); + verify(exportS3ObjectsTotalCounter, never()).increment(1); } @Test @@ -193,7 +268,8 @@ void test_shutDown() { final ExecutorService executorService = Executors.newSingleThreadExecutor(); executorService.submit(exportScheduler); exportScheduler.shutdown(); - verifyNoMoreInteractions(sourceCoordinator, snapshotManager, exportTaskManager, s3Client); + verifyNoMoreInteractions(sourceCoordinator, snapshotManager, exportTaskManager, s3Client, + exportJobSuccessCounter, exportJobFailureCounter, exportS3ObjectsTotalCounter); executorService.shutdownNow(); } diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/BinlogEventListenerTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/BinlogEventListenerTest.java index 406a89cec9..30f622c5d7 100644 --- a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/BinlogEventListenerTest.java +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/BinlogEventListenerTest.java @@ -14,6 +14,7 @@ import org.mockito.Answers; import org.mockito.Mock; import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.buffer.Buffer; import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.model.record.Record; @@ -33,6 +34,9 @@ class BinlogEventListenerTest { @Mock(answer = Answers.RETURNS_DEEP_STUBS) private RdsSourceConfig sourceConfig; + @Mock + private PluginMetrics pluginMetrics; + @Mock(answer = Answers.RETURNS_DEEP_STUBS) private com.github.shyiko.mysql.binlog.event.Event binlogEvent; @@ -87,6 +91,6 @@ void test_given_DeleteRows_event_then_calls_correct_handler(EventType eventType) } private BinlogEventListener createObjectUnderTest() { - return new BinlogEventListener(buffer, sourceConfig); + return new BinlogEventListener(buffer, sourceConfig, pluginMetrics); } } \ No newline at end of file From e66f9bc0129770727d1922e4b95119c6763bbc2a Mon Sep 17 00:00:00 2001 From: Krishna Kondaka <41027584+kkondaka@users.noreply.github.com> Date: Tue, 30 Jul 2024 13:33:51 -0700 Subject: [PATCH 09/25] Cleanup resources properly when Opensearch sink fails to initialize (#4758) * dplive1.yaml Signed-off-by: Krishna Kondaka * rebased to latest Signed-off-by: Krishna Kondaka * removed unnecessary file Signed-off-by: Krishna Kondaka --------- Signed-off-by: Krishna Kondaka Co-authored-by: Krishna Kondaka --- .../sink/opensearch/OpenSearchClientRefresher.java | 5 ++++- .../plugins/sink/opensearch/OpenSearchSink.java | 7 ++++--- .../sink/opensearch/OpenSearchClientRefresherTest.java | 9 ++++++++- 3 files changed, 16 insertions(+), 5 deletions(-) diff --git a/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/OpenSearchClientRefresher.java b/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/OpenSearchClientRefresher.java index 6859dee8be..b697fb26bf 100644 --- a/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/OpenSearchClientRefresher.java +++ b/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/OpenSearchClientRefresher.java @@ -30,7 +30,7 @@ public OpenSearchClientRefresher(final PluginMetrics pluginMetrics, final Function clientFunction) { this.clientFunction = clientFunction; this.currentConfig = connectionConfiguration; - this.currentClient = clientFunction.apply(connectionConfiguration); + this.currentClient = null; credentialsChangeCounter = pluginMetrics.counter(CREDENTIALS_CHANGED); clientRefreshErrorsCounter = pluginMetrics.counter(CLIENT_REFRESH_ERRORS); } @@ -44,6 +44,9 @@ public Class getComponentClass() { public OpenSearchClient get() { readWriteLock.readLock().lock(); try { + if (currentClient == null) { + currentClient = clientFunction.apply(currentConfig); + } return currentClient; } finally { readWriteLock.readLock().unlock(); diff --git a/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/OpenSearchSink.java b/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/OpenSearchSink.java index 199b4e1e0e..1b6f44fde6 100644 --- a/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/OpenSearchSink.java +++ b/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/OpenSearchSink.java @@ -186,6 +186,7 @@ public OpenSearchSink(final PluginSetting pluginSetting, this.bulkRequestMap = new ConcurrentHashMap<>(); this.lastFlushTimeMap = new ConcurrentHashMap<>(); this.pluginConfigObservable = pluginConfigObservable; + this.objectMapper = new ObjectMapper(); final Optional dlqConfig = openSearchSinkConfig.getRetryConfiguration().getDlq(); if (dlqConfig.isPresent()) { @@ -201,7 +202,7 @@ public void doInitialize() { doInitializeInternal(); } catch (IOException e) { LOG.warn("Failed to initialize OpenSearch sink, retrying: {} ", e.getMessage()); - closeFiles(); + this.shutdown(); } catch (InvalidPluginConfigurationException e) { LOG.error("Failed to initialize OpenSearch sink due to a configuration error.", e); this.shutdown(); @@ -212,7 +213,7 @@ public void doInitialize() { throw e; } catch (Exception e) { LOG.warn("Failed to initialize OpenSearch sink with a retryable exception. ", e); - closeFiles(); + this.shutdown(); } } @@ -279,7 +280,6 @@ private void doInitializeInternal() throws IOException { bulkRequestSupplier, pluginSetting); - objectMapper = new ObjectMapper(); this.initialized = true; LOG.info("Initialized OpenSearch sink"); } @@ -615,6 +615,7 @@ private void closeFiles() { public void shutdown() { super.shutdown(); closeFiles(); + openSearchClient.shutdown(); } private void maybeUpdateServerlessNetworkPolicy() { diff --git a/data-prepper-plugins/opensearch/src/test/java/org/opensearch/dataprepper/plugins/sink/opensearch/OpenSearchClientRefresherTest.java b/data-prepper-plugins/opensearch/src/test/java/org/opensearch/dataprepper/plugins/sink/opensearch/OpenSearchClientRefresherTest.java index b9326c606f..584051dff6 100644 --- a/data-prepper-plugins/opensearch/src/test/java/org/opensearch/dataprepper/plugins/sink/opensearch/OpenSearchClientRefresherTest.java +++ b/data-prepper-plugins/opensearch/src/test/java/org/opensearch/dataprepper/plugins/sink/opensearch/OpenSearchClientRefresherTest.java @@ -71,6 +71,7 @@ void testGet() { @Test void testGetAfterUpdateWithDeprecatedBasicAuthUnchanged() { final OpenSearchClientRefresher objectUnderTest = createObjectUnderTest(); + assertThat(objectUnderTest.get(), equalTo(openSearchClient)); verify(clientFunction, times(1)).apply(any()); when(connectionConfiguration.getUsername()).thenReturn(TEST_USERNAME); when(connectionConfiguration.getPassword()).thenReturn(TEST_PASSWORD); @@ -91,6 +92,7 @@ void testGetAfterUpdateWithDeprecatedBasicAuthUnchanged() { @Test void testGetAfterUpdateWithBasicAuthUnchanged() { final OpenSearchClientRefresher objectUnderTest = createObjectUnderTest(); + assertThat(objectUnderTest.get(), equalTo(openSearchClient)); verify(clientFunction, times(1)).apply(any()); when(connectionConfiguration.getAuthConfig()).thenReturn(authConfig); when(authConfig.getUsername()).thenReturn(TEST_USERNAME); @@ -115,6 +117,7 @@ void testGetAfterUpdateWithBasicAuthUnchanged() { void testGetAfterUpdateWithDeprecatedUsernameChanged() { when(pluginMetrics.counter(CREDENTIALS_CHANGED)).thenReturn(credentialsChangeCounter); final OpenSearchClientRefresher objectUnderTest = createObjectUnderTest(); + assertThat(objectUnderTest.get(), equalTo(openSearchClient)); verify(clientFunction, times(1)).apply(any()); assertThat(objectUnderTest.get(), equalTo(openSearchClient)); when(connectionConfiguration.getUsername()).thenReturn(TEST_USERNAME); @@ -138,6 +141,7 @@ void testGetAfterUpdateWithDeprecatedUsernameChanged() { void testGetAfterUpdateWithUsernameChanged() { when(pluginMetrics.counter(CREDENTIALS_CHANGED)).thenReturn(credentialsChangeCounter); final OpenSearchClientRefresher objectUnderTest = createObjectUnderTest(); + assertThat(objectUnderTest.get(), equalTo(openSearchClient)); verify(clientFunction, times(1)).apply(any()); assertThat(objectUnderTest.get(), equalTo(openSearchClient)); when(connectionConfiguration.getAuthConfig()).thenReturn(authConfig); @@ -165,6 +169,7 @@ void testGetAfterUpdateWithUsernameChanged() { void testGetAfterUpdateWithDeprecatedPasswordChanged() { when(pluginMetrics.counter(CREDENTIALS_CHANGED)).thenReturn(credentialsChangeCounter); final OpenSearchClientRefresher objectUnderTest = createObjectUnderTest(); + assertThat(objectUnderTest.get(), equalTo(openSearchClient)); verify(clientFunction, times(1)).apply(any()); assertThat(objectUnderTest.get(), equalTo(openSearchClient)); when(connectionConfiguration.getUsername()).thenReturn(TEST_USERNAME); @@ -190,6 +195,7 @@ void testGetAfterUpdateWithDeprecatedPasswordChanged() { void testGetAfterUpdateWithPasswordChanged() { when(pluginMetrics.counter(CREDENTIALS_CHANGED)).thenReturn(credentialsChangeCounter); final OpenSearchClientRefresher objectUnderTest = createObjectUnderTest(); + assertThat(objectUnderTest.get(), equalTo(openSearchClient)); verify(clientFunction, times(1)).apply(any()); assertThat(objectUnderTest.get(), equalTo(openSearchClient)); when(connectionConfiguration.getAuthConfig()).thenReturn(authConfig); @@ -219,6 +225,7 @@ void testGetAfterUpdateClientFailure() { when(pluginMetrics.counter(CREDENTIALS_CHANGED)).thenReturn(credentialsChangeCounter); when(pluginMetrics.counter(CLIENT_REFRESH_ERRORS)).thenReturn(clientRefreshErrorsCounter); final OpenSearchClientRefresher objectUnderTest = createObjectUnderTest(); + assertThat(objectUnderTest.get(), equalTo(openSearchClient)); verify(clientFunction, times(1)).apply(any()); assertThat(objectUnderTest.get(), equalTo(openSearchClient)); when(connectionConfiguration.getUsername()).thenReturn(TEST_USERNAME); @@ -240,4 +247,4 @@ void testGetAfterUpdateClientFailure() { verify(clientRefreshErrorsCounter).increment(); verify(clientFunction, times(2)).apply(any()); } -} \ No newline at end of file +} From fd87e3d0c9359d5745f391e536e910c255292400 Mon Sep 17 00:00:00 2001 From: Krishna Kondaka <41027584+kkondaka@users.noreply.github.com> Date: Wed, 31 Jul 2024 18:45:39 -0700 Subject: [PATCH 10/25] Increase timeout in Acknowledgement IT tests (#4774) Increase timeout for acknowledgement IT tests Signed-off-by: Krishna Kondaka Signed-off-by: Krishna Kondaka Co-authored-by: Krishna Kondaka --- .../integration/PipelinesWithAcksIT.java | 24 +++++++++---------- 1 file changed, 12 insertions(+), 12 deletions(-) diff --git a/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/integration/PipelinesWithAcksIT.java b/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/integration/PipelinesWithAcksIT.java index 744105d46d..7d3a73d7a5 100644 --- a/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/integration/PipelinesWithAcksIT.java +++ b/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/integration/PipelinesWithAcksIT.java @@ -68,7 +68,7 @@ void simple_pipeline_with_single_record() { final int numRecords = 1; inMemorySourceAccessor.submit(IN_MEMORY_IDENTIFIER, numRecords); - await().atMost(20000, TimeUnit.MILLISECONDS) + await().atMost(40000, TimeUnit.MILLISECONDS) .untilAsserted(() -> { List> outputRecords = inMemorySinkAccessor.get(IN_MEMORY_IDENTIFIER); assertThat(outputRecords, not(empty())); @@ -84,7 +84,7 @@ void simple_pipeline_with_multiple_records() { final int numRecords = 100; inMemorySourceAccessor.submit(IN_MEMORY_IDENTIFIER, numRecords); - await().atMost(20000, TimeUnit.MILLISECONDS) + await().atMost(40000, TimeUnit.MILLISECONDS) .untilAsserted(() -> { List> outputRecords = inMemorySinkAccessor.get(IN_MEMORY_IDENTIFIER); assertThat(outputRecords, not(empty())); @@ -99,7 +99,7 @@ void two_pipelines_with_multiple_records() { final int numRecords = 100; inMemorySourceAccessor.submit(IN_MEMORY_IDENTIFIER, numRecords); - await().atMost(20000, TimeUnit.MILLISECONDS) + await().atMost(40000, TimeUnit.MILLISECONDS) .untilAsserted(() -> { List> outputRecords = inMemorySinkAccessor.get(IN_MEMORY_IDENTIFIER); assertThat(outputRecords, not(empty())); @@ -114,7 +114,7 @@ void three_pipelines_with_multiple_records() { final int numRecords = 100; inMemorySourceAccessor.submit(IN_MEMORY_IDENTIFIER, numRecords); - await().atMost(20000, TimeUnit.MILLISECONDS) + await().atMost(40000, TimeUnit.MILLISECONDS) .untilAsserted(() -> { List> outputRecords = inMemorySinkAccessor.get(IN_MEMORY_IDENTIFIER); assertThat(outputRecords, not(empty())); @@ -129,7 +129,7 @@ void three_pipelines_with_all_unrouted_records() { final int numRecords = 2; inMemorySourceAccessor.submitWithStatus(IN_MEMORY_IDENTIFIER, numRecords); - await().atMost(20000, TimeUnit.MILLISECONDS) + await().atMost(40000, TimeUnit.MILLISECONDS) .untilAsserted(() -> { assertTrue(inMemorySourceAccessor != null); assertTrue(inMemorySourceAccessor.getAckReceived() != null); @@ -145,7 +145,7 @@ void three_pipelines_with_route_and_multiple_records() { final int numRecords = 100; inMemorySourceAccessor.submitWithStatus(IN_MEMORY_IDENTIFIER, numRecords); - await().atMost(20000, TimeUnit.MILLISECONDS) + await().atMost(40000, TimeUnit.MILLISECONDS) .untilAsserted(() -> { List> outputRecords = inMemorySinkAccessor.get(IN_MEMORY_IDENTIFIER); assertThat(outputRecords, not(empty())); @@ -161,7 +161,7 @@ void three_pipelines_with_default_route_and_multiple_records() { inMemorySourceAccessor.submitWithStatus(IN_MEMORY_IDENTIFIER, numRecords); - await().atMost(20000, TimeUnit.MILLISECONDS) + await().atMost(40000, TimeUnit.MILLISECONDS) .untilAsserted(() -> { List> outputRecords = inMemorySinkAccessor.get(IN_MEMORY_IDENTIFIER); assertThat(outputRecords, not(empty())); @@ -176,7 +176,7 @@ void two_parallel_pipelines_multiple_records() { final int numRecords = 100; inMemorySourceAccessor.submit(IN_MEMORY_IDENTIFIER, numRecords); - await().atMost(20000, TimeUnit.MILLISECONDS) + await().atMost(40000, TimeUnit.MILLISECONDS) .untilAsserted(() -> { List> outputRecords = inMemorySinkAccessor.get(IN_MEMORY_IDENTIFIER); assertThat(outputRecords, not(empty())); @@ -191,7 +191,7 @@ void three_pipelines_multi_sink_multiple_records() { final int numRecords = 100; inMemorySourceAccessor.submit(IN_MEMORY_IDENTIFIER, numRecords); - await().atMost(20000, TimeUnit.MILLISECONDS) + await().atMost(40000, TimeUnit.MILLISECONDS) .untilAsserted(() -> { List> outputRecords = inMemorySinkAccessor.get(IN_MEMORY_IDENTIFIER); assertThat(outputRecords, not(empty())); @@ -206,7 +206,7 @@ void one_pipeline_three_sinks_multiple_records() { final int numRecords = 100; inMemorySourceAccessor.submit(IN_MEMORY_IDENTIFIER, numRecords); - await().atMost(20000, TimeUnit.MILLISECONDS) + await().atMost(40000, TimeUnit.MILLISECONDS) .untilAsserted(() -> { List> outputRecords = inMemorySinkAccessor.get(IN_MEMORY_IDENTIFIER); assertThat(outputRecords, not(empty())); @@ -221,7 +221,7 @@ void one_pipeline_ack_expiry_multiple_records() { final int numRecords = 100; inMemorySourceAccessor.submit(IN_MEMORY_IDENTIFIER, numRecords); - await().atMost(20000, TimeUnit.MILLISECONDS) + await().atMost(40000, TimeUnit.MILLISECONDS) .untilAsserted(() -> { List> outputRecords = inMemorySinkAccessor.get(IN_MEMORY_IDENTIFIER); assertThat(outputRecords, not(empty())); @@ -237,7 +237,7 @@ void one_pipeline_three_sinks_negative_ack_multiple_records() { inMemorySourceAccessor.submit(IN_MEMORY_IDENTIFIER, numRecords); inMemorySinkAccessor.setResult(false); - await().atMost(20000, TimeUnit.MILLISECONDS) + await().atMost(40000, TimeUnit.MILLISECONDS) .untilAsserted(() -> { List> outputRecords = inMemorySinkAccessor.get(IN_MEMORY_IDENTIFIER); assertThat(outputRecords, not(empty())); From ecc3bfb93070b6a82f81413a045fa969ce283399 Mon Sep 17 00:00:00 2001 From: David Venable Date: Thu, 1 Aug 2024 16:11:53 -0500 Subject: [PATCH 11/25] Removes @asifsmohammed from the CODEOWNERS to allow the release to proceed. (#4800) Signed-off-by: David Venable --- .github/CODEOWNERS | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/CODEOWNERS b/.github/CODEOWNERS index babe6b3d56..54c084c0f8 100644 --- a/.github/CODEOWNERS +++ b/.github/CODEOWNERS @@ -1,2 +1,2 @@ # This should match the owning team set up in https://github.com/orgs/opensearch-project/teams -* @chenqi0805 @engechas @graytaylor0 @dinujoh @kkondaka @asifsmohammed @KarstenSchnitter @dlvenable @oeyh \ No newline at end of file +* @chenqi0805 @engechas @graytaylor0 @dinujoh @kkondaka @KarstenSchnitter @dlvenable @oeyh \ No newline at end of file From e22e96916bfe1cb314a021ed7f0bf33d6dd8371c Mon Sep 17 00:00:00 2001 From: David Venable Date: Fri, 2 Aug 2024 11:28:55 -0500 Subject: [PATCH 12/25] Using Awaitility and mocks in the LogGeneratorSourceTest to attempt to improve reliability. (#4746) Signed-off-by: David Venable --- .../loggenerator/LogGeneratorSourceTest.java | 57 +++++++++++-------- 1 file changed, 32 insertions(+), 25 deletions(-) diff --git a/data-prepper-plugins/log-generator-source/src/test/java/org/opensearch/dataprepper/plugins/source/loggenerator/LogGeneratorSourceTest.java b/data-prepper-plugins/log-generator-source/src/test/java/org/opensearch/dataprepper/plugins/source/loggenerator/LogGeneratorSourceTest.java index 180fcae952..28d58f8cff 100644 --- a/data-prepper-plugins/log-generator-source/src/test/java/org/opensearch/dataprepper/plugins/source/loggenerator/LogGeneratorSourceTest.java +++ b/data-prepper-plugins/log-generator-source/src/test/java/org/opensearch/dataprepper/plugins/source/loggenerator/LogGeneratorSourceTest.java @@ -6,36 +6,37 @@ package org.opensearch.dataprepper.plugins.source.loggenerator; +import org.junit.jupiter.api.AfterEach; +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.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.buffer.Buffer; import org.opensearch.dataprepper.model.configuration.PluginModel; import org.opensearch.dataprepper.model.configuration.PluginSetting; import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.model.event.JacksonEvent; import org.opensearch.dataprepper.model.plugin.PluginFactory; import org.opensearch.dataprepper.model.record.Record; -import org.junit.jupiter.api.AfterEach; -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.buffer.blockingbuffer.BlockingBuffer; import java.time.Duration; import java.util.HashMap; import java.util.Map; import java.util.UUID; +import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; -import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.awaitility.Awaitility.await; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyInt; import static org.mockito.Mockito.atLeast; import static org.mockito.Mockito.lenient; import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.spy; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.verifyNoInteractions; import static org.mockito.Mockito.when; import static org.opensearch.dataprepper.plugins.source.loggenerator.LogGeneratorSourceConfig.INFINITE_LOG_COUNT; @@ -50,6 +51,9 @@ public class LogGeneratorSourceTest { private LogGeneratorSource logGeneratorSource; @Mock private PluginModel mockLogPluginModel; + @Mock + private Buffer buffer; + @BeforeEach public void setup() { when(sourceConfig.getLogType()).thenReturn(mockLogPluginModel); @@ -79,21 +83,23 @@ private LogGeneratorSource createObjectUnderTest() { @Test void GIVEN_logGeneratorSourceAndBlockingBuffer_WHEN_noLimit_THEN_keepsWritingToBufferUntilStopped() - throws InterruptedException, TimeoutException { + throws TimeoutException { logGeneratorSource = createObjectUnderTest(); - BlockingBuffer> spyBuffer = spy(new BlockingBuffer>("SamplePipeline")); - Duration interval = Duration.ofMillis(100); lenient().when(sourceConfig.getInterval()).thenReturn(interval); lenient().when(sourceConfig.getCount()).thenReturn(INFINITE_LOG_COUNT); // no limit to log count - logGeneratorSource.start(spyBuffer); - Thread.sleep((long) (interval.toMillis() * 1.5)); - verify(spyBuffer, atLeast(1)).write(any(Record.class), anyInt()); - Thread.sleep((long) (interval.toMillis() * 0.7)); - verify(spyBuffer, atLeast(2)).write(any(Record.class), anyInt()); + logGeneratorSource.start(buffer); + await() + .atMost((long) (interval.toMillis() * 1.5), TimeUnit.MILLISECONDS) + .untilAsserted(() -> verify(buffer, atLeast(1)).write(any(Record.class), anyInt())); + verify(buffer, atLeast(1)).write(any(Record.class), anyInt()); + await() + .atMost((long) (interval.toMillis() * 1.5), TimeUnit.MILLISECONDS) + .untilAsserted(() -> verify(buffer, atLeast(1)).write(any(Record.class), anyInt())); + verify(buffer, atLeast(2)).write(any(Record.class), anyInt()); } @Test @@ -101,20 +107,21 @@ void GIVEN_logGeneratorSourceAndBlockingBuffer_WHEN_reachedLimit_THEN_stopsWriti throws InterruptedException, TimeoutException { logGeneratorSource = createObjectUnderTest(); - BlockingBuffer> spyBuffer = spy(new BlockingBuffer>("SamplePipeline")); - Duration interval = Duration.ofMillis(100); lenient().when(sourceConfig.getInterval()).thenReturn(interval); lenient().when(sourceConfig.getCount()).thenReturn(1); // max log count of 1 in logGeneratorSource - assertEquals(spyBuffer.isEmpty(), true); - logGeneratorSource.start(spyBuffer); - Thread.sleep((long) (interval.toMillis() * 1.1)); + verifyNoInteractions(buffer); - verify(spyBuffer, times(1)).write(any(Record.class), anyInt()); + logGeneratorSource.start(buffer); - Thread.sleep(interval.toMillis()); - verify(spyBuffer, times(1)).write(any(Record.class), anyInt()); + await() + .atMost(interval.multipliedBy(3)) + .untilAsserted(() -> verify(buffer, atLeast(1)).write(any(Record.class), anyInt())); + verify(buffer, times(1)).write(any(Record.class), anyInt()); + + Thread.sleep((long) (interval.toMillis() * 1.1)); + verify(buffer, times(1)).write(any(Record.class), anyInt()); } } \ No newline at end of file From 642db0d49bb2ac9581c1c58602d9417cec1eb9d8 Mon Sep 17 00:00:00 2001 From: Qi Chen Date: Fri, 2 Aug 2024 14:44:32 -0500 Subject: [PATCH 13/25] ADD: data prepper plugin schema generation (#4777) * ADD: data-prepper-plugin-schema Signed-off-by: George Chen --- config/checkstyle/checkstyle-suppressions.xml | 2 + .../model/configuration/PipelineModel.java | 15 +- data-prepper-plugin-schema-cli/README.md | 12 ++ data-prepper-plugin-schema-cli/build.gradle | 29 ++++ .../DataPrepperPluginSchemaExecute.java | 74 ++++++++++ .../schemas/JsonSchemaConverter.java | 52 +++++++ .../PluginConfigsJsonSchemaConverter.java | 135 ++++++++++++++++++ .../schemas/module/CustomJacksonModule.java | 31 ++++ .../schemas/JsonSchemaConverterTest.java | 60 ++++++++ .../PluginConfigsJsonSchemaConverterIT.java | 80 +++++++++++ .../PluginConfigsJsonSchemaConverterTest.java | 110 ++++++++++++++ settings.gradle | 1 + 12 files changed, 596 insertions(+), 5 deletions(-) create mode 100644 data-prepper-plugin-schema-cli/README.md create mode 100644 data-prepper-plugin-schema-cli/build.gradle create mode 100644 data-prepper-plugin-schema-cli/src/main/java/org/opensearch/dataprepper/schemas/DataPrepperPluginSchemaExecute.java create mode 100644 data-prepper-plugin-schema-cli/src/main/java/org/opensearch/dataprepper/schemas/JsonSchemaConverter.java create mode 100644 data-prepper-plugin-schema-cli/src/main/java/org/opensearch/dataprepper/schemas/PluginConfigsJsonSchemaConverter.java create mode 100644 data-prepper-plugin-schema-cli/src/main/java/org/opensearch/dataprepper/schemas/module/CustomJacksonModule.java create mode 100644 data-prepper-plugin-schema-cli/src/test/java/org/opensearch/dataprepper/schemas/JsonSchemaConverterTest.java create mode 100644 data-prepper-plugin-schema-cli/src/test/java/org/opensearch/dataprepper/schemas/PluginConfigsJsonSchemaConverterIT.java create mode 100644 data-prepper-plugin-schema-cli/src/test/java/org/opensearch/dataprepper/schemas/PluginConfigsJsonSchemaConverterTest.java diff --git a/config/checkstyle/checkstyle-suppressions.xml b/config/checkstyle/checkstyle-suppressions.xml index 42c37e7dd5..ab3ba001a9 100644 --- a/config/checkstyle/checkstyle-suppressions.xml +++ b/config/checkstyle/checkstyle-suppressions.xml @@ -14,4 +14,6 @@ + + diff --git a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/configuration/PipelineModel.java b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/configuration/PipelineModel.java index 1c8221f899..7af56175a0 100644 --- a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/configuration/PipelineModel.java +++ b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/configuration/PipelineModel.java @@ -24,25 +24,30 @@ * @since 1.2 */ public class PipelineModel { + public static final String SOURCE_PLUGIN_TYPE = "source"; + public static final String PROCESSOR_PLUGIN_TYPE = "processor"; + public static final String BUFFER_PLUGIN_TYPE = "buffer"; + public static final String ROUTE_PLUGIN_TYPE = "route"; + public static final String SINK_PLUGIN_TYPE = "sink"; private static final Logger LOG = LoggerFactory.getLogger(PipelineModel.class); - @JsonProperty("source") + @JsonProperty(SOURCE_PLUGIN_TYPE) private final PluginModel source; - @JsonProperty("processor") + @JsonProperty(PROCESSOR_PLUGIN_TYPE) @JsonInclude(JsonInclude.Include.NON_NULL) private final List processors; - @JsonProperty("buffer") + @JsonProperty(BUFFER_PLUGIN_TYPE) @JsonInclude(JsonInclude.Include.NON_NULL) private final PluginModel buffer; @JsonProperty("routes") - @JsonAlias("route") + @JsonAlias(ROUTE_PLUGIN_TYPE) @JsonInclude(JsonInclude.Include.NON_EMPTY) private final List routes; - @JsonProperty("sink") + @JsonProperty(SINK_PLUGIN_TYPE) private final List sinks; @JsonProperty("workers") diff --git a/data-prepper-plugin-schema-cli/README.md b/data-prepper-plugin-schema-cli/README.md new file mode 100644 index 0000000000..7a4d9bc11b --- /dev/null +++ b/data-prepper-plugin-schema-cli/README.md @@ -0,0 +1,12 @@ +# Data Prepper Plugin Schema CLI + +This module includes the SDK and CLI for generating schemas for Data Prepper pipeline plugins. + +## CLI Usage + +``` +./gradlew :data-prepper-plugin-schema-cli:run --args='--plugin_type=processor --plugin_names=grok' +``` + +* plugin_type: A required parameter specifies type of processor. Valid options are `source`, `buffer`, `processor`, `route`, `sink`. +* plugin_names: An optional parameter filters the result by plugin names separated by `,`, e.g. `grok,date`. diff --git a/data-prepper-plugin-schema-cli/build.gradle b/data-prepper-plugin-schema-cli/build.gradle new file mode 100644 index 0000000000..2c2db93ee6 --- /dev/null +++ b/data-prepper-plugin-schema-cli/build.gradle @@ -0,0 +1,29 @@ +plugins { + id 'data-prepper.publish' + id 'application' +} + +application { + mainClass = 'org.opensearch.dataprepper.schemas.DataPrepperPluginSchemaExecute' +} + +dependencies { + implementation project(':data-prepper-plugins') + implementation project(':data-prepper-plugin-framework') + implementation 'com.fasterxml.jackson.core:jackson-databind' + implementation 'org.reflections:reflections:0.10.2' + implementation 'com.github.victools:jsonschema-maven-plugin:4.35.0' + implementation 'com.github.victools:jsonschema-generator:4.35.0' + implementation 'com.github.victools:jsonschema-module-jackson:4.35.0' + implementation 'com.github.victools:jsonschema-module-jakarta-validation:4.35.0' + implementation 'javax.inject:javax.inject:1' + implementation 'info.picocli:picocli:4.6.1' + implementation(libs.spring.core) { + exclude group: 'commons-logging', module: 'commons-logging' + } + implementation(libs.spring.context) { + exclude group: 'commons-logging', module: 'commons-logging' + } + testImplementation(platform("org.junit:junit-bom:5.9.1")) + testImplementation("org.junit.jupiter:junit-jupiter") +} \ No newline at end of file diff --git a/data-prepper-plugin-schema-cli/src/main/java/org/opensearch/dataprepper/schemas/DataPrepperPluginSchemaExecute.java b/data-prepper-plugin-schema-cli/src/main/java/org/opensearch/dataprepper/schemas/DataPrepperPluginSchemaExecute.java new file mode 100644 index 0000000000..a1a76c0510 --- /dev/null +++ b/data-prepper-plugin-schema-cli/src/main/java/org/opensearch/dataprepper/schemas/DataPrepperPluginSchemaExecute.java @@ -0,0 +1,74 @@ +package org.opensearch.dataprepper.schemas; + +import com.github.victools.jsonschema.generator.Module; +import com.github.victools.jsonschema.generator.OptionPreset; +import com.github.victools.jsonschema.generator.SchemaVersion; +import com.github.victools.jsonschema.module.jakarta.validation.JakartaValidationModule; +import com.github.victools.jsonschema.module.jakarta.validation.JakartaValidationOption; +import org.opensearch.dataprepper.schemas.module.CustomJacksonModule; +import org.reflections.Reflections; +import org.reflections.scanners.Scanners; +import org.reflections.util.ClasspathHelper; +import org.reflections.util.ConfigurationBuilder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import picocli.CommandLine; + +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static com.github.victools.jsonschema.module.jackson.JacksonOption.RESPECT_JSONPROPERTY_REQUIRED; + +public class DataPrepperPluginSchemaExecute implements Runnable { + private static final Logger LOG = LoggerFactory.getLogger(DataPrepperPluginSchemaExecute.class); + static final String DEFAULT_PLUGINS_CLASSPATH = "org.opensearch.dataprepper.plugins"; + + @CommandLine.Option(names = {"--plugin_type"}, required = true) + private String pluginTypeName; + + @CommandLine.Option(names = {"--plugin_names"}) + private String pluginNames; + + @CommandLine.Option(names = {"--site.url"}, defaultValue = "https://opensearch.org") + private String siteUrl; + @CommandLine.Option(names = {"--site.baseurl"}, defaultValue = "/docs/latest") + private String siteBaseUrl; + + public static void main(String[] args) { + final int exitCode = new CommandLine(new DataPrepperPluginSchemaExecute()).execute(args); + System.exit(exitCode); + } + + @Override + public void run() { + final List modules = List.of( + new CustomJacksonModule(RESPECT_JSONPROPERTY_REQUIRED), + new JakartaValidationModule(JakartaValidationOption.NOT_NULLABLE_FIELD_IS_REQUIRED, + JakartaValidationOption.INCLUDE_PATTERN_EXPRESSIONS) + ); + final Reflections reflections = new Reflections(new ConfigurationBuilder() + .setUrls(ClasspathHelper.forPackage(DEFAULT_PLUGINS_CLASSPATH)) + .setScanners(Scanners.TypesAnnotated, Scanners.SubTypes)); + final PluginConfigsJsonSchemaConverter pluginConfigsJsonSchemaConverter = new PluginConfigsJsonSchemaConverter( + reflections, new JsonSchemaConverter(modules), siteUrl, siteBaseUrl); + final Class pluginType = pluginConfigsJsonSchemaConverter.pluginTypeNameToPluginType(pluginTypeName); + final Map pluginNameToJsonSchemaMap = pluginConfigsJsonSchemaConverter.convertPluginConfigsIntoJsonSchemas( + SchemaVersion.DRAFT_2020_12, OptionPreset.PLAIN_JSON, pluginType); + if (pluginNames == null) { + pluginNameToJsonSchemaMap.values().forEach(System.out::println); + } else { + final Set pluginNamesSet = Set.of(pluginNames.split(",")); + final List result = pluginNamesSet.stream().flatMap(name -> { + if (!pluginNameToJsonSchemaMap.containsKey(name)) { + LOG.error("plugin name: {} not found", name); + return Stream.empty(); + } + return Stream.of(pluginNameToJsonSchemaMap.get(name)); + }).collect(Collectors.toList()); + result.forEach(System.out::println); + } + } +} diff --git a/data-prepper-plugin-schema-cli/src/main/java/org/opensearch/dataprepper/schemas/JsonSchemaConverter.java b/data-prepper-plugin-schema-cli/src/main/java/org/opensearch/dataprepper/schemas/JsonSchemaConverter.java new file mode 100644 index 0000000000..fe08825af4 --- /dev/null +++ b/data-prepper-plugin-schema-cli/src/main/java/org/opensearch/dataprepper/schemas/JsonSchemaConverter.java @@ -0,0 +1,52 @@ +package org.opensearch.dataprepper.schemas; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.node.ObjectNode; +import com.github.victools.jsonschema.generator.FieldScope; +import com.github.victools.jsonschema.generator.Module; +import com.github.victools.jsonschema.generator.OptionPreset; +import com.github.victools.jsonschema.generator.SchemaGenerator; +import com.github.victools.jsonschema.generator.SchemaGeneratorConfig; +import com.github.victools.jsonschema.generator.SchemaGeneratorConfigBuilder; +import com.github.victools.jsonschema.generator.SchemaGeneratorConfigPart; +import com.github.victools.jsonschema.generator.SchemaVersion; + +import java.util.List; + +public class JsonSchemaConverter { + static final String DEPRECATED_SINCE_KEY = "deprecated"; + private final List jsonSchemaGeneratorModules; + + public JsonSchemaConverter(final List jsonSchemaGeneratorModules) { + this.jsonSchemaGeneratorModules = jsonSchemaGeneratorModules; + } + + public ObjectNode convertIntoJsonSchema( + final SchemaVersion schemaVersion, final OptionPreset optionPreset, final Class clazz) + throws JsonProcessingException { + final SchemaGeneratorConfigBuilder configBuilder = new SchemaGeneratorConfigBuilder( + schemaVersion, optionPreset); + loadJsonSchemaGeneratorModules(configBuilder); + final SchemaGeneratorConfigPart scopeSchemaGeneratorConfigPart = configBuilder.forFields(); + overrideInstanceAttributeWithDeprecated(scopeSchemaGeneratorConfigPart); + + final SchemaGeneratorConfig config = configBuilder.build(); + final SchemaGenerator generator = new SchemaGenerator(config); + return generator.generateSchema(clazz); + } + + private void loadJsonSchemaGeneratorModules(final SchemaGeneratorConfigBuilder configBuilder) { + jsonSchemaGeneratorModules.forEach(configBuilder::with); + } + + private void overrideInstanceAttributeWithDeprecated( + final SchemaGeneratorConfigPart scopeSchemaGeneratorConfigPart) { + scopeSchemaGeneratorConfigPart.withInstanceAttributeOverride((node, field, context) -> { + final Deprecated deprecatedAnnotation = field.getAnnotationConsideringFieldAndGetter( + Deprecated.class); + if (deprecatedAnnotation != null) { + node.put(DEPRECATED_SINCE_KEY, deprecatedAnnotation.since()); + } + }); + } +} diff --git a/data-prepper-plugin-schema-cli/src/main/java/org/opensearch/dataprepper/schemas/PluginConfigsJsonSchemaConverter.java b/data-prepper-plugin-schema-cli/src/main/java/org/opensearch/dataprepper/schemas/PluginConfigsJsonSchemaConverter.java new file mode 100644 index 0000000000..b7f4c1a531 --- /dev/null +++ b/data-prepper-plugin-schema-cli/src/main/java/org/opensearch/dataprepper/schemas/PluginConfigsJsonSchemaConverter.java @@ -0,0 +1,135 @@ +package org.opensearch.dataprepper.schemas; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.node.ObjectNode; +import com.github.victools.jsonschema.generator.OptionPreset; +import com.github.victools.jsonschema.generator.SchemaVersion; +import org.opensearch.dataprepper.model.annotations.DataPrepperPlugin; +import org.opensearch.dataprepper.model.buffer.Buffer; +import org.opensearch.dataprepper.model.configuration.ConditionalRoute; +import org.opensearch.dataprepper.model.processor.Processor; +import org.opensearch.dataprepper.model.sink.Sink; +import org.opensearch.dataprepper.model.source.Source; +import org.reflections.Reflections; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.opensearch.dataprepper.model.configuration.PipelineModel.BUFFER_PLUGIN_TYPE; +import static org.opensearch.dataprepper.model.configuration.PipelineModel.PROCESSOR_PLUGIN_TYPE; +import static org.opensearch.dataprepper.model.configuration.PipelineModel.ROUTE_PLUGIN_TYPE; +import static org.opensearch.dataprepper.model.configuration.PipelineModel.SINK_PLUGIN_TYPE; +import static org.opensearch.dataprepper.model.configuration.PipelineModel.SOURCE_PLUGIN_TYPE; + +public class PluginConfigsJsonSchemaConverter { + private static final Logger LOG = LoggerFactory.getLogger(PluginConfigsJsonSchemaConverter.class); + static final String SITE_URL_PLACEHOLDER = "{{site.url}}"; + static final String SITE_BASE_URL_PLACEHOLDER = "{{site.baseurl}}"; + static final String DOCUMENTATION_LINK_KEY = "documentation"; + static final String PLUGIN_NAME_KEY = "name"; + static final String PLUGIN_DOCUMENTATION_URL_FORMAT = + "%s%s/data-prepper/pipelines/configuration/%s/%s/"; + static final Map, String> PLUGIN_TYPE_TO_URI_PARAMETER_MAP = Map.of( + Source.class, "sources", + Processor.class, "processors", + ConditionalRoute.class, "processors", + Buffer.class, "buffers", + Sink.class, "sinks" + ); + static final String CONDITIONAL_ROUTE_PROCESSOR_NAME = "routes"; + static final Map> PLUGIN_TYPE_NAME_TO_CLASS_MAP = Map.of( + SOURCE_PLUGIN_TYPE, Source.class, + PROCESSOR_PLUGIN_TYPE, Processor.class, + ROUTE_PLUGIN_TYPE, ConditionalRoute.class, + BUFFER_PLUGIN_TYPE, Buffer.class, + SINK_PLUGIN_TYPE, Sink.class); + + private final String siteUrl; + private final String siteBaseUrl; + private final Reflections reflections; + private final JsonSchemaConverter jsonSchemaConverter; + + public PluginConfigsJsonSchemaConverter( + final Reflections reflections, + final JsonSchemaConverter jsonSchemaConverter, + final String siteUrl, + final String siteBaseUrl) { + this.reflections = reflections; + this.jsonSchemaConverter = jsonSchemaConverter; + this.siteUrl = siteUrl == null ? SITE_URL_PLACEHOLDER : siteUrl; + this.siteBaseUrl = siteBaseUrl == null ? SITE_BASE_URL_PLACEHOLDER : siteBaseUrl; + } + + public Set validPluginTypeNames() { + return PLUGIN_TYPE_NAME_TO_CLASS_MAP.keySet(); + } + + public Class pluginTypeNameToPluginType(final String pluginTypeName) { + final Class pluginType = PLUGIN_TYPE_NAME_TO_CLASS_MAP.get(pluginTypeName); + if (pluginType == null) { + throw new IllegalArgumentException(String.format("Invalid plugin type name: %s.", pluginTypeName)); + } + return pluginType; + } + + public Map convertPluginConfigsIntoJsonSchemas( + final SchemaVersion schemaVersion, final OptionPreset optionPreset, final Class pluginType) { + final Map> nameToConfigClass = scanForPluginConfigs(pluginType); + return nameToConfigClass.entrySet().stream() + .flatMap(entry -> { + final String pluginName = entry.getKey(); + String value; + try { + final ObjectNode jsonSchemaNode = jsonSchemaConverter.convertIntoJsonSchema( + schemaVersion, optionPreset, entry.getValue()); + addPluginName(jsonSchemaNode, pluginName); + addDocumentationLink(jsonSchemaNode, pluginName, pluginType); + value = jsonSchemaNode.toPrettyString(); + } catch (JsonProcessingException e) { + LOG.error("Encountered error retrieving JSON schema for {}", pluginName); + return Stream.empty(); + } + return Stream.of(Map.entry(entry.getKey(), value)); + }) + .filter(entry -> Objects.nonNull(entry.getValue())) + .collect(Collectors.toMap( + Map.Entry::getKey, + Map.Entry::getValue + )); + } + + private Map> scanForPluginConfigs(final Class pluginType) { + if (ConditionalRoute.class.equals(pluginType)) { + return Map.of(CONDITIONAL_ROUTE_PROCESSOR_NAME, ConditionalRoute.class); + } + return reflections.getTypesAnnotatedWith(DataPrepperPlugin.class).stream() + .map(clazz -> clazz.getAnnotation(DataPrepperPlugin.class)) + .filter(dataPrepperPlugin -> pluginType.equals(dataPrepperPlugin.pluginType())) + .collect(Collectors.toMap( + DataPrepperPlugin::name, + DataPrepperPlugin::pluginConfigurationType + )); + } + + private void addDocumentationLink(final ObjectNode jsonSchemaNode, + final String pluginName, + final Class pluginType) { + jsonSchemaNode.put(DOCUMENTATION_LINK_KEY, + String.format( + PLUGIN_DOCUMENTATION_URL_FORMAT, + siteUrl, + siteBaseUrl, + PLUGIN_TYPE_TO_URI_PARAMETER_MAP.get(pluginType), + pluginName)); + } + + private void addPluginName(final ObjectNode jsonSchemaNode, + final String pluginName) { + jsonSchemaNode.put(PLUGIN_NAME_KEY, pluginName); + } +} diff --git a/data-prepper-plugin-schema-cli/src/main/java/org/opensearch/dataprepper/schemas/module/CustomJacksonModule.java b/data-prepper-plugin-schema-cli/src/main/java/org/opensearch/dataprepper/schemas/module/CustomJacksonModule.java new file mode 100644 index 0000000000..09c649cc4c --- /dev/null +++ b/data-prepper-plugin-schema-cli/src/main/java/org/opensearch/dataprepper/schemas/module/CustomJacksonModule.java @@ -0,0 +1,31 @@ +package org.opensearch.dataprepper.schemas.module; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.databind.PropertyNamingStrategies; +import com.github.victools.jsonschema.generator.MemberScope; +import com.github.victools.jsonschema.module.jackson.JacksonModule; +import com.github.victools.jsonschema.module.jackson.JacksonOption; + +public class CustomJacksonModule extends JacksonModule { + + public CustomJacksonModule() { + super(); + } + + public CustomJacksonModule(JacksonOption... options) { + super(options); + } + + @Override + protected String getPropertyNameOverrideBasedOnJsonPropertyAnnotation(MemberScope member) { + JsonProperty annotation = member.getAnnotationConsideringFieldAndGetter(JsonProperty.class); + if (annotation != null) { + String nameOverride = annotation.value(); + // check for invalid overrides + if (nameOverride != null && !nameOverride.isEmpty() && !nameOverride.equals(member.getDeclaredName())) { + return nameOverride; + } + } + return PropertyNamingStrategies.SNAKE_CASE.nameForField(null, null, member.getName()); + } +} diff --git a/data-prepper-plugin-schema-cli/src/test/java/org/opensearch/dataprepper/schemas/JsonSchemaConverterTest.java b/data-prepper-plugin-schema-cli/src/test/java/org/opensearch/dataprepper/schemas/JsonSchemaConverterTest.java new file mode 100644 index 0000000000..d5d172f8c0 --- /dev/null +++ b/data-prepper-plugin-schema-cli/src/test/java/org/opensearch/dataprepper/schemas/JsonSchemaConverterTest.java @@ -0,0 +1,60 @@ +package org.opensearch.dataprepper.schemas; + +import com.fasterxml.jackson.annotation.JsonClassDescription; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.node.ObjectNode; +import com.github.victools.jsonschema.generator.Module; +import com.github.victools.jsonschema.generator.OptionPreset; +import com.github.victools.jsonschema.generator.SchemaVersion; +import org.junit.jupiter.api.Test; +import org.opensearch.dataprepper.schemas.module.CustomJacksonModule; + +import java.util.Collections; +import java.util.List; + +import static org.hamcrest.CoreMatchers.instanceOf; +import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.MatcherAssert.assertThat; + +class JsonSchemaConverterTest { + + public JsonSchemaConverter createObjectUnderTest(final List modules) { + return new JsonSchemaConverter(modules); + } + + @Test + void testConvertIntoJsonSchemaWithDefaultModules() throws JsonProcessingException { + final JsonSchemaConverter jsonSchemaConverter = createObjectUnderTest(Collections.emptyList()); + final ObjectNode jsonSchemaNode = jsonSchemaConverter.convertIntoJsonSchema( + SchemaVersion.DRAFT_2020_12, OptionPreset.PLAIN_JSON, TestConfig.class); + assertThat(jsonSchemaNode, instanceOf(ObjectNode.class)); + } + + @Test + void testConvertIntoJsonSchemaWithCustomJacksonModule() throws JsonProcessingException { + final JsonSchemaConverter jsonSchemaConverter = createObjectUnderTest( + Collections.singletonList(new CustomJacksonModule())); + final ObjectNode jsonSchemaNode = jsonSchemaConverter.convertIntoJsonSchema( + SchemaVersion.DRAFT_2020_12, OptionPreset.PLAIN_JSON, TestConfig.class); + assertThat(jsonSchemaNode, instanceOf(ObjectNode.class)); + assertThat(jsonSchemaNode.has("description"), is(true)); + final JsonNode propertiesNode = jsonSchemaNode.at("/properties"); + assertThat(propertiesNode, instanceOf(ObjectNode.class)); + assertThat(propertiesNode.has("test_attribute_with_getter"), is(true)); + assertThat(propertiesNode.has("custom_test_attribute"), is(true)); + } + + @JsonClassDescription("test config") + static class TestConfig { + private String testAttributeWithGetter; + + @JsonProperty("custom_test_attribute") + private String testAttributeWithJsonPropertyAnnotation; + + public String getTestAttributeWithGetter() { + return testAttributeWithGetter; + } + } +} \ No newline at end of file diff --git a/data-prepper-plugin-schema-cli/src/test/java/org/opensearch/dataprepper/schemas/PluginConfigsJsonSchemaConverterIT.java b/data-prepper-plugin-schema-cli/src/test/java/org/opensearch/dataprepper/schemas/PluginConfigsJsonSchemaConverterIT.java new file mode 100644 index 0000000000..71e9bf5faa --- /dev/null +++ b/data-prepper-plugin-schema-cli/src/test/java/org/opensearch/dataprepper/schemas/PluginConfigsJsonSchemaConverterIT.java @@ -0,0 +1,80 @@ +package org.opensearch.dataprepper.schemas; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.github.victools.jsonschema.generator.Module; +import com.github.victools.jsonschema.generator.OptionPreset; +import com.github.victools.jsonschema.generator.SchemaVersion; +import com.github.victools.jsonschema.module.jackson.JacksonModule; +import com.github.victools.jsonschema.module.jakarta.validation.JakartaValidationModule; +import com.github.victools.jsonschema.module.jakarta.validation.JakartaValidationOption; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; +import org.reflections.Reflections; +import org.reflections.scanners.Scanners; +import org.reflections.util.ClasspathHelper; +import org.reflections.util.ConfigurationBuilder; + +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.stream.Stream; + +import static com.github.victools.jsonschema.module.jackson.JacksonOption.RESPECT_JSONPROPERTY_REQUIRED; +import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.CoreMatchers.notNullValue; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.opensearch.dataprepper.schemas.PluginConfigsJsonSchemaConverter.DOCUMENTATION_LINK_KEY; +import static org.opensearch.dataprepper.schemas.PluginConfigsJsonSchemaConverter.PLUGIN_NAME_KEY; + +class PluginConfigsJsonSchemaConverterIT { + static final String DEFAULT_PLUGINS_CLASSPATH = "org.opensearch.dataprepper.plugins"; + private static final String TEST_URL = String.format("https://%s/", UUID.randomUUID()); + private static final String TEST_BASE_URL = String.format("/%s", UUID.randomUUID()); + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + static final TypeReference> MAP_TYPE_REFERENCE = new TypeReference<>() {}; + + private PluginConfigsJsonSchemaConverter objectUnderTest; + + @BeforeEach + void setUp() { + final List modules = List.of( + new JacksonModule(RESPECT_JSONPROPERTY_REQUIRED), + new JakartaValidationModule(JakartaValidationOption.NOT_NULLABLE_FIELD_IS_REQUIRED, + JakartaValidationOption.INCLUDE_PATTERN_EXPRESSIONS) + ); + final Reflections reflections = new Reflections(new ConfigurationBuilder() + .setUrls(ClasspathHelper.forPackage(DEFAULT_PLUGINS_CLASSPATH)) + .setScanners(Scanners.TypesAnnotated, Scanners.SubTypes)); + objectUnderTest = new PluginConfigsJsonSchemaConverter( + reflections, new JsonSchemaConverter(modules), TEST_URL, TEST_BASE_URL); + } + + @ParameterizedTest + @MethodSource("getValidPluginTypes") + void testConvertPluginConfigsIntoJsonSchemas(final Class pluginType) { + final Map result = objectUnderTest.convertPluginConfigsIntoJsonSchemas( + SchemaVersion.DRAFT_2020_12, OptionPreset.PLAIN_JSON, pluginType); + assertThat(result.isEmpty(), is(false)); + result.values().forEach(schema -> { + final Map schemaMap; + try { + schemaMap = OBJECT_MAPPER.readValue(schema, MAP_TYPE_REFERENCE); + } catch (JsonProcessingException e) { + throw new RuntimeException(e); + } + assertThat(schemaMap, notNullValue()); + assertThat(schemaMap.containsKey(PLUGIN_NAME_KEY), is(true)); + assertThat(((String) schemaMap.get(DOCUMENTATION_LINK_KEY)).startsWith(TEST_URL + TEST_BASE_URL), + is(true)); + }); + } + + private static Stream getValidPluginTypes() { + return PluginConfigsJsonSchemaConverter.PLUGIN_TYPE_TO_URI_PARAMETER_MAP.keySet() + .stream().flatMap(clazz -> Stream.of(Arguments.of(clazz))); + } +} \ No newline at end of file diff --git a/data-prepper-plugin-schema-cli/src/test/java/org/opensearch/dataprepper/schemas/PluginConfigsJsonSchemaConverterTest.java b/data-prepper-plugin-schema-cli/src/test/java/org/opensearch/dataprepper/schemas/PluginConfigsJsonSchemaConverterTest.java new file mode 100644 index 0000000000..3d1c1b585a --- /dev/null +++ b/data-prepper-plugin-schema-cli/src/test/java/org/opensearch/dataprepper/schemas/PluginConfigsJsonSchemaConverterTest.java @@ -0,0 +1,110 @@ +package org.opensearch.dataprepper.schemas; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.node.ObjectNode; +import com.github.victools.jsonschema.generator.OptionPreset; +import com.github.victools.jsonschema.generator.SchemaVersion; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.InjectMocks; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.dataprepper.model.annotations.DataPrepperPlugin; +import org.reflections.Reflections; + +import java.util.Map; +import java.util.Set; +import java.util.UUID; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.CoreMatchers.notNullValue; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; +import static org.opensearch.dataprepper.schemas.PluginConfigsJsonSchemaConverter.DOCUMENTATION_LINK_KEY; +import static org.opensearch.dataprepper.schemas.PluginConfigsJsonSchemaConverter.PLUGIN_NAME_KEY; +import static org.opensearch.dataprepper.schemas.PluginConfigsJsonSchemaConverter.PLUGIN_TYPE_NAME_TO_CLASS_MAP; + +@ExtendWith(MockitoExtension.class) +class PluginConfigsJsonSchemaConverterTest { + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + static final TypeReference> MAP_TYPE_REFERENCE = new TypeReference<>() {}; + + @Mock + private JsonSchemaConverter jsonSchemaConverter; + + @Mock + private Reflections reflections; + + @InjectMocks + private PluginConfigsJsonSchemaConverter objectUnderTest; + + @Test + void testValidPluginTypeNames() { + assertThat(PLUGIN_TYPE_NAME_TO_CLASS_MAP.keySet().containsAll(objectUnderTest.validPluginTypeNames()), + is(true)); + } + + @Test + void testPluginTypeNameToPluginTypeWithValidInput() { + objectUnderTest.validPluginTypeNames().forEach( + pluginType -> assertThat(objectUnderTest.pluginTypeNameToPluginType(pluginType), + equalTo(PLUGIN_TYPE_NAME_TO_CLASS_MAP.get(pluginType)))); + } + + @Test + void testPluginTypeNameToPluginTypeWithInValidInput() { + final String inValidPluginType = "invalid-" + UUID.randomUUID(); + assertThrows( + IllegalArgumentException.class, () -> objectUnderTest.pluginTypeNameToPluginType(inValidPluginType)); + } + + @Test + void testConvertPluginConfigsIntoJsonSchemasHappyPath() throws JsonProcessingException { + when(reflections.getTypesAnnotatedWith(eq(DataPrepperPlugin.class))).thenReturn(Set.of(TestPlugin.class)); + final ObjectNode objectNode = OBJECT_MAPPER.createObjectNode(); + when(jsonSchemaConverter.convertIntoJsonSchema( + any(SchemaVersion.class), any(OptionPreset.class), eq(TestPluginConfig.class))).thenReturn(objectNode); + final Map result = objectUnderTest.convertPluginConfigsIntoJsonSchemas( + SchemaVersion.DRAFT_2020_12, OptionPreset.PLAIN_JSON, TestPluginType.class); + assertThat(result.size(), equalTo(1)); + final Map schemaMap = OBJECT_MAPPER.readValue(result.get("test_plugin"), MAP_TYPE_REFERENCE); + assertThat(schemaMap, notNullValue()); + assertThat(schemaMap.get(DOCUMENTATION_LINK_KEY), equalTo( + "{{site.url}}{{site.baseurl}}/data-prepper/pipelines/configuration/null/test_plugin/" + )); + assertThat(schemaMap.containsKey(PLUGIN_NAME_KEY), is(true)); + } + + @Test + void testConvertPluginConfigsIntoJsonSchemasWithError() throws JsonProcessingException { + when(reflections.getTypesAnnotatedWith(eq(DataPrepperPlugin.class))).thenReturn(Set.of(TestPlugin.class)); + final JsonProcessingException jsonProcessingException = mock(JsonProcessingException.class); + when(jsonSchemaConverter.convertIntoJsonSchema( + any(SchemaVersion.class), any(OptionPreset.class), eq(TestPluginConfig.class))).thenThrow( + jsonProcessingException); + final Map result = objectUnderTest.convertPluginConfigsIntoJsonSchemas( + SchemaVersion.DRAFT_2020_12, OptionPreset.PLAIN_JSON, TestPluginType.class); + assertThat(result.isEmpty(), is(true)); + } + + @DataPrepperPlugin( + name = "test_plugin", pluginType = TestPluginType.class, pluginConfigurationType = TestPluginConfig.class) + static class TestPlugin { + + } + + static class TestPluginConfig { + + } + + static class TestPluginType { + + } +} \ No newline at end of file diff --git a/settings.gradle b/settings.gradle index cb7e888c53..18ccd4dc7b 100644 --- a/settings.gradle +++ b/settings.gradle @@ -104,6 +104,7 @@ include 'data-prepper-core' include 'data-prepper-main' include 'data-prepper-pipeline-parser' include 'data-prepper-plugin-framework' +include 'data-prepper-plugin-schema-cli' include 'data-prepper-plugins:common' include 'data-prepper-plugins:armeria-common' include 'data-prepper-plugins:anomaly-detector-processor' From 04de9eb7a9e2086078521c921bbbbc5866bc9e3d Mon Sep 17 00:00:00 2001 From: Hai Yan <8153134+oeyh@users.noreply.github.com> Date: Thu, 8 Aug 2024 16:51:12 -0500 Subject: [PATCH 14/25] Several improvements to RDS source (#4810) * Add schema manager to query database Signed-off-by: Hai Yan * Get real primary keys for export Signed-off-by: Hai Yan * Get binlog start position for stream Signed-off-by: Hai Yan * Refactor SnapshotStrategy to RdsApiStrategy Signed-off-by: Hai Yan * Update unit tests Signed-off-by: Hai Yan * address comments Signed-off-by: Hai Yan * Add retry to database queries Signed-off-by: Hai Yan * Handle describe exceptions Signed-off-by: Hai Yan * Address more comments Signed-off-by: Hai Yan --------- Signed-off-by: Hai Yan --- data-prepper-plugins/rds-source/build.gradle | 1 + .../plugins/source/rds/RdsService.java | 32 +++-- .../state/DataFileProgressState.java | 17 +++ .../state/ExportProgressState.java | 15 +++ .../source/rds/export/DataFileLoader.java | 7 +- .../source/rds/export/ExportScheduler.java | 25 ++-- .../source/rds/export/SnapshotManager.java | 5 +- .../ClusterApiStrategy.java} | 42 +++++-- .../InstanceApiStrategy.java} | 43 +++++-- .../source/rds/leader/LeaderScheduler.java | 34 +++++- .../RdsApiStrategy.java} | 14 ++- .../plugins/source/rds/model/DbMetadata.java | 44 +++++++ .../source/rds/schema/ConnectionManager.java | 47 ++++++++ .../source/rds/schema/SchemaManager.java | 79 +++++++++++++ .../rds/stream/BinlogClientFactory.java | 48 ++------ .../source/rds/stream/StreamWorker.java | 29 +++-- .../plugins/source/rds/RdsServiceTest.java | 33 +++++- .../rds/export/SnapshotManagerTest.java | 3 +- .../ClusterApiStrategyTest.java} | 40 ++++++- .../InstanceApiStrategyTest.java} | 42 ++++++- .../rds/leader/LeaderSchedulerTest.java | 10 +- .../rds/schema/ConnectionManagerTest.java | 80 +++++++++++++ .../source/rds/schema/SchemaManagerTest.java | 111 ++++++++++++++++++ .../rds/stream/BinlogClientFactoryTest.java | 30 ++--- .../source/rds/stream/StreamWorkerTest.java | 11 +- 25 files changed, 705 insertions(+), 137 deletions(-) rename data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/{export/ClusterSnapshotStrategy.java => leader/ClusterApiStrategy.java} (55%) rename data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/{export/InstanceSnapshotStrategy.java => leader/InstanceApiStrategy.java} (53%) rename data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/{export/SnapshotStrategy.java => leader/RdsApiStrategy.java} (61%) create mode 100644 data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/model/DbMetadata.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/SchemaManager.java rename data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/{export/ClusterSnapshotStrategyTest.java => leader/ClusterApiStrategyTest.java} (76%) rename data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/{export/InstanceSnapshotStrategyTest.java => leader/InstanceApiStrategyTest.java} (72%) create mode 100644 data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/schema/ConnectionManagerTest.java create mode 100644 data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/schema/SchemaManagerTest.java diff --git a/data-prepper-plugins/rds-source/build.gradle b/data-prepper-plugins/rds-source/build.gradle index 14c851f645..0203cccf10 100644 --- a/data-prepper-plugins/rds-source/build.gradle +++ b/data-prepper-plugins/rds-source/build.gradle @@ -22,6 +22,7 @@ dependencies { implementation 'com.fasterxml.jackson.core:jackson-databind' implementation 'com.zendesk:mysql-binlog-connector-java:0.29.2' + implementation 'com.mysql:mysql-connector-j:8.4.0' testImplementation project(path: ':data-prepper-test-common') testImplementation 'com.fasterxml.jackson.dataformat:jackson-dataformat-yaml' 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 73b71e0085..005fee88fd 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 @@ -13,14 +13,17 @@ import org.opensearch.dataprepper.model.event.EventFactory; import org.opensearch.dataprepper.model.record.Record; import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourceCoordinator; -import org.opensearch.dataprepper.plugins.source.rds.export.ClusterSnapshotStrategy; 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; -import org.opensearch.dataprepper.plugins.source.rds.export.InstanceSnapshotStrategy; import org.opensearch.dataprepper.plugins.source.rds.export.SnapshotManager; -import org.opensearch.dataprepper.plugins.source.rds.export.SnapshotStrategy; +import org.opensearch.dataprepper.plugins.source.rds.leader.ClusterApiStrategy; +import org.opensearch.dataprepper.plugins.source.rds.leader.InstanceApiStrategy; import org.opensearch.dataprepper.plugins.source.rds.leader.LeaderScheduler; +import org.opensearch.dataprepper.plugins.source.rds.leader.RdsApiStrategy; +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.SchemaManager; import org.opensearch.dataprepper.plugins.source.rds.stream.BinlogClientFactory; import org.opensearch.dataprepper.plugins.source.rds.stream.StreamScheduler; import org.slf4j.Logger; @@ -77,13 +80,16 @@ public RdsService(final EnhancedSourceCoordinator sourceCoordinator, public void start(Buffer> buffer) { LOG.info("Start running RDS service"); final List runnableList = new ArrayList<>(); - leaderScheduler = new LeaderScheduler(sourceCoordinator, sourceConfig); + + final RdsApiStrategy rdsApiStrategy = sourceConfig.isCluster() ? + new ClusterApiStrategy(rdsClient) : new InstanceApiStrategy(rdsClient); + final DbMetadata dbMetadata = rdsApiStrategy.describeDb(sourceConfig.getDbIdentifier()); + leaderScheduler = new LeaderScheduler( + sourceCoordinator, sourceConfig, getSchemaManager(sourceConfig, dbMetadata), dbMetadata); runnableList.add(leaderScheduler); if (sourceConfig.isExportEnabled()) { - final SnapshotStrategy snapshotStrategy = sourceConfig.isCluster() ? - new ClusterSnapshotStrategy(rdsClient) : new InstanceSnapshotStrategy(rdsClient); - final SnapshotManager snapshotManager = new SnapshotManager(snapshotStrategy); + final SnapshotManager snapshotManager = new SnapshotManager(rdsApiStrategy); final ExportTaskManager exportTaskManager = new ExportTaskManager(rdsClient); exportScheduler = new ExportScheduler( sourceCoordinator, snapshotManager, exportTaskManager, s3Client, pluginMetrics); @@ -94,7 +100,7 @@ public void start(Buffer> buffer) { } if (sourceConfig.isStreamEnabled()) { - BinaryLogClient binaryLogClient = new BinlogClientFactory(sourceConfig, rdsClient).create(); + BinaryLogClient binaryLogClient = new BinlogClientFactory(sourceConfig, rdsClient, dbMetadata).create(); if (sourceConfig.getTlsConfig() == null || !sourceConfig.getTlsConfig().isInsecure()) { binaryLogClient.setSSLMode(SSLMode.REQUIRED); } else { @@ -128,4 +134,14 @@ public void shutdown() { executor.shutdownNow(); } } + + private SchemaManager getSchemaManager(final RdsSourceConfig sourceConfig, final DbMetadata dbMetadata) { + final ConnectionManager connectionManager = new ConnectionManager( + dbMetadata.getHostName(), + dbMetadata.getPort(), + sourceConfig.getAuthenticationConfig().getUsername(), + sourceConfig.getAuthenticationConfig().getPassword(), + !sourceConfig.getTlsConfig().isInsecure()); + return new SchemaManager(connectionManager); + } } diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/state/DataFileProgressState.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/state/DataFileProgressState.java index 4f94f57d15..9fee60105f 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/state/DataFileProgressState.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/state/DataFileProgressState.java @@ -7,6 +7,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.List; +import java.util.Map; + public class DataFileProgressState { @JsonProperty("isLoaded") @@ -21,6 +24,12 @@ public class DataFileProgressState { @JsonProperty("sourceTable") private String sourceTable; + /** + * Map of table name to primary keys + */ + @JsonProperty("primaryKeyMap") + private Map> primaryKeyMap; + @JsonProperty("snapshotTime") private long snapshotTime; @@ -63,4 +72,12 @@ public long getSnapshotTime() { public void setSnapshotTime(long snapshotTime) { this.snapshotTime = snapshotTime; } + + public Map> getPrimaryKeyMap() { + return primaryKeyMap; + } + + public void setPrimaryKeyMap(Map> primaryKeyMap) { + this.primaryKeyMap = primaryKeyMap; + } } diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/state/ExportProgressState.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/state/ExportProgressState.java index b224f63a97..e4bbeb4c98 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/state/ExportProgressState.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/state/ExportProgressState.java @@ -8,6 +8,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; +import java.util.Map; /** * Progress state for an EXPORT partition @@ -32,6 +33,12 @@ public class ExportProgressState { @JsonProperty("tables") private List tables; + /** + * Map of table name to primary keys + */ + @JsonProperty("primaryKeyMap") + private Map> primaryKeyMap; + @JsonProperty("kmsKeyId") private String kmsKeyId; @@ -89,6 +96,14 @@ public void setTables(List tables) { this.tables = tables; } + public Map> getPrimaryKeyMap() { + return primaryKeyMap; + } + + public void setPrimaryKeyMap(Map> primaryKeyMap) { + this.primaryKeyMap = primaryKeyMap; + } + public String getKmsKeyId() { return kmsKeyId; } diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileLoader.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileLoader.java index 42a5b3a0d5..16bd559848 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileLoader.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileLoader.java @@ -93,8 +93,8 @@ public void run() { DataFileProgressState progressState = dataFilePartition.getProgressState().get(); - // TODO: primary key to be obtained by querying database schema - final String primaryKeyName = "id"; + final String fullTableName = progressState.getSourceDatabase() + "." + progressState.getSourceTable(); + final List primaryKeys = progressState.getPrimaryKeyMap().getOrDefault(fullTableName, List.of()); final long snapshotTime = progressState.getSnapshotTime(); final long eventVersionNumber = snapshotTime - VERSION_OVERLAP_TIME_FOR_EXPORT.toMillis(); @@ -103,13 +103,14 @@ public void run() { record, progressState.getSourceDatabase(), progressState.getSourceTable(), - List.of(primaryKeyName), + primaryKeys, snapshotTime, eventVersionNumber)); bufferAccumulator.add(transformedRecord); eventCount.getAndIncrement(); bytesProcessedSummary.record(bytes); } catch (Exception e) { + LOG.error("Failed to process record from object s3://{}/{}", bucket, objectKey, e); throw new RuntimeException(e); } }); diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/ExportScheduler.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/ExportScheduler.java index 343ade8b85..ce534747e1 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/ExportScheduler.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/ExportScheduler.java @@ -30,6 +30,7 @@ import java.time.Instant; import java.util.ArrayList; import java.util.List; +import java.util.Map; import java.util.Optional; import java.util.concurrent.Callable; import java.util.concurrent.CompletableFuture; @@ -189,12 +190,15 @@ private SnapshotInfo checkSnapshotStatus(String snapshotId, Duration timeout) { LOG.debug("Start checking status of snapshot {}", snapshotId); while (Instant.now().isBefore(endTime)) { SnapshotInfo snapshotInfo = snapshotManager.checkSnapshotStatus(snapshotId); - String status = snapshotInfo.getStatus(); - // Valid snapshot statuses are: available, copying, creating - // The status should never be "copying" here - if (SnapshotStatus.AVAILABLE.getStatusName().equals(status)) { - LOG.info("Snapshot {} is available.", snapshotId); - return snapshotInfo; + + if (snapshotInfo != null) { + String status = snapshotInfo.getStatus(); + // Valid snapshot statuses are: available, copying, creating + // The status should never be "copying" here + if (SnapshotStatus.AVAILABLE.getStatusName().equals(status)) { + LOG.info("Snapshot {} is available.", snapshotId); + return snapshotInfo; + } } LOG.debug("Snapshot {} is still creating. Wait and check later", snapshotId); @@ -272,7 +276,7 @@ private BiConsumer completeExport(ExportPartition exportParti // Create data file partitions for processing S3 files List dataFileObjectKeys = getDataFileObjectKeys(bucket, prefix, exportTaskId); - createDataFilePartitions(bucket, exportTaskId, dataFileObjectKeys, snapshotTime); + createDataFilePartitions(bucket, exportTaskId, dataFileObjectKeys, snapshotTime, state.getPrimaryKeyMap()); completeExportPartition(exportPartition); } @@ -301,7 +305,11 @@ private List getDataFileObjectKeys(String bucket, String prefix, String return objectKeys; } - private void createDataFilePartitions(String bucket, String exportTaskId, List dataFileObjectKeys, long snapshotTime) { + private void createDataFilePartitions(String bucket, + String exportTaskId, + List dataFileObjectKeys, + long snapshotTime, + Map> primaryKeyMap) { LOG.info("Total of {} data files generated for export {}", dataFileObjectKeys.size(), exportTaskId); AtomicInteger totalFiles = new AtomicInteger(); for (final String objectKey : dataFileObjectKeys) { @@ -313,6 +321,7 @@ private void createDataFilePartitions(String bucket, String exportTaskId, List> getPrimaryKeyMap() { + return sourceConfig.getTableNames().stream() + .collect(Collectors.toMap( + fullTableName -> fullTableName, + fullTableName -> schemaManager.getPrimaryKeys(fullTableName.split("\\.")[0], fullTableName.split("\\.")[1]) + )); + } + private void createStreamPartition(RdsSourceConfig sourceConfig) { final StreamProgressState progressState = new StreamProgressState(); progressState.setWaitForExport(sourceConfig.isExportEnabled()); + getCurrentBinlogPosition().ifPresent(progressState::setStartPosition); StreamPartition streamPartition = new StreamPartition(sourceConfig.getDbIdentifier(), progressState); sourceCoordinator.createPartition(streamPartition); } + + private Optional getCurrentBinlogPosition() { + return schemaManager.getCurrentBinaryLogPosition(); + } } diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/SnapshotStrategy.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/leader/RdsApiStrategy.java similarity index 61% rename from data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/SnapshotStrategy.java rename to data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/leader/RdsApiStrategy.java index af2f0507f7..5b8fe157c1 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/SnapshotStrategy.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/leader/RdsApiStrategy.java @@ -3,14 +3,22 @@ * SPDX-License-Identifier: Apache-2.0 */ -package org.opensearch.dataprepper.plugins.source.rds.export; +package org.opensearch.dataprepper.plugins.source.rds.leader; +import org.opensearch.dataprepper.plugins.source.rds.model.DbMetadata; import org.opensearch.dataprepper.plugins.source.rds.model.SnapshotInfo; /** - * Provides a strategy for creating and describing RDS snapshots. + * Provides a strategy for running RDS APIs. */ -public interface SnapshotStrategy { +public interface RdsApiStrategy { + /** + * Describes an RDS instance or cluster. + * @param dbIdentifier The identifier of the RDS instance or cluster to describe + * @return An {@link DbMetadata} object describing the instance or cluster + */ + DbMetadata describeDb(String dbIdentifier); + /** * Creates a snapshot of an RDS instance or cluster. * diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/model/DbMetadata.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/model/DbMetadata.java new file mode 100644 index 0000000000..02918e6da4 --- /dev/null +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/model/DbMetadata.java @@ -0,0 +1,44 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.rds.model; + +import java.util.Map; + +public class DbMetadata { + + private static final String DB_IDENTIFIER_KEY = "dbIdentifier"; + private static final String HOST_NAME_KEY = "hostName"; + private static final String PORT_KEY = "port"; + private final String dbIdentifier; + private final String hostName; + private final int port; + + public DbMetadata(final String dbIdentifier, final String hostName, final int port) { + this.dbIdentifier = dbIdentifier; + this.hostName = hostName; + this.port = port; + } + + public String getDbIdentifier() { + return dbIdentifier; + } + + public String getHostName() { + return hostName; + } + + public int getPort() { + return port; + } + + public Map toMap() { + return Map.of( + DB_IDENTIFIER_KEY, dbIdentifier, + HOST_NAME_KEY, hostName, + PORT_KEY, port + ); + } +} 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..90a93428c0 --- /dev/null +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/schema/ConnectionManager.java @@ -0,0 +1,47 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.rds.schema; + +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.SQLException; +import java.util.Properties; + +public class ConnectionManager { + static final String JDBC_URL_FORMAT = "jdbc:mysql://%s:%d"; + private final String hostName; + private final int port; + private final String username; + private final String password; + private final boolean requireSSL; + + public ConnectionManager(String hostName, int port, String username, String password, boolean requireSSL) { + this.hostName = hostName; + this.port = port; + this.username = username; + this.password = password; + this.requireSSL = requireSSL; + } + + public Connection getConnection() throws SQLException { + final Properties props = new Properties(); + props.setProperty("user", username); + props.setProperty("password", password); + if (requireSSL) { + props.setProperty("useSSL", "true"); + props.setProperty("requireSSL", "true"); + } else { + props.setProperty("useSSL", "false"); + } + final String jdbcUrl = String.format(JDBC_URL_FORMAT, hostName, port); + 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/SchemaManager.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/schema/SchemaManager.java new file mode 100644 index 0000000000..b7623fc85c --- /dev/null +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/schema/SchemaManager.java @@ -0,0 +1,79 @@ +/* + * 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.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.sql.Connection; +import java.sql.ResultSet; +import java.sql.Statement; +import java.util.ArrayList; +import java.util.List; +import java.util.Optional; + +public class SchemaManager { + private static final Logger LOG = LoggerFactory.getLogger(SchemaManager.class); + 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; + 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()) { + 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 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(); + } + + 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/stream/BinlogClientFactory.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/BinlogClientFactory.java index 7c28eea8db..24ed45d7c2 100644 --- 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 @@ -7,60 +7,28 @@ import com.github.shyiko.mysql.binlog.BinaryLogClient; 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 software.amazon.awssdk.services.rds.model.DBCluster; -import software.amazon.awssdk.services.rds.model.DBInstance; -import software.amazon.awssdk.services.rds.model.DescribeDbClustersRequest; -import software.amazon.awssdk.services.rds.model.DescribeDbClustersResponse; -import software.amazon.awssdk.services.rds.model.DescribeDbInstancesRequest; -import software.amazon.awssdk.services.rds.model.DescribeDbInstancesResponse; public class BinlogClientFactory { private final RdsSourceConfig sourceConfig; - private final RdsClient rdsClient; + private final DbMetadata dbMetadata; - public BinlogClientFactory(final RdsSourceConfig sourceConfig, final RdsClient rdsClient) { + public BinlogClientFactory(final RdsSourceConfig sourceConfig, + final RdsClient rdsClient, + final DbMetadata dbMetadata) { this.sourceConfig = sourceConfig; this.rdsClient = rdsClient; + this.dbMetadata = dbMetadata; } public BinaryLogClient create() { - // TODO: refactor SnapshotStrategy to RdsApiStrategy to accommodate more APIs for clusters and instances - String hostName; - int port; - if (sourceConfig.isCluster()) { - DBCluster dbCluster = describeDbCluster(sourceConfig.getDbIdentifier()); - hostName = dbCluster.endpoint(); - port = dbCluster.port(); - } else { - DBInstance dbInstance = describeDbInstance(sourceConfig.getDbIdentifier()); - hostName = dbInstance.endpoint().address(); - port = dbInstance.endpoint().port(); - } return new BinaryLogClient( - hostName, - port, + dbMetadata.getHostName(), + dbMetadata.getPort(), sourceConfig.getAuthenticationConfig().getUsername(), sourceConfig.getAuthenticationConfig().getPassword()); } - - private DBInstance describeDbInstance(final String dbInstanceIdentifier) { - DescribeDbInstancesRequest request = DescribeDbInstancesRequest.builder() - .dbInstanceIdentifier(dbInstanceIdentifier) - .build(); - - DescribeDbInstancesResponse response = rdsClient.describeDBInstances(request); - return response.dbInstances().get(0); - } - - private DBCluster describeDbCluster(final String dbClusterIdentifier) { - DescribeDbClustersRequest request = DescribeDbClustersRequest.builder() - .dbClusterIdentifier(dbClusterIdentifier) - .build(); - - DescribeDbClustersResponse response = rdsClient.describeDBClusters(request); - return response.dbClusters().get(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 e1e169dea5..303ecc2a53 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 @@ -41,29 +41,19 @@ public static StreamWorker create(final EnhancedSourceCoordinator sourceCoordina } public void processStream(final StreamPartition streamPartition) { - // get current binlog position - BinlogCoordinate currentBinlogCoords = streamPartition.getProgressState().get().getCurrentPosition(); - - // set start of binlog stream to current position if exists - if (currentBinlogCoords != null) { - final String binlogFilename = currentBinlogCoords.getBinlogFilename(); - final long binlogPosition = currentBinlogCoords.getBinlogPosition(); - LOG.debug("Will start binlog stream from binlog file {} and position {}.", binlogFilename, binlogPosition); - binaryLogClient.setBinlogFilename(binlogFilename); - binaryLogClient.setBinlogPosition(binlogPosition); - } - while (shouldWaitForExport(streamPartition) && !Thread.currentThread().isInterrupted()) { LOG.info("Initial load not completed yet for {}, waiting...", streamPartition.getPartitionKey()); try { Thread.sleep(DEFAULT_EXPORT_COMPLETE_WAIT_INTERVAL_MILLIS); } catch (final InterruptedException ex) { - LOG.info("The StreamScheduler was interrupted while waiting to retry, stopping processing"); + LOG.info("The Stream Scheduler was interrupted while waiting to retry, stopping processing"); Thread.currentThread().interrupt(); break; } } + setStartBinlogPosition(streamPartition); + try { LOG.info("Connect to database to read change events."); binaryLogClient.connect(); @@ -92,4 +82,17 @@ private boolean isExportDone(StreamPartition streamPartition) { Optional globalStatePartition = sourceCoordinator.getPartition("stream-for-" + dbIdentifier); return globalStatePartition.isPresent(); } + + private void setStartBinlogPosition(final StreamPartition streamPartition) { + final BinlogCoordinate startBinlogPosition = streamPartition.getProgressState().get().getStartPosition(); + + // set start of binlog stream to current position if exists + if (startBinlogPosition != null) { + final String binlogFilename = startBinlogPosition.getBinlogFilename(); + final long binlogPosition = startBinlogPosition.getBinlogPosition(); + LOG.debug("Will start binlog stream from binlog file {} and position {}.", binlogFilename, binlogPosition); + binaryLogClient.setBinlogFilename(binlogFilename); + binaryLogClient.setBinlogPosition(binlogPosition); + } + } } 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 275d5c0c38..e0b69746c2 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 @@ -76,9 +76,23 @@ void setUp() { @Test void test_normal_service_start_when_export_is_enabled() { - RdsService rdsService = createObjectUnderTest(); + final String dbIdentifier = UUID.randomUUID().toString(); + final String host = UUID.randomUUID().toString(); + final int port = 3306; + final DescribeDbInstancesResponse describeDbInstancesResponse = DescribeDbInstancesResponse.builder() + .dbInstances(DBInstance.builder() + .endpoint(Endpoint.builder() + .address(host) + .port(port) + .build()) + .build()) + .build(); when(sourceConfig.isExportEnabled()).thenReturn(true); when(sourceConfig.isStreamEnabled()).thenReturn(false); + when(sourceConfig.getDbIdentifier()).thenReturn(dbIdentifier); + when(rdsClient.describeDBInstances(any(DescribeDbInstancesRequest.class))).thenReturn(describeDbInstancesResponse); + + final RdsService rdsService = createObjectUnderTest(); try (final MockedStatic executorsMockedStatic = mockStatic(Executors.class)) { executorsMockedStatic.when(() -> Executors.newFixedThreadPool(anyInt())).thenReturn(executor); rdsService.start(buffer); @@ -92,7 +106,6 @@ void test_normal_service_start_when_export_is_enabled() { @Test void test_normal_service_start_when_stream_is_enabled() { - RdsService rdsService = createObjectUnderTest(); when(sourceConfig.isStreamEnabled()).thenReturn(true); when(sourceConfig.isExportEnabled()).thenReturn(false); final String dbIdentifier = UUID.randomUUID().toString(); @@ -107,12 +120,14 @@ void test_normal_service_start_when_stream_is_enabled() { .endpoint(hostEndpoint) .build()); when(rdsClient.describeDBInstances(any(DescribeDbInstancesRequest.class))).thenReturn(describeDbInstancesResponse); + final RdsSourceConfig.AuthenticationConfig authConfig = mock(RdsSourceConfig.AuthenticationConfig.class); when(authConfig.getUsername()).thenReturn(UUID.randomUUID().toString()); when(authConfig.getPassword()).thenReturn(UUID.randomUUID().toString()); when(sourceConfig.getAuthenticationConfig()).thenReturn(authConfig); when(sourceConfig.getTlsConfig()).thenReturn(mock(TlsConfig.class)); + final RdsService rdsService = createObjectUnderTest(); try (final MockedStatic executorsMockedStatic = mockStatic(Executors.class)) { executorsMockedStatic.when(() -> Executors.newFixedThreadPool(anyInt())).thenReturn(executor); rdsService.start(buffer); @@ -126,6 +141,20 @@ void test_normal_service_start_when_stream_is_enabled() { @Test void test_service_shutdown_calls_executor_shutdownNow() { + final String dbIdentifier = UUID.randomUUID().toString(); + final String host = UUID.randomUUID().toString(); + final int port = 3306; + final DescribeDbInstancesResponse describeDbInstancesResponse = DescribeDbInstancesResponse.builder() + .dbInstances(DBInstance.builder() + .endpoint(Endpoint.builder() + .address(host) + .port(port) + .build()) + .build()) + .build(); + when(sourceConfig.getDbIdentifier()).thenReturn(dbIdentifier); + when(rdsClient.describeDBInstances(any(DescribeDbInstancesRequest.class))).thenReturn(describeDbInstancesResponse); + RdsService rdsService = createObjectUnderTest(); try (final MockedStatic executorsMockedStatic = mockStatic(Executors.class)) { executorsMockedStatic.when(() -> Executors.newFixedThreadPool(anyInt())).thenReturn(executor); diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/SnapshotManagerTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/SnapshotManagerTest.java index d88531bb65..d517a20700 100644 --- a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/SnapshotManagerTest.java +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/SnapshotManagerTest.java @@ -10,6 +10,7 @@ import org.junit.jupiter.api.extension.ExtendWith; import org.mockito.Mock; import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.dataprepper.plugins.source.rds.leader.RdsApiStrategy; import java.util.UUID; @@ -21,7 +22,7 @@ class SnapshotManagerTest { @Mock - private SnapshotStrategy snapshotStrategy; + private RdsApiStrategy snapshotStrategy; private SnapshotManager snapshotManager; diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/ClusterSnapshotStrategyTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/leader/ClusterApiStrategyTest.java similarity index 76% rename from data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/ClusterSnapshotStrategyTest.java rename to data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/leader/ClusterApiStrategyTest.java index a0965efad2..a64604e035 100644 --- a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/ClusterSnapshotStrategyTest.java +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/leader/ClusterApiStrategyTest.java @@ -3,7 +3,7 @@ * SPDX-License-Identifier: Apache-2.0 */ -package org.opensearch.dataprepper.plugins.source.rds.export; +package org.opensearch.dataprepper.plugins.source.rds.leader; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -11,16 +11,21 @@ import org.mockito.ArgumentCaptor; import org.mockito.Mock; import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.dataprepper.plugins.source.rds.model.DbMetadata; import org.opensearch.dataprepper.plugins.source.rds.model.SnapshotInfo; import software.amazon.awssdk.services.rds.RdsClient; import software.amazon.awssdk.services.rds.model.CreateDbClusterSnapshotRequest; import software.amazon.awssdk.services.rds.model.CreateDbClusterSnapshotResponse; +import software.amazon.awssdk.services.rds.model.DBCluster; import software.amazon.awssdk.services.rds.model.DBClusterSnapshot; import software.amazon.awssdk.services.rds.model.DescribeDbClusterSnapshotsRequest; import software.amazon.awssdk.services.rds.model.DescribeDbClusterSnapshotsResponse; +import software.amazon.awssdk.services.rds.model.DescribeDbClustersRequest; +import software.amazon.awssdk.services.rds.model.DescribeDbClustersResponse; import java.time.Instant; import java.util.List; +import java.util.Random; import java.util.UUID; import static org.hamcrest.MatcherAssert.assertThat; @@ -32,18 +37,43 @@ import static org.mockito.Mockito.when; @ExtendWith(MockitoExtension.class) -class ClusterSnapshotStrategyTest { +class ClusterApiStrategyTest { @Mock private RdsClient rdsClient; - private ClusterSnapshotStrategy objectUnderTest; + private ClusterApiStrategy objectUnderTest; + + private final Random random = new Random(); @BeforeEach void setUp() { objectUnderTest = createObjectUnderTest(); } + @Test + void test_describeDb_returns_correct_results() { + final String dbClusterId = UUID.randomUUID().toString(); + final String host = UUID.randomUUID().toString(); + final int port = random.nextInt(); + final DescribeDbClustersRequest describeDbClustersRequest = DescribeDbClustersRequest.builder() + .dbClusterIdentifier(dbClusterId) + .build(); + final DescribeDbClustersResponse describeDbClustersResponse = DescribeDbClustersResponse.builder() + .dbClusters(DBCluster.builder() + .endpoint(host) + .port(port) + .build()) + .build(); + when(rdsClient.describeDBClusters(describeDbClustersRequest)).thenReturn(describeDbClustersResponse); + + DbMetadata dbMetadata = objectUnderTest.describeDb(dbClusterId); + + assertThat(dbMetadata.getDbIdentifier(), equalTo(dbClusterId)); + assertThat(dbMetadata.getHostName(), equalTo(host)); + assertThat(dbMetadata.getPort(), equalTo(port)); + } + @Test void test_create_snapshot_with_success() { final String dbInstanceId = UUID.randomUUID().toString(); @@ -112,7 +142,7 @@ void test_check_snapshot_status_returns_correct_result() { assertThat(snapshotInfo.getCreateTime(), equalTo(createTime)); } - private ClusterSnapshotStrategy createObjectUnderTest() { - return new ClusterSnapshotStrategy(rdsClient); + private ClusterApiStrategy createObjectUnderTest() { + return new ClusterApiStrategy(rdsClient); } } \ No newline at end of file diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/InstanceSnapshotStrategyTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/leader/InstanceApiStrategyTest.java similarity index 72% rename from data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/InstanceSnapshotStrategyTest.java rename to data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/leader/InstanceApiStrategyTest.java index c9a87ea3a2..8699c4b4be 100644 --- a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/InstanceSnapshotStrategyTest.java +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/leader/InstanceApiStrategyTest.java @@ -3,7 +3,7 @@ * SPDX-License-Identifier: Apache-2.0 */ -package org.opensearch.dataprepper.plugins.source.rds.export; +package org.opensearch.dataprepper.plugins.source.rds.leader; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -11,16 +11,22 @@ import org.mockito.ArgumentCaptor; import org.mockito.Mock; import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.dataprepper.plugins.source.rds.model.DbMetadata; import org.opensearch.dataprepper.plugins.source.rds.model.SnapshotInfo; import software.amazon.awssdk.services.rds.RdsClient; import software.amazon.awssdk.services.rds.model.CreateDbSnapshotRequest; import software.amazon.awssdk.services.rds.model.CreateDbSnapshotResponse; +import software.amazon.awssdk.services.rds.model.DBInstance; import software.amazon.awssdk.services.rds.model.DBSnapshot; +import software.amazon.awssdk.services.rds.model.DescribeDbInstancesRequest; +import software.amazon.awssdk.services.rds.model.DescribeDbInstancesResponse; import software.amazon.awssdk.services.rds.model.DescribeDbSnapshotsRequest; import software.amazon.awssdk.services.rds.model.DescribeDbSnapshotsResponse; +import software.amazon.awssdk.services.rds.model.Endpoint; import java.time.Instant; import java.util.List; +import java.util.Random; import java.util.UUID; import static org.hamcrest.MatcherAssert.assertThat; @@ -32,18 +38,44 @@ import static org.mockito.Mockito.when; @ExtendWith(MockitoExtension.class) -class InstanceSnapshotStrategyTest { +class InstanceApiStrategyTest { @Mock private RdsClient rdsClient; - private InstanceSnapshotStrategy objectUnderTest; + private InstanceApiStrategy objectUnderTest; + private final Random random = new Random(); @BeforeEach void setUp() { objectUnderTest = createObjectUnderTest(); } + @Test + void test_describeDb_returns_correct_results() { + final String dbInstanceId = UUID.randomUUID().toString(); + final String host = UUID.randomUUID().toString(); + final int port = random.nextInt(); + final DescribeDbInstancesRequest describeDbInstancesRequest = DescribeDbInstancesRequest.builder() + .dbInstanceIdentifier(dbInstanceId) + .build(); + final DescribeDbInstancesResponse describeDbInstancesResponse = DescribeDbInstancesResponse.builder() + .dbInstances(DBInstance.builder() + .endpoint(Endpoint.builder() + .address(host) + .port(port) + .build()) + .build()) + .build(); + when(rdsClient.describeDBInstances(describeDbInstancesRequest)).thenReturn(describeDbInstancesResponse); + + DbMetadata dbMetadata = objectUnderTest.describeDb(dbInstanceId); + + assertThat(dbMetadata.getDbIdentifier(), equalTo(dbInstanceId)); + assertThat(dbMetadata.getHostName(), equalTo(host)); + assertThat(dbMetadata.getPort(), equalTo(port)); + } + @Test void test_create_snapshot_with_success() { final String dbInstanceId = UUID.randomUUID().toString(); @@ -112,7 +144,7 @@ void test_check_snapshot_status_returns_correct_result() { assertThat(snapshotInfo.getCreateTime(), equalTo(createTime)); } - private InstanceSnapshotStrategy createObjectUnderTest() { - return new InstanceSnapshotStrategy(rdsClient); + private InstanceApiStrategy createObjectUnderTest() { + return new InstanceApiStrategy(rdsClient); } } \ No newline at end of file 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 e844cc0ff4..e4b9cb1304 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 @@ -19,6 +19,8 @@ import org.opensearch.dataprepper.plugins.source.rds.coordination.partition.GlobalState; 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.DbMetadata; +import org.opensearch.dataprepper.plugins.source.rds.schema.SchemaManager; import java.time.Duration; import java.util.Optional; @@ -46,6 +48,12 @@ class LeaderSchedulerTest { @Mock(answer = Answers.RETURNS_DEFAULTS) private RdsSourceConfig sourceConfig; + @Mock + private SchemaManager schemaManager; + + @Mock + private DbMetadata dbMetadata; + @Mock private LeaderPartition leaderPartition; @@ -130,6 +138,6 @@ void test_shutDown() { } private LeaderScheduler createObjectUnderTest() { - return new LeaderScheduler(sourceCoordinator, sourceConfig); + return new LeaderScheduler(sourceCoordinator, sourceConfig, schemaManager, dbMetadata); } } \ No newline at end of file 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/ConnectionManagerTest.java new file mode 100644 index 0000000000..7705ac967b --- /dev/null +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/schema/ConnectionManagerTest.java @@ -0,0 +1,80 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +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 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; + + +class ConnectionManagerTest { + + private String hostName; + private int port; + private String username; + private String password; + private boolean requireSSL; + private final Random random = new Random(); + + @BeforeEach + void setUp() { + hostName = UUID.randomUUID().toString(); + port = random.nextInt(); + username = UUID.randomUUID().toString(); + password = UUID.randomUUID().toString(); + } + + @Test + void test_getConnection_when_requireSSL_is_true() throws SQLException { + requireSSL = true; + final ConnectionManager 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))); + final Properties properties = propertiesArgumentCaptor.getValue(); + assertThat(properties.getProperty("user"), is(username)); + assertThat(properties.getProperty("password"), is(password)); + assertThat(properties.getProperty("useSSL"), is("true")); + assertThat(properties.getProperty("requireSSL"), is("true")); + } + + @Test + void test_getConnection_when_requireSSL_is_false() throws SQLException { + requireSSL = false; + final ConnectionManager 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))); + final Properties properties = propertiesArgumentCaptor.getValue(); + assertThat(properties.getProperty("user"), is(username)); + assertThat(properties.getProperty("password"), is(password)); + assertThat(properties.getProperty("useSSL"), is("false")); + } + + private ConnectionManager createObjectUnderTest() { + return new ConnectionManager(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/SchemaManagerTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/schema/SchemaManagerTest.java new file mode 100644 index 0000000000..2661fd2bf0 --- /dev/null +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/schema/SchemaManagerTest.java @@ -0,0 +1,111 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +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.model.BinlogCoordinate; + +import java.sql.Connection; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Statement; +import java.util.List; +import java.util.Optional; +import java.util.UUID; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.contains; +import static org.hamcrest.Matchers.empty; +import static org.hamcrest.Matchers.is; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; +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; + +@ExtendWith(MockitoExtension.class) +class SchemaManagerTest { + + @Mock + private ConnectionManager connectionManager; + + @Mock(answer = Answers.RETURNS_DEEP_STUBS) + private Connection connection; + + @Mock + private ResultSet resultSet; + + private SchemaManager schemaManager; + + @BeforeEach + void setUp() { + schemaManager = createObjectUnderTest(); + } + + @Test + void test_getPrimaryKeys_returns_primary_keys() throws SQLException { + final String databaseName = UUID.randomUUID().toString(); + final String tableName = UUID.randomUUID().toString(); + final String primaryKey = UUID.randomUUID().toString(); + when(connectionManager.getConnection()).thenReturn(connection); + when(connection.getMetaData().getPrimaryKeys(databaseName, null, tableName)).thenReturn(resultSet); + when(resultSet.next()).thenReturn(true, false); + when(resultSet.getString(COLUMN_NAME)).thenReturn(primaryKey); + + final List primaryKeys = schemaManager.getPrimaryKeys(databaseName, tableName); + + assertThat(primaryKeys, contains(primaryKey)); + } + + @Test + void test_getPrimaryKeys_throws_exception_then_returns_empty_list() throws SQLException { + final String databaseName = UUID.randomUUID().toString(); + final String tableName = UUID.randomUUID().toString(); + when(connectionManager.getConnection()).thenThrow(SQLException.class); + + final List primaryKeys = schemaManager.getPrimaryKeys(databaseName, tableName); + + assertThat(primaryKeys, empty()); + } + + @Test + void test_getCurrentBinaryLogPosition_returns_binlog_coords() throws SQLException { + final Statement statement = mock(Statement.class); + final String binlogFile = UUID.randomUUID().toString(); + final long binlogPosition = 123L; + when(connectionManager.getConnection()).thenReturn(connection); + when(connection.createStatement()).thenReturn(statement); + when(statement.executeQuery(BINLOG_STATUS_QUERY)).thenReturn(resultSet); + when(resultSet.next()).thenReturn(true); + when(resultSet.getString(BINLOG_FILE)).thenReturn(binlogFile); + when(resultSet.getLong(BINLOG_POSITION)).thenReturn(binlogPosition); + + final Optional binlogCoordinate = schemaManager.getCurrentBinaryLogPosition(); + + assertThat(binlogCoordinate.isPresent(), is(true)); + assertThat(binlogCoordinate.get().getBinlogFilename(), is(binlogFile)); + assertThat(binlogCoordinate.get().getBinlogPosition(), is(binlogPosition)); + } + + @Test + void test_getCurrentBinaryLogPosition_throws_exception_then_returns_empty() throws SQLException { + when(connectionManager.getConnection()).thenThrow(SQLException.class); + + final Optional binlogCoordinate = schemaManager.getCurrentBinaryLogPosition(); + + assertThat(binlogCoordinate.isPresent(), is(false)); + } + + private SchemaManager createObjectUnderTest() { + return new SchemaManager(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/BinlogClientFactoryTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/BinlogClientFactoryTest.java index 52ba312b5a..326ea4eb67 100644 --- 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 @@ -11,18 +11,13 @@ 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 software.amazon.awssdk.services.rds.model.DBInstance; -import software.amazon.awssdk.services.rds.model.DescribeDbInstancesRequest; -import software.amazon.awssdk.services.rds.model.DescribeDbInstancesResponse; -import java.util.List; import java.util.Random; -import java.util.UUID; -import static org.mockito.ArgumentMatchers.any; -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) @@ -34,6 +29,9 @@ class BinlogClientFactoryTest { @Mock private RdsClient rdsClient; + @Mock + private DbMetadata dbMetadata; + private BinlogClientFactory binlogClientFactory; private Random random; @@ -45,22 +43,18 @@ void setUp() { @Test void test_create() { - DescribeDbInstancesResponse describeDbInstancesResponse = mock(DescribeDbInstancesResponse.class); - DBInstance dbInstance = mock(DBInstance.class, RETURNS_DEEP_STUBS); - final String address = UUID.randomUUID().toString(); - final Integer port = random.nextInt(); - when(dbInstance.endpoint().address()).thenReturn(address); - when(dbInstance.endpoint().port()).thenReturn(port); - when(describeDbInstancesResponse.dbInstances()).thenReturn(List.of(dbInstance)); - when(sourceConfig.getDbIdentifier()).thenReturn(UUID.randomUUID().toString()); - when(rdsClient.describeDBInstances(any(DescribeDbInstancesRequest.class))).thenReturn(describeDbInstancesResponse); - RdsSourceConfig.AuthenticationConfig authenticationConfig = mock(RdsSourceConfig.AuthenticationConfig.class); + final RdsSourceConfig.AuthenticationConfig authenticationConfig = mock(RdsSourceConfig.AuthenticationConfig.class); when(sourceConfig.getAuthenticationConfig()).thenReturn(authenticationConfig); binlogClientFactory.create(); + + verify(dbMetadata).getHostName(); + verify(dbMetadata).getPort(); + verify(authenticationConfig).getUsername(); + verify(authenticationConfig).getPassword(); } private BinlogClientFactory createBinlogClientFactory() { - return new BinlogClientFactory(sourceConfig, rdsClient); + return new BinlogClientFactory(sourceConfig, rdsClient, dbMetadata); } } \ No newline at end of file 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 4dd3930466..1392c852e8 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 @@ -19,6 +19,7 @@ import java.io.IOException; import java.util.Optional; +import java.util.UUID; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.never; @@ -49,11 +50,11 @@ void setUp() { @Test void test_processStream_with_given_binlog_coordinates() throws IOException { - StreamProgressState streamProgressState = mock(StreamProgressState.class); + final StreamProgressState streamProgressState = mock(StreamProgressState.class); + final String binlogFilename = UUID.randomUUID().toString(); + final long binlogPosition = 100L; when(streamPartition.getProgressState()).thenReturn(Optional.of(streamProgressState)); - final String binlogFilename = "binlog-001"; - final Long binlogPosition = 100L; - when(streamProgressState.getCurrentPosition()).thenReturn(new BinlogCoordinate(binlogFilename, binlogPosition)); + when(streamProgressState.getStartPosition()).thenReturn(new BinlogCoordinate(binlogFilename, binlogPosition)); when(streamProgressState.shouldWaitForExport()).thenReturn(false); streamWorker.processStream(streamPartition); @@ -69,7 +70,7 @@ void test_processStream_without_current_binlog_coordinates() throws IOException when(streamPartition.getProgressState()).thenReturn(Optional.of(streamProgressState)); final String binlogFilename = "binlog-001"; final Long binlogPosition = 100L; - when(streamProgressState.getCurrentPosition()).thenReturn(null); + when(streamProgressState.getStartPosition()).thenReturn(null); when(streamProgressState.shouldWaitForExport()).thenReturn(false); streamWorker.processStream(streamPartition); From 1ddebf68f50b730d3e495f50d53ea6fe9129c7b2 Mon Sep 17 00:00:00 2001 From: Krishna Kondaka <41027584+kkondaka@users.noreply.github.com> Date: Sat, 10 Aug 2024 12:04:18 -0700 Subject: [PATCH 15/25] Fix null document in DLQ object (#4814) * dplive1.yaml Signed-off-by: Krishna Kondaka * Delete .github/workflows/static.yml Signed-off-by: Krishna Kondaka * Fix null document in DLQ object Signed-off-by: Krishna Kondaka --------- Signed-off-by: Krishna Kondaka Co-authored-by: Krishna Kondaka --- .../dataprepper/plugins/sink/opensearch/OpenSearchSink.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/OpenSearchSink.java b/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/OpenSearchSink.java index 1b6f44fde6..2248ba669a 100644 --- a/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/OpenSearchSink.java +++ b/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/OpenSearchSink.java @@ -448,9 +448,9 @@ public void doOutput(final Collection> records) { } SerializedJson serializedJsonNode = null; - if (StringUtils.equals(action, OpenSearchBulkActions.UPDATE.toString()) || - StringUtils.equals(action, OpenSearchBulkActions.UPSERT.toString()) || - StringUtils.equals(action, OpenSearchBulkActions.DELETE.toString())) { + if (StringUtils.equals(eventAction, OpenSearchBulkActions.UPDATE.toString()) || + StringUtils.equals(eventAction, OpenSearchBulkActions.UPSERT.toString()) || + StringUtils.equals(eventAction, OpenSearchBulkActions.DELETE.toString())) { serializedJsonNode = SerializedJson.fromJsonNode(event.getJsonNode(), document); } BulkOperation bulkOperation; From 176d8288eaea408304ecde56caa60cbea9a626d8 Mon Sep 17 00:00:00 2001 From: David Venable Date: Mon, 12 Aug 2024 10:20:48 -0500 Subject: [PATCH 16/25] Fixes a regex expression bug. When the left-hand side of the operation is null, always return false rather than throwing an exception. Resolves #4763. (#4798) Signed-off-by: David Venable --- .../expression/GenericRegexMatchOperator.java | 2 + ...ericExpressionEvaluator_ConditionalIT.java | 213 +++++++++--------- .../expression/RegexEqualOperatorTest.java | 6 + .../expression/RegexNotEqualOperatorTest.java | 6 + 4 files changed, 123 insertions(+), 104 deletions(-) diff --git a/data-prepper-expression/src/main/java/org/opensearch/dataprepper/expression/GenericRegexMatchOperator.java b/data-prepper-expression/src/main/java/org/opensearch/dataprepper/expression/GenericRegexMatchOperator.java index 1154978116..b4d34dadc1 100644 --- a/data-prepper-expression/src/main/java/org/opensearch/dataprepper/expression/GenericRegexMatchOperator.java +++ b/data-prepper-expression/src/main/java/org/opensearch/dataprepper/expression/GenericRegexMatchOperator.java @@ -37,6 +37,8 @@ public int getSymbol() { @Override public Boolean evaluate(final Object ... args) { checkArgument(args.length == 2, displayName + " requires operands length needs to be 2."); + if(args[0] == null) + return false; checkArgument(args[0] instanceof String, displayName + " requires left operand to be String."); checkArgument(args[1] instanceof String, displayName + " requires right operand to be String."); try { diff --git a/data-prepper-expression/src/test/java/org/opensearch/dataprepper/expression/GenericExpressionEvaluator_ConditionalIT.java b/data-prepper-expression/src/test/java/org/opensearch/dataprepper/expression/GenericExpressionEvaluator_ConditionalIT.java index 0bef1a65a0..a8fc7971f3 100644 --- a/data-prepper-expression/src/test/java/org/opensearch/dataprepper/expression/GenericExpressionEvaluator_ConditionalIT.java +++ b/data-prepper-expression/src/test/java/org/opensearch/dataprepper/expression/GenericExpressionEvaluator_ConditionalIT.java @@ -35,6 +35,7 @@ import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.equalTo; import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.params.provider.Arguments.arguments; class GenericExpressionEvaluator_ConditionalIT { /** @@ -145,76 +146,80 @@ private static Stream validExpressionArguments() { int testStringLength = random.nextInt(10); String testString = RandomStringUtils.randomAlphabetic(testStringLength); return Stream.of( - Arguments.of("true", event("{}"), true), - Arguments.of("/status_code == 200", event("{\"status_code\": 200}"), true), - Arguments.of("/status_code == 200", longEvent, true), - Arguments.of("/status_code != 300", event("{\"status_code\": 200}"), true), - Arguments.of("/status_code == 200", event("{}"), false), - Arguments.of("/success == /status_code", event("{\"success\": true, \"status_code\": 200}"), false), - Arguments.of("/success != /status_code", event("{\"success\": true, \"status_code\": 200}"), true), - Arguments.of("/part1@part2.part3 != 111", event("{\"success\": true, \"part1@part2.part3\":111, \"status_code\": 200}"), false), - Arguments.of("/part1.part2@part3 != 111", event("{\"success\": true, \"part1.part2@part3\":222, \"status_code\": 200}"), true), - Arguments.of("/pi == 3.14159", event("{\"pi\": 3.14159}"), true), - Arguments.of("/value == 12345.678", event("{\"value\": 12345.678}"), true), - Arguments.of("/value == 12345.678E12", event("{\"value\": 12345.678E12}"), true), - Arguments.of("/value == 12345.678e-12", event("{\"value\": 12345.678e-12}"), true), - Arguments.of("/value == 12345.0000012", event("{\"value\": 12345.0000012}"), true), - Arguments.of("/value == 12345.00012E6", event("{\"value\": 12345.00012E6}"), true), - Arguments.of("true == (/is_cool == true)", event("{\"is_cool\": true}"), true), - Arguments.of("not /is_cool", event("{\"is_cool\": true}"), false), - Arguments.of("/status_code < 300", event("{\"status_code\": 200}"), true), - Arguments.of("/status_code != null", event("{\"status_code\": 200}"), true), - Arguments.of("null != /status_code", event("{\"status_code\": 200}"), true), - Arguments.of("/status_code == null", event("{\"status_code\": null}"), true), - Arguments.of("/response == null", event("{\"status_code\": 200}"), true), - Arguments.of("null == /response", event("{\"status_code\": 200}"), true), - Arguments.of("/response != null", event("{\"status_code\": 200}"), false), - Arguments.of("/status_code <= 0", event("{\"status_code\": 200}"), false), - Arguments.of("/status_code > 0", event("{\"status_code\": 200}"), true), - Arguments.of("/status_code >= 300", event("{\"status_code\": 200}"), false), - Arguments.of("-/status_code == -200", event("{\"status_code\": 200}"), true), - Arguments.of("/success and /status_code == 200", event("{\"success\": true, \"status_code\": 200}"), true), - Arguments.of("/success or /status_code == 200", event("{\"success\": false, \"status_code\": 200}"), true), - Arguments.of("(/success == true) or (/status_code == 200)", event("{\"success\": false, \"status_code\": 200}"), true), - Arguments.of("/should_drop", event("{\"should_drop\": true}"), true), - Arguments.of("/should_drop", event("{\"should_drop\": false}"), false), - Arguments.of("/logs/2/should_drop", event("{\"logs\": [{}, {}, {\"should_drop\": true}]}"), true), - Arguments.of( + arguments("true", event("{}"), true), + arguments("/status_code == 200", event("{\"status_code\": 200}"), true), + arguments("/status_code == 200", longEvent, true), + arguments("/status_code != 300", event("{\"status_code\": 200}"), true), + arguments("/status_code == 200", event("{}"), false), + arguments("/success == /status_code", event("{\"success\": true, \"status_code\": 200}"), false), + arguments("/success != /status_code", event("{\"success\": true, \"status_code\": 200}"), true), + arguments("/part1@part2.part3 != 111", event("{\"success\": true, \"part1@part2.part3\":111, \"status_code\": 200}"), false), + arguments("/part1.part2@part3 != 111", event("{\"success\": true, \"part1.part2@part3\":222, \"status_code\": 200}"), true), + arguments("/pi == 3.14159", event("{\"pi\": 3.14159}"), true), + arguments("/value == 12345.678", event("{\"value\": 12345.678}"), true), + arguments("/value == 12345.678E12", event("{\"value\": 12345.678E12}"), true), + arguments("/value == 12345.678e-12", event("{\"value\": 12345.678e-12}"), true), + arguments("/value == 12345.0000012", event("{\"value\": 12345.0000012}"), true), + arguments("/value == 12345.00012E6", event("{\"value\": 12345.00012E6}"), true), + arguments("true == (/is_cool == true)", event("{\"is_cool\": true}"), true), + arguments("not /is_cool", event("{\"is_cool\": true}"), false), + arguments("/status_code < 300", event("{\"status_code\": 200}"), true), + arguments("/status_code != null", event("{\"status_code\": 200}"), true), + arguments("null != /status_code", event("{\"status_code\": 200}"), true), + arguments("/status_code == null", event("{\"status_code\": null}"), true), + arguments("/response == null", event("{\"status_code\": 200}"), true), + arguments("null == /response", event("{\"status_code\": 200}"), true), + arguments("/response != null", event("{\"status_code\": 200}"), false), + arguments("/status_code <= 0", event("{\"status_code\": 200}"), false), + arguments("/status_code > 0", event("{\"status_code\": 200}"), true), + arguments("/status_code >= 300", event("{\"status_code\": 200}"), false), + arguments("-/status_code == -200", event("{\"status_code\": 200}"), true), + arguments("/success and /status_code == 200", event("{\"success\": true, \"status_code\": 200}"), true), + arguments("/success or /status_code == 200", event("{\"success\": false, \"status_code\": 200}"), true), + arguments("(/success == true) or (/status_code == 200)", event("{\"success\": false, \"status_code\": 200}"), true), + arguments("/should_drop", event("{\"should_drop\": true}"), true), + arguments("/should_drop", event("{\"should_drop\": false}"), false), + arguments("/logs/2/should_drop", event("{\"logs\": [{}, {}, {\"should_drop\": true}]}"), true), + arguments( escapedJsonPointer(ALL_JACKSON_EVENT_GET_SUPPORTED_CHARACTERS) + " == true", complexEvent(ALL_JACKSON_EVENT_GET_SUPPORTED_CHARACTERS, true), true), - Arguments.of("/durationInNanos > 5000000000", event("{\"durationInNanos\": 6000000000}"), true), - Arguments.of("/response == \"OK\"", event("{\"response\": \"OK\"}"), true), - Arguments.of("length(/response) == "+testStringLength, event("{\"response\": \""+testString+"\"}"), true), - Arguments.of("hasTags(\""+ testTag1+"\")", longEvent, true), - Arguments.of("hasTags(\""+ testTag1+"\",\""+testTag2+"\")", longEvent, true), - Arguments.of("hasTags(\""+ testTag1+"\", \""+testTag2+"\", \""+testTag3+"\")", longEvent, true), - Arguments.of("hasTags(\""+ testTag4+"\")", longEvent, false), - Arguments.of("hasTags(\""+ testTag3+"\",\""+testTag4+"\")", longEvent, false), - Arguments.of("contains(\""+ strValue+"\",\""+strValue.substring(1,5)+"\")", longEvent, true), - Arguments.of("contains(/status,\""+strValue.substring(0,2)+"\")", event("{\"status\":\""+strValue+"\"}"), true), - Arguments.of("contains(\""+strValue+strValue+"\",/status)", event("{\"status\":\""+strValue+"\"}"), true), - Arguments.of("contains(/message,/status)", event("{\"status\":\""+strValue+"\", \"message\":\""+strValue+strValue+"\"}"), true), - Arguments.of("contains(/unknown,/status)", event("{\"status\":\""+strValue+"\", \"message\":\""+strValue+strValue+"\"}"), false), - Arguments.of("contains(/status,/unknown)", event("{\"status\":\""+strValue+"\", \"message\":\""+strValue+strValue+"\"}"), false), - Arguments.of("getMetadata(\"key1\") == \""+strValue+"\"", longEvent, true), - Arguments.of("getMetadata(\"key2\") == "+value4, longEvent, true), - Arguments.of("getMetadata(\"key3\") == "+value5, longEvent, true), - Arguments.of("getMetadata(\"/key1\") == \""+strValue+"\"", longEvent, true), - Arguments.of("getMetadata(\"/key2\") == "+value4, longEvent, true), - Arguments.of("getMetadata(\"key3\") == "+value5, longEvent, true), - Arguments.of("getMetadata(\"/key6\") == \""+value5+"\"", longEvent, false), - Arguments.of("getMetadata(\"key6\") == "+value5, longEvent, false), - Arguments.of("cidrContains(/sourceIp,\"192.0.2.0/24\")", event("{\"sourceIp\": \"192.0.2.3\"}"), true), - Arguments.of("cidrContains(/sourceIp,\"192.0.2.0/24\",\"192.1.1.0/24\")", event("{\"sourceIp\": \"192.0.2.3\"}"), true), - Arguments.of("cidrContains(/sourceIp,\"192.0.2.0/24\",\"192.1.1.0/24\")", event("{\"sourceIp\": \"192.2.2.3\"}"), false), - Arguments.of("cidrContains(/sourceIp,\"2001:0db8::/32\")", event("{\"sourceIp\": \"2001:0db8:aaaa:bbbb::\"}"), true), - Arguments.of("cidrContains(/sourceIp,\"2001:0db8::/32\",\"2001:aaaa::/32\")", event("{\"sourceIp\": \"2001:0db8:aaaa:bbbb::\"}"), true), - Arguments.of("cidrContains(/sourceIp,\"2001:0db8::/32\",\"2001:aaaa::/32\")", event("{\"sourceIp\": \"2001:abcd:aaaa:bbbb::\"}"), false), - Arguments.of("/sourceIp != null", event("{\"sourceIp\": [10, 20]}"), true), - Arguments.of("/sourceIp == null", event("{\"sourceIp\": [\"test\", \"test_two\"]}"), false), - Arguments.of("/sourceIp == null", event("{\"sourceIp\": {\"test\": \"test_two\"}}"), false), - Arguments.of("/sourceIp != null", event("{\"sourceIp\": {\"test\": \"test_two\"}}"), true) + arguments("/durationInNanos > 5000000000", event("{\"durationInNanos\": 6000000000}"), true), + arguments("/response == \"OK\"", event("{\"response\": \"OK\"}"), true), + arguments("length(/response) == "+testStringLength, event("{\"response\": \""+testString+"\"}"), true), + arguments("hasTags(\""+ testTag1+"\")", longEvent, true), + arguments("hasTags(\""+ testTag1+"\",\""+testTag2+"\")", longEvent, true), + arguments("hasTags(\""+ testTag1+"\", \""+testTag2+"\", \""+testTag3+"\")", longEvent, true), + arguments("hasTags(\""+ testTag4+"\")", longEvent, false), + arguments("hasTags(\""+ testTag3+"\",\""+testTag4+"\")", longEvent, false), + arguments("contains(\""+ strValue+"\",\""+strValue.substring(1,5)+"\")", longEvent, true), + arguments("contains(/status,\""+strValue.substring(0,2)+"\")", event("{\"status\":\""+strValue+"\"}"), true), + arguments("contains(\""+strValue+strValue+"\",/status)", event("{\"status\":\""+strValue+"\"}"), true), + arguments("contains(/message,/status)", event("{\"status\":\""+strValue+"\", \"message\":\""+strValue+strValue+"\"}"), true), + arguments("contains(/unknown,/status)", event("{\"status\":\""+strValue+"\", \"message\":\""+strValue+strValue+"\"}"), false), + arguments("contains(/status,/unknown)", event("{\"status\":\""+strValue+"\", \"message\":\""+strValue+strValue+"\"}"), false), + arguments("getMetadata(\"key1\") == \""+strValue+"\"", longEvent, true), + arguments("getMetadata(\"key2\") == "+value4, longEvent, true), + arguments("getMetadata(\"key3\") == "+value5, longEvent, true), + arguments("getMetadata(\"/key1\") == \""+strValue+"\"", longEvent, true), + arguments("getMetadata(\"/key2\") == "+value4, longEvent, true), + arguments("getMetadata(\"key3\") == "+value5, longEvent, true), + arguments("getMetadata(\"/key6\") == \""+value5+"\"", longEvent, false), + arguments("getMetadata(\"key6\") == "+value5, longEvent, false), + arguments("cidrContains(/sourceIp,\"192.0.2.0/24\")", event("{\"sourceIp\": \"192.0.2.3\"}"), true), + arguments("cidrContains(/sourceIp,\"192.0.2.0/24\",\"192.1.1.0/24\")", event("{\"sourceIp\": \"192.0.2.3\"}"), true), + arguments("cidrContains(/sourceIp,\"192.0.2.0/24\",\"192.1.1.0/24\")", event("{\"sourceIp\": \"192.2.2.3\"}"), false), + arguments("cidrContains(/sourceIp,\"2001:0db8::/32\")", event("{\"sourceIp\": \"2001:0db8:aaaa:bbbb::\"}"), true), + arguments("cidrContains(/sourceIp,\"2001:0db8::/32\",\"2001:aaaa::/32\")", event("{\"sourceIp\": \"2001:0db8:aaaa:bbbb::\"}"), true), + arguments("cidrContains(/sourceIp,\"2001:0db8::/32\",\"2001:aaaa::/32\")", event("{\"sourceIp\": \"2001:abcd:aaaa:bbbb::\"}"), false), + arguments("/sourceIp != null", event("{\"sourceIp\": [10, 20]}"), true), + arguments("/sourceIp == null", event("{\"sourceIp\": [\"test\", \"test_two\"]}"), false), + arguments("/sourceIp == null", event("{\"sourceIp\": {\"test\": \"test_two\"}}"), false), + arguments("/sourceIp != null", event("{\"sourceIp\": {\"test\": \"test_two\"}}"), true), + arguments("/name =~ \".*dataprepper-[0-9]+\"", event("{\"name\": \"dataprepper-0\"}"), true), + arguments("/name =~ \".*dataprepper-[0-9]+\"", event("{\"name\": \"dataprepper-212\"}"), true), + arguments("/name =~ \".*dataprepper-[0-9]+\"", event("{\"name\": \"dataprepper-abc\"}"), false), + arguments("/name =~ \".*dataprepper-[0-9]+\"", event("{\"other\": \"dataprepper-abc\"}"), false) ); } @@ -236,43 +241,43 @@ private static Stream invalidExpressionArguments() { int testStringLength = random.nextInt(10); String testString = RandomStringUtils.randomAlphabetic(testStringLength); return Stream.of( - Arguments.of("/missing", event("{}")), - Arguments.of("/success < /status_code", event("{\"success\": true, \"status_code\": 200}")), - Arguments.of("/success <= /status_code", event("{\"success\": true, \"status_code\": 200}")), - Arguments.of("/success > /status_code", event("{\"success\": true, \"status_code\": 200}")), - Arguments.of("/success >= /status_code", event("{\"success\": true, \"status_code\": 200}")), - Arguments.of("/success > null", event("{\"success\": true, \"status_code\": 200}")), - Arguments.of("/success >= null", event("{\"success\": true, \"status_code\": 200}")), - Arguments.of("/status_code < null", event("{\"success\": true, \"status_code\": 200}")), - Arguments.of("/status_code <= null", event("{\"success\": true, \"status_code\": 200}")), - Arguments.of("not /status_code", event("{\"status_code\": 200}")), - Arguments.of("/status_code >= 200 and 3", event("{\"status_code\": 200}")), - Arguments.of("", event("{}")), - Arguments.of("-false", event("{}")), - Arguments.of("not 5", event("{}")), - Arguments.of("not null", event("{}")), - Arguments.of("not/status_code", event("{\"status_code\": 200}")), - Arguments.of("trueand/status_code", event("{\"status_code\": 200}")), - Arguments.of("trueor/status_code", event("{\"status_code\": 200}")), - Arguments.of("length(\""+testString+") == "+testStringLength, event("{\"response\": \""+testString+"\"}")), - Arguments.of("length(\""+testString+"\") == "+testStringLength, event("{\"response\": \""+testString+"\"}")), - Arguments.of("hasTags(10)", tagEvent), - Arguments.of("hasTags("+ testTag1+")", tagEvent), - Arguments.of("hasTags(\""+ testTag1+")", tagEvent), - Arguments.of("hasTags(\""+ testTag1+"\","+testTag2+"\")", tagEvent), - Arguments.of("hasTags(,\""+testTag2+"\")", tagEvent), - Arguments.of("hasTags(\""+testTag2+"\",)", tagEvent), - Arguments.of("contains(\""+testTag2+"\",)", tagEvent), - Arguments.of("contains(\""+testTag2+"\")", tagEvent), - Arguments.of("contains(/intField, /strField)", event("{\"intField\":1234,\"strField\":\"string\"}")), - Arguments.of("contains(1234, /strField)", event("{\"intField\":1234,\"strField\":\"string\"}")), - Arguments.of("contains(str, /strField)", event("{\"intField\":1234,\"strField\":\"string\"}")), - Arguments.of("contains(/strField, 1234)", event("{\"intField\":1234,\"strField\":\"string\"}")), - Arguments.of("getMetadata(10)", tagEvent), - Arguments.of("getMetadata("+ testMetadataKey+ ")", tagEvent), - Arguments.of("getMetadata(\""+ testMetadataKey+")", tagEvent), - Arguments.of("cidrContains(/sourceIp)", event("{\"sourceIp\": \"192.0.2.3\"}")), - Arguments.of("cidrContains(/sourceIp,123)", event("{\"sourceIp\": \"192.0.2.3\"}")) + arguments("/missing", event("{}")), + arguments("/success < /status_code", event("{\"success\": true, \"status_code\": 200}")), + arguments("/success <= /status_code", event("{\"success\": true, \"status_code\": 200}")), + arguments("/success > /status_code", event("{\"success\": true, \"status_code\": 200}")), + arguments("/success >= /status_code", event("{\"success\": true, \"status_code\": 200}")), + arguments("/success > null", event("{\"success\": true, \"status_code\": 200}")), + arguments("/success >= null", event("{\"success\": true, \"status_code\": 200}")), + arguments("/status_code < null", event("{\"success\": true, \"status_code\": 200}")), + arguments("/status_code <= null", event("{\"success\": true, \"status_code\": 200}")), + arguments("not /status_code", event("{\"status_code\": 200}")), + arguments("/status_code >= 200 and 3", event("{\"status_code\": 200}")), + arguments("", event("{}")), + arguments("-false", event("{}")), + arguments("not 5", event("{}")), + arguments("not null", event("{}")), + arguments("not/status_code", event("{\"status_code\": 200}")), + arguments("trueand/status_code", event("{\"status_code\": 200}")), + arguments("trueor/status_code", event("{\"status_code\": 200}")), + arguments("length(\""+testString+") == "+testStringLength, event("{\"response\": \""+testString+"\"}")), + arguments("length(\""+testString+"\") == "+testStringLength, event("{\"response\": \""+testString+"\"}")), + arguments("hasTags(10)", tagEvent), + arguments("hasTags("+ testTag1+")", tagEvent), + arguments("hasTags(\""+ testTag1+")", tagEvent), + arguments("hasTags(\""+ testTag1+"\","+testTag2+"\")", tagEvent), + arguments("hasTags(,\""+testTag2+"\")", tagEvent), + arguments("hasTags(\""+testTag2+"\",)", tagEvent), + arguments("contains(\""+testTag2+"\",)", tagEvent), + arguments("contains(\""+testTag2+"\")", tagEvent), + arguments("contains(/intField, /strField)", event("{\"intField\":1234,\"strField\":\"string\"}")), + arguments("contains(1234, /strField)", event("{\"intField\":1234,\"strField\":\"string\"}")), + arguments("contains(str, /strField)", event("{\"intField\":1234,\"strField\":\"string\"}")), + arguments("contains(/strField, 1234)", event("{\"intField\":1234,\"strField\":\"string\"}")), + arguments("getMetadata(10)", tagEvent), + arguments("getMetadata("+ testMetadataKey+ ")", tagEvent), + arguments("getMetadata(\""+ testMetadataKey+")", tagEvent), + arguments("cidrContains(/sourceIp)", event("{\"sourceIp\": \"192.0.2.3\"}")), + arguments("cidrContains(/sourceIp,123)", event("{\"sourceIp\": \"192.0.2.3\"}")) ); } diff --git a/data-prepper-expression/src/test/java/org/opensearch/dataprepper/expression/RegexEqualOperatorTest.java b/data-prepper-expression/src/test/java/org/opensearch/dataprepper/expression/RegexEqualOperatorTest.java index 46fcdd9ccf..bb92cd1e49 100644 --- a/data-prepper-expression/src/test/java/org/opensearch/dataprepper/expression/RegexEqualOperatorTest.java +++ b/data-prepper-expression/src/test/java/org/opensearch/dataprepper/expression/RegexEqualOperatorTest.java @@ -12,6 +12,7 @@ import org.mockito.junit.jupiter.MockitoExtension; import org.opensearch.dataprepper.expression.antlr.DataPrepperExpressionParser; +import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.is; import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -64,4 +65,9 @@ void testEvalInValidArgType() { void testEvalInValidPattern() { assertThrows(IllegalArgumentException.class, () -> objectUnderTest.evaluate("a", "*")); } + + @Test + void evaluate_with_null_lhs_returns_false() { + assertThat(objectUnderTest.evaluate(null, "a*"), equalTo(false)); + } } diff --git a/data-prepper-expression/src/test/java/org/opensearch/dataprepper/expression/RegexNotEqualOperatorTest.java b/data-prepper-expression/src/test/java/org/opensearch/dataprepper/expression/RegexNotEqualOperatorTest.java index 221c354eb7..30bc199413 100644 --- a/data-prepper-expression/src/test/java/org/opensearch/dataprepper/expression/RegexNotEqualOperatorTest.java +++ b/data-prepper-expression/src/test/java/org/opensearch/dataprepper/expression/RegexNotEqualOperatorTest.java @@ -12,6 +12,7 @@ import org.mockito.junit.jupiter.MockitoExtension; import org.opensearch.dataprepper.expression.antlr.DataPrepperExpressionParser; +import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.is; import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -64,4 +65,9 @@ void testEvalInValidArgType() { void testEvalInValidPattern() { assertThrows(IllegalArgumentException.class, () -> objectUnderTest.evaluate("a", "*")); } + + @Test + void evaluate_with_null_lhs_returns_false() { + assertThat(objectUnderTest.evaluate(null, "a*"), equalTo(false)); + } } From 0e00df68a60f9dd57ad34d1182065c0ba4917067 Mon Sep 17 00:00:00 2001 From: Jayesh Parmar <89792517+jayeshjeh@users.noreply.github.com> Date: Mon, 12 Aug 2024 21:26:55 +0530 Subject: [PATCH 17/25] Create docker-compose-dataprepper.yaml (#4756) * Create docker-compose-dataprepper.yaml Signed-off-by: Jayesh Parmar <89792517+jayeshjeh@users.noreply.github.com> Signed-off-by: jayeshjeh * Necessary chnages made Signed-off-by: jayeshjeh --------- Signed-off-by: Jayesh Parmar <89792517+jayeshjeh@users.noreply.github.com> Signed-off-by: jayeshjeh --- examples/log-ingestion/data-prepper-config.yaml | 6 ++++++ .../docker-compose-dataprepper.yaml | 16 ++++++++++++++++ examples/log-ingestion/docker-compose.yaml | 1 + examples/log-ingestion/fluent-bit.conf | 4 +++- examples/log-ingestion/log_pipeline.yaml | 2 +- 5 files changed, 27 insertions(+), 2 deletions(-) create mode 100644 examples/log-ingestion/data-prepper-config.yaml create mode 100644 examples/log-ingestion/docker-compose-dataprepper.yaml diff --git a/examples/log-ingestion/data-prepper-config.yaml b/examples/log-ingestion/data-prepper-config.yaml new file mode 100644 index 0000000000..c42ba72275 --- /dev/null +++ b/examples/log-ingestion/data-prepper-config.yaml @@ -0,0 +1,6 @@ +ssl: false +serverPort: 4900 +authentication: + http_basic: + username: admin + password: admin diff --git a/examples/log-ingestion/docker-compose-dataprepper.yaml b/examples/log-ingestion/docker-compose-dataprepper.yaml new file mode 100644 index 0000000000..29e2fdcc64 --- /dev/null +++ b/examples/log-ingestion/docker-compose-dataprepper.yaml @@ -0,0 +1,16 @@ +version: '3.7' +services: + data-prepper: + image: opensearchproject/data-prepper:2 + container_name: data-prepper + volumes: + - ./log_pipeline.yaml:/usr/share/data-prepper/pipelines/log_pipeline.yaml + - ./data-prepper-config.yaml:/usr/share/data-prepper/config/data-prepper-config.yaml + ports: + - 2021:2021 + networks: + - opensearch-net + +networks: + opensearch-net: + driver: bridge diff --git a/examples/log-ingestion/docker-compose.yaml b/examples/log-ingestion/docker-compose.yaml index 099ae707cf..c7864d50dd 100644 --- a/examples/log-ingestion/docker-compose.yaml +++ b/examples/log-ingestion/docker-compose.yaml @@ -15,6 +15,7 @@ services: - discovery.type=single-node - bootstrap.memory_lock=true # along with the memlock settings below, disables swapping - "OPENSEARCH_JAVA_OPTS=-Xms512m -Xmx512m" # minimum and maximum Java heap size, recommend setting both to 50% of system RAM + - "OPENSEARCH_INITIAL_ADMIN_PASSWORD=Developer@123" ulimits: memlock: soft: -1 diff --git a/examples/log-ingestion/fluent-bit.conf b/examples/log-ingestion/fluent-bit.conf index b7a1a6c5dc..5ea9fdbf96 100644 --- a/examples/log-ingestion/fluent-bit.conf +++ b/examples/log-ingestion/fluent-bit.conf @@ -10,4 +10,6 @@ Host data-prepper Port 2021 URI /log/ingest - Format json \ No newline at end of file + Format json + HTTP_User admin + HTTP_Passwd admin diff --git a/examples/log-ingestion/log_pipeline.yaml b/examples/log-ingestion/log_pipeline.yaml index ee40619d6b..6f357239d2 100644 --- a/examples/log-ingestion/log_pipeline.yaml +++ b/examples/log-ingestion/log_pipeline.yaml @@ -11,5 +11,5 @@ log-pipeline: hosts: [ "https://opensearch:9200" ] insecure: true username: admin - password: admin + password: Developer@123 index: apache_logs From fedd07073f8a5b89b69456a116eed3cf935ba398 Mon Sep 17 00:00:00 2001 From: David Venable Date: Mon, 12 Aug 2024 11:25:22 -0500 Subject: [PATCH 18/25] Release notes for Data Prepper 2.8.1 (#4807) Signed-off-by: David Venable --- .../data-prepper.release-notes-2.8.1.md | 12 ++++++++++++ 1 file changed, 12 insertions(+) create mode 100644 release/release-notes/data-prepper.release-notes-2.8.1.md diff --git a/release/release-notes/data-prepper.release-notes-2.8.1.md b/release/release-notes/data-prepper.release-notes-2.8.1.md new file mode 100644 index 0000000000..72f9989f2f --- /dev/null +++ b/release/release-notes/data-prepper.release-notes-2.8.1.md @@ -0,0 +1,12 @@ +## 2024-08-01 Version 2.8.1 + +--- + +### Bug Fixes +* Jackson 2.17.0 LockFreePool causes memory issues ([#4729](https://github.com/opensearch-project/data-prepper/issues/4729)) + + +### Maintenance +* Updates Jackson to 2.17.2 ([#4753](https://github.com/opensearch-project/data-prepper/pull/4753)) +* Updates to Armeria 1.29.0 ([#4741](https://github.com/opensearch-project/data-prepper/pull/4741)) +* Parquet codec tests fix ([#4742](https://github.com/opensearch-project/data-prepper/pull/4742)) From be85abe67e195c9757c560a9377c7a17f5e2df82 Mon Sep 17 00:00:00 2001 From: Qi Chen Date: Mon, 12 Aug 2024 11:51:51 -0500 Subject: [PATCH 19/25] ENH: add folder path as output for schema generation (#4820) * ENH: add folder path as output Signed-off-by: George Chen --- data-prepper-plugin-schema-cli/README.md | 3 +- .../DataPrepperPluginSchemaExecute.java | 69 ++++++++++++++++--- 2 files changed, 61 insertions(+), 11 deletions(-) diff --git a/data-prepper-plugin-schema-cli/README.md b/data-prepper-plugin-schema-cli/README.md index 7a4d9bc11b..30b0612603 100644 --- a/data-prepper-plugin-schema-cli/README.md +++ b/data-prepper-plugin-schema-cli/README.md @@ -5,8 +5,9 @@ This module includes the SDK and CLI for generating schemas for Data Prepper pip ## CLI Usage ``` -./gradlew :data-prepper-plugin-schema-cli:run --args='--plugin_type=processor --plugin_names=grok' +./gradlew :data-prepper-plugin-schema-cli:run --args='--plugin_type=processor --plugin_names=grok --output_folder=/path/to/schemas' ``` * plugin_type: A required parameter specifies type of processor. Valid options are `source`, `buffer`, `processor`, `route`, `sink`. * plugin_names: An optional parameter filters the result by plugin names separated by `,`, e.g. `grok,date`. +* output_folder: An optional parameter to specify the output folder path. diff --git a/data-prepper-plugin-schema-cli/src/main/java/org/opensearch/dataprepper/schemas/DataPrepperPluginSchemaExecute.java b/data-prepper-plugin-schema-cli/src/main/java/org/opensearch/dataprepper/schemas/DataPrepperPluginSchemaExecute.java index a1a76c0510..a505a013c4 100644 --- a/data-prepper-plugin-schema-cli/src/main/java/org/opensearch/dataprepper/schemas/DataPrepperPluginSchemaExecute.java +++ b/data-prepper-plugin-schema-cli/src/main/java/org/opensearch/dataprepper/schemas/DataPrepperPluginSchemaExecute.java @@ -14,11 +14,16 @@ import org.slf4j.LoggerFactory; import picocli.CommandLine; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.Collection; import java.util.List; import java.util.Map; import java.util.Set; +import java.util.function.Function; import java.util.stream.Collectors; -import java.util.stream.Stream; import static com.github.victools.jsonschema.module.jackson.JacksonOption.RESPECT_JSONPROPERTY_REQUIRED; @@ -37,6 +42,9 @@ public class DataPrepperPluginSchemaExecute implements Runnable { @CommandLine.Option(names = {"--site.baseurl"}, defaultValue = "/docs/latest") private String siteBaseUrl; + @CommandLine.Option(names = {"--output_folder"}) + private String folderPath; + public static void main(String[] args) { final int exitCode = new CommandLine(new DataPrepperPluginSchemaExecute()).execute(args); System.exit(exitCode); @@ -57,18 +65,59 @@ public void run() { final Class pluginType = pluginConfigsJsonSchemaConverter.pluginTypeNameToPluginType(pluginTypeName); final Map pluginNameToJsonSchemaMap = pluginConfigsJsonSchemaConverter.convertPluginConfigsIntoJsonSchemas( SchemaVersion.DRAFT_2020_12, OptionPreset.PLAIN_JSON, pluginType); + Map filteredPluginNameToJsonSchemaMap; if (pluginNames == null) { - pluginNameToJsonSchemaMap.values().forEach(System.out::println); + filteredPluginNameToJsonSchemaMap = pluginNameToJsonSchemaMap; } else { final Set pluginNamesSet = Set.of(pluginNames.split(",")); - final List result = pluginNamesSet.stream().flatMap(name -> { - if (!pluginNameToJsonSchemaMap.containsKey(name)) { - LOG.error("plugin name: {} not found", name); - return Stream.empty(); - } - return Stream.of(pluginNameToJsonSchemaMap.get(name)); - }).collect(Collectors.toList()); - result.forEach(System.out::println); + filteredPluginNameToJsonSchemaMap = pluginNamesSet.stream() + .filter(name -> { + if (!pluginNameToJsonSchemaMap.containsKey(name)) { + LOG.error("plugin name: {} not found", name); + return false; + } + return true; + }) + .collect(Collectors.toMap( + Function.identity(), + pluginNameToJsonSchemaMap::get + )); + } + + if (folderPath == null) { + writeCollectionToConsole(filteredPluginNameToJsonSchemaMap.values()); + } else { + writeMapToFiles(filteredPluginNameToJsonSchemaMap, folderPath); + } + } + + private static void writeCollectionToConsole(final Collection values) { + values.forEach(System.out::println); + } + + private static void writeMapToFiles(final Map map, final String folderPath) { + // Ensure the directory exists + final Path directory = Paths.get(folderPath); + if (!Files.exists(directory)) { + try { + Files.createDirectories(directory); + } catch (IOException e) { + System.err.println("Error creating directory: " + e.getMessage()); + return; + } + } + + // Iterate through the map and write each entry to a file + for (final Map.Entry entry : map.entrySet()) { + final String fileName = entry.getKey() + ".json"; + final Path filePath = directory.resolve(fileName); + + try { + Files.write(filePath, entry.getValue().getBytes()); + System.out.println("Written file: " + filePath); + } catch (IOException e) { + System.err.println("Error writing file " + fileName + ": " + e.getMessage()); + } } } } From aa50a1d1fbf2499c5edfe4da8aaa806cf474da86 Mon Sep 17 00:00:00 2001 From: Qi Chen Date: Mon, 12 Aug 2024 13:24:50 -0500 Subject: [PATCH 20/25] FIX: build service map relationship even when trace group is missing (#4822) Signed-off-by: George Chen --- .../processor/ServiceMapStatefulProcessor.java | 2 +- .../ServiceMapStatefulProcessorTest.java | 15 ++++++++++----- 2 files changed, 11 insertions(+), 6 deletions(-) diff --git a/data-prepper-plugins/service-map-stateful/src/main/java/org/opensearch/dataprepper/plugins/processor/ServiceMapStatefulProcessor.java b/data-prepper-plugins/service-map-stateful/src/main/java/org/opensearch/dataprepper/plugins/processor/ServiceMapStatefulProcessor.java index 75041a09b4..667b8ea882 100644 --- a/data-prepper-plugins/service-map-stateful/src/main/java/org/opensearch/dataprepper/plugins/processor/ServiceMapStatefulProcessor.java +++ b/data-prepper-plugins/service-map-stateful/src/main/java/org/opensearch/dataprepper/plugins/processor/ServiceMapStatefulProcessor.java @@ -242,7 +242,7 @@ private Collection> iterateProcessorState(final MapDbProcessorStat } final String traceGroupName = getTraceGroupName(child.traceId); - if (traceGroupName == null || parent == null || parent.serviceName.equals(child.serviceName)) { + if (parent == null || parent.serviceName.equals(child.serviceName)) { return; } diff --git a/data-prepper-plugins/service-map-stateful/src/test/java/org/opensearch/dataprepper/plugins/processor/ServiceMapStatefulProcessorTest.java b/data-prepper-plugins/service-map-stateful/src/test/java/org/opensearch/dataprepper/plugins/processor/ServiceMapStatefulProcessorTest.java index b565642e19..cae81c8134 100644 --- a/data-prepper-plugins/service-map-stateful/src/test/java/org/opensearch/dataprepper/plugins/processor/ServiceMapStatefulProcessorTest.java +++ b/data-prepper-plugins/service-map-stateful/src/test/java/org/opensearch/dataprepper/plugins/processor/ServiceMapStatefulProcessorTest.java @@ -40,6 +40,7 @@ import static io.opentelemetry.proto.trace.v1.Span.SpanKind.SPAN_KIND_CLIENT; import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.nullValue; import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -301,7 +302,6 @@ public void testTraceGroupsWithEventRecordData() throws Exception { assertThat(relationshipCountMeasurement.getValue(), equalTo((double)relationshipsFound.size())); - //Make sure that future relationships that are equivalent are caught by cache final byte[] rootSpanId3Bytes = ServiceMapTestUtils.getRandomBytes(8); final byte[] traceId3Bytes = ServiceMapTestUtils.getRandomBytes(16); final String rootSpanId3 = Hex.encodeHexString(rootSpanId3Bytes); @@ -312,19 +312,24 @@ public void testTraceGroupsWithEventRecordData() throws Exception { AUTHENTICATION_SERVICE, "reset", Hex.encodeHexString(ServiceMapTestUtils.getRandomBytes(8)), frontendSpans3.getSpanId(), traceId3, io.opentelemetry.proto.trace.v1.Span.SpanKind.SPAN_KIND_SERVER); + // relationship missing traceGroupName when(clock.millis()).thenReturn(450L); Future> r7 = ServiceMapTestUtils.startExecuteAsync(threadpool, serviceMapStateful1, Collections.singletonList(new Record<>(frontendSpans3))); Future> r8 = ServiceMapTestUtils.startExecuteAsync(threadpool, serviceMapStateful2, Collections.singletonList(new Record<>(authenticationSpansServer2))); - assertTrue(r7.get().isEmpty()); - assertTrue(r8.get().isEmpty()); + final Set relationshipsFoundWithNoTraceGroupName = new HashSet<>(); + relationshipsFoundWithNoTraceGroupName.addAll(r7.get()); + relationshipsFoundWithNoTraceGroupName.addAll(r8.get()); when(clock.millis()).thenReturn(560L); Future> r9 = ServiceMapTestUtils.startExecuteAsync(threadpool, serviceMapStateful1, Arrays.asList()); Future> r10 = ServiceMapTestUtils.startExecuteAsync(threadpool, serviceMapStateful2, Arrays.asList()); - assertTrue(r9.get().isEmpty()); - assertTrue(r10.get().isEmpty()); + relationshipsFoundWithNoTraceGroupName.addAll(r9.get()); + relationshipsFoundWithNoTraceGroupName.addAll(r10.get()); + assertThat(relationshipsFoundWithNoTraceGroupName.size(), equalTo(4)); + relationshipsFoundWithNoTraceGroupName.forEach( + relationship -> assertThat(relationship.getTraceGroupName(), nullValue())); serviceMapStateful1.shutdown(); serviceMapStateful2.shutdown(); } From 2f21a437091b2a41cfe6a53df9978ec0b456fd2c Mon Sep 17 00:00:00 2001 From: Taylor Gray Date: Mon, 12 Aug 2024 16:21:05 -0500 Subject: [PATCH 21/25] Add delete_source parameter to the csv processor (#4828) Signed-off-by: Taylor Gray --- .../plugins/processor/csv/CsvProcessor.java | 4 ++++ .../processor/csv/CsvProcessorConfig.java | 6 ++++++ .../processor/csv/CsvProcessorTest.java | 19 +++++++++++++++++++ 3 files changed, 29 insertions(+) diff --git a/data-prepper-plugins/csv-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/csv/CsvProcessor.java b/data-prepper-plugins/csv-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/csv/CsvProcessor.java index a9f99e5862..3e8780a417 100644 --- a/data-prepper-plugins/csv-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/csv/CsvProcessor.java +++ b/data-prepper-plugins/csv-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/csv/CsvProcessor.java @@ -99,6 +99,10 @@ public Collection> doExecute(final Collection> recor if (thisEventHasHeaderSource && Boolean.TRUE.equals(config.isDeleteHeader())) { event.delete(config.getColumnNamesSourceKey()); } + + if (config.isDeleteSource()) { + event.delete(config.getSource()); + } } catch (final IOException e) { csvInvalidEventsCounter.increment(); LOG.error(EVENT, "An exception occurred while reading event [{}]", event, e); diff --git a/data-prepper-plugins/csv-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/csv/CsvProcessorConfig.java b/data-prepper-plugins/csv-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/csv/CsvProcessorConfig.java index 8c770b597a..fb803798b2 100644 --- a/data-prepper-plugins/csv-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/csv/CsvProcessorConfig.java +++ b/data-prepper-plugins/csv-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/csv/CsvProcessorConfig.java @@ -62,6 +62,10 @@ public class CsvProcessorConfig { "the processor should be applied to the event.") private String csvWhen; + @JsonPropertyDescription("If true, the configured source field will be deleted after the CSV data is parsed into separate fields.") + @JsonProperty + private boolean deleteSource = false; + /** * The field of the Event that contains the CSV data to be processed. * @@ -120,6 +124,8 @@ public List getColumnNames() { public String getCsvWhen() { return csvWhen; } + public Boolean isDeleteSource() { return deleteSource; } + @AssertTrue(message = "delimiter must be exactly one character.") boolean isValidDelimiter() { return delimiter.length() == 1; diff --git a/data-prepper-plugins/csv-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/csv/CsvProcessorTest.java b/data-prepper-plugins/csv-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/csv/CsvProcessorTest.java index 5239679fab..cacfedb609 100644 --- a/data-prepper-plugins/csv-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/csv/CsvProcessorTest.java +++ b/data-prepper-plugins/csv-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/csv/CsvProcessorTest.java @@ -56,6 +56,7 @@ void setup() { lenient().when(processorConfig.getQuoteCharacter()).thenReturn(defaultConfig.getQuoteCharacter()); lenient().when(processorConfig.getColumnNamesSourceKey()).thenReturn(defaultConfig.getColumnNamesSourceKey()); lenient().when(processorConfig.getColumnNames()).thenReturn(defaultConfig.getColumnNames()); + lenient().when(processorConfig.isDeleteSource()).thenReturn(false); lenient().when(pluginMetrics.counter(CsvProcessor.CSV_INVALID_EVENTS)).thenReturn(csvInvalidEventsCounter); @@ -66,6 +67,24 @@ private CsvProcessor createObjectUnderTest() { return new CsvProcessor(pluginMetrics, processorConfig, expressionEvaluator); } + @Test + void delete_source_true_deletes_the_source() { + when(processorConfig.isDeleteSource()).thenReturn(true); + + when(processorConfig.getSource()).thenReturn("different_source"); + + final Map eventData = new HashMap<>(); + eventData.put("different_source","1,2,3"); + final Record eventUnderTest = buildRecordWithEvent(eventData); + + final List> editedEvents = (List>) csvProcessor.doExecute(Collections.singletonList(eventUnderTest)); + final Event parsedEvent = getSingleEvent(editedEvents); + assertThat(parsedEvent.containsKey("different_source"), equalTo(false)); + assertThatKeyEquals(parsedEvent, "column1", "1"); + assertThatKeyEquals(parsedEvent, "column2", "2"); + assertThatKeyEquals(parsedEvent, "column3", "3"); + } + @Test void do_nothing_when_source_is_null_value_or_does_not_exist_in_the_Event() { From 1487973d61f1f92625e75774f94dc6aa41278136 Mon Sep 17 00:00:00 2001 From: Krishna Kondaka <41027584+kkondaka@users.noreply.github.com> Date: Tue, 13 Aug 2024 13:41:22 -0700 Subject: [PATCH 22/25] Config description changes for aggregate and anomaly detector processors. (#4829) * dplive1.yaml Signed-off-by: Krishna Kondaka * Delete .github/workflows/static.yml Signed-off-by: Krishna Kondaka * Add json property description for aggregate processor and anomaly detector processors Signed-off-by: Krishna Kondaka * Addressed review comments Signed-off-by: Krishna Kondaka * Fixed build failure Signed-off-by: Krishna Kondaka --------- Signed-off-by: Krishna Kondaka Co-authored-by: Krishna Kondaka --- .../processor/aggregate/AggregateProcessorConfig.java | 8 ++++++++ .../aggregate/actions/AppendAggregateActionConfig.java | 2 ++ .../aggregate/actions/CountAggregateActionConfig.java | 7 +++++++ .../aggregate/actions/HistogramAggregateActionConfig.java | 8 ++++++++ .../actions/PercentSamplerAggregateActionConfig.java | 2 ++ .../actions/RateLimiterAggregateActionConfig.java | 3 +++ .../actions/TailSamplerAggregateActionConfig.java | 4 ++++ .../anomalydetector/AnomalyDetectorProcessorConfig.java | 6 ++++++ .../anomalydetector/modes/RandomCutForestModeConfig.java | 7 +++++++ 9 files changed, 47 insertions(+) diff --git a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateProcessorConfig.java b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateProcessorConfig.java index 7f81a82194..1c2c9fa701 100644 --- a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateProcessorConfig.java +++ b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateProcessorConfig.java @@ -6,6 +6,7 @@ package org.opensearch.dataprepper.plugins.processor.aggregate; import org.opensearch.dataprepper.model.configuration.PluginModel; +import com.fasterxml.jackson.annotation.JsonPropertyDescription; import com.fasterxml.jackson.annotation.JsonProperty; import jakarta.validation.constraints.AssertTrue; import jakarta.validation.constraints.NotEmpty; @@ -18,27 +19,34 @@ public class AggregateProcessorConfig { static int DEFAULT_GROUP_DURATION_SECONDS = 180; + @JsonPropertyDescription("An unordered list by which to group events. Events with the same values as these keys are put into the same group. If an event does not contain one of the identification_keys, then the value of that key is considered to be equal to null. At least one identification_key is required (for example, [\"sourceIp\", \"destinationIp\", \"port\"].") @JsonProperty("identification_keys") @NotEmpty private List identificationKeys; + @JsonPropertyDescription("The amount of time that a group should exist before it is concluded automatically. Supports ISO_8601 notation strings (\"PT20.345S\", \"PT15M\", etc.) as well as simple notation for seconds (\"60s\") and milliseconds (\"1500ms\"). Default value is 180s.") @JsonProperty("group_duration") private Duration groupDuration = Duration.ofSeconds(DEFAULT_GROUP_DURATION_SECONDS); + @JsonPropertyDescription("The action to be performed on each group. One of the available aggregate actions must be provided, or you can create custom aggregate actions. remove_duplicates and put_all are the available actions. For more information, see Creating New Aggregate Actions.") @JsonProperty("action") @NotNull private PluginModel aggregateAction; + @JsonPropertyDescription("When local_mode is set to true, the aggregation is performed locally on each Data Prepper node instead of forwarding events to a specific node based on the identification_keys using a hash function. Default is false.") @JsonProperty("local_mode") @NotNull private Boolean localMode = false; + @JsonPropertyDescription("A boolean indicating if the unaggregated events should be forwarded to the next processor/sink in the chain.") @JsonProperty("output_unaggregated_events") private Boolean outputUnaggregatedEvents = false; + @JsonPropertyDescription("Tag to be used for aggregated events to distinguish aggregated events from unaggregated events.") @JsonProperty("aggregated_events_tag") private String aggregatedEventsTag; + @JsonPropertyDescription("A Data Prepper conditional expression (https://opensearch.org/docs/latest/data-prepper/pipelines/expression-syntax/), such as '/some-key == \"test\"', that will be evaluated to determine whether the processor will be run on the event.") @JsonProperty("aggregate_when") private String whenCondition; diff --git a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/AppendAggregateActionConfig.java b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/AppendAggregateActionConfig.java index 4d021b7c5a..3d4a9b4a86 100644 --- a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/AppendAggregateActionConfig.java +++ b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/AppendAggregateActionConfig.java @@ -8,9 +8,11 @@ import java.util.List; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonPropertyDescription; public class AppendAggregateActionConfig { + @JsonPropertyDescription("List of keys to append.") @JsonProperty("keys_to_append") List keysToAppend; diff --git a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/CountAggregateActionConfig.java b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/CountAggregateActionConfig.java index 1144aee261..a0325ee3a9 100644 --- a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/CountAggregateActionConfig.java +++ b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/CountAggregateActionConfig.java @@ -9,6 +9,7 @@ import java.util.List; import java.util.Set; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonPropertyDescription; public class CountAggregateActionConfig { static final String SUM_METRIC_NAME = "count"; @@ -17,21 +18,27 @@ public class CountAggregateActionConfig { public static final String DEFAULT_END_TIME_KEY = "aggr._end_time"; public static final Set validOutputFormats = new HashSet<>(Set.of(OutputFormat.OTEL_METRICS.toString(), OutputFormat.RAW.toString())); + @JsonPropertyDescription("Key used for storing the count. Default name is aggr._count.") @JsonProperty("count_key") String countKey = DEFAULT_COUNT_KEY; + @JsonPropertyDescription("Metric name to be used when otel format is used.") @JsonProperty("metric_name") String metricName = SUM_METRIC_NAME; + @JsonPropertyDescription("List of unique keys to count.") @JsonProperty("unique_keys") List uniqueKeys = null; + @JsonPropertyDescription("Key used for storing the start time. Default name is aggr._start_time.") @JsonProperty("start_time_key") String startTimeKey = DEFAULT_START_TIME_KEY; + @JsonPropertyDescription("Key used for storing the end time. Default name is aggr._end_time.") @JsonProperty("end_time_key") String endTimeKey = DEFAULT_END_TIME_KEY; + @JsonPropertyDescription("Format of the aggregated event. otel_metrics is the default output format which outputs in OTel metrics SUM type with count as value. Other options is - raw - which generates a JSON object with the count_key field as a count value and the start_time_key field with aggregation start time as value.") @JsonProperty("output_format") String outputFormat = OutputFormat.OTEL_METRICS.toString(); diff --git a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/HistogramAggregateActionConfig.java b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/HistogramAggregateActionConfig.java index 7c998c123d..6d89a1bd8f 100644 --- a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/HistogramAggregateActionConfig.java +++ b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/HistogramAggregateActionConfig.java @@ -9,6 +9,7 @@ import java.util.List; import java.util.HashSet; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonPropertyDescription; import jakarta.validation.constraints.NotNull; public class HistogramAggregateActionConfig { @@ -25,27 +26,34 @@ public class HistogramAggregateActionConfig { public static final String DURATION_KEY = "duration"; public static final Set validOutputFormats = new HashSet<>(Set.of(OutputFormat.OTEL_METRICS.toString(), OutputFormat.RAW.toString())); + @JsonPropertyDescription("Name of the field in the events the histogram generates.") @JsonProperty("key") @NotNull String key; + @JsonPropertyDescription("The name of units for the values in the key. For example, bytes, traces etc") @JsonProperty("units") @NotNull String units; + @JsonPropertyDescription("Metric name to be used when otel format is used.") @JsonProperty("metric_name") String metricName = HISTOGRAM_METRIC_NAME; + @JsonPropertyDescription("Key prefix used by all the fields created in the aggregated event. Having a prefix ensures that the names of the histogram event do not conflict with the field names in the event.") @JsonProperty("generated_key_prefix") String generatedKeyPrefix = DEFAULT_GENERATED_KEY_PREFIX; + @JsonPropertyDescription("A list of buckets (values of type double) indicating the buckets in the histogram.") @JsonProperty("buckets") @NotNull List buckets; + @JsonPropertyDescription("Format of the aggregated event. otel_metrics is the default output format which outputs in OTel metrics SUM type with count as value. Other options is - raw - which generates a JSON object with the count_key field as a count value and the start_time_key field with aggregation start time as value.") @JsonProperty("output_format") String outputFormat = OutputFormat.OTEL_METRICS.toString(); + @JsonPropertyDescription("A Boolean value indicating whether the histogram should include the min and max of the values in the aggregation.") @JsonProperty("record_minmax") boolean recordMinMax = false; diff --git a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/PercentSamplerAggregateActionConfig.java b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/PercentSamplerAggregateActionConfig.java index d0bc2db7d9..be9770400a 100644 --- a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/PercentSamplerAggregateActionConfig.java +++ b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/PercentSamplerAggregateActionConfig.java @@ -7,9 +7,11 @@ import com.fasterxml.jackson.annotation.JsonProperty; import jakarta.validation.constraints.NotNull; +import com.fasterxml.jackson.annotation.JsonPropertyDescription; import jakarta.validation.constraints.AssertTrue; public class PercentSamplerAggregateActionConfig { + @JsonPropertyDescription("Percent value of the sampling to be done. 0.0 < percent < 100.0") @JsonProperty("percent") @NotNull private double percent; diff --git a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/RateLimiterAggregateActionConfig.java b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/RateLimiterAggregateActionConfig.java index 78de6e74b0..158f3b1ac6 100644 --- a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/RateLimiterAggregateActionConfig.java +++ b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/RateLimiterAggregateActionConfig.java @@ -8,15 +8,18 @@ import java.util.Set; import java.util.HashSet; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonPropertyDescription; import jakarta.validation.constraints.NotNull; public class RateLimiterAggregateActionConfig { public static final Set validRateLimiterModes = new HashSet<>(Set.of(RateLimiterMode.BLOCK.toString(), RateLimiterMode.DROP.toString())); + @JsonPropertyDescription("The number of events allowed per second.") @JsonProperty("events_per_second") @NotNull int eventsPerSecond; + @JsonPropertyDescription("Indicates what action the rate_limiter takes when the number of events received is greater than the number of events allowed per second. Default value is block, which blocks the processor from running after the maximum number of events allowed per second is reached until the next second. Alternatively, the drop option drops the excess events received in that second. Default is block") @JsonProperty("when_exceeds") String whenExceedsMode = RateLimiterMode.BLOCK.toString(); diff --git a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/TailSamplerAggregateActionConfig.java b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/TailSamplerAggregateActionConfig.java index 37475a88e7..f86672e3b9 100644 --- a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/TailSamplerAggregateActionConfig.java +++ b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/TailSamplerAggregateActionConfig.java @@ -6,20 +6,24 @@ package org.opensearch.dataprepper.plugins.processor.aggregate.actions; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonPropertyDescription; import jakarta.validation.constraints.NotNull; import jakarta.validation.constraints.AssertTrue; import java.time.Duration; public class TailSamplerAggregateActionConfig { + @JsonPropertyDescription("Period to wait before considering that a trace event is complete") @JsonProperty("wait_period") @NotNull private Duration waitPeriod; + @JsonPropertyDescription("Percent value to use for sampling non error events. 0.0 < percent < 100.0") @JsonProperty("percent") @NotNull private Integer percent; + @JsonPropertyDescription("A Data Prepper conditional expression (https://opensearch.org/docs/latest/data-prepper/pipelines/expression-syntax/), such as '/some-key == \"test\"', that will be evaluated to determine whether the event is an error event or not") @JsonProperty("condition") private String condition; diff --git a/data-prepper-plugins/anomaly-detector-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/anomalydetector/AnomalyDetectorProcessorConfig.java b/data-prepper-plugins/anomaly-detector-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/anomalydetector/AnomalyDetectorProcessorConfig.java index 6331ee1f21..6545bbb552 100644 --- a/data-prepper-plugins/anomaly-detector-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/anomalydetector/AnomalyDetectorProcessorConfig.java +++ b/data-prepper-plugins/anomaly-detector-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/anomalydetector/AnomalyDetectorProcessorConfig.java @@ -6,6 +6,7 @@ package org.opensearch.dataprepper.plugins.processor.anomalydetector; import org.opensearch.dataprepper.model.configuration.PluginModel; +import com.fasterxml.jackson.annotation.JsonPropertyDescription; import com.fasterxml.jackson.annotation.JsonProperty; import jakarta.validation.constraints.NotEmpty; import jakarta.validation.constraints.NotNull; @@ -14,20 +15,25 @@ import java.util.List; public class AnomalyDetectorProcessorConfig { + @JsonPropertyDescription("The ML algorithm (or model) used to detect anomalies. You must provide a mode. See random_cut_forest mode.") @JsonProperty("mode") @NotNull private PluginModel detectorMode; + @JsonPropertyDescription("A non-ordered List that is used as input to the ML algorithm to detect anomalies in the values of the keys in the list. At least one key is required.") @JsonProperty("keys") @NotEmpty private List keys; + @JsonPropertyDescription("If provided, anomalies will be detected within each unique instance of these keys. For example, if you provide the ip field, anomalies will be detected separately for each unique IP address.") @JsonProperty("identification_keys") private List identificationKeys = Collections.emptyList(); + @JsonPropertyDescription("RCF will try to automatically learn and reduce the number of anomalies detected. For example, if latency is consistently between 50 and 100, and then suddenly jumps to around 1000, only the first one or two data points after the transition will be detected (unless there are other spikes/anomalies). Similarly, for repeated spikes to the same level, RCF will likely eliminate many of the spikes after a few initial ones. This is because the default setting is to minimize the number of alerts detected. Setting the verbose setting to true will cause RCF to consistently detect these repeated cases, which may be useful for detecting anomalous behavior that lasts an extended period of time. Default is false.") @JsonProperty("verbose") private Boolean verbose = false; + @JsonPropertyDescription("If using the identification_keys settings, a new ML model will be created for every degree of cardinality. This can cause a large amount of memory usage, so it is helpful to set a limit on the number of models. Default limit is 5000.") @JsonProperty("cardinality_limit") private int cardinalityLimit = 5000; diff --git a/data-prepper-plugins/anomaly-detector-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/anomalydetector/modes/RandomCutForestModeConfig.java b/data-prepper-plugins/anomaly-detector-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/anomalydetector/modes/RandomCutForestModeConfig.java index 60ad080dcd..c477746253 100644 --- a/data-prepper-plugins/anomaly-detector-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/anomalydetector/modes/RandomCutForestModeConfig.java +++ b/data-prepper-plugins/anomaly-detector-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/anomalydetector/modes/RandomCutForestModeConfig.java @@ -6,6 +6,7 @@ package org.opensearch.dataprepper.plugins.processor.anomalydetector.modes; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonPropertyDescription; import jakarta.validation.constraints.AssertTrue; import java.util.Set; @@ -25,25 +26,31 @@ public class RandomCutForestModeConfig { public static final String VERSION_1_0 = "1.0"; + @JsonPropertyDescription("The algorithm version number. Default is 1.0.") @JsonProperty("version") private String version = VERSION_1_0; public static final Set validVersions = new HashSet<>(Set.of(VERSION_1_0)); + @JsonPropertyDescription("The type of data sent to the algorithm. Default is metrics type") @JsonProperty("type") private String type = RandomCutForestType.METRICS.toString(); public static final Set validTypes = new HashSet<>(Set.of(RandomCutForestType.METRICS.toString())); + @JsonPropertyDescription("The shingle size used in the ML algorithm. Default is 60.") @JsonProperty("shingle_size") private int shingleSize = DEFAULT_SHINGLE_SIZE; + @JsonPropertyDescription("The sample size used in the ML algorithm. Default is 256.") @JsonProperty("sample_size") private int sampleSize = DEFAULT_SAMPLE_SIZE; + @JsonPropertyDescription("The time decay value used in the ML algorithm. Used as the mathematical expression timeDecay divided by SampleSize in the ML algorithm. Default is 0.1") @JsonProperty("time_decay") private double timeDecay = DEFAULT_TIME_DECAY; + @JsonPropertyDescription("Output after indicates the number of events to consume before outputting anamolies. Default is 32.") @JsonProperty("output_after") private int outputAfter = DEFAULT_OUTPUT_AFTER; From 38fe2afb91670d5d8ba5a278dd96de0274ad8f58 Mon Sep 17 00:00:00 2001 From: Ivan Tse <115105835+ivan-tse@users.noreply.github.com> Date: Wed, 14 Aug 2024 09:36:12 -0700 Subject: [PATCH 23/25] PersonalizeSink: add client and configuration classes (#4803) PersonalizeSink: add client and configuration classes Signed-off-by: Ivan Tse --- .../personalize-sink/build.gradle | 48 ++++ .../sink/personalize/ClientFactory.java | 58 +++++ .../sink/personalize/PersonalizeSink.java | 80 +++++++ .../personalize/PersonalizeSinkService.java | 68 ++++++ .../AwsAuthenticationOptions.java | 80 +++++++ .../PersonalizeAdvancedValidation.java | 4 + .../PersonalizeSinkConfiguration.java | 137 ++++++++++++ .../dataset/DatasetTypeOptions.java | 33 +++ .../sink/personalize/ClientFactoryTest.java | 135 ++++++++++++ .../sink/personalize/PersonalizeSinkTest.java | 85 ++++++++ .../AwsAuthenticationOptionsTest.java | 129 +++++++++++ .../PersonalizeSinkConfigurationTest.java | 205 ++++++++++++++++++ .../dataset/DatasetTypeOptionsTest.java | 38 ++++ settings.gradle | 1 + 14 files changed, 1101 insertions(+) create mode 100644 data-prepper-plugins/personalize-sink/build.gradle create mode 100644 data-prepper-plugins/personalize-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/personalize/ClientFactory.java create mode 100644 data-prepper-plugins/personalize-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/personalize/PersonalizeSink.java create mode 100644 data-prepper-plugins/personalize-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/personalize/PersonalizeSinkService.java create mode 100644 data-prepper-plugins/personalize-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/personalize/configuration/AwsAuthenticationOptions.java create mode 100644 data-prepper-plugins/personalize-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/personalize/configuration/PersonalizeAdvancedValidation.java create mode 100644 data-prepper-plugins/personalize-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/personalize/configuration/PersonalizeSinkConfiguration.java create mode 100644 data-prepper-plugins/personalize-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/personalize/dataset/DatasetTypeOptions.java create mode 100644 data-prepper-plugins/personalize-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/personalize/ClientFactoryTest.java create mode 100644 data-prepper-plugins/personalize-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/personalize/PersonalizeSinkTest.java create mode 100644 data-prepper-plugins/personalize-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/personalize/configuration/AwsAuthenticationOptionsTest.java create mode 100644 data-prepper-plugins/personalize-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/personalize/configuration/PersonalizeSinkConfigurationTest.java create mode 100644 data-prepper-plugins/personalize-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/personalize/dataset/DatasetTypeOptionsTest.java diff --git a/data-prepper-plugins/personalize-sink/build.gradle b/data-prepper-plugins/personalize-sink/build.gradle new file mode 100644 index 0000000000..bf408a04b8 --- /dev/null +++ b/data-prepper-plugins/personalize-sink/build.gradle @@ -0,0 +1,48 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +dependencies { + implementation project(':data-prepper-api') + implementation project(path: ':data-prepper-plugins:common') + implementation project(':data-prepper-plugins:aws-plugin-api') + implementation 'io.micrometer:micrometer-core' + implementation 'com.fasterxml.jackson.core:jackson-core' + implementation 'com.fasterxml.jackson.core:jackson-databind' + implementation 'org.hibernate.validator:hibernate-validator:8.0.1.Final' + implementation 'software.amazon.awssdk:personalizeevents' + implementation 'software.amazon.awssdk:sts' + implementation 'software.amazon.awssdk:arns' + testImplementation project(':data-prepper-test-common') + testImplementation testLibs.slf4j.simple +} + +sourceSets { + integrationTest { + java { + compileClasspath += main.output + test.output + runtimeClasspath += main.output + test.output + srcDir file('src/integrationTest/java') + } + resources.srcDir file('src/integrationTest/resources') + } +} + +configurations { + integrationTestImplementation.extendsFrom testImplementation + integrationTestRuntime.extendsFrom testRuntime +} + +task integrationTest(type: Test) { + group = 'verification' + testClassesDirs = sourceSets.integrationTest.output.classesDirs + + useJUnitPlatform() + + classpath = sourceSets.integrationTest.runtimeClasspath + + filter { + includeTestsMatching '*IT' + } +} diff --git a/data-prepper-plugins/personalize-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/personalize/ClientFactory.java b/data-prepper-plugins/personalize-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/personalize/ClientFactory.java new file mode 100644 index 0000000000..2c93fc991b --- /dev/null +++ b/data-prepper-plugins/personalize-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/personalize/ClientFactory.java @@ -0,0 +1,58 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.sink.personalize; + +import org.opensearch.dataprepper.aws.api.AwsCredentialsOptions; +import org.opensearch.dataprepper.aws.api.AwsCredentialsSupplier; +import org.opensearch.dataprepper.plugins.sink.personalize.configuration.AwsAuthenticationOptions; +import org.opensearch.dataprepper.plugins.sink.personalize.configuration.PersonalizeSinkConfiguration; +import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; +import software.amazon.awssdk.core.client.config.ClientOverrideConfiguration; +import software.amazon.awssdk.core.retry.RetryPolicy; +import software.amazon.awssdk.regions.Region; +import software.amazon.awssdk.services.personalizeevents.PersonalizeEventsClient; + +final class ClientFactory { + private ClientFactory() { } + + static PersonalizeEventsClient createPersonalizeEventsClient(final PersonalizeSinkConfiguration personalizeSinkConfig, final AwsCredentialsSupplier awsCredentialsSupplier) { + final AwsCredentialsOptions awsCredentialsOptions = convertToCredentialsOptions(personalizeSinkConfig.getAwsAuthenticationOptions()); + final AwsCredentialsProvider awsCredentialsProvider = awsCredentialsSupplier.getProvider(awsCredentialsOptions); + + return PersonalizeEventsClient.builder() + .region(getRegion(personalizeSinkConfig, awsCredentialsSupplier)) + .credentialsProvider(awsCredentialsProvider) + .overrideConfiguration(createOverrideConfiguration(personalizeSinkConfig)).build(); + } + + private static ClientOverrideConfiguration createOverrideConfiguration(final PersonalizeSinkConfiguration personalizeSinkConfig) { + final RetryPolicy retryPolicy = RetryPolicy.builder().numRetries(personalizeSinkConfig.getMaxRetries()).build(); + return ClientOverrideConfiguration.builder() + .retryPolicy(retryPolicy) + .build(); + } + + private static AwsCredentialsOptions convertToCredentialsOptions(final AwsAuthenticationOptions awsAuthenticationOptions) { + if (awsAuthenticationOptions == null) { + return AwsCredentialsOptions.builder().build(); + } + return AwsCredentialsOptions.builder() + .withRegion(awsAuthenticationOptions.getAwsRegion().orElse(null)) + .withStsRoleArn(awsAuthenticationOptions.getAwsStsRoleArn()) + .withStsExternalId(awsAuthenticationOptions.getAwsStsExternalId()) + .withStsHeaderOverrides(awsAuthenticationOptions.getAwsStsHeaderOverrides()) + .build(); + } + + private static Region getRegion(final PersonalizeSinkConfiguration personalizeSinkConfig, final AwsCredentialsSupplier awsCredentialsSupplier) { + Region defaultRegion = awsCredentialsSupplier.getDefaultRegion().orElse(null); + if (personalizeSinkConfig.getAwsAuthenticationOptions() == null) { + return defaultRegion; + } else { + return personalizeSinkConfig.getAwsAuthenticationOptions().getAwsRegion().orElse(defaultRegion); + } + } +} \ No newline at end of file diff --git a/data-prepper-plugins/personalize-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/personalize/PersonalizeSink.java b/data-prepper-plugins/personalize-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/personalize/PersonalizeSink.java new file mode 100644 index 0000000000..a93e58875c --- /dev/null +++ b/data-prepper-plugins/personalize-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/personalize/PersonalizeSink.java @@ -0,0 +1,80 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.sink.personalize; + +import org.opensearch.dataprepper.aws.api.AwsCredentialsSupplier; +import org.opensearch.dataprepper.model.annotations.DataPrepperPlugin; +import org.opensearch.dataprepper.model.annotations.DataPrepperPluginConstructor; +import org.opensearch.dataprepper.model.configuration.PluginSetting; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.plugin.PluginFactory; +import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.model.sink.AbstractSink; +import org.opensearch.dataprepper.model.sink.Sink; +import org.opensearch.dataprepper.model.sink.SinkContext; +import org.opensearch.dataprepper.plugins.sink.personalize.configuration.PersonalizeSinkConfiguration; +import software.amazon.awssdk.services.personalizeevents.PersonalizeEventsClient; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Collection; + +/** + * Implementation class of personalize-sink plugin. It is responsible for receiving the collection of + * {@link Event} and uploading to amazon personalize. + */ +@DataPrepperPlugin(name = "aws_personalize", pluginType = Sink.class, pluginConfigurationType = PersonalizeSinkConfiguration.class) +public class PersonalizeSink extends AbstractSink> { + + private static final Logger LOG = LoggerFactory.getLogger(PersonalizeSink.class); + + private final PersonalizeSinkConfiguration personalizeSinkConfig; + private volatile boolean sinkInitialized; + private final PersonalizeSinkService personalizeSinkService; + private final SinkContext sinkContext; + + /** + * @param pluginSetting dp plugin settings. + * @param personalizeSinkConfig personalize sink configurations. + * @param sinkContext sink context + * @param awsCredentialsSupplier aws credentials + * @param pluginFactory dp plugin factory. + */ + @DataPrepperPluginConstructor + public PersonalizeSink(final PluginSetting pluginSetting, + final PersonalizeSinkConfiguration personalizeSinkConfig, + final PluginFactory pluginFactory, + final SinkContext sinkContext, + final AwsCredentialsSupplier awsCredentialsSupplier) { + super(pluginSetting); + this.personalizeSinkConfig = personalizeSinkConfig; + this.sinkContext = sinkContext; + + sinkInitialized = false; + + final PersonalizeEventsClient personalizeEventsClient = ClientFactory.createPersonalizeEventsClient(personalizeSinkConfig, awsCredentialsSupplier); + + personalizeSinkService = new PersonalizeSinkService(personalizeSinkConfig, pluginMetrics); + } + + @Override + public boolean isReady() { + return sinkInitialized; + } + + @Override + public void doInitialize() { + sinkInitialized = true; + } + + /** + * @param records Records to be output + */ + @Override + public void doOutput(final Collection> records) { + personalizeSinkService.output(records); + } +} \ No newline at end of file diff --git a/data-prepper-plugins/personalize-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/personalize/PersonalizeSinkService.java b/data-prepper-plugins/personalize-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/personalize/PersonalizeSinkService.java new file mode 100644 index 0000000000..80ea94bcf1 --- /dev/null +++ b/data-prepper-plugins/personalize-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/personalize/PersonalizeSinkService.java @@ -0,0 +1,68 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.sink.personalize; + +import io.micrometer.core.instrument.Counter; +import io.micrometer.core.instrument.Timer; +import org.opensearch.dataprepper.plugins.sink.personalize.configuration.PersonalizeSinkConfiguration; +import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.record.Record; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Collection; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; + +/** + * Class responsible for creating PersonalizeEventsClient object, check thresholds, + * get new buffer and write records into buffer. + */ +class PersonalizeSinkService { + + private static final Logger LOG = LoggerFactory.getLogger(PersonalizeSinkService.class); + public static final String RECORDS_SUCCEEDED = "personalizeRecordsSucceeded"; + public static final String RECORDS_FAILED = "personalizeRecordsFailed"; + public static final String RECORDS_INVALID = "personalizeRecordsInvalid"; + public static final String REQUESTS_THROTTLED = "personalizeRequestsThrottled"; + public static final String REQUEST_LATENCY = "personalizeRequestLatency"; + + private final PersonalizeSinkConfiguration personalizeSinkConfig; + private final Lock reentrantLock; + private final int maxRetries; + private final Counter recordsSucceededCounter; + private final Counter recordsFailedCounter; + private final Counter recordsInvalidCounter; + private final Counter requestsThrottledCounter; + private final Timer requestLatencyTimer; + + /** + * @param personalizeSinkConfig personalize sink related configuration. + * @param pluginMetrics metrics. + */ + public PersonalizeSinkService(final PersonalizeSinkConfiguration personalizeSinkConfig, + final PluginMetrics pluginMetrics) { + this.personalizeSinkConfig = personalizeSinkConfig; + reentrantLock = new ReentrantLock(); + + maxRetries = personalizeSinkConfig.getMaxRetries(); + + recordsSucceededCounter = pluginMetrics.counter(RECORDS_SUCCEEDED); + recordsFailedCounter = pluginMetrics.counter(RECORDS_FAILED); + recordsInvalidCounter = pluginMetrics.counter(RECORDS_INVALID); + requestsThrottledCounter = pluginMetrics.counter(REQUESTS_THROTTLED); + requestLatencyTimer = pluginMetrics.timer(REQUEST_LATENCY); + } + + /** + * @param records received records and add into buffer. + */ + void output(Collection> records) { + LOG.trace("{} records received", records.size()); + return; + } +} \ No newline at end of file diff --git a/data-prepper-plugins/personalize-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/personalize/configuration/AwsAuthenticationOptions.java b/data-prepper-plugins/personalize-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/personalize/configuration/AwsAuthenticationOptions.java new file mode 100644 index 0000000000..ba7e96d43d --- /dev/null +++ b/data-prepper-plugins/personalize-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/personalize/configuration/AwsAuthenticationOptions.java @@ -0,0 +1,80 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.sink.personalize.configuration; + +import com.fasterxml.jackson.annotation.JsonProperty; +import jakarta.validation.GroupSequence; +import jakarta.validation.constraints.AssertTrue; +import jakarta.validation.constraints.Size; +import software.amazon.awssdk.regions.Region; +import software.amazon.awssdk.arns.Arn; + +import java.util.Map; +import java.util.Optional; + +@GroupSequence({AwsAuthenticationOptions.class, PersonalizeAdvancedValidation.class}) +public class AwsAuthenticationOptions { + private static final String AWS_IAM_ROLE = "role"; + private static final String AWS_IAM = "iam"; + + @JsonProperty("region") + @Size(min = 1, message = "Region cannot be empty string") + private String awsRegion; + + @JsonProperty("sts_role_arn") + @Size(min = 20, max = 2048, message = "awsStsRoleArn length should be between 1 and 2048 characters") + private String awsStsRoleArn; + + @JsonProperty("sts_external_id") + @Size(min = 2, max = 1224, message = "awsStsExternalId length should be between 2 and 1224 characters") + private String awsStsExternalId; + + @JsonProperty("sts_header_overrides") + @Size(max = 5, message = "sts_header_overrides supports a maximum of 5 headers to override") + private Map awsStsHeaderOverrides; + + @AssertTrue(message = "sts_role_arn must be an IAM Role", groups = PersonalizeAdvancedValidation.class) + boolean isValidStsRoleArn() { + if (awsStsRoleArn == null) { + return true; + } + final Arn arn = getArn(); + boolean status = true; + if (!AWS_IAM.equals(arn.service())) { + status = false; + } + final Optional resourceType = arn.resource().resourceType(); + if (resourceType.isEmpty() || !resourceType.get().equals(AWS_IAM_ROLE)) { + status = false; + } + return status; + } + + private Arn getArn() { + try { + return Arn.fromString(awsStsRoleArn); + } catch (final Exception e) { + throw new IllegalArgumentException(String.format("Invalid ARN format for awsStsRoleArn. Check the format of %s", awsStsRoleArn)); + } + } + + public Optional getAwsRegion() { + Region region = awsRegion != null ? Region.of(awsRegion) : null; + return Optional.ofNullable(region); + } + + public String getAwsStsRoleArn() { + return awsStsRoleArn; + } + + public String getAwsStsExternalId() { + return awsStsExternalId; + } + + public Map getAwsStsHeaderOverrides() { + return awsStsHeaderOverrides; + } +} \ No newline at end of file diff --git a/data-prepper-plugins/personalize-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/personalize/configuration/PersonalizeAdvancedValidation.java b/data-prepper-plugins/personalize-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/personalize/configuration/PersonalizeAdvancedValidation.java new file mode 100644 index 0000000000..f48c1d9466 --- /dev/null +++ b/data-prepper-plugins/personalize-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/personalize/configuration/PersonalizeAdvancedValidation.java @@ -0,0 +1,4 @@ +package org.opensearch.dataprepper.plugins.sink.personalize.configuration; + +interface PersonalizeAdvancedValidation { +} diff --git a/data-prepper-plugins/personalize-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/personalize/configuration/PersonalizeSinkConfiguration.java b/data-prepper-plugins/personalize-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/personalize/configuration/PersonalizeSinkConfiguration.java new file mode 100644 index 0000000000..95c9f1d5c9 --- /dev/null +++ b/data-prepper-plugins/personalize-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/personalize/configuration/PersonalizeSinkConfiguration.java @@ -0,0 +1,137 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.sink.personalize.configuration; + +import jakarta.validation.GroupSequence; +import org.opensearch.dataprepper.plugins.sink.personalize.dataset.DatasetTypeOptions; +import com.fasterxml.jackson.annotation.JsonProperty; +import jakarta.validation.Valid; +import jakarta.validation.constraints.NotNull; +import jakarta.validation.constraints.AssertTrue; +import software.amazon.awssdk.arns.Arn; + +import java.util.List; +import java.util.Optional; + +/** + * personalize sink configuration class contains properties, used to read yaml configuration. + */ +@GroupSequence({PersonalizeSinkConfiguration.class, PersonalizeAdvancedValidation.class}) +public class PersonalizeSinkConfiguration { + private static final int DEFAULT_RETRIES = 10; + private static final String AWS_PERSONALIZE = "personalize"; + private static final String AWS_PERSONALIZE_DATASET = "dataset"; + private static final List DATASET_ARN_REQUIRED_LIST = List.of(DatasetTypeOptions.USERS, DatasetTypeOptions.ITEMS); + + @JsonProperty("aws") + @Valid + private AwsAuthenticationOptions awsAuthenticationOptions; + + @JsonProperty("dataset_type") + @NotNull + @Valid + private DatasetTypeOptions datasetType; + + @JsonProperty("dataset_arn") + private String datasetArn; + + @JsonProperty("tracking_id") + private String trackingId; + + @JsonProperty("document_root_key") + private String documentRootKey; + + @JsonProperty("max_retries") + private int maxRetries = DEFAULT_RETRIES; + + @AssertTrue(message = "A dataset arn is required for items and users datasets.", groups = PersonalizeAdvancedValidation.class) + boolean isDatasetArnProvidedWhenNeeded() { + if (DATASET_ARN_REQUIRED_LIST.contains(datasetType)) { + return datasetArn != null; + } + return true; + } + + @AssertTrue(message = "dataset_arn must be a Personalize Dataset arn", groups = PersonalizeAdvancedValidation.class) + boolean isValidDatasetArn() { + if (datasetArn == null) { + return true; + } + final Arn arn = getArn(); + boolean status = true; + if (!AWS_PERSONALIZE.equals(arn.service())) { + status = false; + } + final Optional resourceType = arn.resource().resourceType(); + if (resourceType.isEmpty() || !resourceType.get().equals(AWS_PERSONALIZE_DATASET)) { + status = false; + } + return status; + } + + private Arn getArn() { + try { + return Arn.fromString(datasetArn); + } catch (final Exception e) { + throw new IllegalArgumentException(String.format("Invalid ARN format for datasetArn. Check the format of %s", datasetArn), e); + } + } + + @AssertTrue(message = "A tracking id is required for interactions dataset.", groups = PersonalizeAdvancedValidation.class) + boolean isTrackingIdProvidedWhenNeeded() { + if (DatasetTypeOptions.INTERACTIONS.equals(datasetType)) { + return trackingId != null; + } + return true; + } + + /** + * Aws Authentication configuration Options. + * @return aws authentication options. + */ + public AwsAuthenticationOptions getAwsAuthenticationOptions() { + return awsAuthenticationOptions; + } + + /** + * Dataset type configuration Options. + * @return dataset type option object. + */ + public DatasetTypeOptions getDatasetType() { + return datasetType; + } + + /** + * Dataset arn for Personalize Dataset. + * @return dataset arn string. + */ + public String getDatasetArn() { + return datasetArn; + } + + /** + * Tracking id for Personalize Event Tracker. + * @return tracking id string. + */ + public String getTrackingId() { + return trackingId; + } + + /** + * Tracking id for Personalize Event Tracker. + * @return document root key string. + */ + public String getDocumentRootKey() { + return documentRootKey; + } + + /** + * Personalize client retries configuration Options. + * @return maximum retries value. + */ + public int getMaxRetries() { + return maxRetries; + } +} \ No newline at end of file diff --git a/data-prepper-plugins/personalize-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/personalize/dataset/DatasetTypeOptions.java b/data-prepper-plugins/personalize-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/personalize/dataset/DatasetTypeOptions.java new file mode 100644 index 0000000000..cc6791f0a6 --- /dev/null +++ b/data-prepper-plugins/personalize-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/personalize/dataset/DatasetTypeOptions.java @@ -0,0 +1,33 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.sink.personalize.dataset; + +import com.fasterxml.jackson.annotation.JsonCreator; +import java.util.Arrays; +import java.util.Map; +import java.util.stream.Collectors; + +/** + * Defines all the dataset types enumerations. + */ +public enum DatasetTypeOptions { + USERS("users"), + ITEMS("items"), + INTERACTIONS("interactions"); + + private final String option; + private static final Map OPTIONS_MAP = Arrays.stream(DatasetTypeOptions.values()) + .collect(Collectors.toMap(value -> value.option, value -> value)); + + DatasetTypeOptions(final String option) { + this.option = option.toLowerCase(); + } + + @JsonCreator + static DatasetTypeOptions fromOptionValue(final String option) { + return OPTIONS_MAP.get(option); + } +} \ No newline at end of file diff --git a/data-prepper-plugins/personalize-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/personalize/ClientFactoryTest.java b/data-prepper-plugins/personalize-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/personalize/ClientFactoryTest.java new file mode 100644 index 0000000000..6b1ad7f80a --- /dev/null +++ b/data-prepper-plugins/personalize-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/personalize/ClientFactoryTest.java @@ -0,0 +1,135 @@ +package org.opensearch.dataprepper.plugins.sink.personalize; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; +import org.mockito.ArgumentCaptor; +import org.mockito.Mock; +import org.mockito.MockedStatic; +import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.dataprepper.aws.api.AwsCredentialsOptions; +import org.opensearch.dataprepper.aws.api.AwsCredentialsSupplier; +import org.opensearch.dataprepper.plugins.sink.personalize.configuration.PersonalizeSinkConfiguration; +import org.opensearch.dataprepper.plugins.sink.personalize.configuration.AwsAuthenticationOptions; +import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; +import software.amazon.awssdk.core.client.config.ClientOverrideConfiguration; +import software.amazon.awssdk.regions.Region; +import software.amazon.awssdk.services.personalizeevents.PersonalizeEventsClient; +import software.amazon.awssdk.services.personalizeevents.PersonalizeEventsClientBuilder; + +import java.util.Collections; +import java.util.Map; +import java.util.Optional; +import java.util.UUID; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.notNullValue; +import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.mockStatic; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +@ExtendWith(MockitoExtension.class) +class ClientFactoryTest { + @Mock + private PersonalizeSinkConfiguration personalizeSinkConfig; + @Mock + private AwsCredentialsSupplier awsCredentialsSupplier; + + @Mock + private AwsAuthenticationOptions awsAuthenticationOptions; + + @BeforeEach + void setUp() { + when(personalizeSinkConfig.getAwsAuthenticationOptions()).thenReturn(awsAuthenticationOptions); + } + + @Test + void createPersonalizeEventsClient_with_real_PersonalizeEventsClient() { + when(awsAuthenticationOptions.getAwsRegion()).thenReturn(Optional.of(Region.US_EAST_1)); + final PersonalizeEventsClient personalizeEventsClient = ClientFactory.createPersonalizeEventsClient(personalizeSinkConfig, awsCredentialsSupplier); + + assertThat(personalizeEventsClient, notNullValue()); + } + + @Test + void createPersonalizeEventsClient_provides_correct_inputs_for_null_awsAuthenticationOptions() { + when(personalizeSinkConfig.getAwsAuthenticationOptions()).thenReturn(awsAuthenticationOptions); + final AwsCredentialsProvider expectedCredentialsProvider = mock(AwsCredentialsProvider.class); + when(awsCredentialsSupplier.getProvider(any())).thenReturn(expectedCredentialsProvider); + + final PersonalizeEventsClientBuilder personalizeEventsClientBuilder = mock(PersonalizeEventsClientBuilder.class); + when(personalizeEventsClientBuilder.region(any())).thenReturn(personalizeEventsClientBuilder); + when(personalizeEventsClientBuilder.credentialsProvider(any())).thenReturn(personalizeEventsClientBuilder); + when(personalizeEventsClientBuilder.overrideConfiguration(any(ClientOverrideConfiguration.class))).thenReturn(personalizeEventsClientBuilder); + try(final MockedStatic personalizeEventsClientMockedStatic = mockStatic(PersonalizeEventsClient.class)) { + personalizeEventsClientMockedStatic.when(PersonalizeEventsClient::builder) + .thenReturn(personalizeEventsClientBuilder); + ClientFactory.createPersonalizeEventsClient(personalizeSinkConfig, awsCredentialsSupplier); + } + + final ArgumentCaptor credentialsProviderArgumentCaptor = ArgumentCaptor.forClass(AwsCredentialsProvider.class); + verify(personalizeEventsClientBuilder).credentialsProvider(credentialsProviderArgumentCaptor.capture()); + + final AwsCredentialsProvider actualCredentialsProvider = credentialsProviderArgumentCaptor.getValue(); + + assertThat(actualCredentialsProvider, equalTo(expectedCredentialsProvider)); + + final ArgumentCaptor optionsArgumentCaptor = ArgumentCaptor.forClass(AwsCredentialsOptions.class); + verify(awsCredentialsSupplier).getProvider(optionsArgumentCaptor.capture()); + + final AwsCredentialsOptions actualCredentialsOptions = optionsArgumentCaptor.getValue(); + assertThat(actualCredentialsOptions, is(notNullValue())); + assertThat(actualCredentialsOptions.getRegion(), equalTo(null)); + assertThat(actualCredentialsOptions.getStsRoleArn(), equalTo(null)); + assertThat(actualCredentialsOptions.getStsExternalId(), equalTo(null)); + assertThat(actualCredentialsOptions.getStsHeaderOverrides(), equalTo(Collections.emptyMap())); + } + + @ParameterizedTest + @ValueSource(strings = {"us-east-1", "us-west-2", "eu-central-1"}) + void createPersonalizeEventsClient_provides_correct_inputs(final String regionString) { + final Region region = Region.of(regionString); + final String stsRoleArn = UUID.randomUUID().toString(); + final String externalId = UUID.randomUUID().toString(); + final Map stsHeaderOverrides = Map.of(UUID.randomUUID().toString(), UUID.randomUUID().toString()); + when(awsAuthenticationOptions.getAwsRegion()).thenReturn(Optional.of(region)); + when(awsAuthenticationOptions.getAwsStsRoleArn()).thenReturn(stsRoleArn); + when(awsAuthenticationOptions.getAwsStsExternalId()).thenReturn(externalId); + when(awsAuthenticationOptions.getAwsStsHeaderOverrides()).thenReturn(stsHeaderOverrides); + + final AwsCredentialsProvider expectedCredentialsProvider = mock(AwsCredentialsProvider.class); + when(awsCredentialsSupplier.getProvider(any())).thenReturn(expectedCredentialsProvider); + + final PersonalizeEventsClientBuilder personalizeEventsClientBuilder = mock(PersonalizeEventsClientBuilder.class); + when(personalizeEventsClientBuilder.region(region)).thenReturn(personalizeEventsClientBuilder); + when(personalizeEventsClientBuilder.credentialsProvider(any())).thenReturn(personalizeEventsClientBuilder); + when(personalizeEventsClientBuilder.overrideConfiguration(any(ClientOverrideConfiguration.class))).thenReturn(personalizeEventsClientBuilder); + try(final MockedStatic personalizeEventsClientMockedStatic = mockStatic(PersonalizeEventsClient.class)) { + personalizeEventsClientMockedStatic.when(PersonalizeEventsClient::builder) + .thenReturn(personalizeEventsClientBuilder); + ClientFactory.createPersonalizeEventsClient(personalizeSinkConfig, awsCredentialsSupplier); + } + + final ArgumentCaptor credentialsProviderArgumentCaptor = ArgumentCaptor.forClass(AwsCredentialsProvider.class); + verify(personalizeEventsClientBuilder).credentialsProvider(credentialsProviderArgumentCaptor.capture()); + + final AwsCredentialsProvider actualCredentialsProvider = credentialsProviderArgumentCaptor.getValue(); + + assertThat(actualCredentialsProvider, equalTo(expectedCredentialsProvider)); + + final ArgumentCaptor optionsArgumentCaptor = ArgumentCaptor.forClass(AwsCredentialsOptions.class); + verify(awsCredentialsSupplier).getProvider(optionsArgumentCaptor.capture()); + + final AwsCredentialsOptions actualCredentialsOptions = optionsArgumentCaptor.getValue(); + assertThat(actualCredentialsOptions.getRegion(), equalTo(region)); + assertThat(actualCredentialsOptions.getStsRoleArn(), equalTo(stsRoleArn)); + assertThat(actualCredentialsOptions.getStsExternalId(), equalTo(externalId)); + assertThat(actualCredentialsOptions.getStsHeaderOverrides(), equalTo(stsHeaderOverrides)); + } +} \ No newline at end of file diff --git a/data-prepper-plugins/personalize-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/personalize/PersonalizeSinkTest.java b/data-prepper-plugins/personalize-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/personalize/PersonalizeSinkTest.java new file mode 100644 index 0000000000..852e75630f --- /dev/null +++ b/data-prepper-plugins/personalize-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/personalize/PersonalizeSinkTest.java @@ -0,0 +1,85 @@ +package org.opensearch.dataprepper.plugins.sink.personalize; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.opensearch.dataprepper.aws.api.AwsCredentialsSupplier; +import org.opensearch.dataprepper.model.configuration.PluginSetting; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.plugin.PluginFactory; +import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.model.sink.SinkContext; +import org.opensearch.dataprepper.plugins.sink.personalize.configuration.AwsAuthenticationOptions; +import org.opensearch.dataprepper.plugins.sink.personalize.configuration.PersonalizeSinkConfiguration; +import org.opensearch.dataprepper.plugins.sink.personalize.dataset.DatasetTypeOptions; +import software.amazon.awssdk.regions.Region; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Optional; + +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +class PersonalizeSinkTest { + public static final int MAX_RETRIES = 10; + public static final String REGION = "us-east-1"; + public static final String SINK_PLUGIN_NAME = "personalize"; + public static final String SINK_PIPELINE_NAME = "personalize-sink-pipeline"; + public static final String DATASET_ARN = "arn:aws:iam::123456789012:dataset/test"; + public static final String TRACKING_ID = "1233513241"; + private PersonalizeSinkConfiguration personalizeSinkConfig; + private PersonalizeSink personalizeSink; + private PluginSetting pluginSetting; + private PluginFactory pluginFactory; + private AwsCredentialsSupplier awsCredentialsSupplier; + private SinkContext sinkContext; + + @BeforeEach + void setup() { + personalizeSinkConfig = mock(PersonalizeSinkConfiguration.class); + sinkContext = mock(SinkContext.class); + AwsAuthenticationOptions awsAuthenticationOptions = mock(AwsAuthenticationOptions.class); + pluginSetting = mock(PluginSetting.class); + pluginFactory = mock(PluginFactory.class); + awsCredentialsSupplier = mock(AwsCredentialsSupplier.class); + + when(personalizeSinkConfig.getMaxRetries()).thenReturn(MAX_RETRIES); + when(personalizeSinkConfig.getDatasetArn()).thenReturn(DATASET_ARN); + when(personalizeSinkConfig.getDatasetType()).thenReturn(DatasetTypeOptions.USERS); + when(personalizeSinkConfig.getAwsAuthenticationOptions()).thenReturn(awsAuthenticationOptions); + when(awsAuthenticationOptions.getAwsRegion()).thenReturn(Optional.of(Region.of(REGION))); + when(pluginSetting.getName()).thenReturn(SINK_PLUGIN_NAME); + when(pluginSetting.getPipelineName()).thenReturn(SINK_PIPELINE_NAME); + } + + private PersonalizeSink createObjectUnderTest() { + return new PersonalizeSink(pluginSetting, personalizeSinkConfig, pluginFactory, sinkContext, awsCredentialsSupplier); + } + + @Test + void test_personalize_sink_plugin_isReady_positive() { + personalizeSink = createObjectUnderTest(); + Assertions.assertNotNull(personalizeSink); + personalizeSink.doInitialize(); + assertTrue(personalizeSink.isReady(), "Expected the personalize sink to be ready, but it is reporting it is not ready."); + } + + @Test + void test_personalize_Sink_plugin_isReady_negative() { + personalizeSink = createObjectUnderTest(); + Assertions.assertNotNull(personalizeSink); + assertFalse(personalizeSink.isReady(), "Expected the personalize sink to report that it is not ready, but it is reporting it is ready."); + } + + @Test + void test_doOutput_with_empty_records() { + personalizeSink = createObjectUnderTest(); + Assertions.assertNotNull(personalizeSink); + personalizeSink.doInitialize(); + Collection> records = new ArrayList<>(); + personalizeSink.doOutput(records); + } +} diff --git a/data-prepper-plugins/personalize-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/personalize/configuration/AwsAuthenticationOptionsTest.java b/data-prepper-plugins/personalize-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/personalize/configuration/AwsAuthenticationOptionsTest.java new file mode 100644 index 0000000000..29be309622 --- /dev/null +++ b/data-prepper-plugins/personalize-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/personalize/configuration/AwsAuthenticationOptionsTest.java @@ -0,0 +1,129 @@ +package org.opensearch.dataprepper.plugins.sink.personalize.configuration; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; +import software.amazon.awssdk.regions.Region; + +import java.util.Collections; +import java.util.Map; +import java.util.Optional; +import java.util.UUID; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.nullValue; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.assertFalse; + +class AwsAuthenticationOptionsTest { + private ObjectMapper objectMapper; + + @BeforeEach + void setUp() { + objectMapper = new ObjectMapper(); + } + + @ParameterizedTest + @ValueSource(strings = {"us-east-1", "us-west-2", "eu-central-1"}) + void getAwsRegion_returns_Region_of(final String regionString) { + final Optional expectedRegionObject = Optional.of(Region.of(regionString)); + final Map jsonMap = Map.of("region", regionString); + final AwsAuthenticationOptions objectUnderTest = objectMapper.convertValue(jsonMap, AwsAuthenticationOptions.class); + assertThat(objectUnderTest.getAwsRegion(), equalTo(expectedRegionObject)); + } + + @Test + void getAwsRegion_returns_null_when_region_is_null() { + final Map jsonMap = Collections.emptyMap(); + final AwsAuthenticationOptions objectUnderTest = objectMapper.convertValue(jsonMap, AwsAuthenticationOptions.class); + assertThat(objectUnderTest.getAwsRegion(), equalTo(Optional.empty())); + } + + @Test + void getAwsStsRoleArn_returns_value_from_deserialized_JSON() { + final String stsRoleArn = UUID.randomUUID().toString(); + final Map jsonMap = Map.of("sts_role_arn", stsRoleArn); + final AwsAuthenticationOptions objectUnderTest = objectMapper.convertValue(jsonMap, AwsAuthenticationOptions.class); + assertThat(objectUnderTest.getAwsStsRoleArn(), equalTo(stsRoleArn)); + } + + @Test + void getAwsStsRoleArn_returns_null_if_not_in_JSON() { + final Map jsonMap = Collections.emptyMap(); + final AwsAuthenticationOptions objectUnderTest = objectMapper.convertValue(jsonMap, AwsAuthenticationOptions.class); + assertThat(objectUnderTest.getAwsStsRoleArn(), nullValue()); + } + + @Test + void isValidStsRoleArn_returns_true_for_valid_IAM_role() { + final String stsRoleArn = "arn:aws:iam::123456789012:role/test"; + final Map jsonMap = Map.of("sts_role_arn", stsRoleArn); + final AwsAuthenticationOptions objectUnderTest = objectMapper.convertValue(jsonMap, AwsAuthenticationOptions.class); + assertTrue(objectUnderTest.isValidStsRoleArn()); + } + + @Test + void isValidStsRoleArn_returns_true_for_null() { + final Map jsonMap = Collections.emptyMap(); + final AwsAuthenticationOptions objectUnderTest = objectMapper.convertValue(jsonMap, AwsAuthenticationOptions.class); + assertTrue(objectUnderTest.isValidStsRoleArn()); + } + + @Test + void isValidStsRoleArn_returns_false_when_arn_service_is_not_IAM() { + final String stsRoleArn = "arn:aws:personalize::123456789012:role/test"; + final Map jsonMap = Map.of("sts_role_arn", stsRoleArn); + final AwsAuthenticationOptions objectUnderTest = objectMapper.convertValue(jsonMap, AwsAuthenticationOptions.class); + assertFalse(objectUnderTest.isValidStsRoleArn()); + } + + @Test + void isValidStsRoleArn_returns_false_when_arn_resource_is_not_role() { + final String stsRoleArn = "arn:aws:iam::123456789012:dataset/test"; + final Map jsonMap = Map.of("sts_role_arn", stsRoleArn); + final AwsAuthenticationOptions objectUnderTest = objectMapper.convertValue(jsonMap, AwsAuthenticationOptions.class); + assertFalse(objectUnderTest.isValidStsRoleArn()); + } + + @Test + void isValidStsRoleArn_invalid_arn_throws_IllegalArgumentException() { + final String stsRoleArn = UUID.randomUUID().toString(); + final Map jsonMap = Map.of("sts_role_arn", stsRoleArn); + final AwsAuthenticationOptions objectUnderTest = objectMapper.convertValue(jsonMap, AwsAuthenticationOptions.class); + assertThrows(IllegalArgumentException.class, () -> objectUnderTest.isValidStsRoleArn()); + } + + @Test + void getAwsStsExternalId_returns_value_from_deserialized_JSON() { + final String stsExternalId = UUID.randomUUID().toString(); + final Map jsonMap = Map.of("sts_external_id", stsExternalId); + final AwsAuthenticationOptions objectUnderTest = objectMapper.convertValue(jsonMap, AwsAuthenticationOptions.class); + assertThat(objectUnderTest.getAwsStsExternalId(), equalTo(stsExternalId)); + } + + @Test + void getAwsStsExternalId_returns_null_if_not_in_JSON() { + final Map jsonMap = Collections.emptyMap(); + final AwsAuthenticationOptions objectUnderTest = objectMapper.convertValue(jsonMap, AwsAuthenticationOptions.class); + assertThat(objectUnderTest.getAwsStsExternalId(), nullValue()); + } + + @Test + void getAwsStsHeaderOverrides_returns_value_from_deserialized_JSON() { + final Map stsHeaderOverrides = Map.of(UUID.randomUUID().toString(), UUID.randomUUID().toString()); + final Map jsonMap = Map.of("sts_header_overrides", stsHeaderOverrides); + final AwsAuthenticationOptions objectUnderTest = objectMapper.convertValue(jsonMap, AwsAuthenticationOptions.class); + assertThat(objectUnderTest.getAwsStsHeaderOverrides(), equalTo(stsHeaderOverrides)); + } + + @Test + void getAwsStsHeaderOverrides_returns_null_if_not_in_JSON() { + final Map jsonMap = Collections.emptyMap(); + final AwsAuthenticationOptions objectUnderTest = objectMapper.convertValue(jsonMap, AwsAuthenticationOptions.class); + assertThat(objectUnderTest.getAwsStsHeaderOverrides(), nullValue()); + } +} diff --git a/data-prepper-plugins/personalize-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/personalize/configuration/PersonalizeSinkConfigurationTest.java b/data-prepper-plugins/personalize-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/personalize/configuration/PersonalizeSinkConfigurationTest.java new file mode 100644 index 0000000000..67bc690623 --- /dev/null +++ b/data-prepper-plugins/personalize-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/personalize/configuration/PersonalizeSinkConfigurationTest.java @@ -0,0 +1,205 @@ +package org.opensearch.dataprepper.plugins.sink.personalize.configuration; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.opensearch.dataprepper.plugins.sink.personalize.dataset.DatasetTypeOptions; + +import java.util.Collections; +import java.util.Map; +import java.util.UUID; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.nullValue; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; + +class PersonalizeSinkConfigurationTest { + private static final int DEFAULT_RETRIES = 10; + private ObjectMapper objectMapper; + + @BeforeEach + void setUp() { + objectMapper = new ObjectMapper(); + } + + @Test + void getDatasetType_returns_value_from_deserialized_JSON() { + final String datasetType = "users"; + final Map jsonMap = Map.of("dataset_type", datasetType); + final PersonalizeSinkConfiguration objectUnderTest = objectMapper.convertValue(jsonMap, PersonalizeSinkConfiguration.class); + assertThat(objectUnderTest.getDatasetType(), equalTo(DatasetTypeOptions.USERS)); + } + + @Test + void getDatasetArn_returns_null_when_datasetArn_is_null() { + final Map jsonMap = Collections.emptyMap(); + final PersonalizeSinkConfiguration objectUnderTest = objectMapper.convertValue(jsonMap, PersonalizeSinkConfiguration.class); + assertThat(objectUnderTest.getDatasetArn(), nullValue()); + } + + @Test + void getDatasetArn_returns_value_from_deserialized_JSON() { + final String datasetArn = UUID.randomUUID().toString(); + final Map jsonMap = Map.of("dataset_arn", datasetArn); + final PersonalizeSinkConfiguration objectUnderTest = objectMapper.convertValue(jsonMap, PersonalizeSinkConfiguration.class); + assertThat(objectUnderTest.getDatasetArn(), equalTo(datasetArn)); + } + + @Test + void isDatasetArnProvidedWhenNeeded_returns_true_when_datasetType_is_interactions_and_datasetArn_is_null() { + final String datasetType = "interactions"; + final Map jsonMap = Map.of("dataset_type", datasetType); + final PersonalizeSinkConfiguration objectUnderTest = objectMapper.convertValue(jsonMap, PersonalizeSinkConfiguration.class); + assertTrue(objectUnderTest.isDatasetArnProvidedWhenNeeded()); + } + + @Test + void isDatasetArnProvidedWhenNeeded_returns_true_when_datasetType_is_users_and_datasetArn_is_provided() { + final String datasetType = "users"; + final String datasetArn = UUID.randomUUID().toString(); + final Map jsonMap = Map.of("dataset_type", datasetType, "dataset_arn", datasetArn); + final PersonalizeSinkConfiguration objectUnderTest = objectMapper.convertValue(jsonMap, PersonalizeSinkConfiguration.class); + assertTrue(objectUnderTest.isDatasetArnProvidedWhenNeeded()); + } + + @Test + void isDatasetArnProvidedWhenNeeded_returns_false_when_datasetType_is_users_and_datasetArn_is_not_provided() { + final String datasetType = "users"; + final Map jsonMap = Map.of("dataset_type", datasetType); + final PersonalizeSinkConfiguration objectUnderTest = objectMapper.convertValue(jsonMap, PersonalizeSinkConfiguration.class); + assertFalse(objectUnderTest.isDatasetArnProvidedWhenNeeded()); + } + + @Test + void isDatasetArnProvidedWhenNeeded_returns_true_when_datasetType_is_items_and_datasetArn_is_provided() { + final String datasetType = "items"; + final String datasetArn = UUID.randomUUID().toString(); + final Map jsonMap = Map.of("dataset_type", datasetType, "dataset_arn", datasetArn); + final PersonalizeSinkConfiguration objectUnderTest = objectMapper.convertValue(jsonMap, PersonalizeSinkConfiguration.class); + assertTrue(objectUnderTest.isDatasetArnProvidedWhenNeeded()); + } + + @Test + void isDatasetArnProvidedWhenNeeded_returns_false_when_datasetType_is_items_and_datasetArn_is_not_provided() { + final String datasetType = "items"; + final Map jsonMap = Map.of("dataset_type", datasetType); + final PersonalizeSinkConfiguration objectUnderTest = objectMapper.convertValue(jsonMap, PersonalizeSinkConfiguration.class); + assertFalse(objectUnderTest.isDatasetArnProvidedWhenNeeded()); + } + + @Test + void isValidDatasetArn_returns_true_for_valid_dataset_arn() { + final String datasetArn = "arn:aws:personalize::123456789012:dataset/test"; + final Map jsonMap = Map.of("dataset_arn", datasetArn); + final PersonalizeSinkConfiguration objectUnderTest = objectMapper.convertValue(jsonMap, PersonalizeSinkConfiguration.class); + assertTrue(objectUnderTest.isValidDatasetArn()); + } + + @Test + void isValidDatasetArn_returns_false_when_arn_service_is_not_personalize() { + final String datasetArn = "arn:aws:iam::123456789012:dataset/test"; + final Map jsonMap = Map.of("dataset_arn", datasetArn); + final PersonalizeSinkConfiguration objectUnderTest = objectMapper.convertValue(jsonMap, PersonalizeSinkConfiguration.class); + assertFalse(objectUnderTest.isValidDatasetArn()); + } + + @Test + void isValidDatasetArn_returns_false_when_arn_resource_is_not_dataset() { + final String datasetArn = "arn:aws:personalize::123456789012:role/test"; + final Map jsonMap = Map.of("dataset_arn", datasetArn); + final PersonalizeSinkConfiguration objectUnderTest = objectMapper.convertValue(jsonMap, PersonalizeSinkConfiguration.class); + assertFalse(objectUnderTest.isValidDatasetArn()); + } + + @Test + void isValidStsRoleArn_invalid_arn_throws_IllegalArgumentException() { + final String datasetArn = UUID.randomUUID().toString(); + final Map jsonMap = Map.of("dataset_arn", datasetArn); + final PersonalizeSinkConfiguration objectUnderTest = objectMapper.convertValue(jsonMap, PersonalizeSinkConfiguration.class); + assertThrows(IllegalArgumentException.class, () -> objectUnderTest.isValidDatasetArn()); + } + + + + @Test + void getTrackingId_returns_null_when_trackingId_is_null() { + final Map jsonMap = Collections.emptyMap(); + final PersonalizeSinkConfiguration objectUnderTest = objectMapper.convertValue(jsonMap, PersonalizeSinkConfiguration.class); + assertThat(objectUnderTest.getTrackingId(), nullValue()); + } + + @Test + void getTrackingId_returns_value_from_deserialized_JSON() { + final String trackingId = UUID.randomUUID().toString();; + final Map jsonMap = Map.of("tracking_id", trackingId); + final PersonalizeSinkConfiguration objectUnderTest = objectMapper.convertValue(jsonMap, PersonalizeSinkConfiguration.class); + assertThat(objectUnderTest.getTrackingId(), equalTo(trackingId)); + } + + @Test + void isTrackingIdProvidedWhenNeeded_returns_false_when_datasetType_is_interactions_and_trackingId_is_not_provided() { + final String datasetType = "interactions"; + final Map jsonMap = Map.of("dataset_type", datasetType); + final PersonalizeSinkConfiguration objectUnderTest = objectMapper.convertValue(jsonMap, PersonalizeSinkConfiguration.class); + assertFalse(objectUnderTest.isTrackingIdProvidedWhenNeeded()); + } + + @Test + void isTrackingIdProvidedWhenNeeded_returns_true_when_datasetType_is_interactions_and_trackingId_is_provided() { + final String datasetType = "interactions"; + final String trackingId = UUID.randomUUID().toString(); + final Map jsonMap = Map.of("dataset_type", datasetType, "tracking_id", trackingId); + final PersonalizeSinkConfiguration objectUnderTest = objectMapper.convertValue(jsonMap, PersonalizeSinkConfiguration.class); + assertTrue(objectUnderTest.isTrackingIdProvidedWhenNeeded()); + } + + @Test + void isTrackingIdProvidedWhenNeeded_returns_true_when_datasetType_is_users_and_trackingId_is_not_provided() { + final String datasetType = "users"; + final Map jsonMap = Map.of("dataset_type", datasetType); + final PersonalizeSinkConfiguration objectUnderTest = objectMapper.convertValue(jsonMap, PersonalizeSinkConfiguration.class); + assertTrue(objectUnderTest.isTrackingIdProvidedWhenNeeded()); + } + + @Test + void isTrackingIdProvidedWhenNeeded_returns_true_when_datasetType_is_items_and_trackingId_is_not_provided() { + final String datasetType = "items"; + final Map jsonMap = Map.of("dataset_type", datasetType); + final PersonalizeSinkConfiguration objectUnderTest = objectMapper.convertValue(jsonMap, PersonalizeSinkConfiguration.class); + assertTrue(objectUnderTest.isTrackingIdProvidedWhenNeeded()); + } + + + @Test + void getDocumentRootKey_returns_null_when_documentRootKey_is_null() { + final Map jsonMap = Collections.emptyMap(); + final PersonalizeSinkConfiguration objectUnderTest = objectMapper.convertValue(jsonMap, PersonalizeSinkConfiguration.class); + assertThat(objectUnderTest.getDocumentRootKey(), nullValue()); + } + + @Test + void getDocumentRootKey_returns_value_from_deserialized_JSON() { + final String documentRootKey = UUID.randomUUID().toString();; + final Map jsonMap = Map.of("document_root_key", documentRootKey); + final PersonalizeSinkConfiguration objectUnderTest = objectMapper.convertValue(jsonMap, PersonalizeSinkConfiguration.class); + assertThat(objectUnderTest.getDocumentRootKey(), equalTo(documentRootKey)); + } + + @Test + void getMaxRetries_returns_default_when_maxRetries_is_null() { + final Map jsonMap = Collections.emptyMap(); + final PersonalizeSinkConfiguration objectUnderTest = objectMapper.convertValue(jsonMap, PersonalizeSinkConfiguration.class); + assertThat(objectUnderTest.getMaxRetries(), equalTo(DEFAULT_RETRIES)); + } + + @Test + void getMaxRetries_returns_value_from_deserialized_JSON() { + final int maxRetries = 3; + final Map jsonMap = Map.of("max_retries", maxRetries); + final PersonalizeSinkConfiguration objectUnderTest = objectMapper.convertValue(jsonMap, PersonalizeSinkConfiguration.class); + assertThat(objectUnderTest.getMaxRetries(), equalTo(maxRetries)); + } +} diff --git a/data-prepper-plugins/personalize-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/personalize/dataset/DatasetTypeOptionsTest.java b/data-prepper-plugins/personalize-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/personalize/dataset/DatasetTypeOptionsTest.java new file mode 100644 index 0000000000..40b1821d0a --- /dev/null +++ b/data-prepper-plugins/personalize-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/personalize/dataset/DatasetTypeOptionsTest.java @@ -0,0 +1,38 @@ +package org.opensearch.dataprepper.plugins.sink.personalize.dataset; + +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.junit.jupiter.MockitoExtension; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.jupiter.api.Assertions.assertNotNull; + +@ExtendWith(MockitoExtension.class) +class DatasetTypeOptionsTest { + @Test + void notNull_test() { + assertNotNull(DatasetTypeOptions.ITEMS); + } + + @Test + void fromOptionValue_users_test() { + DatasetTypeOptions datasetTypeOptions = DatasetTypeOptions.fromOptionValue("users"); + assertNotNull(datasetTypeOptions); + assertThat(datasetTypeOptions.toString(), equalTo("USERS")); + } + + @Test + void fromOptionValue_items_test() { + DatasetTypeOptions datasetTypeOptions = DatasetTypeOptions.fromOptionValue("items"); + assertNotNull(datasetTypeOptions); + assertThat(datasetTypeOptions.toString(), equalTo("ITEMS")); + } + + @Test + void fromOptionValue_interactions_test() { + DatasetTypeOptions datasetTypeOptions = DatasetTypeOptions.fromOptionValue("interactions"); + assertNotNull(datasetTypeOptions); + assertThat(datasetTypeOptions.toString(), equalTo("INTERACTIONS")); + } +} diff --git a/settings.gradle b/settings.gradle index 18ccd4dc7b..63e7ad2a9f 100644 --- a/settings.gradle +++ b/settings.gradle @@ -170,6 +170,7 @@ include 'data-prepper-plugins:buffer-common' //include 'data-prepper-plugins:http-sink' //include 'data-prepper-plugins:sns-sink' //include 'data-prepper-plugins:prometheus-sink' +include 'data-prepper-plugins:personalize-sink' include 'data-prepper-plugins:dissect-processor' include 'data-prepper-plugins:dynamodb-source' include 'data-prepper-plugins:decompress-processor' From 00cc2a576ecb7bb0e624e2ee77d7eaa28644c7c6 Mon Sep 17 00:00:00 2001 From: Qi Chen Date: Wed, 14 Aug 2024 11:38:44 -0500 Subject: [PATCH 24/25] FIX: include schema cli into release (#4833) MAINT: include schema cli into release Signed-off-by: George Chen --- build-resources.gradle | 5 +++-- data-prepper-plugin-schema-cli/build.gradle | 11 +++++++++++ 2 files changed, 14 insertions(+), 2 deletions(-) diff --git a/build-resources.gradle b/build-resources.gradle index b8a7bd5879..446f1d97b2 100644 --- a/build-resources.gradle +++ b/build-resources.gradle @@ -14,5 +14,6 @@ ext.coreProjects = [ project(':data-prepper-plugins'), project(':data-prepper-test-common'), project(':data-prepper-test-event'), - project(':data-prepper-plugin-framework') -] \ No newline at end of file + project(':data-prepper-plugin-framework'), + project(':data-prepper-plugin-schema-cli') +] diff --git a/data-prepper-plugin-schema-cli/build.gradle b/data-prepper-plugin-schema-cli/build.gradle index 2c2db93ee6..2108fad681 100644 --- a/data-prepper-plugin-schema-cli/build.gradle +++ b/data-prepper-plugin-schema-cli/build.gradle @@ -26,4 +26,15 @@ dependencies { } testImplementation(platform("org.junit:junit-bom:5.9.1")) testImplementation("org.junit.jupiter:junit-jupiter") +} + +jacocoTestCoverageVerification { + afterEvaluate { + classDirectories.from = files(classDirectories.files.collect { + fileTree(dir: it, exclude: [ + // Exclude main class + 'org/opensearch/dataprepper/schemas/DataPrepperPluginSchemaExecute.class' + ]) + }) + } } \ No newline at end of file From 1bfed0d11d8d0b436ed96867d715dcb6b3aa17f8 Mon Sep 17 00:00:00 2001 From: Krishna Kondaka <41027584+kkondaka@users.noreply.github.com> Date: Wed, 14 Aug 2024 11:49:12 -0700 Subject: [PATCH 25/25] Http chunking fixes (#4823) * dplive1.yaml Signed-off-by: Krishna Kondaka * Delete .github/workflows/static.yml Signed-off-by: Krishna Kondaka * Fix http message chunking bug Signed-off-by: Krishna Kondaka * Modified tests to test for chunks correctly Signed-off-by: Krishna Kondaka * Added comments Signed-off-by: Krishna Kondaka * Addressed offline review comments Signed-off-by: Krishna Kondaka * Addressed review comments Signed-off-by: Krishna Kondaka * Added tests Signed-off-by: Krishna Kondaka * Added tests Signed-off-by: Krishna Kondaka * Addressed review comments Signed-off-by: Krishna Kondaka --------- Signed-off-by: Krishna Kondaka Co-authored-by: Krishna Kondaka --- .../dataprepper/model/buffer/Buffer.java | 14 ++++ .../dataprepper/model/buffer/BufferTest.java | 6 ++ .../parser/MultiBufferDecorator.java | 6 ++ .../parser/MultiBufferDecoratorTest.java | 11 ++- .../dataprepper/http/codec/JsonCodec.java | 12 ++- .../dataprepper/http/codec/JsonCodecTest.java | 77 ++++++++++++++---- .../source/loghttp/LogHTTPService.java | 23 +++++- .../source/loghttp/LogHTTPServiceTest.java | 81 ++++++++++++++++++- .../plugins/kafka/buffer/KafkaBuffer.java | 6 ++ 9 files changed, 210 insertions(+), 26 deletions(-) diff --git a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/buffer/Buffer.java b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/buffer/Buffer.java index eaaa978230..874a9d350e 100644 --- a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/buffer/Buffer.java +++ b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/buffer/Buffer.java @@ -84,10 +84,24 @@ default boolean isByteBuffer() { return false; } + /** + * returns max request size of an entry in the buffer + * + * @return Optional value of the buffer's max request size + */ default Optional getMaxRequestSize() { return Optional.empty(); } + /** + * returns optimal request size of an entry in the buffer + * + * @return Optional value of the buffer's optimal request size + */ + default Optional getOptimalRequestSize() { + return Optional.empty(); + } + /** * Checks if the buffer enables acknowledgements for the pipeline * diff --git a/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/buffer/BufferTest.java b/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/buffer/BufferTest.java index 2236f0ba33..0d9aa51296 100644 --- a/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/buffer/BufferTest.java +++ b/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/buffer/BufferTest.java @@ -37,6 +37,12 @@ void testMaxRequestSize() { assertEquals(buffer.getMaxRequestSize(), Optional.empty()); } + @Test + void testOptimalRequestSize() { + final Buffer> buffer = createObjectUnderTest(); + assertEquals(buffer.getOptimalRequestSize(), Optional.empty()); + } + @Test void testShutdown() { final Buffer> buffer = createObjectUnderTest(); diff --git a/data-prepper-core/src/main/java/org/opensearch/dataprepper/parser/MultiBufferDecorator.java b/data-prepper-core/src/main/java/org/opensearch/dataprepper/parser/MultiBufferDecorator.java index eaa6c09491..76440c0c56 100644 --- a/data-prepper-core/src/main/java/org/opensearch/dataprepper/parser/MultiBufferDecorator.java +++ b/data-prepper-core/src/main/java/org/opensearch/dataprepper/parser/MultiBufferDecorator.java @@ -50,6 +50,12 @@ public Optional getMaxRequestSize() { return maxRequestSize.isPresent() ? Optional.of(maxRequestSize.getAsInt()) : Optional.empty(); } + @Override + public Optional getOptimalRequestSize() { + OptionalInt optimalRequestSize = allBuffers.stream().filter(b -> b.getOptimalRequestSize().isPresent()).mapToInt(b -> (Integer)b.getOptimalRequestSize().get()).min(); + return optimalRequestSize.isPresent() ? Optional.of(optimalRequestSize.getAsInt()) : Optional.empty(); + } + @Override public void shutdown() { allBuffers.forEach(Buffer::shutdown); diff --git a/data-prepper-core/src/test/java/org/opensearch/dataprepper/parser/MultiBufferDecoratorTest.java b/data-prepper-core/src/test/java/org/opensearch/dataprepper/parser/MultiBufferDecoratorTest.java index bae4cb763b..896268fcf4 100644 --- a/data-prepper-core/src/test/java/org/opensearch/dataprepper/parser/MultiBufferDecoratorTest.java +++ b/data-prepper-core/src/test/java/org/opensearch/dataprepper/parser/MultiBufferDecoratorTest.java @@ -231,6 +231,15 @@ void test_getMaxRequestSize() { assertThat(multiBufferDecorator.getMaxRequestSize(), equalTo(Optional.empty())); } + @Test + void test_getOptimalRequestSize() { + when(primaryBuffer.getOptimalRequestSize()).thenReturn(Optional.empty()); + when(secondaryBuffer.getOptimalRequestSize()).thenReturn(Optional.empty()); + + final MultiBufferDecorator multiBufferDecorator = createObjectUnderTest(2); + assertThat(multiBufferDecorator.getOptimalRequestSize(), equalTo(Optional.empty())); + } + private MultiBufferDecorator createObjectUnderTest(final int secondaryBufferCount) { final List secondaryBuffers = IntStream.range(0, secondaryBufferCount) .mapToObj(i -> secondaryBuffer) @@ -238,4 +247,4 @@ private MultiBufferDecorator createObjectUnderTest(final int secondaryBufferCoun return new MultiBufferDecorator(primaryBuffer, secondaryBuffers); } -} \ No newline at end of file +} diff --git a/data-prepper-plugins/http-source-common/src/main/java/org/opensearch/dataprepper/http/codec/JsonCodec.java b/data-prepper-plugins/http-source-common/src/main/java/org/opensearch/dataprepper/http/codec/JsonCodec.java index 4c0020a83e..9a74cdb767 100644 --- a/data-prepper-plugins/http-source-common/src/main/java/org/opensearch/dataprepper/http/codec/JsonCodec.java +++ b/data-prepper-plugins/http-source-common/src/main/java/org/opensearch/dataprepper/http/codec/JsonCodec.java @@ -47,17 +47,21 @@ public List> parse(HttpData httpData, int maxSize) throws IOExcepti List> jsonList = new ArrayList<>(); final List> logList = mapper.readValue(httpData.toInputStream(), LIST_OF_MAP_TYPE_REFERENCE); - int size = OVERHEAD_CHARACTERS.length(); List innerJsonList = new ArrayList<>(); - for (final Map log: logList) { + int size = OVERHEAD_CHARACTERS.length(); + for (Map log: logList) { final String recordString = mapper.writeValueAsString(log); - if (size + recordString.length() > maxSize) { + final int nextRecordLength = recordString.getBytes(Charset.defaultCharset()).length; + // It is possible that the first record is larger than maxSize, then + // innerJsonList size would be zero. + if (size + nextRecordLength > maxSize && !innerJsonList.isEmpty()) { jsonList.add(innerJsonList); innerJsonList = new ArrayList<>(); size = OVERHEAD_CHARACTERS.length(); } + // The following may result in a innerJsonList with larger than "maxSize" length recordString innerJsonList.add(recordString); - size += recordString.getBytes(Charset.defaultCharset()).length + COMMA_OVERHEAD_LENGTH; + size += nextRecordLength + COMMA_OVERHEAD_LENGTH; } if (size > OVERHEAD_CHARACTERS.length()) { jsonList.add(innerJsonList); diff --git a/data-prepper-plugins/http-source-common/src/test/java/org/opensearch/dataprepper/http/codec/JsonCodecTest.java b/data-prepper-plugins/http-source-common/src/test/java/org/opensearch/dataprepper/http/codec/JsonCodecTest.java index 8843d8d6e7..ec095e04e2 100644 --- a/data-prepper-plugins/http-source-common/src/test/java/org/opensearch/dataprepper/http/codec/JsonCodecTest.java +++ b/data-prepper-plugins/http-source-common/src/test/java/org/opensearch/dataprepper/http/codec/JsonCodecTest.java @@ -24,6 +24,7 @@ import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.greaterThanOrEqualTo; import static org.hamcrest.Matchers.lessThanOrEqualTo; +import static org.hamcrest.Matchers.greaterThanOrEqualTo; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.params.provider.Arguments.arguments; @@ -69,20 +70,28 @@ public void testParseSuccessWithMaxSize() throws IOException { @ParameterizedTest @ArgumentsSource(JsonArrayWithKnownFirstArgumentsProvider.class) public void parse_should_return_lists_smaller_than_provided_length( - final String inputJsonArray, final String knownFirstPart) throws IOException { + final String inputJsonArray, final String knownFirstPart, final int maxSize, final List> expectedChunks, final List exceedsMaxSize) throws IOException { final int knownSingleBodySize = knownFirstPart.getBytes(Charset.defaultCharset()).length; - final int maxSize = (knownSingleBodySize * 2) + 3; final List> chunkedBodies = objectUnderTest.parse(HttpData.ofUtf8(inputJsonArray), maxSize); assertThat(chunkedBodies, notNullValue()); - assertThat(chunkedBodies.size(), greaterThanOrEqualTo(1)); - final String firstReconstructed = chunkedBodies.get(0).stream().collect(Collectors.joining(",", "[", "]")); - assertThat(firstReconstructed.getBytes(Charset.defaultCharset()).length, - lessThanOrEqualTo(maxSize)); - - assertThat(chunkedBodies.get(0).size(), greaterThanOrEqualTo(1)); - assertThat(chunkedBodies.get(0).get(0), equalTo(knownFirstPart)); + assertThat(chunkedBodies.size(), equalTo(expectedChunks.size())); + + for (int i = 0; i < expectedChunks.size(); i++) { + final String reconstructed = chunkedBodies.get(i).stream().collect(Collectors.joining(",", "[", "]")); + if (exceedsMaxSize.get(i)) { + assertThat(reconstructed.getBytes(Charset.defaultCharset()).length, + greaterThanOrEqualTo(maxSize)); + } else { + assertThat(reconstructed.getBytes(Charset.defaultCharset()).length, + lessThanOrEqualTo(maxSize)); + } + + for (int j = 0; j < expectedChunks.get(i).size(); j++) { + assertThat(chunkedBodies.get(i).get(j), equalTo(expectedChunks.get(i).get(j))); + } + } } @Test @@ -103,14 +112,50 @@ public void testParseNonJsonFailure() { static class JsonArrayWithKnownFirstArgumentsProvider implements ArgumentsProvider { @Override public Stream provideArguments(ExtensionContext extensionContext) throws Exception { + // First test, all chunks smaller than maxSize, output has 3 lists, all smaller than maxSize + String chunk11 = "{\"ὊὊὊ1\":\"ὊὊὊ1\"}"; + String chunk12 = "{\"ὊὊὊ2\":\"ὊὊὊO2\"}"; + String chunk13 = "{\"a3\":\"b3\"}"; + String chunk14 = "{\"ὊὊὊ4\":\"ὊὊὊ4\"}"; + // Second test, all chunks smaller than maxSize, output has 2 lists, all smaller than maxSize + String chunk21 = "{\"aaa1\":\"aaa1\"}"; + String chunk22 = "{\"aaa2\":\"aaa2\"}"; + String chunk23 = "{\"a3\":\"b3\"}"; + String chunk24 = "{\"bbb4\":\"bbb4\"}"; + // Third test, all chunks larger than maxSize, output has 4 lists, all larger than maxSize + String chunk31 = "{\"ὊὊὊ1\":\"ὊὊὊ01\"}"; + String chunk32 = "{\"ὊὊὊ2\":\"ὊὊὊO2\"}"; + String chunk33 = "{\"ὊὊὊ3\":\"ὊὊὊO3\"}"; + String chunk34 = "{\"ὊὊὊ4\":\"ὊὊὊO4\"}"; + // Fourth test, only first chunk larger than maxSize, output has 3 lists, with first chunk larger than maxSize and others smaller + String chunk41 = "{\"aaaaaaaaaaa1\":\"aaaaaaaaaaa1\"}"; + String chunk42 = "{\"aaa2\":\"aaa2\"}"; + String chunk43 = "{\"a3\":\"b3\"}"; + String chunk44 = "{\"bbb4\":\"bbb4\"}"; + // Fifth test, only second chunk larger than maxSize, output has 3 lists, with second chunk larger than maxSize and others smaller + String chunk51 = "{\"aaa2\":\"aaa2\"}"; + String chunk52 = "{\"aaaaaaaaaaa1\":\"aaaaaaaaaaa1\"}"; + String chunk53 = "{\"a3\":\"b3\"}"; + String chunk54 = "{\"bb4\":\"bb4\"}"; + // Sixth test, only last chunk larger than maxSize, output has 3 lists, with last chunk larger than maxSize and others smaller + String chunk61 = "{\"aaa2\":\"aaa2\"}"; + String chunk62 = "{\"a3\":\"b3\"}"; + String chunk63 = "{\"bbb4\":\"bbb4\"}"; + String chunk64 = "{\"aaaaaaaaaaa1\":\"aaaaaaaaaaa1\"}"; + final int maxSize1 = chunk11.getBytes(Charset.defaultCharset()).length * 2 + 3; + final int maxSize2 = chunk21.getBytes(Charset.defaultCharset()).length * 2 + 3; + final int maxSize3 = chunk31.getBytes(Charset.defaultCharset()).length - 1; + final int maxSize4 = chunk42.getBytes(Charset.defaultCharset()).length * 2 + 3; + final int maxSize5 = chunk51.getBytes(Charset.defaultCharset()).length * 2 + 3; + final int maxSize6 = chunk61.getBytes(Charset.defaultCharset()).length * 2 + 3; return Stream.of( - arguments( - "[{\"ὊὊὊ1\":\"ὊὊὊ1\"}, {\"ὊὊὊ2\":\"ὊὊὊ2\"}, {\"a3\":\"b3\"}, {\"ὊὊὊ4\":\"ὊὊὊ4\"}]", - "{\"ὊὊὊ1\":\"ὊὊὊ1\"}"), - arguments( - "[{\"aaa1\":\"aaa1\"}, {\"aaa2\":\"aaa2\"}, {\"a3\":\"b3\"}, {\"bbb4\":\"bbb4\"}]", - "{\"aaa1\":\"aaa1\"}") + arguments("["+chunk11+","+chunk12+","+chunk13+","+chunk14+"]", chunk11, maxSize1, List.of(List.of(chunk11), List.of(chunk12, chunk13), List.of(chunk14)), List.of(false, false, false)), + arguments("["+chunk21+","+chunk22+","+chunk23+","+chunk24+"]", chunk21, maxSize2, List.of(List.of(chunk21, chunk22), List.of(chunk23, chunk24)), List.of(false, false)), + arguments("["+chunk31+","+chunk32+","+chunk33+","+chunk34+"]", chunk31, maxSize3, List.of(List.of(chunk31), List.of(chunk32), List.of(chunk33), List.of(chunk34)), List.of(true, true, true, true)), + arguments("["+chunk41+","+chunk42+","+chunk43+","+chunk44+"]", chunk41, maxSize4, List.of(List.of(chunk41), List.of(chunk42, chunk43), List.of(chunk44)), List.of(true, false, false)), + arguments("["+chunk51+","+chunk52+","+chunk53+","+chunk54+"]", chunk51, maxSize5, List.of(List.of(chunk51), List.of(chunk52), List.of(chunk53,chunk54)), List.of(false, true, false)), + arguments("["+chunk61+","+chunk62+","+chunk63+","+chunk64+"]", chunk61, maxSize6, List.of(List.of(chunk61,chunk62), List.of(chunk63), List.of(chunk64)), List.of(false, false, true)) ); } } -} \ No newline at end of file +} diff --git a/data-prepper-plugins/http-source/src/main/java/org/opensearch/dataprepper/plugins/source/loghttp/LogHTTPService.java b/data-prepper-plugins/http-source/src/main/java/org/opensearch/dataprepper/plugins/source/loghttp/LogHTTPService.java index 8384315aa4..1eac99ed92 100644 --- a/data-prepper-plugins/http-source/src/main/java/org/opensearch/dataprepper/plugins/source/loghttp/LogHTTPService.java +++ b/data-prepper-plugins/http-source/src/main/java/org/opensearch/dataprepper/plugins/source/loghttp/LogHTTPService.java @@ -39,6 +39,8 @@ public class LogHTTPService { private static final int SERIALIZATION_OVERHEAD = 1024; public static final String REQUESTS_RECEIVED = "requestsReceived"; public static final String SUCCESS_REQUESTS = "successRequests"; + public static final String REQUESTS_OVER_OPTIMAL_SIZE = "requestsOverOptimalSize"; + public static final String REQUESTS_OVER_MAXIMUM_SIZE = "requestsOverMaximumSize"; public static final String PAYLOAD_SIZE = "payloadSize"; public static final String REQUEST_PROCESS_DURATION = "requestProcessDuration"; @@ -50,9 +52,12 @@ public class LogHTTPService { private final int bufferWriteTimeoutInMillis; private final Counter requestsReceivedCounter; private final Counter successRequestsCounter; + private final Counter requestsOverOptimalSizeCounter; + private final Counter requestsOverMaximumSizeCounter; private final DistributionSummary payloadSizeSummary; private final Timer requestProcessDuration; private Integer maxRequestLength; + private Integer optimalRequestLength; public LogHTTPService(final int bufferWriteTimeoutInMillis, final Buffer> buffer, @@ -61,8 +66,11 @@ public LogHTTPService(final int bufferWriteTimeoutInMillis, this.buffer = buffer; this.bufferWriteTimeoutInMillis = bufferWriteTimeoutInMillis; this.maxRequestLength = buffer.getMaxRequestSize().isPresent() ? buffer.getMaxRequestSize().get(): null; + this.optimalRequestLength = buffer.getOptimalRequestSize().isPresent() ? buffer.getOptimalRequestSize().get(): null; requestsReceivedCounter = pluginMetrics.counter(REQUESTS_RECEIVED); successRequestsCounter = pluginMetrics.counter(SUCCESS_REQUESTS); + requestsOverOptimalSizeCounter = pluginMetrics.counter(REQUESTS_OVER_OPTIMAL_SIZE); + requestsOverMaximumSizeCounter = pluginMetrics.counter(REQUESTS_OVER_MAXIMUM_SIZE); payloadSizeSummary = pluginMetrics.summary(PAYLOAD_SIZE); requestProcessDuration = pluginMetrics.timer(REQUEST_PROCESS_DURATION); } @@ -91,24 +99,33 @@ private void sendJsonList(List jsonList) throws Exception { } sb.append("]"); if (sb.toString().getBytes().length > maxRequestLength) { + requestsOverMaximumSizeCounter.increment(); throw new RuntimeException("Request length "+ sb.toString().getBytes().length + " exceeds maxRequestLength "+ maxRequestLength); + } else if (sb.toString().getBytes().length > optimalRequestLength) { + requestsOverOptimalSizeCounter.increment(); } buffer.writeBytes(sb.toString().getBytes(), key, bufferWriteTimeoutInMillis); } - private HttpResponse processRequest(final AggregatedHttpRequest aggregatedHttpRequest) throws Exception { + HttpResponse processRequest(final AggregatedHttpRequest aggregatedHttpRequest) throws Exception { final HttpData content = aggregatedHttpRequest.content(); List> jsonList; + boolean isJsonListSplit = false; try { - jsonList = (maxRequestLength == null) ? jsonCodec.parse(content) : jsonCodec.parse(content, maxRequestLength - SERIALIZATION_OVERHEAD); + if (buffer.isByteBuffer() && maxRequestLength != null && optimalRequestLength != null) { + jsonList = jsonCodec.parse(content, optimalRequestLength - SERIALIZATION_OVERHEAD); + isJsonListSplit = true; + } else { + jsonList = jsonCodec.parse(content); + } } catch (IOException e) { LOG.error("Failed to parse the request of size {} due to: {}", content.length(), e.getMessage()); throw new IOException("Bad request data format. Needs to be json array.", e.getCause()); } try { if (buffer.isByteBuffer()) { - if (maxRequestLength != null && content.array().length > maxRequestLength) { + if (isJsonListSplit && content.array().length > optimalRequestLength) { for (final List innerJsonList: jsonList) { sendJsonList(innerJsonList); } diff --git a/data-prepper-plugins/http-source/src/test/java/org/opensearch/dataprepper/plugins/source/loghttp/LogHTTPServiceTest.java b/data-prepper-plugins/http-source/src/test/java/org/opensearch/dataprepper/plugins/source/loghttp/LogHTTPServiceTest.java index bd289de769..2e9b802f32 100644 --- a/data-prepper-plugins/http-source/src/test/java/org/opensearch/dataprepper/plugins/source/loghttp/LogHTTPServiceTest.java +++ b/data-prepper-plugins/http-source/src/test/java/org/opensearch/dataprepper/plugins/source/loghttp/LogHTTPServiceTest.java @@ -40,10 +40,14 @@ import java.util.Collections; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.UUID; import java.util.concurrent.Callable; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeoutException; +import java.nio.charset.StandardCharsets; +import java.io.ByteArrayInputStream; +import java.io.InputStream; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -51,6 +55,11 @@ import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; +import static org.mockito.Mockito.lenient; +import static org.mockito.Mockito.mock; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.ArgumentMatchers.isNull; @ExtendWith(MockitoExtension.class) class LogHTTPServiceTest { @@ -78,14 +87,17 @@ class LogHTTPServiceTest { private LogHTTPService logHTTPService; + @Mock + private Buffer> byteBuffer; + @BeforeEach public void setUp() throws Exception { when(pluginMetrics.counter(LogHTTPService.REQUESTS_RECEIVED)).thenReturn(requestsReceivedCounter); when(pluginMetrics.counter(LogHTTPService.SUCCESS_REQUESTS)).thenReturn(successRequestsCounter); when(pluginMetrics.summary(LogHTTPService.PAYLOAD_SIZE)).thenReturn(payloadSizeSummary); when(pluginMetrics.timer(LogHTTPService.REQUEST_PROCESS_DURATION)).thenReturn(requestProcessDuration); - when(serviceRequestContext.isTimedOut()).thenReturn(false); - when(requestProcessDuration.recordCallable(ArgumentMatchers.>any())).thenAnswer( + lenient().when(serviceRequestContext.isTimedOut()).thenReturn(false); + lenient().when(requestProcessDuration.recordCallable(ArgumentMatchers.>any())).thenAnswer( (Answer) invocation -> { final Object[] args = invocation.getArguments(); @SuppressWarnings("unchecked") @@ -170,6 +182,71 @@ public void testHTTPRequestTimeout() throws Exception { verify(requestProcessDuration, times(2)).recordCallable(ArgumentMatchers.>any()); } + @Test + public void testChunking() throws Exception { + byteBuffer = mock(Buffer.class); + when(byteBuffer.isByteBuffer()).thenReturn(true); + when(byteBuffer.getMaxRequestSize()).thenReturn(Optional.of(4*1024*1024)); + when(byteBuffer.getOptimalRequestSize()).thenReturn(Optional.of(1024*1024)); + + logHTTPService = new LogHTTPService(TEST_TIMEOUT_IN_MILLIS, byteBuffer, null, pluginMetrics); + AggregatedHttpRequest aggregatedHttpRequest = mock(AggregatedHttpRequest.class); + HttpData httpData = mock(HttpData.class); + // Test small json data + String testString ="{\"key1\":\"value1\"},{\"key2\":\"value2\"},{\"key3\":\"value3\"},{\"key4\":\"value4\"},{\"key5\":\"value5\"}"; + String exampleString = "[ " + testString + "]"; + when(httpData.array()).thenReturn(exampleString.getBytes()); + InputStream stream = new ByteArrayInputStream(exampleString.getBytes(StandardCharsets.UTF_8)); + when(httpData.toInputStream()).thenReturn(stream); + + when(aggregatedHttpRequest.content()).thenReturn(httpData); + logHTTPService.processRequest(aggregatedHttpRequest); + verify(byteBuffer, times(1)).writeBytes(any(), (String)isNull(), any(Integer.class)); + + // Test more than 1MB json data + StringBuilder sb = new StringBuilder(1024*1024+10240); + for (int i =0; i < 12500; i++) { + sb.append(testString); + if (i+1 != 12500) + sb.append(","); + } + String largeTestString = sb.toString(); + exampleString = "[" + largeTestString + "]"; + when(httpData.array()).thenReturn(exampleString.getBytes()); + stream = new ByteArrayInputStream(exampleString.getBytes(StandardCharsets.UTF_8)); + when(httpData.toInputStream()).thenReturn(stream); + + when(aggregatedHttpRequest.content()).thenReturn(httpData); + logHTTPService.processRequest(aggregatedHttpRequest); + verify(byteBuffer, times(2)).writeBytes(any(), anyString(), any(Integer.class)); + // Test more than 4MB json data + exampleString = "[" + largeTestString + "," + largeTestString + ","+largeTestString +","+largeTestString+"]"; + when(httpData.array()).thenReturn(exampleString.getBytes()); + stream = new ByteArrayInputStream(exampleString.getBytes(StandardCharsets.UTF_8)); + when(httpData.toInputStream()).thenReturn(stream); + + when(aggregatedHttpRequest.content()).thenReturn(httpData); + logHTTPService.processRequest(aggregatedHttpRequest); + verify(byteBuffer, times(7)).writeBytes(any(), anyString(), any(Integer.class)); + + // Test more than 4MB single json object, should throw exception + int length = 3*1024*1024; + sb = new StringBuilder(length); + for (int i = 0; i < length; i++) { + sb.append('A'); + } + String value = sb.toString(); + exampleString = "[{\"key\":\""+value+"\"}]"; + + when(httpData.array()).thenReturn(exampleString.getBytes()); + stream = new ByteArrayInputStream(exampleString.getBytes(StandardCharsets.UTF_8)); + when(httpData.toInputStream()).thenReturn(stream); + + when(aggregatedHttpRequest.content()).thenReturn(httpData); + assertThrows(RuntimeException.class, () -> logHTTPService.processRequest(aggregatedHttpRequest)); + + } + private AggregatedHttpRequest generateRandomValidHTTPRequest(int numJson) throws JsonProcessingException, ExecutionException, InterruptedException { RequestHeaders requestHeaders = RequestHeaders.builder() diff --git a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/buffer/KafkaBuffer.java b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/buffer/KafkaBuffer.java index 02e24709eb..f8ec9c4d91 100644 --- a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/buffer/KafkaBuffer.java +++ b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/buffer/KafkaBuffer.java @@ -32,6 +32,7 @@ import org.opensearch.dataprepper.plugins.kafka.consumer.KafkaCustomConsumerFactory; import org.opensearch.dataprepper.plugins.kafka.producer.KafkaCustomProducer; import org.opensearch.dataprepper.plugins.kafka.producer.KafkaCustomProducerFactory; +import org.opensearch.dataprepper.plugins.kafka.configuration.KafkaProducerProperties; import org.opensearch.dataprepper.plugins.kafka.service.TopicServiceFactory; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -96,6 +97,11 @@ public Optional getMaxRequestSize() { return Optional.of(producer.getMaxRequestSize()); } + @Override + public Optional getOptimalRequestSize() { + return Optional.of(KafkaProducerProperties.DEFAULT_MAX_REQUEST_SIZE); + } + @Override public void writeBytes(final byte[] bytes, final String key, int timeoutInMillis) throws Exception { try {