diff --git a/build.gradle b/build.gradle index d1a3aa8d48..006e641354 100644 --- a/build.gradle +++ b/build.gradle @@ -177,6 +177,8 @@ subprojects { } } else if (details.requested.group == 'log4j' && details.requested.name == 'log4j') { details.useTarget group: 'org.apache.logging.log4j', name: 'log4j-1.2-api', version: '2.17.1' + } else if (details.requested.group == 'org.xerial.snappy' && details.requested.name == 'snappy-java') { + details.useTarget group: 'org.xerial.snappy', name: 'snappy-java', version: '1.1.10.1' } } } diff --git a/data-prepper-api/build.gradle b/data-prepper-api/build.gradle index 6234c99214..42032769b9 100644 --- a/data-prepper-api/build.gradle +++ b/data-prepper-api/build.gradle @@ -14,7 +14,7 @@ dependencies { testImplementation testLibs.junit.vintage testImplementation project(':data-prepper-test-common') testImplementation 'org.skyscreamer:jsonassert:1.5.1' - testImplementation 'commons-io:commons-io:2.11.0' + testImplementation 'commons-io:commons-io:2.13.0' } jacocoTestCoverageVerification { diff --git a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/codec/OutputCodec.java b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/codec/OutputCodec.java index 06056f5608..39c7a1490f 100644 --- a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/codec/OutputCodec.java +++ b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/codec/OutputCodec.java @@ -5,14 +5,21 @@ package org.opensearch.dataprepper.model.codec; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.log.JacksonLog; import org.opensearch.dataprepper.model.sink.Sink; import java.io.IOException; import java.io.OutputStream; +import java.util.Map; public interface OutputCodec { + static final ObjectMapper objectMapper = new ObjectMapper(); + /** * this method get called from {@link Sink} to do initial wrapping in {@link OutputStream} * Implementors should do initial wrapping according to the implementation @@ -26,11 +33,12 @@ public interface OutputCodec { * this method get called from {@link Sink} to write event in {@link OutputStream} * Implementors should do get data from event and write to the {@link OutputStream} * - * @param event event Record event - * @param outputStream outputStream param to hold the event data + * @param event event Record event + * @param outputStream outputStream param to hold the event data + * @param tagsTargetKey to add tags to the record * @throws IOException throws IOException when not able to write data to {@link OutputStream} */ - void writeEvent(Event event, OutputStream outputStream) throws IOException; + void writeEvent(Event event, OutputStream outputStream, String tagsTargetKey) throws IOException; /** * this method get called from {@link Sink} to do final wrapping in {@link OutputStream} @@ -47,4 +55,11 @@ public interface OutputCodec { * @return String */ String getExtension(); + + default Event addTagsToEvent(Event event, String tagsTargetKey) throws JsonProcessingException { + String eventJsonString = event.jsonBuilder().includeTags(tagsTargetKey).toJsonString(); + Map eventData = objectMapper.readValue(eventJsonString, new TypeReference<>() { + }); + return JacksonLog.builder().withData(eventData).build(); + } } diff --git a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/JacksonEvent.java b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/JacksonEvent.java index 047e4c4a07..8b4b2a729f 100644 --- a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/JacksonEvent.java +++ b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/JacksonEvent.java @@ -373,19 +373,9 @@ private String trimKey(final String key) { } private boolean isValidKey(final String key) { - char previous = ' '; - char next = ' '; for (int i = 0; i < key.length(); i++) { char c = key.charAt(i); - if (i < key.length() - 1) { - next = key.charAt(i + 1); - } - - if ((i == 0 || i == key.length() - 1 || previous == '/' || next == '/') && (c == '_' || c == '.' || c == '-')) { - return false; - } - if (!(c >= 48 && c <= 57 || c >= 65 && c <= 90 || c >= 97 && c <= 122 @@ -397,7 +387,6 @@ private boolean isValidKey(final String key) { return false; } - previous = c; } return true; } diff --git a/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/codec/OutputCodecTest.java b/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/codec/OutputCodecTest.java new file mode 100644 index 0000000000..461907d9a4 --- /dev/null +++ b/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/codec/OutputCodecTest.java @@ -0,0 +1,69 @@ +package org.opensearch.dataprepper.model.codec; + +import com.fasterxml.jackson.core.JsonProcessingException; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.opensearch.dataprepper.model.event.DefaultEventMetadata; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.event.EventMetadata; +import org.opensearch.dataprepper.model.event.EventType; +import org.opensearch.dataprepper.model.event.JacksonEvent; +import org.opensearch.dataprepper.model.log.JacksonLog; + +import java.io.IOException; +import java.io.OutputStream; +import java.util.Arrays; +import java.util.LinkedHashMap; +import java.util.Map; +import java.util.Set; +import java.util.UUID; + +import static org.junit.Assert.assertNotEquals; + +public class OutputCodecTest { + + @BeforeEach + public void setUp() { + } + + @Test + public void testWriteMetrics() throws JsonProcessingException { + OutputCodec outputCodec = new OutputCodec() { + @Override + public void start(OutputStream outputStream) throws IOException { + } + + @Override + public void writeEvent(Event event, OutputStream outputStream, String tagsTargetKey) throws IOException { + } + + @Override + public void complete(OutputStream outputStream) throws IOException { + } + + @Override + public String getExtension() { + return null; + } + }; + + final Set testTags = Set.of("tag1"); + final EventMetadata defaultEventMetadata = DefaultEventMetadata.builder(). + withEventType(EventType.LOG.toString()). + withTags(testTags).build(); + Map json = generateJson(); + final JacksonEvent event = JacksonLog.builder().withData(json).withEventMetadata(defaultEventMetadata).build(); + Event tagsToEvent = outputCodec.addTagsToEvent(event, "Tag"); + assertNotEquals(event.toJsonString(), tagsToEvent.toJsonString()); + } + + private static Map generateJson() { + final Map jsonObject = new LinkedHashMap<>(); + for (int i = 0; i < 2; i++) { + jsonObject.put(UUID.randomUUID().toString(), UUID.randomUUID().toString()); + } + jsonObject.put(UUID.randomUUID().toString(), Arrays.asList(UUID.randomUUID().toString(), + UUID.randomUUID().toString(), UUID.randomUUID().toString())); + return jsonObject; + } +} diff --git a/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/event/JacksonEventTest.java b/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/event/JacksonEventTest.java index 9de73495f9..92b181ac8c 100644 --- a/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/event/JacksonEventTest.java +++ b/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/event/JacksonEventTest.java @@ -323,9 +323,8 @@ public void testIsValueAList_withNull() { } @ParameterizedTest - @ValueSource(strings = {"", "withSpecialChars*$%", "-withPrefixDash", "\\-withEscapeChars", "\\\\/withMultipleEscapeChars", - "withDashSuffix-", "withDashSuffix-/nestedKey", "withDashPrefix/-nestedKey", "_withUnderscorePrefix", "withUnderscoreSuffix_", - ".withDotPrefix", "withDotSuffix.", "with,Comma", "with:Colon", "with[Bracket", "with|Brace"}) + @ValueSource(strings = {"", "withSpecialChars*$%", "\\-withEscapeChars", "\\\\/withMultipleEscapeChars", + "with,Comma", "with:Colon", "with[Bracket", "with|Brace"}) void testKey_withInvalidKey_throwsIllegalArgumentException(final String invalidKey) { assertThrowsForKeyCheck(IllegalArgumentException.class, invalidKey); } diff --git a/data-prepper-plugins/armeria-common/src/main/java/org/opensearch/dataprepper/armeria/authentication/ArmeriaHttpAuthenticationProvider.java b/data-prepper-plugins/armeria-common/src/main/java/org/opensearch/dataprepper/armeria/authentication/ArmeriaHttpAuthenticationProvider.java index 489ddcd125..dc1e157ca1 100644 --- a/data-prepper-plugins/armeria-common/src/main/java/org/opensearch/dataprepper/armeria/authentication/ArmeriaHttpAuthenticationProvider.java +++ b/data-prepper-plugins/armeria-common/src/main/java/org/opensearch/dataprepper/armeria/authentication/ArmeriaHttpAuthenticationProvider.java @@ -30,6 +30,7 @@ public interface ArmeriaHttpAuthenticationProvider { * Gets an authentication decorator to an Armeria {@link ServerBuilder}. * * @since 2.0 + * @return returns authentication decorator */ default Optional> getAuthenticationDecorator() { return Optional.empty(); diff --git a/data-prepper-plugins/armeria-common/src/main/java/org/opensearch/dataprepper/armeria/authentication/GrpcAuthenticationProvider.java b/data-prepper-plugins/armeria-common/src/main/java/org/opensearch/dataprepper/armeria/authentication/GrpcAuthenticationProvider.java index e933caf7c3..cde38b31fe 100644 --- a/data-prepper-plugins/armeria-common/src/main/java/org/opensearch/dataprepper/armeria/authentication/GrpcAuthenticationProvider.java +++ b/data-prepper-plugins/armeria-common/src/main/java/org/opensearch/dataprepper/armeria/authentication/GrpcAuthenticationProvider.java @@ -29,6 +29,7 @@ public interface GrpcAuthenticationProvider { /** * Returns a {@link ServerInterceptor} that does authentication * @since 1.2 + * @return returns authentication interceptor */ ServerInterceptor getAuthenticationInterceptor(); @@ -36,8 +37,9 @@ public interface GrpcAuthenticationProvider { * Allows implementors to provide an {@link HttpService} to either intercept the HTTP request prior to validation, * or to perform validation on the HTTP request. This may be optional, in which case it is not used. * @since 1.5 + * @return returns http authentication service */ default Optional> getHttpAuthenticationService() { return Optional.empty(); } -} \ No newline at end of file +} diff --git a/data-prepper-plugins/avro-codecs/src/main/java/org/opensearch/dataprepper/plugins/codec/avro/AvroOutputCodec.java b/data-prepper-plugins/avro-codecs/src/main/java/org/opensearch/dataprepper/plugins/codec/avro/AvroOutputCodec.java index 081c6dd0e2..8129594162 100644 --- a/data-prepper-plugins/avro-codecs/src/main/java/org/opensearch/dataprepper/plugins/codec/avro/AvroOutputCodec.java +++ b/data-prepper-plugins/avro-codecs/src/main/java/org/opensearch/dataprepper/plugins/codec/avro/AvroOutputCodec.java @@ -31,7 +31,7 @@ public void start(final OutputStream outputStream) throws IOException { } @Override - public void writeEvent(final Event event, final OutputStream outputStream) throws IOException { + public void writeEvent(final Event event, final OutputStream outputStream,final String tagsTargetKey) throws IOException { // TODO: write event data to the outputstream } diff --git a/data-prepper-plugins/cloudwatch-logs/build.gradle b/data-prepper-plugins/cloudwatch-logs/build.gradle new file mode 100644 index 0000000000..2d00cbd2d0 --- /dev/null +++ b/data-prepper-plugins/cloudwatch-logs/build.gradle @@ -0,0 +1,38 @@ +plugins { + id 'java' + id 'java-library' +} + +repositories { + mavenCentral() +} + +dependencies { + api project(':data-prepper-api') + implementation project(':data-prepper-plugins:aws-plugin-api') + implementation project(path: ':data-prepper-plugins:common') + testImplementation 'org.junit.jupiter:junit-jupiter' + implementation 'com.fasterxml.jackson.core:jackson-core' + implementation 'com.fasterxml.jackson.core:jackson-databind' + implementation 'com.fasterxml.jackson.dataformat:jackson-dataformat-yaml' + implementation 'software.amazon.awssdk:cloudwatch' + implementation 'software.amazon.awssdk:cloudwatchlogs' + implementation 'org.apache.commons:commons-lang3:3.12.0' + testImplementation project(path: ':data-prepper-test-common') + testImplementation project(path: ':data-prepper-test-common') +} + +jacocoTestCoverageVerification { + dependsOn jacocoTestReport + violationRules { + rule { //in addition to core projects rule + limit { + minimum = 0.90 + } + } + } +} + +test { + useJUnitPlatform() +} \ No newline at end of file diff --git a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/config/AwsConfig.java b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/config/AwsConfig.java new file mode 100644 index 0000000000..6a2a85557e --- /dev/null +++ b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/config/AwsConfig.java @@ -0,0 +1,48 @@ +package org.opensearch.dataprepper.plugins.sink.config; + +import com.fasterxml.jackson.annotation.JsonProperty; +import jakarta.validation.constraints.Size; +import software.amazon.awssdk.regions.Region; + +import java.util.Map; + +/** + * AwsConfig is based on the S3-Sink AwsAuthenticationOptions + * where the configuration allows the sink to fetch Aws credentials + * and resources. + */ +public class AwsConfig { + public static int DEFAULT_CONNECTION_ATTEMPTS = 5; + + @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_header_overrides") + @Size(max = 5, message = "sts_header_overrides supports a maximum of 5 headers to override") + private Map awsStsHeaderOverrides; + + @JsonProperty("sts_external_id") + @Size(min = 2, max = 1224, message = "awsStsExternalId length should be between 2 and 1224 characters") + private String awsStsExternalId; + + public Region getAwsRegion() { + return awsRegion != null ? Region.of(awsRegion) : null; + } + + public String getAwsStsRoleArn() { + return awsStsRoleArn; + } + + public String getAwsStsExternalId() { + return awsStsExternalId; + } + + public Map getAwsStsHeaderOverrides() { + return awsStsHeaderOverrides; + } +} diff --git a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/config/CwlSinkConfig.java b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/config/CwlSinkConfig.java new file mode 100644 index 0000000000..230512bde5 --- /dev/null +++ b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/config/CwlSinkConfig.java @@ -0,0 +1,52 @@ +package org.opensearch.dataprepper.plugins.sink.config; + +import com.fasterxml.jackson.annotation.JsonProperty; +import jakarta.validation.Valid; +import jakarta.validation.constraints.NotEmpty; +import jakarta.validation.constraints.NotNull; + +public class CwlSinkConfig { + public static final String DEFAULT_BUFFER_TYPE = "in_memory"; + + @JsonProperty("aws") + @NotNull + @Valid + private AwsConfig awsConfig; + + @JsonProperty("threshold") + @NotNull + private ThresholdConfig thresholdConfig; + + @JsonProperty("buffer_type") + private String bufferType = DEFAULT_BUFFER_TYPE; + + @JsonProperty("log_group") + @NotEmpty + @NotNull + private String logGroup; + + @JsonProperty("log_stream") + @NotEmpty + @NotNull + private String logStream; + + public AwsConfig getAwsConfig() { + return awsConfig; + } + + public ThresholdConfig getThresholdConfig() { + return thresholdConfig; + } + + public String getBufferType() { + return bufferType; + } + + public String getLogGroup() { + return logGroup; + } + + public String getLogStream() { + return logStream; + } +} diff --git a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/config/ThresholdConfig.java b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/config/ThresholdConfig.java new file mode 100644 index 0000000000..77571a2c29 --- /dev/null +++ b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/config/ThresholdConfig.java @@ -0,0 +1,66 @@ +package org.opensearch.dataprepper.plugins.sink.config; + +import com.fasterxml.jackson.annotation.JsonProperty; +import jakarta.validation.constraints.Size; + +/** + * The threshold config holds the different configurations for + * buffer restrictions, retransmission restrictions and timeout + * restrictions. + */ +public class ThresholdConfig { + public static final int DEFAULT_BATCH_SIZE = 100; + public static final int DEFAULT_EVENT_SIZE = 50; + public static final int DEFAULT_SIZE_OF_REQUEST = 524288; + public static final int DEFAULT_RETRY_COUNT = 5; + public static final int DEFAULT_LOG_SEND_INTERVAL_TIME = 60; + public static final int DEFAULT_BACKOFF_TIME = 5000; + + @JsonProperty("batch_size") + @Size(min = 1, max = 10000, message = "batch_size amount should be between 1 to 10000") + private int batchSize = DEFAULT_BATCH_SIZE; + + @JsonProperty("max_event_size") + @Size(min = 1, max = 256, message = "max_event_size amount should be between 1 to 256 kilobytes") + private int maxEventSize = DEFAULT_EVENT_SIZE; + + @JsonProperty("max_request_size") + @Size(min = 1, max = 1048576, message = "max_batch_request_size amount should be between 1 and 1048576 bytes") + private int maxRequestSize = DEFAULT_SIZE_OF_REQUEST; + + @JsonProperty("retry_count") + @Size(min = 1, max = 15, message = "retry_count amount should be between 1 and 15") + private int retryCount = DEFAULT_RETRY_COUNT; + + @JsonProperty("log_send_interval") + @Size(min = 5, max = 300, message = "log_send_interval amount should be between 5 and 300 seconds") + private int logSendInterval = DEFAULT_LOG_SEND_INTERVAL_TIME; + + @JsonProperty("back_off_time") + @Size(min = 500, max = 1000, message = "back_off_time amount should be between 500 and 1000 milliseconds") + private int backOffTime = DEFAULT_BACKOFF_TIME; + + public int getBatchSize() { + return batchSize; + } + + public int getMaxEventSize() { + return maxEventSize; + } + + public int getMaxRequestSize() { + return maxRequestSize; + } + + public int getRetryCount() { + return retryCount; + } + + public int getLogSendInterval() { + return logSendInterval; + } + + public int getBackOffTime() { + return backOffTime; + } +} diff --git a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/configuration/AwsConfigTest.java b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/configuration/AwsConfigTest.java new file mode 100644 index 0000000000..8ab03d575d --- /dev/null +++ b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/configuration/AwsConfigTest.java @@ -0,0 +1,87 @@ +package org.opensearch.dataprepper.plugins.sink.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 org.opensearch.dataprepper.plugins.sink.config.AwsConfig; +import software.amazon.awssdk.regions.Region; + +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; + +public class AwsConfigTest { + 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 Region expectedRegionObject = Region.of(regionString); + final Map jsonMap = Map.of("region", regionString); + final AwsConfig objectUnderTest = objectMapper.convertValue(jsonMap, AwsConfig.class); + assertThat(objectUnderTest.getAwsRegion(), equalTo(expectedRegionObject)); + } + + @Test + void getAwsRegion_returns_null_when_region_is_null() { + final Map jsonMap = Collections.emptyMap(); + final AwsConfig objectUnderTest = objectMapper.convertValue(jsonMap, AwsConfig.class); + assertThat(objectUnderTest.getAwsRegion(), nullValue()); + } + + @Test + void getAwsStsRoleArn_returns_value_from_deserialized_JSON() { + final String stsRoleArn = UUID.randomUUID().toString(); + final Map jsonMap = Map.of("sts_role_arn", stsRoleArn); + final AwsConfig objectUnderTest = objectMapper.convertValue(jsonMap, AwsConfig.class); + assertThat(objectUnderTest.getAwsStsRoleArn(), equalTo(stsRoleArn)); + } + + @Test + void getAwsStsRoleArn_returns_null_if_not_in_JSON() { + final Map jsonMap = Collections.emptyMap(); + final AwsConfig objectUnderTest = objectMapper.convertValue(jsonMap, AwsConfig.class); + assertThat(objectUnderTest.getAwsStsRoleArn(), nullValue()); + } + + @Test + void getAwsStsExternalId_returns_value_from_deserialized_JSON() { + final String stsExternalId = UUID.randomUUID().toString(); + final Map jsonMap = Map.of("sts_external_id", stsExternalId); + final AwsConfig objectUnderTest = objectMapper.convertValue(jsonMap, AwsConfig.class); + assertThat(objectUnderTest.getAwsStsExternalId(), equalTo(stsExternalId)); + } + + @Test + void getAwsStsExternalId_returns_null_if_not_in_JSON() { + final Map jsonMap = Collections.emptyMap(); + final AwsConfig objectUnderTest = objectMapper.convertValue(jsonMap, AwsConfig.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 AwsConfig objectUnderTest = objectMapper.convertValue(jsonMap, AwsConfig.class); + assertThat(objectUnderTest.getAwsStsHeaderOverrides(), equalTo(stsHeaderOverrides)); + } + + @Test + void getAwsStsHeaderOverrides_returns_null_if_not_in_JSON() { + final Map jsonMap = Collections.emptyMap(); + final AwsConfig objectUnderTest = objectMapper.convertValue(jsonMap, AwsConfig.class); + assertThat(objectUnderTest.getAwsStsHeaderOverrides(), nullValue()); + } +} diff --git a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/configuration/CwlSinkConfigTest.java b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/configuration/CwlSinkConfigTest.java new file mode 100644 index 0000000000..9842a333ee --- /dev/null +++ b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/configuration/CwlSinkConfigTest.java @@ -0,0 +1,63 @@ +package org.opensearch.dataprepper.plugins.sink.configuration; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.opensearch.dataprepper.plugins.sink.config.AwsConfig; +import org.opensearch.dataprepper.plugins.sink.config.CwlSinkConfig; +import org.opensearch.dataprepper.plugins.sink.config.ThresholdConfig; +import org.opensearch.dataprepper.test.helper.ReflectivelySetField; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.equalTo; + +public class CwlSinkConfigTest { + private CwlSinkConfig cwlSinkConfig; + private AwsConfig awsConfig; + private ThresholdConfig thresholdConfig; + private final String LOG_GROUP = "testLogGroup"; + private final String LOG_STREAM = "testLogStream"; + + @BeforeEach + void setUp() { + cwlSinkConfig = new CwlSinkConfig(); + awsConfig = new AwsConfig(); + thresholdConfig = new ThresholdConfig(); + } + + @Test + void check_null_auth_config_test() { + assertThat(new CwlSinkConfig().getAwsConfig(), equalTo(null)); + } + + @Test + void check_default_buffer_type_test() { + assertThat(new CwlSinkConfig().getBufferType(), equalTo(CwlSinkConfig.DEFAULT_BUFFER_TYPE)); + } + + @Test + void check_null_log_group_test() { + assertThat(new CwlSinkConfig().getLogGroup(), equalTo(null)); + } + @Test + void check_null_log_stream_test() { + assertThat(new CwlSinkConfig().getLogStream(), equalTo(null)); + } + + @Test + void check_valid_log_group_and_log_stream_test() throws NoSuchFieldException, IllegalAccessException { + ReflectivelySetField.setField(cwlSinkConfig.getClass(), cwlSinkConfig, "logGroup", LOG_GROUP); + ReflectivelySetField.setField(cwlSinkConfig.getClass(), cwlSinkConfig, "logStream", LOG_STREAM); + + assertThat(cwlSinkConfig.getLogGroup(), equalTo(LOG_GROUP)); + assertThat(cwlSinkConfig.getLogStream(), equalTo(LOG_STREAM)); + } + + @Test + void check_valid_sub_config_test() throws NoSuchFieldException, IllegalAccessException { + ReflectivelySetField.setField(cwlSinkConfig.getClass(), cwlSinkConfig, "thresholdConfig", thresholdConfig); + ReflectivelySetField.setField(cwlSinkConfig.getClass(), cwlSinkConfig, "awsConfig", awsConfig); + + assertThat(cwlSinkConfig.getAwsConfig(), equalTo(awsConfig)); + assertThat(cwlSinkConfig.getThresholdConfig(), equalTo(thresholdConfig)); + } +} diff --git a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/configuration/ThresholdConfigTest.java b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/configuration/ThresholdConfigTest.java new file mode 100644 index 0000000000..e6af96e08d --- /dev/null +++ b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/configuration/ThresholdConfigTest.java @@ -0,0 +1,82 @@ +package org.opensearch.dataprepper.plugins.sink.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 org.opensearch.dataprepper.plugins.sink.config.ThresholdConfig; + +import java.util.Map; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.equalTo; + +public class ThresholdConfigTest { + private ObjectMapper objectMapper; + + @BeforeEach + void setUp() { + objectMapper = new ObjectMapper(); + } + + @Test + void check_default_values() { + final ThresholdConfig thresholdConfig = new ThresholdConfig(); + + assertThat(thresholdConfig.getBackOffTime(), equalTo(ThresholdConfig.DEFAULT_BACKOFF_TIME)); + assertThat(thresholdConfig.getRetryCount(), equalTo(ThresholdConfig.DEFAULT_RETRY_COUNT)); + assertThat(thresholdConfig.getBatchSize(), equalTo(ThresholdConfig.DEFAULT_BATCH_SIZE)); + assertThat(thresholdConfig.getMaxEventSize(), equalTo(ThresholdConfig.DEFAULT_EVENT_SIZE)); + assertThat(thresholdConfig.getMaxRequestSize(), equalTo(ThresholdConfig.DEFAULT_SIZE_OF_REQUEST)); + assertThat(thresholdConfig.getLogSendInterval(), equalTo(ThresholdConfig.DEFAULT_LOG_SEND_INTERVAL_TIME)); + } + + @ParameterizedTest + @ValueSource(ints = {1, 10, 10000}) + void check_valid_batch_size(final int batchSize) { + final Map jsonMap = Map.of("batch_size", batchSize); + final ThresholdConfig thresholdConfigTest = objectMapper.convertValue(jsonMap, ThresholdConfig.class); + assertThat(thresholdConfigTest.getBatchSize(), equalTo(batchSize)); + } + + @ParameterizedTest + @ValueSource(ints = {1, 10, 256}) + void check_valid_max_event_size(final int max_event_size) { + final Map jsonMap = Map.of("max_event_size", max_event_size); + final ThresholdConfig thresholdConfigTest = objectMapper.convertValue(jsonMap, ThresholdConfig.class); + assertThat(thresholdConfigTest.getMaxEventSize(), equalTo(max_event_size)); + } + + @ParameterizedTest + @ValueSource(ints = {1, 100, 1048576}) + void check_valid_request_size(final int max_batch_request_size) { + final Map jsonMap = Map.of("max_request_size", max_batch_request_size); + final ThresholdConfig thresholdConfigTest = objectMapper.convertValue(jsonMap, ThresholdConfig.class); + assertThat(thresholdConfigTest.getMaxRequestSize(), equalTo(max_batch_request_size)); + } + + @ParameterizedTest + @ValueSource(ints = {1, 10, 15}) + void check_valid_retry_count(final int retry_count) { + final Map jsonMap = Map.of("retry_count", retry_count); + final ThresholdConfig thresholdConfigTest = objectMapper.convertValue(jsonMap, ThresholdConfig.class); + assertThat(thresholdConfigTest.getRetryCount(), equalTo(retry_count)); + } + + @ParameterizedTest + @ValueSource(ints = {5, 10, 300}) + void check_valid_log_send_interval(final int log_send_interval) { + final Map jsonMap = Map.of("log_send_interval", log_send_interval); + final ThresholdConfig thresholdConfigTest = objectMapper.convertValue(jsonMap, ThresholdConfig.class); + assertThat(thresholdConfigTest.getLogSendInterval(), equalTo(log_send_interval)); + } + + @ParameterizedTest + @ValueSource(ints = {0, 100, 5000}) + void check_valid_back_off_time(final int back_off_time) { + final Map jsonMap = Map.of("back_off_time", back_off_time); + final ThresholdConfig thresholdConfigTest = objectMapper.convertValue(jsonMap, ThresholdConfig.class); + assertThat(thresholdConfigTest.getBackOffTime(), equalTo(back_off_time)); + } +} diff --git a/data-prepper-plugins/common/build.gradle b/data-prepper-plugins/common/build.gradle index 24a4fca124..9a2041e5c6 100644 --- a/data-prepper-plugins/common/build.gradle +++ b/data-prepper-plugins/common/build.gradle @@ -13,7 +13,7 @@ dependencies { implementation "commons-io:commons-io:2.12.0" implementation 'software.amazon.awssdk:s3' implementation 'software.amazon.awssdk:acm' - implementation 'org.apache.commons:commons-compress:1.21' + implementation 'org.apache.commons:commons-compress:1.23.0' implementation "org.apache.commons:commons-lang3:3.12.0" implementation "org.bouncycastle:bcprov-jdk15on:1.70" implementation "org.bouncycastle:bcpkix-jdk15on:1.70" @@ -21,7 +21,7 @@ dependencies { implementation 'io.micrometer:micrometer-core' testImplementation testLibs.junit.vintage implementation 'org.apache.parquet:parquet-common:1.12.3' - implementation 'org.xerial.snappy:snappy-java:1.1.9.1' + implementation 'org.xerial.snappy:snappy-java:1.1.10.1' testImplementation project(':data-prepper-plugins:blocking-buffer') testImplementation 'commons-io:commons-io:2.12.0' testImplementation testLibs.mockito.inline diff --git a/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/accumulator/Buffer.java b/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/accumulator/Buffer.java new file mode 100644 index 0000000000..c6f6018a4f --- /dev/null +++ b/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/accumulator/Buffer.java @@ -0,0 +1,25 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.accumulator; + +import java.io.IOException; + +/** + * A buffer can hold data before flushing it any Sink. + */ +public interface Buffer { + + /** + * Gets the current size of the buffer. This should be the number of bytes. + * @return buffer size. + */ + long getSize(); + int getEventCount(); + long getDuration(); + + byte[] getSinkBufferData() throws IOException; + void writeEvent(byte[] bytes) throws IOException; +} diff --git a/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/accumulator/BufferFactory.java b/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/accumulator/BufferFactory.java new file mode 100644 index 0000000000..2a99d63aaf --- /dev/null +++ b/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/accumulator/BufferFactory.java @@ -0,0 +1,10 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.accumulator; + +public interface BufferFactory { + Buffer getBuffer(); +} diff --git a/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/accumulator/BufferTypeOptions.java b/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/accumulator/BufferTypeOptions.java new file mode 100644 index 0000000000..e2a2ad7158 --- /dev/null +++ b/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/accumulator/BufferTypeOptions.java @@ -0,0 +1,40 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.accumulator; + +import com.fasterxml.jackson.annotation.JsonCreator; + +import java.util.Arrays; +import java.util.Map; +import java.util.stream.Collectors; + +/** + * Defines all the buffer types enumerations. + */ +public enum BufferTypeOptions { + + INMEMORY("in_memory", new InMemoryBufferFactory()), + LOCALFILE("local_file", new LocalFileBufferFactory()); + + private final String option; + private final BufferFactory bufferType; + private static final Map OPTIONS_MAP = Arrays.stream(BufferTypeOptions.values()) + .collect(Collectors.toMap(value -> value.option, value -> value)); + + BufferTypeOptions(final String option, final BufferFactory bufferType) { + this.option = option.toLowerCase(); + this.bufferType = bufferType; + } + + public BufferFactory getBufferType() { + return bufferType; + } + + @JsonCreator + static BufferTypeOptions fromOptionValue(final String option) { + return OPTIONS_MAP.get(option); + } +} \ No newline at end of file diff --git a/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/accumulator/InMemoryBuffer.java b/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/accumulator/InMemoryBuffer.java new file mode 100644 index 0000000000..e583e54260 --- /dev/null +++ b/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/accumulator/InMemoryBuffer.java @@ -0,0 +1,64 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.accumulator; + +import org.apache.commons.lang3.time.StopWatch; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.util.concurrent.TimeUnit; + +/** + * A buffer can hold in memory data and flushing it to any Sink. + */ +public class InMemoryBuffer implements Buffer { + + private static final ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream(); + private int eventCount; + private final StopWatch watch; + + InMemoryBuffer() { + byteArrayOutputStream.reset(); + eventCount = 0; + watch = new StopWatch(); + watch.start(); + } + + @Override + public long getSize() { + return byteArrayOutputStream.size(); + } + + @Override + public int getEventCount() { + return eventCount; + } + + public long getDuration() { + return watch.getTime(TimeUnit.SECONDS); + } + + /** + * collect current buffer data. + * @throws IOException while collecting current buffer data. + */ + @Override + public byte[] getSinkBufferData() throws IOException { + return byteArrayOutputStream.toByteArray(); + } + + /** + * write byte array to output stream. + * + * @param bytes byte array. + * @throws IOException while writing to output stream fails. + */ + @Override + public void writeEvent(byte[] bytes) throws IOException { + byteArrayOutputStream.write(bytes); + byteArrayOutputStream.write(System.lineSeparator().getBytes()); + eventCount++; + } +} \ No newline at end of file diff --git a/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/accumulator/InMemoryBufferFactory.java b/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/accumulator/InMemoryBufferFactory.java new file mode 100644 index 0000000000..ef6ec5daf9 --- /dev/null +++ b/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/accumulator/InMemoryBufferFactory.java @@ -0,0 +1,13 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.accumulator; + +public class InMemoryBufferFactory implements BufferFactory { + @Override + public Buffer getBuffer() { + return new InMemoryBuffer(); + } +} diff --git a/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/accumulator/LocalFileBuffer.java b/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/accumulator/LocalFileBuffer.java new file mode 100644 index 0000000000..9f9b4a3aac --- /dev/null +++ b/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/accumulator/LocalFileBuffer.java @@ -0,0 +1,109 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.accumulator; + +import org.apache.commons.lang3.time.StopWatch; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.BufferedOutputStream; +import java.io.File; +import java.io.FileNotFoundException; +import java.io.OutputStream; +import java.io.FileOutputStream; +import java.io.IOException; + +import java.nio.file.Files; +import java.nio.file.Paths; +import java.util.concurrent.TimeUnit; + +/** + * A buffer can hold local file data and flushing it to any Sink. + */ +public class LocalFileBuffer implements Buffer { + + private static final Logger LOG = LoggerFactory.getLogger(LocalFileBuffer.class); + private final OutputStream outputStream; + private int eventCount; + private final StopWatch watch; + private final File localFile; + + LocalFileBuffer(File tempFile) throws FileNotFoundException { + localFile = tempFile; + outputStream = new BufferedOutputStream(new FileOutputStream(tempFile)); + eventCount = 0; + watch = new StopWatch(); + watch.start(); + } + + @Override + public long getSize() { + try { + outputStream.flush(); + } catch (IOException e) { + LOG.error("An exception occurred while flushing data to buffered output stream :", e); + } + return localFile.length(); + } + + @Override + public int getEventCount() { + return eventCount; + } + + @Override + public long getDuration(){ + return watch.getTime(TimeUnit.SECONDS); + } + + /** + * collect current buffer data. + * @throws IOException while collecting current buffer data. + */ + @Override + public byte[] getSinkBufferData() throws IOException { + final byte[] fileData = Files.readAllBytes(localFile.toPath()); + removeTemporaryFile(); + return fileData; + } + + /** + * write byte array to output stream. + * @param bytes byte array. + * @throws IOException while writing to output stream fails. + */ + @Override + public void writeEvent(byte[] bytes) throws IOException { + outputStream.write(bytes); + outputStream.write(System.lineSeparator().getBytes()); + eventCount++; + } + + /** + * Flushing the buffered data into the output stream. + */ + protected void flushAndCloseStream(){ + try { + outputStream.flush(); + outputStream.close(); + } catch (IOException e) { + LOG.error("An exception occurred while flushing data to buffered output stream :", e); + } + } + + /** + * Remove the local temp file after flushing data to Sink. + */ + protected void removeTemporaryFile() { + if (localFile != null) { + try { + Files.deleteIfExists(Paths.get(localFile.toString())); + } catch (IOException e) { + LOG.error("Unable to delete Local file {}", localFile, e); + } + } + } +} \ No newline at end of file diff --git a/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/accumulator/LocalFileBufferFactory.java b/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/accumulator/LocalFileBufferFactory.java new file mode 100644 index 0000000000..cf11ba2a39 --- /dev/null +++ b/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/accumulator/LocalFileBufferFactory.java @@ -0,0 +1,31 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.accumulator; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.IOException; + +public class LocalFileBufferFactory implements BufferFactory { + + private static final Logger LOG = LoggerFactory.getLogger(LocalFileBufferFactory.class); + public static final String PREFIX = "local"; + public static final String SUFFIX = ".log"; + @Override + public Buffer getBuffer() { + File tempFile = null; + Buffer localfileBuffer = null; + try { + tempFile = File.createTempFile(PREFIX, SUFFIX); + localfileBuffer = new LocalFileBuffer(tempFile); + } catch (IOException e) { + LOG.error("Unable to create temp file ", e); + } + return localfileBuffer; + } +} diff --git a/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/s3keyindex/S3ObjectIndexUtility.java b/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/s3keyindex/S3ObjectIndexUtility.java index 27d54cf44b..bfcac56130 100644 --- a/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/s3keyindex/S3ObjectIndexUtility.java +++ b/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/s3keyindex/S3ObjectIndexUtility.java @@ -38,6 +38,8 @@ public class S3ObjectIndexUtility { /** * Create Object Name with date,time and UniqueID prepended. + * @param indexAlias index alias + * @return returns string of object name with date time */ public static String getObjectNameWithDateTimeId(final String indexAlias) { DateTimeFormatter dateFormatter = validateAndGetDateTimeFormatter(indexAlias); @@ -48,6 +50,8 @@ public static String getObjectNameWithDateTimeId(final String indexAlias) { /** * Create Object path prefix. + * @param indexAlias index alias + * @return returns string of object path prefix */ public static String getObjectPathPrefix(final String indexAlias) { DateTimeFormatter dateFormatter = validateAndGetDateTimeFormatter(indexAlias); @@ -57,6 +61,7 @@ public static String getObjectPathPrefix(final String indexAlias) { /** * Creates epoch seconds. + * @return returns time since epoch in nanoseconds */ public static long getTimeNanos() { Instant time = Instant.now(); @@ -67,6 +72,8 @@ public static long getTimeNanos() { /** * Validate the index with the regular expression pattern. Throws exception if validation fails + * @param indexAlias index alias + * @return returns date time formatter */ public static DateTimeFormatter validateAndGetDateTimeFormatter(final String indexAlias) { final Pattern pattern = Pattern.compile(TIME_PATTERN_INTERNAL_EXTRACTOR_REGULAR_EXPRESSION); @@ -128,8 +135,9 @@ public static void validateTimePatternGranularity(String timePattern) { /** * Returns the current UTC Date and Time + * @return returns current UTC time */ public static ZonedDateTime getCurrentUtcTime() { return LocalDateTime.now().atZone(ZoneId.systemDefault()).withZoneSameInstant(UTC_ZONE_ID); } -} \ No newline at end of file +} diff --git a/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/sink/FileSink.java b/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/sink/FileSink.java index fc3df248fa..e0f2a94b0d 100644 --- a/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/sink/FileSink.java +++ b/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/sink/FileSink.java @@ -48,6 +48,7 @@ public class FileSink implements Sink> { * pluginSetting file. * * @param fileSinkConfig The file sink configuration + * @param sinkContext sink context */ @DataPrepperPluginConstructor public FileSink(final FileSinkConfig fileSinkConfig, final SinkContext sinkContext) { diff --git a/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/sink/StdOutSink.java b/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/sink/StdOutSink.java index e43c25f658..0396af3176 100644 --- a/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/sink/StdOutSink.java +++ b/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/sink/StdOutSink.java @@ -6,6 +6,7 @@ package org.opensearch.dataprepper.plugins.sink; 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.event.EventHandle; @@ -27,7 +28,9 @@ public class StdOutSink implements Sink> { * pluginSetting file. * * @param pluginSetting instance with metadata information from pipeline pluginSetting file. + * @param sinkContext sink context */ + @DataPrepperPluginConstructor public StdOutSink(final PluginSetting pluginSetting, final SinkContext sinkContext) { this(Objects.nonNull(sinkContext) ? sinkContext.getTagsTargetKey() : null); } diff --git a/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/sink/ThresholdCheck.java b/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/sink/ThresholdCheck.java new file mode 100644 index 0000000000..1792aea483 --- /dev/null +++ b/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/sink/ThresholdCheck.java @@ -0,0 +1,37 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.sink; + +import org.opensearch.dataprepper.model.types.ByteCount; +import org.opensearch.dataprepper.plugins.accumulator.Buffer; + +/** + * Check threshold limits. + */ +public class ThresholdCheck { + + private ThresholdCheck() { + } + + /** + * Check threshold exceeds. + * @param currentBuffer current buffer. + * @param maxEvents maximum event provided by user as threshold. + * @param maxBytes maximum bytes provided by user as threshold. + * @param maxCollectionDuration maximum event collection duration provided by user as threshold. + * @return boolean value whether the threshold are met. + */ + public static boolean checkThresholdExceed(final Buffer currentBuffer, final int maxEvents, final ByteCount maxBytes, final long maxCollectionDuration) { + if (maxEvents > 0) { + return currentBuffer.getEventCount() + 1 > maxEvents || + currentBuffer.getDuration() > maxCollectionDuration || + currentBuffer.getSize() > maxBytes.getBytes(); + } else { + return currentBuffer.getDuration() > maxCollectionDuration || + currentBuffer.getSize() > maxBytes.getBytes(); + } + } +} diff --git a/data-prepper-plugins/common/src/test/java/org/opensearch/dataprepper/plugins/accumulator/BufferTypeOptionsTest.java b/data-prepper-plugins/common/src/test/java/org/opensearch/dataprepper/plugins/accumulator/BufferTypeOptionsTest.java new file mode 100644 index 0000000000..2211085b33 --- /dev/null +++ b/data-prepper-plugins/common/src/test/java/org/opensearch/dataprepper/plugins/accumulator/BufferTypeOptionsTest.java @@ -0,0 +1,35 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.accumulator; + +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 BufferTypeOptionsTest { + + @Test + void notNull_test() { + assertNotNull(BufferTypeOptions.INMEMORY); + } + + @Test + void get_buffer_type_test() { + assertNotNull(BufferTypeOptions.INMEMORY.getBufferType()); + } + + @Test + void fromOptionValue_test() { + BufferTypeOptions bufferTypeOptions = BufferTypeOptions.fromOptionValue("in_memory"); + assertNotNull(bufferTypeOptions); + assertThat(bufferTypeOptions.toString(), equalTo("INMEMORY")); + } +} \ No newline at end of file diff --git a/data-prepper-plugins/common/src/test/java/org/opensearch/dataprepper/plugins/accumulator/InMemoryBufferFactoryTest.java b/data-prepper-plugins/common/src/test/java/org/opensearch/dataprepper/plugins/accumulator/InMemoryBufferFactoryTest.java new file mode 100644 index 0000000000..ab533f4015 --- /dev/null +++ b/data-prepper-plugins/common/src/test/java/org/opensearch/dataprepper/plugins/accumulator/InMemoryBufferFactoryTest.java @@ -0,0 +1,30 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.accumulator; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import static org.hamcrest.CoreMatchers.instanceOf; +import static org.hamcrest.MatcherAssert.assertThat; + +class InMemoryBufferFactoryTest { + + @Test + void test_inMemoryBufferFactory_notNull(){ + InMemoryBufferFactory inMemoryBufferFactory = new InMemoryBufferFactory(); + Assertions.assertNotNull(inMemoryBufferFactory); + } + + @Test + void test_buffer_notNull(){ + InMemoryBufferFactory inMemoryBufferFactory = new InMemoryBufferFactory(); + Assertions.assertNotNull(inMemoryBufferFactory); + Buffer buffer = inMemoryBufferFactory.getBuffer(); + Assertions.assertNotNull(buffer); + assertThat(buffer, instanceOf(Buffer.class)); + } +} \ No newline at end of file diff --git a/data-prepper-plugins/common/src/test/java/org/opensearch/dataprepper/plugins/accumulator/InMemoryBufferTest.java b/data-prepper-plugins/common/src/test/java/org/opensearch/dataprepper/plugins/accumulator/InMemoryBufferTest.java new file mode 100644 index 0000000000..ad07cc4011 --- /dev/null +++ b/data-prepper-plugins/common/src/test/java/org/opensearch/dataprepper/plugins/accumulator/InMemoryBufferTest.java @@ -0,0 +1,56 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.accumulator; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.junit.jupiter.MockitoExtension; + +import java.io.IOException; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.greaterThanOrEqualTo; +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; + +@ExtendWith(MockitoExtension.class) +class InMemoryBufferTest { + + public static final int MAX_EVENTS = 55; + + private InMemoryBuffer inMemoryBuffer; + + @Test + void test_with_write_event_into_buffer() throws IOException { + inMemoryBuffer = new InMemoryBuffer(); + + while (inMemoryBuffer.getEventCount() < MAX_EVENTS) { + inMemoryBuffer.writeEvent(generateByteArray()); + } + assertThat(inMemoryBuffer.getSize(), greaterThanOrEqualTo(54110L)); + assertThat(inMemoryBuffer.getEventCount(), equalTo(MAX_EVENTS)); + assertThat(inMemoryBuffer.getDuration(), greaterThanOrEqualTo(0L)); + + } + + @Test + void test_getSinkData_success() { + inMemoryBuffer = new InMemoryBuffer(); + Assertions.assertNotNull(inMemoryBuffer); + assertDoesNotThrow(() -> { + inMemoryBuffer.getSinkBufferData(); + }); + } + + private byte[] generateByteArray() { + byte[] bytes = new byte[1000]; + for (int i = 0; i < 1000; i++) { + bytes[i] = (byte) i; + } + return bytes; + } +} \ No newline at end of file diff --git a/data-prepper-plugins/common/src/test/java/org/opensearch/dataprepper/plugins/accumulator/LocalFileBufferFactoryTest.java b/data-prepper-plugins/common/src/test/java/org/opensearch/dataprepper/plugins/accumulator/LocalFileBufferFactoryTest.java new file mode 100644 index 0000000000..def5990028 --- /dev/null +++ b/data-prepper-plugins/common/src/test/java/org/opensearch/dataprepper/plugins/accumulator/LocalFileBufferFactoryTest.java @@ -0,0 +1,29 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.accumulator; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import static org.hamcrest.CoreMatchers.instanceOf; +import static org.hamcrest.MatcherAssert.assertThat; +class LocalFileBufferFactoryTest { + + @Test + void test_localFileBufferFactory_notNull() { + LocalFileBufferFactory localFileBufferFactory = new LocalFileBufferFactory(); + Assertions.assertNotNull(localFileBufferFactory); + } + + @Test + void test_buffer_notNull() { + LocalFileBufferFactory localFileBufferFactory = new LocalFileBufferFactory(); + Assertions.assertNotNull(localFileBufferFactory); + Buffer buffer = localFileBufferFactory.getBuffer(); + Assertions.assertNotNull(buffer); + assertThat(buffer, instanceOf(LocalFileBuffer.class)); + } +} \ No newline at end of file diff --git a/data-prepper-plugins/common/src/test/java/org/opensearch/dataprepper/plugins/accumulator/LocalFileBufferTest.java b/data-prepper-plugins/common/src/test/java/org/opensearch/dataprepper/plugins/accumulator/LocalFileBufferTest.java new file mode 100644 index 0000000000..53c556e75c --- /dev/null +++ b/data-prepper-plugins/common/src/test/java/org/opensearch/dataprepper/plugins/accumulator/LocalFileBufferTest.java @@ -0,0 +1,84 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.accumulator; + +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +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 java.io.File; +import java.io.IOException; +import java.util.UUID; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.greaterThan; +import static org.hamcrest.Matchers.equalTo; + +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; +import static org.junit.jupiter.api.Assertions.assertFalse; + +@ExtendWith(MockitoExtension.class) +class LocalFileBufferTest { + + public static final String KEY = UUID.randomUUID().toString() + ".log"; + public static final String PREFIX = "local"; + public static final String SUFFIX = ".log"; + + private LocalFileBuffer localFileBuffer; + private File tempFile; + + @BeforeEach + void setUp() throws IOException { + tempFile = File.createTempFile(PREFIX, SUFFIX); + localFileBuffer = new LocalFileBuffer(tempFile); + } + + @Test + void test_with_write_events_into_buffer() throws IOException { + while (localFileBuffer.getEventCount() < 55) { + localFileBuffer.writeEvent(generateByteArray()); + } + assertThat(localFileBuffer.getSize(), greaterThan(1l)); + assertThat(localFileBuffer.getEventCount(), equalTo(55)); + assertThat(localFileBuffer.getDuration(), equalTo(0L)); + localFileBuffer.flushAndCloseStream(); + localFileBuffer.removeTemporaryFile(); + assertFalse(tempFile.exists(), "The temp file has not been deleted."); + } + + @Test + void test_without_write_events_into_buffer() { + assertThat(localFileBuffer.getSize(), equalTo(0L)); + assertThat(localFileBuffer.getEventCount(), equalTo(0)); + assertThat(localFileBuffer.getDuration(), equalTo(0L)); + localFileBuffer.flushAndCloseStream(); + localFileBuffer.removeTemporaryFile(); + assertFalse(tempFile.exists(), "The temp file has not been deleted."); + } + + @Test + void test_getSinkData_success() throws IOException{ + Assertions.assertNotNull(localFileBuffer); + assertDoesNotThrow(() -> { + localFileBuffer.getSinkBufferData(); + }); + } + + @AfterEach + void cleanup() { + tempFile.deleteOnExit(); + } + + private byte[] generateByteArray() { + byte[] bytes = new byte[1000]; + for (int i = 0; i < 1000; i++) { + bytes[i] = (byte) i; + } + return bytes; + } +} \ No newline at end of file diff --git a/data-prepper-plugins/common/src/test/java/org/opensearch/dataprepper/plugins/sink/ThresholdCheckTest.java b/data-prepper-plugins/common/src/test/java/org/opensearch/dataprepper/plugins/sink/ThresholdCheckTest.java new file mode 100644 index 0000000000..23494ecb3d --- /dev/null +++ b/data-prepper-plugins/common/src/test/java/org/opensearch/dataprepper/plugins/sink/ThresholdCheckTest.java @@ -0,0 +1,126 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.sink; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.opensearch.dataprepper.model.types.ByteCount; +import org.opensearch.dataprepper.plugins.accumulator.Buffer; +import org.opensearch.dataprepper.plugins.accumulator.InMemoryBufferFactory; + +import java.io.IOException; + +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; + +class ThresholdCheckTest { + + private Buffer inMemoryBuffer; + + @BeforeEach + void setUp() throws IOException { + inMemoryBuffer = new InMemoryBufferFactory().getBuffer(); + + while (inMemoryBuffer.getEventCount() < 100) { + inMemoryBuffer.writeEvent(generateByteArray()); + } + } + + @Test + void test_exceedThreshold_true_dueTo_maxEvents_is_less_than_buffered_event_count() { + final int maxEvents = 95; + final ByteCount maxBytes = ByteCount.parse("50kb"); + final long maxCollectionDuration = 15; + boolean isThresholdExceed = ThresholdCheck.checkThresholdExceed(inMemoryBuffer, maxEvents, + maxBytes, maxCollectionDuration); + assertTrue(isThresholdExceed, "Threshold not exceeded"); + } + + @Test + void test_exceedThreshold_false_dueTo_maxEvents_is_greater_than_buffered_event_count() { + final int maxEvents = 105; + final ByteCount maxBytes = ByteCount.parse("50mb"); + final long maxCollectionDuration = 50; + boolean isThresholdExceed = ThresholdCheck.checkThresholdExceed(inMemoryBuffer, maxEvents, maxBytes, + maxCollectionDuration); + assertFalse(isThresholdExceed, "Threshold exceeded"); + } + + @Test + void test_exceedThreshold_ture_dueTo_maxBytes_is_less_than_buffered_byte_count() { + final int maxEvents = 500; + final ByteCount maxBytes = ByteCount.parse("1b"); + final long maxCollectionDuration = 15; + boolean isThresholdExceed = ThresholdCheck.checkThresholdExceed(inMemoryBuffer, maxEvents, maxBytes, + maxCollectionDuration); + assertTrue(isThresholdExceed, "Threshold not exceeded"); + } + + @Test + void test_exceedThreshold_false_dueTo_maxBytes_is_greater_than_buffered_byte_count() { + final int maxEvents = 500; + final ByteCount maxBytes = ByteCount.parse("8mb"); + final long maxCollectionDuration = 15; + boolean isThresholdExceed = ThresholdCheck.checkThresholdExceed(inMemoryBuffer, maxEvents, + maxBytes, maxCollectionDuration); + assertFalse(isThresholdExceed, "Threshold exceeded"); + } + + @Test + void test_exceedThreshold_ture_dueTo_maxCollectionDuration_is_less_than_buffered_event_collection_duration() + throws IOException, InterruptedException { + final int maxEvents = 500; + final ByteCount maxBytes = ByteCount.parse("500mb"); + final long maxCollectionDuration = 10; + + inMemoryBuffer = new InMemoryBufferFactory().getBuffer(); + boolean isThresholdExceed = Boolean.FALSE; + synchronized (this) { + while (inMemoryBuffer.getEventCount() < 100) { + inMemoryBuffer.writeEvent(generateByteArray()); + isThresholdExceed = ThresholdCheck.checkThresholdExceed(inMemoryBuffer, maxEvents, + maxBytes, maxCollectionDuration); + if (isThresholdExceed) { + break; + } + wait(5000); + } + } + assertTrue(isThresholdExceed, "Threshold not exceeded"); + } + + @Test + void test_exceedThreshold_ture_dueTo_maxCollectionDuration_is_greater_than_buffered_event_collection_duration() + throws IOException, InterruptedException { + final int maxEvents = 500; + final ByteCount maxBytes = ByteCount.parse("500mb"); + final long maxCollectionDuration = 240; + + inMemoryBuffer = new InMemoryBufferFactory().getBuffer(); + + boolean isThresholdExceed = Boolean.FALSE; + synchronized (this) { + while (inMemoryBuffer.getEventCount() < 100) { + inMemoryBuffer.writeEvent(generateByteArray()); + isThresholdExceed = ThresholdCheck.checkThresholdExceed(inMemoryBuffer, + maxEvents, maxBytes, maxCollectionDuration); + if (isThresholdExceed) { + break; + } + wait(50); + } + } + assertFalse(isThresholdExceed, "Threshold exceeded"); + } + + private byte[] generateByteArray() { + byte[] bytes = new byte[10000]; + for (int i = 0; i < 10000; i++) { + bytes[i] = (byte) i; + } + return bytes; + } +} diff --git a/data-prepper-plugins/csv-processor/src/main/java/org/opensearch/dataprepper/plugins/codec/csv/CsvOutputCodec.java b/data-prepper-plugins/csv-processor/src/main/java/org/opensearch/dataprepper/plugins/codec/csv/CsvOutputCodec.java index 91a82454b2..ef3cc98225 100644 --- a/data-prepper-plugins/csv-processor/src/main/java/org/opensearch/dataprepper/plugins/codec/csv/CsvOutputCodec.java +++ b/data-prepper-plugins/csv-processor/src/main/java/org/opensearch/dataprepper/plugins/codec/csv/CsvOutputCodec.java @@ -30,7 +30,7 @@ public void start(final OutputStream outputStream) throws IOException { } @Override - public void writeEvent(final Event event, final OutputStream outputStream) throws IOException { + public void writeEvent(final Event event, final OutputStream outputStream, String tagsTargetKey) throws IOException { // TODO: validate data according to header and write event data to the outputstream } diff --git a/data-prepper-plugins/http-sink/build.gradle b/data-prepper-plugins/http-sink/build.gradle index da49c79c4b..21e678348f 100644 --- a/data-prepper-plugins/http-sink/build.gradle +++ b/data-prepper-plugins/http-sink/build.gradle @@ -2,11 +2,14 @@ dependencies { implementation project(':data-prepper-api') implementation project(':data-prepper-plugins:aws-plugin-api') + implementation project(':data-prepper-plugins:common') implementation 'com.fasterxml.jackson.core:jackson-core' implementation 'org.hibernate.validator:hibernate-validator:7.0.5.Final' implementation 'com.fasterxml.jackson.dataformat:jackson-dataformat-yaml' implementation 'software.amazon.awssdk:s3' implementation 'software.amazon.awssdk:sts' + implementation group: 'org.apache.httpcomponents.client5', name: 'httpclient5', version: '5.2' + implementation 'org.apache.commons:commons-lang3:3.12.0' } test { diff --git a/data-prepper-plugins/http-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/ClientFactory.java b/data-prepper-plugins/http-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/ClientFactory.java new file mode 100644 index 0000000000..fdb3dcaaa0 --- /dev/null +++ b/data-prepper-plugins/http-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/ClientFactory.java @@ -0,0 +1,13 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.sink; + +/** + * AWS factory class required to create AWS Http Endpoint client for this plugin. + */ +public final class ClientFactory { + private ClientFactory() {} + +} diff --git a/data-prepper-plugins/http-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/HTTPSink.java b/data-prepper-plugins/http-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/HTTPSink.java index 2c32235287..06d057b625 100644 --- a/data-prepper-plugins/http-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/HTTPSink.java +++ b/data-prepper-plugins/http-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/HTTPSink.java @@ -4,6 +4,7 @@ */ package org.opensearch.dataprepper.plugins.sink; +import org.apache.hc.client5.http.impl.classic.CloseableHttpClient; import org.opensearch.dataprepper.aws.api.AwsCredentialsSupplier; import org.opensearch.dataprepper.model.annotations.DataPrepperPlugin; import org.opensearch.dataprepper.model.annotations.DataPrepperPluginConstructor; @@ -15,10 +16,13 @@ import org.opensearch.dataprepper.model.sink.AbstractSink; import org.opensearch.dataprepper.model.sink.Sink; import org.opensearch.dataprepper.plugins.sink.configuration.HttpSinkConfiguration; +import org.opensearch.dataprepper.plugins.sink.configuration.UrlConfigurationOption; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.util.Collection; +import java.util.List; +import java.util.Optional; @DataPrepperPlugin(name = "http", pluginType = Sink.class, pluginConfigurationType = HttpSinkConfiguration.class) public class HTTPSink extends AbstractSink> { @@ -73,4 +77,21 @@ public void doOutput(final Collection> records) { } //TODO: call Service call method } + + + public Optional getAuthHandlerByConfig(final HttpSinkConfiguration sinkConfiguration){ + //TODO: AWS Sigv4 - check + // TODO: call Auth Handlers based on auth Type + + return null; + } + + public List getClassicHttpRequestList(final List urlConfigurationOption){ + // logic for create auth handler for each url based on provided configuration - getAuthHandlerByConfig() + // logic for request preparation for each url + // logic for worker is not there in url level then verify the global workers if global workers also not defined then default 1 + // logic for get the Proxy object if url level proxy enabled else look the global proxy. + // Aws SageMaker headers if headers found in the configuration + return null; + } } \ No newline at end of file diff --git a/data-prepper-plugins/http-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/HttpAuthOptions.java b/data-prepper-plugins/http-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/HttpAuthOptions.java new file mode 100644 index 0000000000..61b20993d8 --- /dev/null +++ b/data-prepper-plugins/http-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/HttpAuthOptions.java @@ -0,0 +1,65 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.sink; + +import org.apache.hc.client5.http.impl.classic.CloseableHttpClient; +import org.apache.hc.core5.http.ClassicHttpRequest; + +public class HttpAuthOptions { + private String url; + + private CloseableHttpClient closeableHttpClient; + + private ClassicHttpRequest classicHttpRequest; + + private int workers; + + private String proxy; + + public CloseableHttpClient getCloseableHttpClient() { + return closeableHttpClient; + } + + public HttpAuthOptions setCloseableHttpClient(CloseableHttpClient closeableHttpClient) { + this.closeableHttpClient = closeableHttpClient; + return this; + } + + public ClassicHttpRequest getClassicHttpRequest() { + return classicHttpRequest; + } + + public HttpAuthOptions setClassicHttpRequest(ClassicHttpRequest classicHttpRequest) { + this.classicHttpRequest = classicHttpRequest; + return this; + } + + public int getWorkers() { + return workers; + } + + public HttpAuthOptions setWorkers(int workers) { + this.workers = workers; + return this; + } + + public String getUrl() { + return url; + } + + public HttpAuthOptions setUrl(String url) { + this.url = url; + return this; + } + + public String getProxy() { + return proxy; + } + + public HttpAuthOptions setProxy(String proxy) { + this.proxy = proxy; + return this; + } +} diff --git a/data-prepper-plugins/http-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/dlq/FailedDlqData.java b/data-prepper-plugins/http-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/dlq/FailedDlqData.java new file mode 100644 index 0000000000..62a85cb709 --- /dev/null +++ b/data-prepper-plugins/http-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/dlq/FailedDlqData.java @@ -0,0 +1,58 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.sink.dlq; + +import com.fasterxml.jackson.annotation.JsonIgnore; +import org.opensearch.dataprepper.model.event.EventHandle; + +import java.util.Objects; + +public class FailedDlqData { + + private final int status; + + private final String message; + + @JsonIgnore + private final EventHandle eventHandle; + + private FailedDlqData(final int status, + final String message, + final EventHandle eventHandle) { + this.status = status; + Objects.requireNonNull(message); + this.message = message; + this.eventHandle = eventHandle; + } + + public int getStatus() { + return status; + } + + public String getMessage() { + return message; + } + public EventHandle getEventHandle() { + return eventHandle; + } + + + public static Builder builder() { + return new Builder(); + } + + public static class Builder { + + private EventHandle eventHandle; + + private int status = 0; + + private String message; + + public FailedDlqData build() { + return new FailedDlqData(status, message, eventHandle); + } + } +} diff --git a/data-prepper-plugins/http-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/handler/BasicAuthHttpSinkHandler.java b/data-prepper-plugins/http-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/handler/BasicAuthHttpSinkHandler.java new file mode 100644 index 0000000000..c84841c31e --- /dev/null +++ b/data-prepper-plugins/http-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/handler/BasicAuthHttpSinkHandler.java @@ -0,0 +1,17 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.sink.handler; + +import org.opensearch.dataprepper.plugins.sink.configuration.HttpSinkConfiguration; + +import java.util.Optional; + +public class BasicAuthHttpSinkHandler implements MultiAuthHttpSinkHandler { + @Override + public Optional authenticate(HttpSinkConfiguration sinkConfiguration) { + // if ssl enabled then set connection manager + return null; + } +} diff --git a/data-prepper-plugins/http-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/handler/BearerTokenAuthHttpSinkHandler.java b/data-prepper-plugins/http-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/handler/BearerTokenAuthHttpSinkHandler.java new file mode 100644 index 0000000000..fec474e6b8 --- /dev/null +++ b/data-prepper-plugins/http-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/handler/BearerTokenAuthHttpSinkHandler.java @@ -0,0 +1,17 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.sink.handler; + +import org.opensearch.dataprepper.plugins.sink.configuration.HttpSinkConfiguration; + +import java.util.Optional; + +public class BearerTokenAuthHttpSinkHandler implements MultiAuthHttpSinkHandler { + @Override + public Optional authenticate(HttpSinkConfiguration sinkConfiguration) { + // if ssl enabled then set connection manager + return null; + } +} diff --git a/data-prepper-plugins/http-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/handler/HttpAuthOptions.java b/data-prepper-plugins/http-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/handler/HttpAuthOptions.java new file mode 100644 index 0000000000..7ff5810b77 --- /dev/null +++ b/data-prepper-plugins/http-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/handler/HttpAuthOptions.java @@ -0,0 +1,61 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.sink.handler; + +import org.apache.hc.client5.http.impl.classic.CloseableHttpClient; +import org.apache.hc.core5.http.ClassicHttpRequest; + +public class HttpAuthOptions { + private String url; + private CloseableHttpClient closeableHttpClient; + private ClassicHttpRequest classicHttpRequest; + private int workers; + private String proxy; + + public CloseableHttpClient getCloseableHttpClient() { + return closeableHttpClient; + } + + public HttpAuthOptions setCloseableHttpClient(CloseableHttpClient closeableHttpClient) { + this.closeableHttpClient = closeableHttpClient; + return this; + } + + public ClassicHttpRequest getClassicHttpRequest() { + return classicHttpRequest; + } + + public HttpAuthOptions setClassicHttpRequest(ClassicHttpRequest classicHttpRequest) { + this.classicHttpRequest = classicHttpRequest; + return this; + } + + public int getWorkers() { + return workers; + } + + public HttpAuthOptions setWorkers(int workers) { + this.workers = workers; + return this; + } + + public String getUrl() { + return url; + } + + public HttpAuthOptions setUrl(String url) { + this.url = url; + return this; + } + + public String getProxy() { + return proxy; + } + + public HttpAuthOptions setProxy(String proxy) { + this.proxy = proxy; + return this; + } +} diff --git a/data-prepper-plugins/http-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/handler/MultiAuthHttpSinkHandler.java b/data-prepper-plugins/http-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/handler/MultiAuthHttpSinkHandler.java new file mode 100644 index 0000000000..e0db436915 --- /dev/null +++ b/data-prepper-plugins/http-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/handler/MultiAuthHttpSinkHandler.java @@ -0,0 +1,14 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.sink.handler; + +import org.opensearch.dataprepper.plugins.sink.configuration.HttpSinkConfiguration; + +import java.util.Optional; + +public interface MultiAuthHttpSinkHandler { + Optional authenticate(final HttpSinkConfiguration sinkConfiguration); + +} diff --git a/data-prepper-plugins/http-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/handler/SecuredAuthHttpSinkHandler.java b/data-prepper-plugins/http-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/handler/SecuredAuthHttpSinkHandler.java new file mode 100644 index 0000000000..9fb58fe223 --- /dev/null +++ b/data-prepper-plugins/http-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/handler/SecuredAuthHttpSinkHandler.java @@ -0,0 +1,18 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.sink.handler; + +import org.opensearch.dataprepper.plugins.sink.configuration.HttpSinkConfiguration; + +import java.util.Optional; + +public class SecuredAuthHttpSinkHandler implements MultiAuthHttpSinkHandler { + @Override + public Optional authenticate(HttpSinkConfiguration sinkConfiguration) { + // logic here to read the certs from ACM/S3/local + // SSL Sigv4 validation and verification and make connection + return null; + } +} diff --git a/data-prepper-plugins/http-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/service/HttpSinkService.java b/data-prepper-plugins/http-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/service/HttpSinkService.java new file mode 100644 index 0000000000..8f97dbb5b0 --- /dev/null +++ b/data-prepper-plugins/http-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/service/HttpSinkService.java @@ -0,0 +1,59 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.sink.service; + +import org.opensearch.dataprepper.model.buffer.Buffer; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.model.types.ByteCount; +import org.opensearch.dataprepper.plugins.accumulator.BufferFactory; +import org.opensearch.dataprepper.plugins.sink.configuration.HttpSinkConfiguration; +import org.opensearch.dataprepper.plugins.sink.handler.HttpAuthOptions; +import org.opensearch.dataprepper.model.codec.OutputCodec; + +import java.util.Collection; +import java.util.List; +public class HttpSinkService { + + private final HttpSinkConfiguration httpSinkConf; + + private final BufferFactory bufferFactory; + + private final List httpAuthOptions; + private OutputCodec codec; + + public HttpSinkService(final OutputCodec codec, + final HttpSinkConfiguration httpSinkConf, + final BufferFactory bufferFactory, + final List httpAuthOptions){ + this.codec= codec; + this.httpSinkConf = httpSinkConf; + this.bufferFactory = bufferFactory; + this.httpAuthOptions = httpAuthOptions; + } + + public void processRecords(Collection> records) { + records.forEach(record -> { + try{ + // logic to fetch the records in batch as per threshold limit - checkThresholdExceed(); + // apply the codec + // push to http end point + }catch(Exception e){ + // In case of any exception, need to write the exception in dlq - logFailureForDlqObjects(); + // In case of any exception, need to push the web hook url- logFailureForWebHook(); + } + //TODO: implement end to end acknowledgement + }); + } + + public static boolean checkThresholdExceed(final Buffer currentBuffer, + final int maxEvents, + final ByteCount maxBytes, + final long maxCollectionDuration) { + // logic for checking the threshold + return true; + } + +} diff --git a/data-prepper-plugins/kafka-plugins/build.gradle b/data-prepper-plugins/kafka-plugins/build.gradle index 831c1c0963..70c635bd08 100644 --- a/data-prepper-plugins/kafka-plugins/build.gradle +++ b/data-prepper-plugins/kafka-plugins/build.gradle @@ -9,6 +9,7 @@ plugins { dependencies { implementation project(':data-prepper-api') + implementation project(':data-prepper-plugins:buffer-common') implementation 'org.apache.kafka:kafka-clients:3.4.0' implementation 'org.apache.avro:avro:1.11.0' implementation 'com.fasterxml.jackson.core:jackson-databind' @@ -16,10 +17,54 @@ dependencies { implementation 'org.apache.commons:commons-lang3:3.12.0' implementation 'io.confluent:kafka-avro-serializer:7.3.3' implementation 'io.confluent:kafka-schema-registry-client:7.3.3' + implementation 'io.confluent:kafka-avro-serializer:7.3.3' + implementation 'io.confluent:kafka-schema-registry-client:7.3.3' + implementation 'io.confluent:kafka-schema-registry:7.3.3:tests' testImplementation 'org.mockito:mockito-inline:4.1.0' testImplementation 'org.yaml:snakeyaml:2.0' testImplementation testLibs.spring.test - testImplementation 'com.fasterxml.jackson.datatype:jackson-datatype-jsr310:2.14.2' + testImplementation 'com.fasterxml.jackson.datatype:jackson-datatype-jsr310:2.15.2' testImplementation project(':data-prepper-test-common') testImplementation project(':data-prepper-plugins:blocking-buffer') + testImplementation project(':data-prepper-core') + testImplementation 'org.mockito:mockito-inline:4.1.0' + testImplementation 'org.apache.kafka:kafka_2.13:3.4.0' + testImplementation 'org.apache.kafka:kafka_2.13:3.4.0:test' + testImplementation 'org.apache.curator:curator-test:5.5.0' + testImplementation 'io.confluent:kafka-schema-registry:7.4.0' + testImplementation 'junit:junit:4.13.1' + testImplementation 'org.apache.kafka:kafka-clients:3.4.0:test' + testImplementation 'org.apache.kafka:connect-json:3.4.0' +} + +test { + useJUnitPlatform() +} + +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() + + filter { + includeTestsMatching '*IT' + } +} + diff --git a/data-prepper-plugins/kafka-plugins/src/integrationTest/java/org/opensearch/dataprepper/plugins/kafka/source/EmbeddedKafkaClusterSingleNode.java b/data-prepper-plugins/kafka-plugins/src/integrationTest/java/org/opensearch/dataprepper/plugins/kafka/source/EmbeddedKafkaClusterSingleNode.java new file mode 100644 index 0000000000..ec791c221f --- /dev/null +++ b/data-prepper-plugins/kafka-plugins/src/integrationTest/java/org/opensearch/dataprepper/plugins/kafka/source/EmbeddedKafkaClusterSingleNode.java @@ -0,0 +1,161 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.kafka.source; + +import io.confluent.kafka.schemaregistry.RestApp; +import io.confluent.kafka.schemaregistry.avro.AvroCompatibilityLevel; +import io.confluent.kafka.schemaregistry.rest.SchemaRegistryConfig; +import kafka.server.KafkaConfig$; +import org.junit.rules.ExternalResource; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.Collections; +import java.util.Map; +import java.util.Properties; + +/** + * Runs an in-memory, "embedded" Kafka cluster with 1 ZooKeeper instance, 1 Kafka broker, and 1 + * Confluent Schema Registry instance. + */ +public class EmbeddedKafkaClusterSingleNode extends ExternalResource { + + private static final Logger log = LoggerFactory.getLogger(EmbeddedKafkaClusterSingleNode.class); + private static final int DEFAULT_BROKER_PORT = 0; + private static final String KAFKA_SCHEMAS_TOPIC = "_schemas"; + private static final String AVRO_COMPATIBILITY_TYPE = AvroCompatibilityLevel.NONE.name; + private static final String KAFKASTORE_OPERATION_TIMEOUT_MS = "60000"; + private static final String KAFKASTORE_DEBUG = "true"; + private static final String KAFKASTORE_INIT_TIMEOUT = "90000"; + + private EmbeddedZooKeeperServer zookeeper; + private EmbeddedKafkaServer broker; + private RestApp schemaRegistry; + private final Properties brokerConfig; + private boolean running; + + public EmbeddedKafkaClusterSingleNode() { + this(new Properties()); + } + + public EmbeddedKafkaClusterSingleNode(final Properties brokerConfig) { + this.brokerConfig = new Properties(); + this.brokerConfig.put(SchemaRegistryConfig.KAFKASTORE_TIMEOUT_CONFIG, KAFKASTORE_OPERATION_TIMEOUT_MS); + this.brokerConfig.putAll(brokerConfig); + } + + /** + * Creates and starts the cluster. + */ + public void start() throws Exception { + log.debug("Initiating embedded Kafka cluster startup"); + log.debug("Starting a ZooKeeper instance..."); + zookeeper = new EmbeddedZooKeeperServer(); + log.debug("ZooKeeper instance is running at {}", zookeeper.connectString()); + + final Properties effectiveBrokerConfig = effectiveBrokerConfigFrom(brokerConfig, zookeeper); + log.debug("Starting a Kafka instance on ...", + effectiveBrokerConfig.getProperty(KafkaConfig$.MODULE$.ZkConnectDoc())); + broker = new EmbeddedKafkaServer(effectiveBrokerConfig); + log.debug("Kafka instance is running at {}, connected to ZooKeeper at {}", + broker.brokerList(), broker.zookeeperConnect()); + + final Properties schemaRegistryProps = new Properties(); + + schemaRegistryProps.put(SchemaRegistryConfig.KAFKASTORE_TIMEOUT_CONFIG, KAFKASTORE_OPERATION_TIMEOUT_MS); + schemaRegistryProps.put(SchemaRegistryConfig.DEBUG_CONFIG, KAFKASTORE_DEBUG); + schemaRegistryProps.put(SchemaRegistryConfig.KAFKASTORE_INIT_TIMEOUT_CONFIG, KAFKASTORE_INIT_TIMEOUT); + schemaRegistryProps.put(SchemaRegistryConfig.KAFKASTORE_BOOTSTRAP_SERVERS_CONFIG, "127.0.0.1:9092"); + + + schemaRegistry = new RestApp(0, zookeeperConnect(), KAFKA_SCHEMAS_TOPIC, "none", schemaRegistryProps); + schemaRegistry.start(); + running = true; + } + + private Properties effectiveBrokerConfigFrom(final Properties brokerConfig, final EmbeddedZooKeeperServer zookeeper) { + final Properties effectiveConfig = new Properties(); + effectiveConfig.putAll(brokerConfig); + effectiveConfig.put(KafkaConfig$.MODULE$.ZkConnectProp(), zookeeper.connectString()); + effectiveConfig.put(KafkaConfig$.MODULE$.ZkSessionTimeoutMsProp(), 30 * 1000); + effectiveConfig.put(KafkaConfig$.MODULE$.ZkConnectionTimeoutMsProp(), 60 * 1000); + effectiveConfig.put(KafkaConfig$.MODULE$.DeleteTopicEnableProp(), true); + effectiveConfig.put(KafkaConfig$.MODULE$.LogCleanerDedupeBufferSizeProp(), 2 * 1024 * 1024L); + effectiveConfig.put(KafkaConfig$.MODULE$.GroupMinSessionTimeoutMsProp(), 0); + effectiveConfig.put(KafkaConfig$.MODULE$.OffsetsTopicReplicationFactorProp(), (short) 1); + effectiveConfig.put(KafkaConfig$.MODULE$.OffsetsTopicPartitionsProp(), 1); + effectiveConfig.put(KafkaConfig$.MODULE$.AutoCreateTopicsEnableProp(), true); + return effectiveConfig; + } + + @Override + protected void before() throws Exception { + start(); + } + + @Override + protected void after() { + stop(); + } + + /** + * Stops the cluster. + */ + public void stop() { + log.info("Stopping Confluent"); + try { + try { + if (schemaRegistry != null) { + schemaRegistry.stop(); + } + } catch (final Exception fatal) { + throw new RuntimeException(fatal); + } + if (broker != null) { + broker.stop(); + } + try { + if (zookeeper != null) { + zookeeper.stop(); + } + } catch (final IOException fatal) { + throw new RuntimeException(fatal); + } + } finally { + running = false; + } + log.info("Confluent Stopped"); + } + + public String bootstrapServers() { + return broker.brokerList(); + } + + public String zookeeperConnect() { + return zookeeper.connectString(); + } + + public String schemaRegistryUrl() { + return schemaRegistry.restConnect; + } + + public void createTopic(final String topic) { + createTopic(topic, 1, (short) 1, Collections.emptyMap()); + } + + public void createTopic(final String topic, final int partitions, final short replication) { + createTopic(topic, partitions, replication, Collections.emptyMap()); + } + + public void createTopic(final String topic, + final int partitions, + final short replication, + final Map topicConfig) { + broker.createTopic(topic, partitions, replication, topicConfig); + } + +} diff --git a/data-prepper-plugins/kafka-plugins/src/integrationTest/java/org/opensearch/dataprepper/plugins/kafka/source/EmbeddedKafkaServer.java b/data-prepper-plugins/kafka-plugins/src/integrationTest/java/org/opensearch/dataprepper/plugins/kafka/source/EmbeddedKafkaServer.java new file mode 100644 index 0000000000..1bb6953ed9 --- /dev/null +++ b/data-prepper-plugins/kafka-plugins/src/integrationTest/java/org/opensearch/dataprepper/plugins/kafka/source/EmbeddedKafkaServer.java @@ -0,0 +1,142 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.kafka.source; + + +import kafka.server.KafkaConfig; +import kafka.server.KafkaConfig$; +import kafka.server.KafkaServer; +import kafka.utils.TestUtils; +import org.apache.kafka.clients.admin.AdminClient; +import org.apache.kafka.clients.admin.AdminClientConfig; +import org.apache.kafka.clients.admin.NewTopic; +import org.apache.kafka.common.errors.UnknownTopicOrPartitionException; +import org.apache.kafka.common.utils.Time; +import org.junit.rules.TemporaryFolder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.IOException; +import java.util.Collections; +import java.util.Map; +import java.util.Properties; +import java.util.concurrent.ExecutionException; + +/** + * Runs an in-memory, "embedded" instance of a Kafka broker, which listens at `127.0.0.1:9092` by + * default. + * + * Requires a running ZooKeeper instance to connect to. By default, it expects a ZooKeeper instance + * running at `127.0.0.1:2181`. + */ +public class EmbeddedKafkaServer { + + private static final Logger log = LoggerFactory.getLogger(EmbeddedKafkaServer.class); + + private static final String DEFAULT_ZK_CONNECT = "127.0.0.1:2181"; + + private final Properties effectiveConfig; + private final File logDir; + private final TemporaryFolder tmpFolder; + private final KafkaServer kafka; + + public EmbeddedKafkaServer(final Properties config) throws IOException { + tmpFolder = new TemporaryFolder(); + tmpFolder.create(); + logDir = tmpFolder.newFolder(); + effectiveConfig = effectiveConfigFrom(config); + final boolean loggingEnabled = true; + + final KafkaConfig kafkaConfig = new KafkaConfig(effectiveConfig, loggingEnabled); + log.info("Starting embedded Kafka broker (with log.dirs={} and ZK ensemble at {}) ...", + logDir, zookeeperConnect()); + kafka = TestUtils.createServer(kafkaConfig, Time.SYSTEM); + log.debug("Startup of embedded Kafka broker at {} completed (with ZK ensemble at {}) ...", + brokerList(), zookeeperConnect()); + } + + private Properties effectiveConfigFrom(final Properties initialConfig) throws IOException { + final Properties effectiveConfig = new Properties(); + effectiveConfig.put(KafkaConfig$.MODULE$.BrokerIdProp(), 1); + effectiveConfig.put(KafkaConfig$.MODULE$.NumPartitionsProp(), 1); + effectiveConfig.put(KafkaConfig$.MODULE$.AutoCreateTopicsEnableProp(), true); + effectiveConfig.put(KafkaConfig$.MODULE$.MessageMaxBytesProp(), 1000000); + effectiveConfig.put(KafkaConfig$.MODULE$.ControlledShutdownEnableProp(), true); + + effectiveConfig.putAll(initialConfig); + effectiveConfig.setProperty(KafkaConfig$.MODULE$.LogDirProp(), logDir.getAbsolutePath()); + return effectiveConfig; + } + + public String brokerList() { + return kafka.config().zkConnect(); + } + + + public String zookeeperConnect() { + return effectiveConfig.getProperty("zookeeper.connect", DEFAULT_ZK_CONNECT); + } + + public void stop() { + log.debug("Shutting down embedded Kafka broker at {} (with ZK ensemble at {}) ...", + brokerList(), zookeeperConnect()); + kafka.shutdown(); + kafka.awaitShutdown(); + log.debug("Removing temp folder {} with logs.dir at {} ...", tmpFolder, logDir); + tmpFolder.delete(); + log.debug("Shutdown of embedded Kafka broker at {} completed (with ZK ensemble at {}) ...", + brokerList(), zookeeperConnect()); + } + + public void createTopic(final String topic) { + createTopic(topic, 1, (short) 1, Collections.emptyMap()); + } + + public void createTopic(final String topic, final int partitions, final short replication) { + createTopic(topic, partitions, replication, Collections.emptyMap()); + } + + public void createTopic(final String topic, + final int partitions, + final short replication, + final Map topicConfig) { + log.debug("Creating topic { name: {}, partitions: {}, replication: {}, config: {} }", + topic, partitions, replication, topicConfig); + + final Properties properties = new Properties(); + properties.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, "127.0.0.1:9092"); + + try (final AdminClient adminClient = AdminClient.create(properties)) { + final NewTopic newTopic = new NewTopic(topic, partitions, replication); + newTopic.configs(topicConfig); + adminClient.createTopics(Collections.singleton(newTopic)).all().get(); + } catch (final InterruptedException | ExecutionException fatal) { + throw new RuntimeException(fatal); + } + + } + + public void deleteTopic(final String topic) { + log.debug("Deleting topic {}", topic); + final Properties properties = new Properties(); + properties.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, brokerList()); + + try (final AdminClient adminClient = AdminClient.create(properties)) { + adminClient.deleteTopics(Collections.singleton(topic)).all().get(); + } catch (final InterruptedException e) { + throw new RuntimeException(e); + } catch (final ExecutionException e) { + if (!(e.getCause() instanceof UnknownTopicOrPartitionException)) { + throw new RuntimeException(e); + } + } + } + + KafkaServer kafkaServer() { + return kafka; + } +} diff --git a/data-prepper-plugins/kafka-plugins/src/integrationTest/java/org/opensearch/dataprepper/plugins/kafka/source/EmbeddedZooKeeperServer.java b/data-prepper-plugins/kafka-plugins/src/integrationTest/java/org/opensearch/dataprepper/plugins/kafka/source/EmbeddedZooKeeperServer.java new file mode 100644 index 0000000000..e4e3d9fdd9 --- /dev/null +++ b/data-prepper-plugins/kafka-plugins/src/integrationTest/java/org/opensearch/dataprepper/plugins/kafka/source/EmbeddedZooKeeperServer.java @@ -0,0 +1,46 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.kafka.source; + +import org.apache.curator.test.TestingServer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; + +/** + * Runs an in-memory, "embedded" instance of a ZooKeeper server. + * + * The ZooKeeper server instance is automatically started when you create a new instance of this class. + */ +public class EmbeddedZooKeeperServer { + + private static final Logger log = LoggerFactory.getLogger(EmbeddedZooKeeperServer.class); + + private final TestingServer server; + + public EmbeddedZooKeeperServer() throws Exception { + log.debug("Starting embedded ZooKeeper server..."); + this.server = new TestingServer(); + log.debug("Embedded ZooKeeper server at {} uses the temp directory at {}", + server.getConnectString(), server.getTempDirectory()); + } + + public void stop() throws IOException { + log.debug("Shutting down embedded ZooKeeper server at {} ...", server.getConnectString()); + server.close(); + log.debug("Shutdown of embedded ZooKeeper server at {} completed", server.getConnectString()); + } + + public String connectString() { + return server.getConnectString(); + } + + public String hostname() { + return connectString().substring(0, connectString().lastIndexOf(':')); + } + +} diff --git a/data-prepper-plugins/kafka-plugins/src/integrationTest/java/org/opensearch/dataprepper/plugins/kafka/source/JSONConsumerIT.java b/data-prepper-plugins/kafka-plugins/src/integrationTest/java/org/opensearch/dataprepper/plugins/kafka/source/JSONConsumerIT.java new file mode 100644 index 0000000000..cc777b25df --- /dev/null +++ b/data-prepper-plugins/kafka-plugins/src/integrationTest/java/org/opensearch/dataprepper/plugins/kafka/source/JSONConsumerIT.java @@ -0,0 +1,115 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.kafka.source; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.datatype.jsr310.JavaTimeModule; +import io.confluent.kafka.serializers.AbstractKafkaAvroSerDeConfig; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.kafka.connect.json.JsonSerializer; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.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.record.Record; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.plugins.kafka.configuration.KafkaSourceConfig; +import org.opensearch.dataprepper.plugins.kafka.configuration.SchemaConfig; +import org.opensearch.dataprepper.plugins.kafka.configuration.TopicConfig; +import org.yaml.snakeyaml.Yaml; + +import java.io.FileReader; +import java.io.IOException; +import java.io.Reader; +import java.io.StringReader; +import java.util.List; +import java.util.Map; +import java.util.Properties; + +@ExtendWith(MockitoExtension.class) +public class JSONConsumerIT { + + private PluginMetrics pluginMetrics; + @Mock + TopicConfig topicConfig; + @Mock + private SchemaConfig schemaConfig; + private KafkaSourceConfig kafkaSourceConfig; + + private KafkaSource kafkaSource; + private Buffer> buffer; + + @ClassRule + public static final EmbeddedKafkaClusterSingleNode CLUSTER = new EmbeddedKafkaClusterSingleNode(); + + @BeforeClass + public static void createTopics() { + CLUSTER.createTopic("test-IT-topic-1"); + } + + @Before + public void configure() throws IOException { + Yaml yaml = new Yaml(); + FileReader fileReader = new FileReader(getClass().getClassLoader().getResource("sample-pipelines-int.yaml").getFile()); + Object data = yaml.load(fileReader); + if(data instanceof Map){ + Map propertyMap = (Map) data; + Map logPipelineMap = (Map) propertyMap.get("log-pipeline"); + Map sourceMap = (Map) logPipelineMap.get("source"); + Map kafkaConfigMap = (Map) sourceMap.get("kafka"); + ObjectMapper mapper = new ObjectMapper(); + mapper.registerModule(new JavaTimeModule()); + String json = mapper.writeValueAsString(kafkaConfigMap); + Reader reader = new StringReader(json); + kafkaSourceConfig = mapper.readValue(reader, KafkaSourceConfig.class); + List topicConfigList = kafkaSourceConfig.getTopics(); + topicConfig = topicConfigList.get(0); + schemaConfig = kafkaSourceConfig.getSchemaConfig(); + } + } + + + @Test + public void testKafkaMessagesForJsonConsumer() throws JsonProcessingException { + produceTestMessages(); + kafkaSource.start(buffer); + } + + private void produceTestMessages() throws JsonProcessingException { + + String value = "{\"writebuffer\":\"true\",\"buffertype\":\"json\"}"; + JsonNode mapper = new ObjectMapper().readTree(value); + + final Properties props = new Properties(); + props.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "127.0.0.1:9092"); + props.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, + StringSerializer.class); + props.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, + JsonSerializer.class); + props.put(AbstractKafkaAvroSerDeConfig.SCHEMA_REGISTRY_URL_CONFIG, CLUSTER.schemaRegistryUrl()); + try (KafkaProducer producer = new KafkaProducer(props)) { + + for (long i = 0; i < 10; i++) { + producer.send(new ProducerRecord<>("test-IT-topic-1", + mapper )); + Thread.sleep(1000L); + } + producer.flush(); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + } +} diff --git a/data-prepper-plugins/kafka-plugins/src/integrationTest/java/org/opensearch/dataprepper/plugins/kafka/source/PlainTextConsumerIT.java b/data-prepper-plugins/kafka-plugins/src/integrationTest/java/org/opensearch/dataprepper/plugins/kafka/source/PlainTextConsumerIT.java new file mode 100644 index 0000000000..a5118e64c5 --- /dev/null +++ b/data-prepper-plugins/kafka-plugins/src/integrationTest/java/org/opensearch/dataprepper/plugins/kafka/source/PlainTextConsumerIT.java @@ -0,0 +1,105 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.kafka.source; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.datatype.jsr310.JavaTimeModule; +import io.confluent.kafka.serializers.AbstractKafkaAvroSerDeConfig; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.serialization.StringSerializer; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Test; +import org.mockito.Mock; +import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.buffer.Buffer; +import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.plugins.kafka.configuration.KafkaSourceConfig; +import org.opensearch.dataprepper.plugins.kafka.configuration.SchemaConfig; +import org.opensearch.dataprepper.plugins.kafka.configuration.TopicConfig; +import org.yaml.snakeyaml.Yaml; + +import java.io.FileReader; +import java.io.IOException; +import java.io.Reader; +import java.io.StringReader; +import java.util.List; +import java.util.Map; +import java.util.Properties; + + +public class PlainTextConsumerIT { + + private PluginMetrics pluginMetrics; + @Mock + TopicConfig topicConfig; + @Mock + private SchemaConfig schemaConfig; + private KafkaSourceConfig kafkaSourceConfig; + + private KafkaSource kafkaSource; + private Buffer> buffer; + + @ClassRule + public static final EmbeddedKafkaClusterSingleNode CLUSTER = new EmbeddedKafkaClusterSingleNode(); + + @BeforeClass + public static void createTopics() { + CLUSTER.createTopic("test-IT-topic"); + } + + @Before + public void configure() throws IOException { + Yaml yaml = new Yaml(); + FileReader fileReader = new FileReader(getClass().getClassLoader().getResource("sample-pipelines-int.yaml").getFile()); + Object data = yaml.load(fileReader); + if(data instanceof Map){ + Map propertyMap = (Map) data; + Map logPipelineMap = (Map) propertyMap.get("log-pipeline"); + Map sourceMap = (Map) logPipelineMap.get("source"); + Map kafkaConfigMap = (Map) sourceMap.get("kafka"); + ObjectMapper mapper = new ObjectMapper(); + mapper.registerModule(new JavaTimeModule()); + String json = mapper.writeValueAsString(kafkaConfigMap); + Reader reader = new StringReader(json); + kafkaSourceConfig = mapper.readValue(reader, KafkaSourceConfig.class); + List topicConfigList = kafkaSourceConfig.getTopics(); + topicConfig = topicConfigList.get(0); + schemaConfig = kafkaSourceConfig.getSchemaConfig(); + } + } + + @Test + public void consumeKafkaMessages_should_return_at_least_one_message() { + produceTestMessages(); + kafkaSource.start(buffer); + } + + private void produceTestMessages() { + + final Properties props = new Properties(); + props.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "127.0.0.1:9092"); + props.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, + StringSerializer.class); + props.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, + StringSerializer.class); + props.put(AbstractKafkaAvroSerDeConfig.SCHEMA_REGISTRY_URL_CONFIG, CLUSTER.schemaRegistryUrl()); + try (KafkaProducer producer = new KafkaProducer(props)) { + for (long i = 0; i < 10; i++) { + producer.send(new ProducerRecord<>("test-IT-topic", + "hello" + i)); + Thread.sleep(1000L); + } + producer.flush(); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + } +} diff --git a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/AwsConfig.java b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/AwsConfig.java new file mode 100644 index 0000000000..4f4fa91cb5 --- /dev/null +++ b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/AwsConfig.java @@ -0,0 +1,19 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.kafka.configuration; + +import com.fasterxml.jackson.annotation.JsonProperty; +import jakarta.validation.constraints.Size; + +public class AwsConfig { + @JsonProperty("msk_arn") + @Size(min = 20, max = 2048, message = "mskArn length should be between 20 and 2048 characters") + private String awsMskArn; + + public String getAwsMskArn() { + return awsMskArn; + } +} diff --git a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/KafkaSourceConfig.java b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/KafkaSourceConfig.java index d4ba49d14c..bf8645c200 100644 --- a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/KafkaSourceConfig.java +++ b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/KafkaSourceConfig.java @@ -11,6 +11,7 @@ import jakarta.validation.constraints.Size; import java.util.List; +import java.time.Duration; /** * * A helper class that helps to read user configuration values from @@ -18,6 +19,7 @@ */ public class KafkaSourceConfig { + public static final Duration DEFAULT_ACKNOWLEDGEMENTS_TIMEOUT = Duration.ofSeconds(30); @JsonProperty("bootstrap_servers") @NotNull @@ -36,6 +38,23 @@ public class KafkaSourceConfig { @JsonProperty("authentication") private AuthConfig authConfig; + @JsonProperty("aws") + private AwsConfig awsConfig; + + @JsonProperty("acknowledgments") + private Boolean acknowledgementsEnabled = false; + + @JsonProperty("acknowledgments_timeout") + private Duration acknowledgementsTimeout = DEFAULT_ACKNOWLEDGEMENTS_TIMEOUT; + + public Boolean getAcknowledgementsEnabled() { + return acknowledgementsEnabled; + } + + public Duration getAcknowledgementsTimeout() { + return acknowledgementsTimeout; + } + public List getTopics() { return topics; } diff --git a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/TopicConfig.java b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/TopicConfig.java index 4a42649fae..b2536170cb 100644 --- a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/TopicConfig.java +++ b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/TopicConfig.java @@ -21,7 +21,7 @@ public class TopicConfig { private static final Duration SESSION_TIMEOUT = Duration.ofSeconds(45); private static final int MAX_RETRY_ATTEMPT = Integer.MAX_VALUE; private static final String AUTO_OFFSET_RESET = "earliest"; - private static final Duration THREAD_WAITING_TIME = Duration.ofSeconds(1); + static final Duration THREAD_WAITING_TIME = Duration.ofSeconds(5); private static final Duration MAX_RECORD_FETCH_TIME = Duration.ofSeconds(4); private static final Duration BUFFER_DEFAULT_TIMEOUT = Duration.ofSeconds(5); private static final Duration MAX_RETRY_DELAY = Duration.ofSeconds(1); @@ -30,7 +30,7 @@ public class TopicConfig { private static final Long FETCH_MIN_BYTES = 1L; private static final Duration RETRY_BACKOFF = Duration.ofSeconds(100); private static final Duration MAX_POLL_INTERVAL = Duration.ofSeconds(300000); - private static final Long CONSUMER_MAX_POLL_RECORDS = 500L; + private static final Integer CONSUMER_MAX_POLL_RECORDS = 500; private static final Integer NUM_OF_WORKERS = 10; private static final Duration HEART_BEAT_INTERVAL_DURATION = Duration.ofSeconds(3); @@ -54,10 +54,10 @@ public class TopicConfig { @Size(min = 1) private Duration maxRetryDelay = MAX_RETRY_DELAY; - @JsonProperty("autocommit") - private String autoCommit = AUTO_COMMIT; + @JsonProperty("auto_commit") + private Boolean autoCommit = false; - @JsonProperty("autocommit_interval") + @JsonProperty("auto_commit_interval") @Size(min = 1) private Duration autoCommitInterval = AUTOCOMMIT_INTERVAL; @@ -100,7 +100,7 @@ public class TopicConfig { private Duration maxPollInterval = MAX_POLL_INTERVAL; @JsonProperty("consumer_max_poll_records") - private Long consumerMaxPollRecords = CONSUMER_MAX_POLL_RECORDS; + private Integer consumerMaxPollRecords = CONSUMER_MAX_POLL_RECORDS; @JsonProperty("heart_beat_interval") @Size(min = 1) @@ -118,7 +118,7 @@ public void setMaxRetryAttempts(Integer maxRetryAttempts) { this.maxRetryAttempts = maxRetryAttempts; } - public String getAutoCommit() { + public Boolean getAutoCommit() { return autoCommit; } @@ -186,7 +186,7 @@ public void setFetchMaxBytes(Long fetchMaxBytes) { this.fetchMaxBytes = fetchMaxBytes; } - public void setAutoCommit(String autoCommit) { + public void setAutoCommit(Boolean autoCommit) { this.autoCommit = autoCommit; } @@ -222,11 +222,11 @@ public void setMaxPollInterval(Duration maxPollInterval) { this.maxPollInterval = maxPollInterval; } - public Long getConsumerMaxPollRecords() { + public Integer getConsumerMaxPollRecords() { return consumerMaxPollRecords; } - public void setConsumerMaxPollRecords(Long consumerMaxPollRecords) { + public void setConsumerMaxPollRecords(Integer consumerMaxPollRecords) { this.consumerMaxPollRecords = consumerMaxPollRecords; } diff --git a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/consumer/KafkaSourceCustomConsumer.java b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/consumer/KafkaSourceCustomConsumer.java index f409193d1f..6bd35391b9 100644 --- a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/consumer/KafkaSourceCustomConsumer.java +++ b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/consumer/KafkaSourceCustomConsumer.java @@ -4,6 +4,10 @@ */ package org.opensearch.dataprepper.plugins.kafka.consumer; +import com.fasterxml.jackson.core.JsonFactory; +import com.fasterxml.jackson.core.JsonParser; +import com.fasterxml.jackson.databind.ObjectMapper; +import io.micrometer.core.instrument.Counter; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecords; import org.apache.kafka.clients.consumer.KafkaConsumer; @@ -11,12 +15,16 @@ import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.clients.consumer.CommitFailedException; import org.apache.kafka.common.TopicPartition; +import org.opensearch.dataprepper.model.log.JacksonLog; import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.buffer.Buffer; import org.opensearch.dataprepper.model.record.Record; -import org.opensearch.dataprepper.plugins.kafka.configuration.KafkaSourceConfig; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSetManager; +import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSet; import org.opensearch.dataprepper.plugins.kafka.configuration.TopicConfig; -import org.opensearch.dataprepper.plugins.kafka.source.KafkaSourceBufferAccumulator; +import org.opensearch.dataprepper.plugins.kafka.configuration.KafkaSourceConfig; +import org.opensearch.dataprepper.buffer.common.BufferAccumulator; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -27,85 +35,204 @@ import java.util.ArrayList; import java.util.Collection; import java.util.List; +import java.util.Objects; import java.util.concurrent.atomic.AtomicBoolean; +import org.opensearch.dataprepper.plugins.kafka.util.MessageFormat; +import org.apache.commons.lang3.Range; /** * * A utility class which will handle the core Kafka consumer operation. */ -public class KafkaSourceCustomConsumer implements ConsumerRebalanceListener { +public class KafkaSourceCustomConsumer implements Runnable, ConsumerRebalanceListener { private static final Logger LOG = LoggerFactory.getLogger(KafkaSourceCustomConsumer.class); - private Map offsetsToCommit = new HashMap<>(); - private long lastReadOffset = 0L; - private volatile long lastCommitTime = System.currentTimeMillis(); - private KafkaConsumer consumer= null; - private AtomicBoolean status = new AtomicBoolean(false); - private Buffer> buffer= null; - private TopicConfig topicConfig = null; - private KafkaSourceConfig kafkaSourceConfig= null; - private PluginMetrics pluginMetrics= null; - private String schemaType= null; + private static final Long COMMIT_OFFSET_INTERVAL_MS = 300000L; + private static final int DEFAULT_NUMBER_OF_RECORDS_TO_ACCUMULATE = 1; + static final String POSITIVE_ACKNOWLEDGEMENT_METRIC_NAME = "positiveAcknowledgementSetCounter"; + static final String NEGATIVE_ACKNOWLEDGEMENT_METRIC_NAME = "negativeAcknowledgementSetCounter"; + static final String DEFAULT_KEY = "message"; - public KafkaSourceCustomConsumer() { - } + private volatile long lastCommitTime; + private KafkaConsumer consumer= null; + private AtomicBoolean shutdownInProgress; + private final String topicName; + private final TopicConfig topicConfig; + private PluginMetrics pluginMetrics= null; + private MessageFormat schema; + private final BufferAccumulator> bufferAccumulator; + private final Buffer> buffer; + private static final ObjectMapper objectMapper = new ObjectMapper(); + private final JsonFactory jsonFactory = new JsonFactory(); + private Map offsetsToCommit; + private final AcknowledgementSetManager acknowledgementSetManager; + private final Map partitionCommitTrackerMap; + private final Counter positiveAcknowledgementSetCounter; + private final Counter negativeAcknowledgementSetCounter; + private final boolean acknowledgementsEnabled; + private final Duration acknowledgementsTimeout; - private KafkaSourceBufferAccumulator kafkaSourceBufferAccumulator= null; - public KafkaSourceCustomConsumer(KafkaConsumer consumer, - AtomicBoolean status, - Buffer> buffer, - TopicConfig topicConfig, - KafkaSourceConfig kafkaSourceConfig, - String schemaType, - PluginMetrics pluginMetrics) { + public KafkaSourceCustomConsumer(final KafkaConsumer consumer, + final AtomicBoolean shutdownInProgress, + final Buffer> buffer, + final KafkaSourceConfig sourceConfig, + final TopicConfig topicConfig, + final String schemaType, + final AcknowledgementSetManager acknowledgementSetManager, + final PluginMetrics pluginMetrics) { + this.topicName = topicConfig.getName(); + this.topicConfig = topicConfig; + this.shutdownInProgress = shutdownInProgress; this.consumer = consumer; - this.status = status; this.buffer = buffer; - this.topicConfig = topicConfig; - this.kafkaSourceConfig = kafkaSourceConfig; - this.schemaType = schemaType; + this.offsetsToCommit = new HashMap<>(); + this.acknowledgementsTimeout = sourceConfig.getAcknowledgementsTimeout(); + // If the timeout value is different from default value, then enable acknowledgements automatically. + this.acknowledgementsEnabled = sourceConfig.getAcknowledgementsEnabled() || acknowledgementsTimeout != KafkaSourceConfig.DEFAULT_ACKNOWLEDGEMENTS_TIMEOUT; + this.acknowledgementSetManager = acknowledgementSetManager; this.pluginMetrics = pluginMetrics; - kafkaSourceBufferAccumulator= new KafkaSourceBufferAccumulator(topicConfig, kafkaSourceConfig, - schemaType, pluginMetrics); + this.partitionCommitTrackerMap = new HashMap<>(); + this.schema = MessageFormat.getByMessageFormatByName(schemaType); + Duration bufferTimeout = Duration.ofSeconds(1); + this.bufferAccumulator = BufferAccumulator.create(buffer, DEFAULT_NUMBER_OF_RECORDS_TO_ACCUMULATE, bufferTimeout); + this.lastCommitTime = System.currentTimeMillis(); + this.positiveAcknowledgementSetCounter = pluginMetrics.counter(POSITIVE_ACKNOWLEDGEMENT_METRIC_NAME); + this.negativeAcknowledgementSetCounter = pluginMetrics.counter(NEGATIVE_ACKNOWLEDGEMENT_METRIC_NAME); } + public void updateOffsetsToCommit(final TopicPartition partition, final OffsetAndMetadata offsetAndMetadata) { + if (Objects.isNull(offsetAndMetadata)) { + return; + } + synchronized (this) { + offsetsToCommit.put(partition, offsetAndMetadata); + } + } - @SuppressWarnings({"rawtypes", "unchecked"}) - public void consumeRecords() { - try { - consumer.subscribe(Arrays.asList(topicConfig.getName())); - do { - offsetsToCommit.clear(); - ConsumerRecords records = poll(consumer); - if (!records.isEmpty() && records.count() > 0) { - iterateRecordPartitions(records); + private AcknowledgementSet createAcknowledgementSet(Map> offsets) { + AcknowledgementSet acknowledgementSet = + acknowledgementSetManager.create((result) -> { + if (result == true) { + positiveAcknowledgementSetCounter.increment(); + offsets.forEach((partition, offsetRange) -> { + int partitionId = partition.partition(); + if (!partitionCommitTrackerMap.containsKey(partitionId)) { + OffsetAndMetadata committedOffsetAndMetadata = consumer.committed(partition); + Long committedOffset = Objects.nonNull(committedOffsetAndMetadata) ? committedOffsetAndMetadata.offset() : null; + + partitionCommitTrackerMap.put(partitionId, new TopicPartitionCommitTracker(partition, committedOffset)); + } + OffsetAndMetadata offsetAndMetadata = partitionCommitTrackerMap.get(partitionId).addCompletedOffsets(offsetRange); + updateOffsetsToCommit(partition, offsetAndMetadata); + }); + } else { + positiveAcknowledgementSetCounter.increment(); } - }while (!status.get()); - } catch (Exception exp) { - LOG.error("Error while reading the records from the topic...", exp); + }, acknowledgementsTimeout); + return acknowledgementSet; + } + + double getPositiveAcknowledgementsCount() { + return positiveAcknowledgementSetCounter.count(); + } + + public void consumeRecords() throws Exception { + ConsumerRecords records = + consumer.poll(topicConfig.getThreadWaitingTime().toMillis()/2); + if (!records.isEmpty() && records.count() > 0) { + Map> offsets = new HashMap<>(); + AcknowledgementSet acknowledgementSet = null; + if (acknowledgementsEnabled) { + acknowledgementSet = createAcknowledgementSet(offsets); + } + iterateRecordPartitions(records, acknowledgementSet, offsets); + if (!acknowledgementsEnabled) { + offsets.forEach((partition, offsetRange) -> + updateOffsetsToCommit(partition, new OffsetAndMetadata(offsetRange.getMaximum() + 1))); + } } } - private void iterateRecordPartitions(ConsumerRecords records) throws Exception { - for (TopicPartition partition : records.partitions()) { - List> kafkaRecords = new ArrayList<>(); - List> partitionRecords = records.records(partition); - iterateConsumerRecords(kafkaRecords, partitionRecords); - if (!kafkaRecords.isEmpty()) { - kafkaSourceBufferAccumulator.writeAllRecordToBuffer(kafkaRecords, buffer, topicConfig); + private void commitOffsets() { + if (topicConfig.getAutoCommit()) { + return; + } + long currentTimeMillis = System.currentTimeMillis(); + if ((currentTimeMillis - lastCommitTime) < COMMIT_OFFSET_INTERVAL_MS) { + return; + } + synchronized (this) { + if (offsetsToCommit.isEmpty()) { + return; + } + try { + consumer.commitSync(); + offsetsToCommit.clear(); + lastCommitTime = currentTimeMillis; + } catch (CommitFailedException e) { + LOG.error("Failed to commit offsets in topic "+topicName); } } - if (!offsetsToCommit.isEmpty() && topicConfig.getAutoCommit().equalsIgnoreCase("false")) { - lastCommitTime = kafkaSourceBufferAccumulator.commitOffsets(consumer, lastCommitTime, offsetsToCommit); + } + + Map getOffsetsToCommit() { + return offsetsToCommit; + } + + @Override + public void run() { + try { + consumer.subscribe(Arrays.asList(topicName)); + while (!shutdownInProgress.get()) { + consumeRecords(); + commitOffsets(); + } + } catch (Exception exp) { + LOG.error("Error while reading the records from the topic...", exp); } } - private void iterateConsumerRecords(List> kafkaRecords, List> partitionRecords) { - for (ConsumerRecord consumerRecord : partitionRecords) { - lastReadOffset = kafkaSourceBufferAccumulator.processConsumerRecords(offsetsToCommit, kafkaRecords, lastReadOffset, consumerRecord, partitionRecords); + private Record getRecord(ConsumerRecord consumerRecord) { + Map data = new HashMap<>(); + Event event; + Object value; + String key = (String)consumerRecord.key(); + if (Objects.isNull(key)) { + key = DEFAULT_KEY; } + if (schema == MessageFormat.JSON || schema == MessageFormat.AVRO) { + value = new HashMap<>(); + try { + final JsonParser jsonParser = jsonFactory.createParser((String)consumerRecord.value().toString()); + value = objectMapper.readValue(jsonParser, Map.class); + } catch (Exception e){ + LOG.error("Failed to parse JSON or AVRO record"); + return null; + } + } else { + value = (String)consumerRecord.value(); + } + data.put(key, value); + event = JacksonLog.builder().withData(data).build(); + return new Record(event); } - private ConsumerRecords poll(final KafkaConsumer consumer) { - return consumer.poll(Duration.ofMillis(1)); + private void iterateRecordPartitions(ConsumerRecords records, final AcknowledgementSet acknowledgementSet, Map> offsets) throws Exception { + for (TopicPartition topicPartition : records.partitions()) { + List> kafkaRecords = new ArrayList<>(); + List> partitionRecords = records.records(topicPartition); + for (ConsumerRecord consumerRecord : partitionRecords) { + Record record = getRecord(consumerRecord); + if (record != null) { + bufferAccumulator.add(record); + if (acknowledgementSet != null) { + acknowledgementSet.add(record.getData()); + } + } + } + long lastOffset = partitionRecords.get(partitionRecords.size() - 1).offset(); + long firstOffset = partitionRecords.get(0).offset(); + Range offsetRange = Range.between(firstOffset, lastOffset); + offsets.put(topicPartition, offsetRange); + } } public void closeConsumer(){ @@ -117,17 +244,13 @@ public void shutdownConsumer(){ } @Override public void onPartitionsAssigned(Collection partitions) { - for (TopicPartition partition : partitions) { - consumer.seek(partition, lastReadOffset); + for (TopicPartition topicPartition : partitions) { + Long committedOffset = consumer.committed(topicPartition).offset(); + consumer.seek(topicPartition, committedOffset); } } @Override public void onPartitionsRevoked(Collection partitions) { - try { - consumer.commitSync(offsetsToCommit); - } catch (CommitFailedException e) { - LOG.error("Failed to commit the record for the Json consumer...", e); - } } } diff --git a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/consumer/MultithreadedConsumer.java b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/consumer/MultithreadedConsumer.java deleted file mode 100644 index cff5286a91..0000000000 --- a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/consumer/MultithreadedConsumer.java +++ /dev/null @@ -1,99 +0,0 @@ -/* - * Copyright OpenSearch Contributors - * SPDX-License-Identifier: Apache-2.0 - */ - -package org.opensearch.dataprepper.plugins.kafka.consumer; - -import com.fasterxml.jackson.databind.JsonNode; -import org.apache.avro.generic.GenericRecord; -import org.apache.kafka.clients.consumer.KafkaConsumer; -import org.apache.kafka.common.errors.WakeupException; -import org.opensearch.dataprepper.metrics.PluginMetrics; -import org.opensearch.dataprepper.model.buffer.Buffer; -import org.opensearch.dataprepper.model.record.Record; -import org.opensearch.dataprepper.plugins.kafka.configuration.KafkaSourceConfig; -import org.opensearch.dataprepper.plugins.kafka.configuration.TopicConfig; -import org.opensearch.dataprepper.plugins.kafka.util.MessageFormat; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.time.LocalDateTime; -import java.util.Objects; -import java.util.Properties; -import java.util.concurrent.atomic.AtomicBoolean; - -/** - * * A Multithreaded helper class which helps to process the records from multiple topics in an - * asynchronous way. - */ -@SuppressWarnings("deprecation") -public class MultithreadedConsumer implements Runnable { - private static final Logger LOG = LoggerFactory.getLogger(MultithreadedConsumer.class); - private final AtomicBoolean status = new AtomicBoolean(false); - private final KafkaSourceConfig sourceConfig; - private final TopicConfig topicConfig; - private final Buffer> buffer; - private final KafkaSourceCustomConsumer customConsumer = new KafkaSourceCustomConsumer(); - private String consumerId; - private String consumerGroupId; - private String schemaType; - private Properties consumerProperties; - private PluginMetrics pluginMetrics; - - public MultithreadedConsumer(String consumerId, - String consumerGroupId, - Properties properties, - TopicConfig topicConfig, - KafkaSourceConfig sourceConfig, - Buffer> buffer, - PluginMetrics pluginMetric, - String schemaType) { - this.consumerProperties = Objects.requireNonNull(properties); - this.consumerId = consumerId; - this.consumerGroupId = consumerGroupId; - this.sourceConfig = sourceConfig; - this.topicConfig = topicConfig; - this.buffer = buffer; - this.schemaType = schemaType; - this.pluginMetrics = pluginMetric; - } - - @SuppressWarnings({"unchecked"}) - @Override - public void run() { - LOG.info("Consumer group : {} and Consumer : {} executed on : {}", consumerGroupId, consumerId, LocalDateTime.now()); - try { - MessageFormat schema = MessageFormat.getByMessageFormatByName(schemaType); - switch (schema) { - case JSON: - new KafkaSourceCustomConsumer(new KafkaConsumer(consumerProperties), status, buffer, topicConfig, sourceConfig, schemaType, pluginMetrics).consumeRecords(); - break; - case AVRO: - new KafkaSourceCustomConsumer(new KafkaConsumer(consumerProperties), status, buffer, topicConfig, sourceConfig, schemaType, pluginMetrics).consumeRecords(); - break; - case PLAINTEXT: - default: - new KafkaSourceCustomConsumer(new KafkaConsumer(consumerProperties), status, buffer, topicConfig, sourceConfig, schemaType, pluginMetrics).consumeRecords(); - break; - } - - } catch (Exception exp) { - if (exp.getCause() instanceof WakeupException && !status.get()) { - LOG.error("Error reading records from the topic...{}", exp.getMessage()); - } - } finally { - LOG.info("Closing the consumer... {}", consumerId); - closeConsumers(); - } - } - - private void closeConsumers() { - customConsumer.closeConsumer(); - } - - public void shutdownConsumer() { - status.set(false); - customConsumer.shutdownConsumer(); - } -} diff --git a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/consumer/TopicPartitionCommitTracker.java b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/consumer/TopicPartitionCommitTracker.java new file mode 100644 index 0000000000..9d10b46611 --- /dev/null +++ b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/consumer/TopicPartitionCommitTracker.java @@ -0,0 +1,79 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.kafka.consumer; + +import java.util.HashMap; +import java.util.Map; +import java.util.Objects; + +import org.apache.commons.lang3.Range; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.apache.kafka.common.TopicPartition; + +public class TopicPartitionCommitTracker { + private long committedOffset; + private final TopicPartition topicPartition; + private final Map> offsetMaxMap; + private final Map> offsetMinMap; + + public TopicPartitionCommitTracker(final TopicPartition topicPartition, Long committedOffset) { + this.topicPartition = topicPartition; + this.committedOffset = Objects.nonNull(committedOffset) ? committedOffset : -1L; + this.offsetMaxMap = new HashMap<>(); + this.offsetMinMap = new HashMap<>(); + this.offsetMaxMap.put(this.committedOffset, Range.between(this.committedOffset, this.committedOffset)); + } + + public TopicPartitionCommitTracker(final String topic, final int partition, Long committedOffset) { + this(new TopicPartition(topic, partition), committedOffset); + } + + public OffsetAndMetadata addCompletedOffsets(final Range offsetRange) { + Long min = offsetRange.getMinimum(); + Long max = offsetRange.getMaximum(); + boolean merged = false; + if (offsetMaxMap.containsKey(min - 1)) { + Range entry = offsetMaxMap.get(min - 1); + offsetMaxMap.remove(min - 1); + offsetMinMap.remove(entry.getMinimum()); + min = entry.getMinimum(); + Range newEntry = Range.between(min, max); + offsetMaxMap.put(max, newEntry); + offsetMinMap.put(min, newEntry); + merged = true; + } + if (offsetMinMap.containsKey(max + 1)) { + Range entry = offsetMinMap.get(max + 1); + offsetMinMap.remove(max + 1); + if (merged) { + offsetMinMap.remove(min); + offsetMaxMap.remove(max); + } + max = entry.getMaximum(); + offsetMaxMap.remove(max); + Range newEntry = Range.between(min, max); + offsetMaxMap.put(max, newEntry); + offsetMinMap.put(min, newEntry); + merged = true; + } + if (!merged) { + offsetMaxMap.put(max, offsetRange); + offsetMinMap.put(min, offsetRange); + return null; + } + if (offsetMinMap.containsKey(committedOffset)) { + Long maxValue = offsetMinMap.get(committedOffset).getMaximum(); + if (maxValue != committedOffset) { + offsetMinMap.remove(committedOffset); + committedOffset = maxValue; + offsetMaxMap.put(committedOffset, Range.between(committedOffset, committedOffset)); + return new OffsetAndMetadata(committedOffset + 1); + } + } + return null; + } + +} diff --git a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/source/KafkaSource.java b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/source/KafkaSource.java index 51b7362eee..6c69c259c2 100644 --- a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/source/KafkaSource.java +++ b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/source/KafkaSource.java @@ -5,6 +5,7 @@ package org.opensearch.dataprepper.plugins.kafka.source; +import org.apache.avro.generic.GenericRecord; import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.ObjectMapper; import io.confluent.kafka.serializers.AbstractKafkaSchemaSerDeConfig; @@ -12,16 +13,19 @@ import io.micrometer.core.instrument.Counter; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.clients.consumer.KafkaConsumer; import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.annotations.DataPrepperPlugin; import org.opensearch.dataprepper.model.annotations.DataPrepperPluginConstructor; import org.opensearch.dataprepper.model.buffer.Buffer; import org.opensearch.dataprepper.model.configuration.PipelineDescription; import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.model.source.Source; +import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSetManager; import org.opensearch.dataprepper.plugins.kafka.configuration.KafkaSourceConfig; import org.opensearch.dataprepper.plugins.kafka.configuration.TopicConfig; -import org.opensearch.dataprepper.plugins.kafka.consumer.MultithreadedConsumer; +import org.opensearch.dataprepper.plugins.kafka.consumer.KafkaSourceCustomConsumer; import org.opensearch.dataprepper.plugins.kafka.util.KafkaSourceJsonDeserializer; import org.opensearch.dataprepper.plugins.kafka.util.MessageFormat; import org.slf4j.Logger; @@ -43,6 +47,7 @@ import java.util.concurrent.TimeUnit; import java.util.stream.IntStream; +import java.util.concurrent.atomic.AtomicBoolean; /** * The starting point of the Kafka-source plugin and the Kafka consumer * properties and kafka multithreaded consumers are being handled here. @@ -50,58 +55,76 @@ @SuppressWarnings("deprecation") @DataPrepperPlugin(name = "kafka", pluginType = Source.class, pluginConfigurationType = KafkaSourceConfig.class) -public class KafkaSource implements Source> { +public class KafkaSource implements Source> { private static final Logger LOG = LoggerFactory.getLogger(KafkaSource.class); private final KafkaSourceConfig sourceConfig; + private AtomicBoolean shutdownInProgress; private ExecutorService executorService; private static final String KAFKA_WORKER_THREAD_PROCESSING_ERRORS = "kafkaWorkerThreadProcessingErrors"; private final Counter kafkaWorkerThreadProcessingErrors; private final PluginMetrics pluginMetrics; - private String consumerGroupID; - private MultithreadedConsumer multithreadedConsumer; - private int totalWorkers; + private KafkaSourceCustomConsumer consumer; private String pipelineName; - private static String schemaType = MessageFormat.PLAINTEXT.toString(); + private String schemaType = MessageFormat.PLAINTEXT.toString(); private static final String SCHEMA_TYPE= "schemaType"; + private final AcknowledgementSetManager acknowledgementSetManager; @DataPrepperPluginConstructor - public KafkaSource(final KafkaSourceConfig sourceConfig, final PluginMetrics pluginMetrics, + public KafkaSource(final KafkaSourceConfig sourceConfig, + final PluginMetrics pluginMetrics, + final AcknowledgementSetManager acknowledgementSetManager, final PipelineDescription pipelineDescription) { this.sourceConfig = sourceConfig; this.pluginMetrics = pluginMetrics; + this.acknowledgementSetManager = acknowledgementSetManager; this.pipelineName = pipelineDescription.getPipelineName(); this.kafkaWorkerThreadProcessingErrors = pluginMetrics.counter(KAFKA_WORKER_THREAD_PROCESSING_ERRORS); + shutdownInProgress = new AtomicBoolean(false); } @Override - public void start(Buffer> buffer) { + public void start(Buffer> buffer) { sourceConfig.getTopics().forEach(topic -> { - totalWorkers = 0; + Properties consumerProperties = getConsumerProperties(topic); + MessageFormat schema = MessageFormat.getByMessageFormatByName(schemaType); + try { - Properties consumerProperties = getConsumerProperties(topic); - totalWorkers = topic.getWorkers(); - consumerGroupID = getGroupId(topic.getName()); - executorService = Executors.newFixedThreadPool(totalWorkers); - IntStream.range(0, totalWorkers + 1).forEach(index -> { - String consumerId = consumerGroupID + "::" + Integer.toString(index + 1); - multithreadedConsumer = new MultithreadedConsumer(consumerId, - consumerGroupID, consumerProperties, topic, sourceConfig, buffer, pluginMetrics, schemaType); - executorService.submit(multithreadedConsumer); + int numWorkers = topic.getWorkers(); + executorService = Executors.newFixedThreadPool(numWorkers); + IntStream.range(0, numWorkers + 1).forEach(index -> { + KafkaConsumer kafkaConsumer; + switch (schema) { + case JSON: + kafkaConsumer = new KafkaConsumer(consumerProperties); + break; + case AVRO: + kafkaConsumer = new KafkaConsumer(consumerProperties); + break; + case PLAINTEXT: + default: + kafkaConsumer = new KafkaConsumer(consumerProperties); + break; + } + consumer = new KafkaSourceCustomConsumer(kafkaConsumer, shutdownInProgress, buffer, sourceConfig, topic, schemaType, acknowledgementSetManager, pluginMetrics); + + executorService.submit(consumer); }); } catch (Exception e) { LOG.error("Failed to setup the Kafka Source Plugin.", e); throw new RuntimeException(); } + LOG.info("Started Kafka source for topic " + topic.getName()); }); } @Override public void stop() { LOG.info("Shutting down Consumers..."); + shutdownInProgress.set(true); executorService.shutdown(); try { if (!executorService.awaitTermination( - calculateLongestThreadWaitingTime(), TimeUnit.MILLISECONDS)) { + calculateLongestThreadWaitingTime(), TimeUnit.SECONDS)) { LOG.info("Consumer threads are waiting for shutting down..."); executorService.shutdownNow(); } @@ -115,10 +138,6 @@ public void stop() { LOG.info("Consumer shutdown successfully..."); } - private String getGroupId(String name) { - return pipelineName + "::" + name; - } - private long calculateLongestThreadWaitingTime() { List topicsList = sourceConfig.getTopics(); return topicsList.stream(). @@ -138,8 +157,12 @@ private Properties getConsumerProperties(TopicConfig topicConfig) { properties.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, sourceConfig.getBootStrapServers()); properties.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, topicConfig.getAutoCommit()); + properties.put(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, + topicConfig.getConsumerMaxPollRecords()); properties.put(ConsumerConfig.GROUP_ID_CONFIG, topicConfig.getGroupId()); - schemaType = getSchemaType(sourceConfig.getSchemaConfig().getRegistryURL(), topicConfig.getName(), sourceConfig.getSchemaConfig().getVersion()); + if (sourceConfig.getSchemaConfig() != null) { + schemaType = getSchemaType(sourceConfig.getSchemaConfig().getRegistryURL(), topicConfig.getName(), sourceConfig.getSchemaConfig().getVersion()); + } if (schemaType.isEmpty()) { schemaType = MessageFormat.PLAINTEXT.toString(); } diff --git a/data-prepper-plugins/kafka-plugins/src/main/resources/sample-pipelines-int.yaml b/data-prepper-plugins/kafka-plugins/src/main/resources/sample-pipelines-int.yaml new file mode 100644 index 0000000000..88efc7b2e5 --- /dev/null +++ b/data-prepper-plugins/kafka-plugins/src/main/resources/sample-pipelines-int.yaml @@ -0,0 +1,34 @@ +log-pipeline: + source: + kafka: + bootstrap_servers: + - 127.0.0.1:9093 + auth_type: plaintext + topics: + - name: my-topic-2 + group_name: kafka-consumer-group-2 + group_id: DPKafkaProj-2 + workers: 10 #optional and default is 10 + autocommit: false #optional and dafault is false + autocommit_interval: 5 #optional and dafault is 5s + session_timeout: 45 #optional and dafault is 45s + max_retry_attempts: 1000 #optional and dafault is 5 + max_retry_delay: 1 #optional and dafault is 5 + auto_offset_reset: earliest #optional and dafault is earliest + thread_waiting_time: 1 #optional and dafault is 1s + max_record_fetch_time: 4 #optional and dafault is 4s + heart_beat_interval: 3 #optional and dafault is 3s + buffer_default_timeout: 5 #optional and dafault is 5s + fetch_max_bytes: 52428800 #optional and dafault is 52428800 + fetch_max_wait: 500 #optional and dafault is 500 + fetch_min_bytes: 1 #optional and dafault is 1 + retry_backoff: 100 #optional and dafault is 10s + max_poll_interval: 300000 #optional and dafault is 300000s + consumer_max_poll_records: 500 #optional and dafault is 500 + - name: my-topic-1 + group_id: DPKafkaProj-1 + schema: + registry_url: http://localhost:8081/ + version: 1 + sink: + - stdout: \ No newline at end of file diff --git a/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/configuration/AwsConfigTest.java b/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/configuration/AwsConfigTest.java new file mode 100644 index 0000000000..d3facb361f --- /dev/null +++ b/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/configuration/AwsConfigTest.java @@ -0,0 +1,42 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.kafka.configuration; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.lang.reflect.Field; +import java.util.UUID; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.MatcherAssert.assertThat; + +class AwsConfigTest { + + private AwsConfig awsConfig; + + @BeforeEach + void setUp() { + awsConfig = new AwsConfig(); + } + + @Test + void getMskArn_notNull() throws NoSuchFieldException, IllegalAccessException { + final String testArn = UUID.randomUUID().toString(); + reflectivelySetField(awsConfig, "awsMskArn", testArn); + assertThat(awsConfig.getAwsMskArn(), equalTo(testArn)); + } + + private void reflectivelySetField(final AwsConfig awsConfig, final String fieldName, final Object value) throws NoSuchFieldException, IllegalAccessException { + final Field field = AwsConfig.class.getDeclaredField(fieldName); + try { + field.setAccessible(true); + field.set(awsConfig, value); + } finally { + field.setAccessible(false); + } + } +} diff --git a/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/configuration/KafkaSourceConfigTest.java b/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/configuration/KafkaSourceConfigTest.java index e7e23c9a04..552baed388 100644 --- a/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/configuration/KafkaSourceConfigTest.java +++ b/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/configuration/KafkaSourceConfigTest.java @@ -17,6 +17,7 @@ import java.util.Arrays; import java.util.Map; import java.util.stream.Collectors; +import java.time.Duration; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.hasItem; @@ -24,6 +25,7 @@ import static org.hamcrest.Matchers.notNullValue; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.mockito.Mockito.mock; +import static org.opensearch.dataprepper.test.helper.ReflectivelySetField.setField; class KafkaSourceConfigTest { @@ -70,11 +72,13 @@ void test_bootStrapServers_not_null(){ @Test void test_topics_not_null(){ + assertEquals(false, kafkaSourceConfig.getAcknowledgementsEnabled()); + assertEquals(KafkaSourceConfig.DEFAULT_ACKNOWLEDGEMENTS_TIMEOUT, kafkaSourceConfig.getAcknowledgementsTimeout()); assertThat(kafkaSourceConfig.getTopics(), notNullValue()); } @Test - void test_setters(){ + void test_setters() throws NoSuchFieldException, IllegalAccessException { kafkaSourceConfig = new KafkaSourceConfig(); kafkaSourceConfig.setBootStrapServers(new ArrayList<>(Arrays.asList("127.0.0.1:9092"))); TopicConfig topicConfig = mock(TopicConfig.class); @@ -82,5 +86,10 @@ void test_setters(){ assertEquals(Arrays.asList("127.0.0.1:9092"), kafkaSourceConfig.getBootStrapServers()); assertEquals(Collections.singletonList(topicConfig), kafkaSourceConfig.getTopics()); + setField(KafkaSourceConfig.class, kafkaSourceConfig, "acknowledgementsEnabled", true); + Duration testTimeout = Duration.ofSeconds(10); + setField(KafkaSourceConfig.class, kafkaSourceConfig, "acknowledgementsTimeout", testTimeout); + assertEquals(true, kafkaSourceConfig.getAcknowledgementsEnabled()); + assertEquals(testTimeout, kafkaSourceConfig.getAcknowledgementsTimeout()); } } diff --git a/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/configuration/TopicConfigTest.java b/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/configuration/TopicConfigTest.java index 3e60bb2771..6c26666adf 100644 --- a/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/configuration/TopicConfigTest.java +++ b/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/configuration/TopicConfigTest.java @@ -69,11 +69,11 @@ void testConfigValues_default() { assertEquals("my-topic-2", topicConfig.getName()); assertEquals("DPKafkaProj-2", topicConfig.getGroupId()); assertEquals("kafka-consumer-group-2", topicConfig.getGroupName()); - assertEquals("false", topicConfig.getAutoCommit()); + assertEquals(false, topicConfig.getAutoCommit()); assertEquals(Duration.ofSeconds(5), topicConfig.getAutoCommitInterval()); assertEquals(Duration.ofSeconds(45), topicConfig.getSessionTimeOut()); assertEquals("earliest", topicConfig.getAutoOffsetReset()); - assertEquals(Duration.ofSeconds(1), topicConfig.getThreadWaitingTime()); + assertEquals(TopicConfig.THREAD_WAITING_TIME, topicConfig.getThreadWaitingTime()); assertEquals(Duration.ofSeconds(4), topicConfig.getMaxRecordFetchTime()); assertEquals(Duration.ofSeconds(5), topicConfig.getBufferDefaultTimeout()); assertEquals(52428800L, topicConfig.getFetchMaxBytes().longValue()); @@ -93,7 +93,7 @@ void testConfigValues_from_yaml() { assertEquals("my-topic-1", topicConfig.getName()); assertEquals("DPKafkaProj-2", topicConfig.getGroupId()); assertEquals("kafka-consumer-group-2", topicConfig.getGroupName()); - assertEquals("false", topicConfig.getAutoCommit()); + assertEquals(false, topicConfig.getAutoCommit()); assertEquals(Duration.ofSeconds(5), topicConfig.getAutoCommitInterval()); assertEquals(Duration.ofSeconds(45), topicConfig.getSessionTimeOut()); assertEquals("earliest", topicConfig.getAutoOffsetReset()); diff --git a/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/consumer/KafkaSourceCustomConsumerTest.java b/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/consumer/KafkaSourceCustomConsumerTest.java index 8567658004..4f4f35d23e 100644 --- a/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/consumer/KafkaSourceCustomConsumerTest.java +++ b/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/consumer/KafkaSourceCustomConsumerTest.java @@ -1,99 +1,128 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + package org.opensearch.dataprepper.plugins.kafka.consumer; import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.datatype.jsr310.JavaTimeModule; +import com.fasterxml.jackson.databind.JsonNode; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecords; import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.TopicPartition; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; -import org.mockito.Mock; import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.CheckpointState; import org.opensearch.dataprepper.model.buffer.Buffer; 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.plugins.buffer.blockingbuffer.BlockingBuffer; import org.opensearch.dataprepper.plugins.kafka.configuration.KafkaSourceConfig; import org.opensearch.dataprepper.plugins.kafka.configuration.TopicConfig; -import org.yaml.snakeyaml.Yaml; +import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSetManager; +import org.opensearch.dataprepper.acknowledgements.DefaultAcknowledgementSetManager; +import io.micrometer.core.instrument.Counter; + +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.ArgumentMatchers.any; +import static org.hamcrest.Matchers.hasEntry; +import static org.hamcrest.MatcherAssert.assertThat; +import org.junit.jupiter.api.Assertions; +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.mockito.junit.jupiter.MockitoSettings; +import org.mockito.quality.Strictness; +import org.mockito.Mock; -import java.io.FileReader; -import java.io.IOException; -import java.io.Reader; -import java.io.StringReader; import java.util.Map; import java.util.HashMap; import java.util.Arrays; import java.util.ArrayList; import java.util.Collection; import java.util.List; -import java.util.LinkedHashMap; -import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; -import java.util.stream.Collectors; - -import static org.mockito.Mockito.spy; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.any; -import static org.mockito.Mockito.doCallRealMethod; - +import java.time.Duration; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +@ExtendWith(MockitoExtension.class) +@MockitoSettings(strictness = Strictness.LENIENT) public class KafkaSourceCustomConsumerTest { + @Mock private KafkaConsumer kafkaConsumer; private AtomicBoolean status; - private Buffer> buffer; + private Buffer> buffer; @Mock private KafkaSourceConfig sourceConfig; + private ExecutorService callbackExecutor; + private AcknowledgementSetManager acknowledgementSetManager; + + @Mock private TopicConfig topicConfig; @Mock private PluginMetrics pluginMetrics; - private String schemaType; - private KafkaSourceCustomConsumer consumer; - private final String TEST_PIPELINE_NAME = "test_pipeline"; - - private Map>> records = new LinkedHashMap>>(); + private ConsumerRecords consumerRecords; + private final String TEST_PIPELINE_NAME = "test_pipeline"; + private AtomicBoolean shutdownInProgress; + private final String testKey1 = "testkey1"; + private final String testKey2 = "testkey2"; + private final String testValue1 = "testValue1"; + private final String testValue2 = "testValue2"; + private final Map testMap1 = Map.of("key1", "value1", "key2", 2); + private final Map testMap2 = Map.of("key3", "value3", "key4", false); + private final String testJsonValue1 = "{ \"key1\": \"value1\", \"key2\": 2}"; + private final String testJsonValue2 = "{ \"key3\": \"value3\", \"key4\": false}"; + private final int testPartition = 0; + private final int testJsonPartition = 1; + private Counter counter; @BeforeEach - public void setUp() throws IOException { - Yaml yaml = new Yaml(); - FileReader fileReader = new FileReader(getClass().getClassLoader().getResource("sample-pipelines.yaml").getFile()); - Object data = yaml.load(fileReader); - if (data instanceof Map) { - Map propertyMap = (Map) data; - Map logPipelineMap = (Map) propertyMap.get("log-pipeline"); - Map sourceMap = (Map) logPipelineMap.get("source"); - Map kafkaConfigMap = (Map) sourceMap.get("kafka"); - ObjectMapper mapper = new ObjectMapper(); - mapper.registerModule(new JavaTimeModule()); - String json = mapper.writeValueAsString(kafkaConfigMap); - Reader reader = new StringReader(json); - sourceConfig = mapper.readValue(reader, KafkaSourceConfig.class); - topicConfig = sourceConfig.getTopics().get(0); - } + public void setUp() { + kafkaConsumer = mock(KafkaConsumer.class); pluginMetrics = mock(PluginMetrics.class); + counter = mock(Counter.class); + topicConfig = mock(TopicConfig.class); + when(topicConfig.getThreadWaitingTime()).thenReturn(Duration.ofSeconds(1)); + when(topicConfig.getAutoCommit()).thenReturn(false); + when(kafkaConsumer.committed(any(TopicPartition.class))).thenReturn(null); + + when(pluginMetrics.counter(anyString())).thenReturn(counter); + doAnswer((i)-> {return null;}).when(counter).increment(); + callbackExecutor = Executors.newFixedThreadPool(2); + acknowledgementSetManager = new DefaultAcknowledgementSetManager(callbackExecutor, Duration.ofMillis(2000)); + + sourceConfig = mock(KafkaSourceConfig.class); buffer = getBuffer(); - status = new AtomicBoolean(true); - kafkaConsumer = mock(KafkaConsumer.class); - schemaType = "plaintext"; - consumer = new KafkaSourceCustomConsumer(kafkaConsumer, status, buffer, topicConfig, sourceConfig, schemaType, pluginMetrics); + shutdownInProgress = new AtomicBoolean(false); + when(topicConfig.getName()).thenReturn("topic1"); } - private BlockingBuffer> getBuffer() { + public KafkaSourceCustomConsumer createObjectUnderTest(String schemaType, boolean acknowledgementsEnabled) { + when(sourceConfig.getAcknowledgementsEnabled()).thenReturn(acknowledgementsEnabled); + when(sourceConfig.getAcknowledgementsTimeout()).thenReturn(KafkaSourceConfig.DEFAULT_ACKNOWLEDGEMENTS_TIMEOUT); + return new KafkaSourceCustomConsumer(kafkaConsumer, shutdownInProgress, buffer, sourceConfig, topicConfig, schemaType, acknowledgementSetManager, pluginMetrics); + } + + private BlockingBuffer> getBuffer() { final HashMap integerHashMap = new HashMap<>(); integerHashMap.put("buffer_size", 10); integerHashMap.put("batch_size", 10); @@ -103,55 +132,130 @@ private BlockingBuffer> getBuffer() { } @Test - public void testConsumeRecords() throws InterruptedException { - + public void testPlainTextConsumeRecords() throws InterruptedException { String topic = topicConfig.getName(); + consumerRecords = createPlainTextRecords(topic); + when(kafkaConsumer.poll(anyLong())).thenReturn(consumerRecords); + consumer = createObjectUnderTest("plaintext", false); - Thread producerThread = new Thread(() -> { - setTopicData(topic); + try { + consumer.consumeRecords(); + } catch (Exception e){} + final Map.Entry>, CheckpointState> bufferRecords = buffer.read(1000); + ArrayList> bufferedRecords = new ArrayList<>(bufferRecords.getKey()); + Assertions.assertEquals(consumerRecords.count(), bufferedRecords.size()); + Map offsetsToCommit = consumer.getOffsetsToCommit(); + Assertions.assertEquals(offsetsToCommit.size(), 1); + offsetsToCommit.forEach((topicPartition, offsetAndMetadata) -> { + Assertions.assertEquals(topicPartition.partition(), testPartition); + Assertions.assertEquals(topicPartition.topic(), topic); + Assertions.assertEquals(offsetAndMetadata.offset(), 2L); }); - producerThread.start(); - TimeUnit.SECONDS.sleep(1); - ConsumerRecords consumerRecords = new ConsumerRecords(records); - when(kafkaConsumer.poll(any())).thenReturn(consumerRecords); - KafkaSourceCustomConsumer spyConsumer = spy(consumer); - spyConsumer.consumeRecords(); - verify(spyConsumer).consumeRecords(); - final Map.Entry>, CheckpointState> bufferRecords = buffer.read(1000); - Assertions.assertEquals(2, new ArrayList<>(bufferRecords.getKey()).size()); - } - private void setTopicData(String topic) { - ConsumerRecord record1 = new ConsumerRecord<>(topic, 0, 0L, "mykey-1", "myvalue-1"); - ConsumerRecord record2 = new ConsumerRecord<>(topic, 0, 0L, "mykey-2", "myvalue-2"); - records.put(new TopicPartition(topic, 1), Arrays.asList(record1, record2)); + for (Record record: bufferedRecords) { + Event event = record.getData(); + String value1 = event.get(testKey1, String.class); + String value2 = event.get(testKey2, String.class); + assertTrue(value1 != null || value2 != null); + if (value1 != null) { + Assertions.assertEquals(value1, testValue1); + } + if (value2 != null) { + Assertions.assertEquals(value2, testValue2); + } + } } @Test - void testOnPartitionsRevoked() { - KafkaSourceCustomConsumer spyConsumer = spy(consumer); - setTopicData(topicConfig.getName()); - final List topicPartitions = records.keySet().stream().collect(Collectors.toList()); - spyConsumer.onPartitionsRevoked(topicPartitions); - verify(spyConsumer).onPartitionsRevoked(topicPartitions); + public void testPlainTextConsumeRecordsWithAcknowledgements() throws InterruptedException { + String topic = topicConfig.getName(); + consumerRecords = createPlainTextRecords(topic); + when(kafkaConsumer.poll(anyLong())).thenReturn(consumerRecords); + consumer = createObjectUnderTest("plaintext", true); + + try { + consumer.consumeRecords(); + } catch (Exception e){} + final Map.Entry>, CheckpointState> bufferRecords = buffer.read(1000); + ArrayList> bufferedRecords = new ArrayList<>(bufferRecords.getKey()); + Assertions.assertEquals(consumerRecords.count(), bufferedRecords.size()); + Map offsetsToCommit = consumer.getOffsetsToCommit(); + Assertions.assertEquals(offsetsToCommit.size(), 0); + + for (Record record: bufferedRecords) { + Event event = record.getData(); + String value1 = event.get(testKey1, String.class); + String value2 = event.get(testKey2, String.class); + assertTrue(value1 != null || value2 != null); + if (value1 != null) { + Assertions.assertEquals(value1, testValue1); + } + if (value2 != null) { + Assertions.assertEquals(value2, testValue2); + } + event.getEventHandle().release(true); + } + // Wait for acknowledgement callback function to run + try { + Thread.sleep(10000); + } catch (Exception e){} + + offsetsToCommit = consumer.getOffsetsToCommit(); + Assertions.assertEquals(offsetsToCommit.size(), 1); + offsetsToCommit.forEach((topicPartition, offsetAndMetadata) -> { + Assertions.assertEquals(topicPartition.partition(), testPartition); + Assertions.assertEquals(topicPartition.topic(), topic); + Assertions.assertEquals(offsetAndMetadata.offset(), 2L); + }); } @Test - void testOnPartitionsAssigned() { - //Map>> records = new LinkedHashMap<>(); - ConsumerRecord record1 = new ConsumerRecord<>("my-topic-1", 0, 0L, "mykey-1", "myvalue-1"); - ConsumerRecord record2 = new ConsumerRecord<>("my-topic-1", 0, 0L, "mykey-2", "myvalue-2"); - //records.put(new TopicPartition("my-topic-1", 1), Arrays.asList(record1, record2)); - TopicPartition partition = new TopicPartition("my-topic-1", 1); - //records.put(partition, Arrays.asList(record1, record2)); - - KafkaSourceCustomConsumer spyConsumer = spy(consumer); - doCallRealMethod().when(spyConsumer).onPartitionsAssigned(Arrays.asList(partition)); - - spyConsumer.onPartitionsAssigned(Arrays.asList(partition)); - verify(spyConsumer).onPartitionsAssigned(Arrays.asList(partition)); - - /*spyConsumer.onPartitionsRevoked(anyList()); - verify(spyConsumer).onPartitionsRevoked(anyList());*/ + public void testJsonConsumeRecords() throws InterruptedException, Exception { + String topic = topicConfig.getName(); + consumerRecords = createJsonRecords(topic); + when(kafkaConsumer.poll(anyLong())).thenReturn(consumerRecords); + consumer = createObjectUnderTest("json", false); + + consumer.consumeRecords(); + final Map.Entry>, CheckpointState> bufferRecords = buffer.read(1000); + ArrayList> bufferedRecords = new ArrayList<>(bufferRecords.getKey()); + Assertions.assertEquals(consumerRecords.count(), bufferedRecords.size()); + Map offsetsToCommit = consumer.getOffsetsToCommit(); + offsetsToCommit.forEach((topicPartition, offsetAndMetadata) -> { + Assertions.assertEquals(topicPartition.partition(), testJsonPartition); + Assertions.assertEquals(topicPartition.topic(), topic); + Assertions.assertEquals(offsetAndMetadata.offset(), 102L); + }); + + for (Record record: bufferedRecords) { + Event event = record.getData(); + Map value1 = event.get(testKey1, Map.class); + Map value2 = event.get(testKey2, Map.class); + assertTrue(value1 != null || value2 != null); + if (value1 != null) { + testMap1.forEach((k, v) -> assertThat(value1, hasEntry(k,v))); + } + if (value2 != null) { + testMap2.forEach((k, v) -> assertThat(value2, hasEntry(k,v))); + } + } + } + + private ConsumerRecords createPlainTextRecords(String topic) { + Map> records = new HashMap<>(); + ConsumerRecord record1 = new ConsumerRecord<>(topic, testPartition, 0L, testKey1, testValue1); + ConsumerRecord record2 = new ConsumerRecord<>(topic, testPartition, 1L, testKey2, testValue2); + records.put(new TopicPartition(topic, testPartition), Arrays.asList(record1, record2)); + return new ConsumerRecords(records); } + + private ConsumerRecords createJsonRecords(String topic) throws Exception { + final ObjectMapper mapper = new ObjectMapper(); + Map> records = new HashMap<>(); + ConsumerRecord record1 = new ConsumerRecord<>(topic, testJsonPartition, 100L, testKey1, mapper.convertValue(testMap1, JsonNode.class)); + ConsumerRecord record2 = new ConsumerRecord<>(topic, testJsonPartition, 101L, testKey2, mapper.convertValue(testMap2, JsonNode.class)); + records.put(new TopicPartition(topic, testJsonPartition), Arrays.asList(record1, record2)); + return new ConsumerRecords(records); + } + } diff --git a/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/consumer/MultithreadedConsumerTest.java b/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/consumer/MultithreadedConsumerTest.java deleted file mode 100644 index da66a26531..0000000000 --- a/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/consumer/MultithreadedConsumerTest.java +++ /dev/null @@ -1,126 +0,0 @@ -package org.opensearch.dataprepper.plugins.kafka.consumer; - -import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.datatype.jsr310.JavaTimeModule; -import org.apache.kafka.clients.consumer.ConsumerConfig; -import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.clients.producer.ProducerConfig; -import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.serialization.StringDeserializer; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; -import org.mockito.Mock; -import org.opensearch.dataprepper.metrics.PluginMetrics; -import org.opensearch.dataprepper.model.buffer.Buffer; -import org.opensearch.dataprepper.model.record.Record; -import org.opensearch.dataprepper.plugins.kafka.configuration.KafkaSourceConfig; -import org.opensearch.dataprepper.plugins.kafka.configuration.TopicConfig; -import org.yaml.snakeyaml.Yaml; - -import java.io.FileReader; -import java.io.IOException; -import java.io.Reader; -import java.io.StringReader; -import java.util.LinkedHashMap; -import java.util.List; -import java.util.Arrays; -import java.util.Properties; -import java.util.Map; -import java.util.concurrent.TimeUnit; - -import static org.mockito.Mockito.spy; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.doCallRealMethod; - - -class MultithreadedConsumerTest { - - @Mock - MultithreadedConsumer multithreadedConsumer; - @Mock - Properties properties; - @Mock - KafkaSourceConfig sourceConfig; - @Mock - TopicConfig topicConfig; - @Mock - Buffer> buffer; - @Mock - PluginMetrics pluginMetrics; - private static final String BOOTSTRAP_SERVERS = "localhost:9092"; - @BeforeEach - void setUp()throws IOException { - Yaml yaml = new Yaml(); - FileReader fileReader = new FileReader(getClass().getClassLoader().getResource("sample-pipelines.yaml").getFile()); - Object data = yaml.load(fileReader); - if (data instanceof Map) { - Map propertyMap = (Map) data; - Map logPipelineMap = (Map) propertyMap.get("log-pipeline"); - Map sourceMap = (Map) logPipelineMap.get("source"); - Map kafkaConfigMap = (Map) sourceMap.get("kafka"); - ObjectMapper mapper = new ObjectMapper(); - mapper.registerModule(new JavaTimeModule()); - String json = mapper.writeValueAsString(kafkaConfigMap); - Reader reader = new StringReader(json); - sourceConfig = mapper.readValue(reader, KafkaSourceConfig.class); - topicConfig = sourceConfig.getTopics().get(0); - } - pluginMetrics = mock(PluginMetrics.class); - buffer = mock(Buffer.class); - } - - private MultithreadedConsumer createObjectUnderTest(String consumerId, - String consumerGroupId, - String schema){ - properties = new Properties(); - properties.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, BOOTSTRAP_SERVERS); - properties.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); - properties.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); - return new MultithreadedConsumer(consumerId, - consumerGroupId, - properties, - topicConfig, - sourceConfig, - buffer, - pluginMetrics, - schema); - } - - // @ParameterizedTest - // @ValueSource(strings = "plaintext") - @Test - void testRunWithPlainText() throws InterruptedException { - String topic = topicConfig.getName(); - //schemaType = "plaintext"; - Map>> records = new LinkedHashMap<>(); - Thread producerThread = new Thread(() -> { - ConsumerRecord record1 = new ConsumerRecord<>(topic, 0, 0L, "mykey-1", "myvalue-1"); - ConsumerRecord record2 = new ConsumerRecord<>(topic, 0, 0L, "mykey-2", "myvalue-2"); - records.put(new TopicPartition(topic, 1), Arrays.asList(record1, record2)); - }); - producerThread.start(); - TimeUnit.SECONDS.sleep(1); - producerThread.join(); - multithreadedConsumer = createObjectUnderTest("DPKafkaProj-1", - "DPKafkaProj-1","plaintext"); - MultithreadedConsumer spySource = spy(multithreadedConsumer); - doCallRealMethod().when(spySource).run(); - // spySource.run(); - //verify(spySource).run(); - } - - - /* @ParameterizedTest - @ValueSource(strings = {"plaintext", "json", "avro"}) - @Test - void testRunWithParameters(String schemaType) { - multithreadedConsumer = createObjectUnderTest("DPKafkaProj-1", - "DPKafkaProj-1", - schemaType); - MultithreadedConsumer spySource = spy(multithreadedConsumer); - doCallRealMethod().when(spySource).run(); - spySource.run(); - verify(spySource).run(); - }*/ - -} \ No newline at end of file diff --git a/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/consumer/TopicPartitionCommitTrackerTest.java b/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/consumer/TopicPartitionCommitTrackerTest.java new file mode 100644 index 0000000000..387ffb909e --- /dev/null +++ b/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/consumer/TopicPartitionCommitTrackerTest.java @@ -0,0 +1,84 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.kafka.consumer; + +import org.apache.commons.lang3.Range; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; +import org.mockito.junit.jupiter.MockitoExtension; + +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.Matchers.greaterThanOrEqualTo; + + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Objects; +import java.util.stream.Stream; + +@ExtendWith(MockitoExtension.class) +class TopicPartitionCommitTrackerTest { + private final String testTopic = "test_topic"; + private final int testPartition = 1; + private TopicPartitionCommitTracker topicPartitionCommitTracker; + public TopicPartitionCommitTracker createObjectUnderTest(String topic, int partition, Long offset) { + return new TopicPartitionCommitTracker(topic, partition, offset); + } + + @ParameterizedTest + @MethodSource("getInputOrder") + public void test(List order) { + topicPartitionCommitTracker = createObjectUnderTest(testTopic, testPartition, -1L); + List> ranges = new ArrayList<>(); + for (int i = 0; i < 10; i++) { + ranges.add(Range.between(i*10L, i*10L+9L)); + } + OffsetAndMetadata result = null; + Long expectedOffset = 10L; + for (Integer i: order) { + result = topicPartitionCommitTracker.addCompletedOffsets(ranges.get(i)); + if (ranges.get(i).getMaximum() == (expectedOffset - 1)) { + assertThat(result.offset(), greaterThanOrEqualTo(expectedOffset)); + expectedOffset = result.offset() + 10L; + } + } + assertTrue(Objects.nonNull(result)); + assertThat(result.offset(), equalTo(100L)); + } + + private static Stream getInputOrder() { + List> orderList = new ArrayList<>(); + for (int i = 0; i < 10; i++) { + List order = new ArrayList<>(); + for (int j = 0; j < 10; j++) { + order.add(j); + } + Collections.shuffle(order); + orderList.add(order); + } + return Stream.of( + Arguments.of(List.of(0, 1, 2, 3, 4, 5, 6, 7, 8, 9)), + Arguments.of(List.of(9, 8, 7, 6, 5, 4, 3, 2, 1, 0)), + Arguments.of(orderList.get(0)), + Arguments.of(orderList.get(1)), + Arguments.of(orderList.get(2)), + Arguments.of(orderList.get(3)), + Arguments.of(orderList.get(4)), + Arguments.of(orderList.get(5)), + Arguments.of(orderList.get(6)), + Arguments.of(orderList.get(7)), + Arguments.of(orderList.get(8)), + Arguments.of(orderList.get(9)) + ); + } + +} diff --git a/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/source/KafkaSourceTest.java b/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/source/KafkaSourceTest.java index ec8d15e9b1..2dbe126691 100644 --- a/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/source/KafkaSourceTest.java +++ b/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/source/KafkaSourceTest.java @@ -5,11 +5,17 @@ package org.opensearch.dataprepper.plugins.kafka.source; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.datatype.jsr310.JavaTimeModule; -import org.apache.kafka.clients.consumer.ConsumerConfig; -import org.apache.kafka.clients.producer.ProducerConfig; -import org.apache.kafka.common.serialization.StringDeserializer; +import org.opensearch.dataprepper.model.buffer.Buffer; +import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.configuration.PipelineDescription; +import org.opensearch.dataprepper.plugins.kafka.configuration.KafkaSourceConfig; +import org.opensearch.dataprepper.plugins.kafka.configuration.SchemaConfig; +import org.opensearch.dataprepper.plugins.kafka.configuration.TopicConfig; +import org.opensearch.dataprepper.plugins.kafka.configuration.PlainTextAuthConfig; +import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSetManager; + import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -18,37 +24,17 @@ import org.mockito.junit.jupiter.MockitoExtension; import org.mockito.junit.jupiter.MockitoSettings; import org.mockito.quality.Strictness; -import org.opensearch.dataprepper.metrics.PluginMetrics; -import org.opensearch.dataprepper.model.configuration.PipelineDescription; -import org.opensearch.dataprepper.plugins.kafka.configuration.KafkaSourceConfig; -import org.opensearch.dataprepper.plugins.kafka.configuration.SchemaConfig; -import org.opensearch.dataprepper.plugins.kafka.configuration.TopicConfig; -import org.opensearch.dataprepper.plugins.kafka.configuration.OAuthConfig; -import org.opensearch.dataprepper.plugins.kafka.configuration.PlainTextAuthConfig; -import org.opensearch.dataprepper.plugins.kafka.consumer.MultithreadedConsumer; -import org.springframework.test.util.ReflectionTestUtils; -import org.yaml.snakeyaml.Yaml; - -import java.io.FileReader; -import java.io.Reader; -import java.io.StringReader; -import java.util.ArrayList; -import java.util.List; -import java.util.Map; -import java.util.Properties; -import java.util.concurrent.ExecutorService; - -import static org.mockito.ArgumentMatchers.any; -import static org.mockito.Mockito.spy; -import static org.mockito.Mockito.doCallRealMethod; -import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; +import java.util.Arrays; +import java.util.List; +import java.time.Duration; @ExtendWith(MockitoExtension.class) @MockitoSettings(strictness = Strictness.LENIENT) class KafkaSourceTest { - @Mock - private KafkaSource source; + private KafkaSource kafkaSource; @Mock private KafkaSourceConfig sourceConfig; @@ -57,73 +43,70 @@ class KafkaSourceTest { private PluginMetrics pluginMetrics; @Mock - private ExecutorService executorService; + private SchemaConfig schemaConfig; @Mock - private SchemaConfig schemaConfig; + private AcknowledgementSetManager acknowledgementSetManager; @Mock private TopicConfig topicConfig; @Mock private PipelineDescription pipelineDescription; @Mock - OAuthConfig oAuthConfig; - @Mock PlainTextAuthConfig plainTextAuthConfig; + @Mock + TopicConfig topic1, topic2; + @Mock + private Buffer> buffer; + private static final String BOOTSTRAP_SERVERS = "localhost:9092"; private static final String TOPIC = "my-topic"; + private static final String TEST_GROUP_ID = "testGroupId"; + + + public KafkaSource createObjectUnderTest() { + return new KafkaSource(sourceConfig, pluginMetrics, acknowledgementSetManager, pipelineDescription); + } @BeforeEach void setUp() throws Exception { - Yaml yaml = new Yaml(); - FileReader fileReader = new FileReader(getClass().getClassLoader().getResource("sample-pipelines.yaml").getFile()); - Object data = yaml.load(fileReader); - if (data instanceof Map) { - Map propertyMap = (Map) data; - Map logPipelineMap = (Map) propertyMap.get("log-pipeline"); - Map sourceMap = (Map) logPipelineMap.get("source"); - Map kafkaConfigMap = (Map) sourceMap.get("kafka"); - ObjectMapper mapper = new ObjectMapper(); - mapper.registerModule(new JavaTimeModule()); - String json = mapper.writeValueAsString(kafkaConfigMap); - Reader reader = new StringReader(json); - sourceConfig = mapper.readValue(reader, KafkaSourceConfig.class); - topicConfig = sourceConfig.getTopics().get(0); - oAuthConfig = sourceConfig.getAuthConfig().getoAuthConfig(); - plainTextAuthConfig = sourceConfig.getAuthConfig().getPlainTextAuthConfig(); - schemaConfig = sourceConfig.getSchemaConfig(); - } + sourceConfig = mock(KafkaSourceConfig.class); + pipelineDescription = mock(PipelineDescription.class); + pluginMetrics = mock(PluginMetrics.class); + acknowledgementSetManager = mock(AcknowledgementSetManager.class); + when(topic1.getName()).thenReturn("topic1"); + when(topic2.getName()).thenReturn("topic2"); + when(topic1.getWorkers()).thenReturn(2); + when(topic2.getWorkers()).thenReturn(3); + when(topic1.getAutoCommitInterval()).thenReturn(Duration.ofSeconds(1)); + when(topic2.getAutoCommitInterval()).thenReturn(Duration.ofSeconds(1)); + when(topic1.getAutoOffsetReset()).thenReturn("earliest"); + when(topic2.getAutoOffsetReset()).thenReturn("earliest"); + when(topic1.getConsumerMaxPollRecords()).thenReturn(1); + when(topic2.getConsumerMaxPollRecords()).thenReturn(1); + when(topic1.getGroupId()).thenReturn(TEST_GROUP_ID); + when(topic2.getGroupId()).thenReturn(TEST_GROUP_ID); + when(topic1.getAutoCommit()).thenReturn(false); + when(topic2.getAutoCommit()).thenReturn(false); + when(topic1.getThreadWaitingTime()).thenReturn(Duration.ofSeconds(10)); + when(topic2.getThreadWaitingTime()).thenReturn(Duration.ofSeconds(10)); + when(sourceConfig.getBootStrapServers()).thenReturn(List.of("http://localhost:1234")); + when(sourceConfig.getTopics()).thenReturn(Arrays.asList(topic1, topic2)); } @Test - void test_kafkaSource_start_execution_catch_block() { - source = new KafkaSource(null, pluginMetrics, pipelineDescription); - KafkaSource spySource = spy(source); - Assertions.assertThrows(Exception.class, () -> spySource.start(any())); + void test_kafkaSource_start_stop() { + kafkaSource = createObjectUnderTest(); + kafkaSource.start(buffer); + try { + Thread.sleep(10); + } catch (Exception e){} + kafkaSource.stop(); } @Test - void test_kafkaSource_stop_execution() throws Exception { - List consumers = buildKafkaSourceConsumer(); - source = new KafkaSource(sourceConfig, pluginMetrics,pipelineDescription); - KafkaSource spySource = spy(source); - ReflectionTestUtils.setField(spySource, "executorService", executorService); - doCallRealMethod().when(spySource).stop(); - spySource.stop(); - verify(spySource).stop(); - } - - private List buildKafkaSourceConsumer() { - List consumers = new ArrayList<>(); - Properties prop = new Properties(); - prop.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, BOOTSTRAP_SERVERS); - prop.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); - prop.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); - MultithreadedConsumer kafkaSourceConsumer = new MultithreadedConsumer( - topicConfig.getGroupId(), - topicConfig.getGroupId(), - prop, null,sourceConfig, null, pluginMetrics,null); - consumers.add(kafkaSourceConsumer); - return consumers; + void test_kafkaSource_start_execution_catch_block() { + kafkaSource = createObjectUnderTest(); + Assertions.assertThrows(Exception.class, () -> kafkaSource.start(null)); } } diff --git a/data-prepper-plugins/kafka-plugins/src/test/resources/sample-pipelines.yaml b/data-prepper-plugins/kafka-plugins/src/test/resources/sample-pipelines.yaml index 2c398a99dc..efd860a8a9 100644 --- a/data-prepper-plugins/kafka-plugins/src/test/resources/sample-pipelines.yaml +++ b/data-prepper-plugins/kafka-plugins/src/test/resources/sample-pipelines.yaml @@ -8,8 +8,8 @@ log-pipeline: group_name: kafka-consumer-group-2 group_id: DPKafkaProj-2 workers: 10 #optional and default is 10 - autocommit: false #optional and dafault is false - autocommit_interval: 5 #optional and dafault is 5s + auto_commit: false #optional and dafault is false + auto_commit_interval: 5 #optional and dafault is 5s session_timeout: 45 #optional and dafault is 45s max_retry_attempts: 1000 #optional and dafault is 5 max_retry_delay: 1 #optional and dafault is 5 @@ -48,4 +48,4 @@ log-pipeline: oauth_sasl_login_callback_handler_class: org.apache.kafka.common.security.oauthbearer.secured.OAuthBearerLoginCallbackHandler oauth_jwks_endpoint_url: https://dev-13650048.okta.com/oauth2/default/v1/keys sink: - - stdout: \ No newline at end of file + - stdout: diff --git a/data-prepper-plugins/key-value-processor/README.md b/data-prepper-plugins/key-value-processor/README.md index a53a818ba7..55dda2dfc2 100644 --- a/data-prepper-plugins/key-value-processor/README.md +++ b/data-prepper-plugins/key-value-processor/README.md @@ -61,6 +61,11 @@ When run, the processor will parse the message into the following output: * There is no default * Cannot be an empty string * Example: `delete_value_regex` is `"\s"`. `{"key1=value1 "}` will parse into `{"key1": "value1"}` +* `transform_key` - Change keys to lowercase, uppercase, or all capitals. + * Default is an empty string (no transformation) + * Example: `transform_key` is `lowercase`. `{"Key1=value1"}` will parse into `{"key1": "value1"}` + * Example: `transform_key` is `uppercase`. `{"key1=value1"}` will parse into `{"Key1": "value1"}` + * Example: `transform_key` is `capitalize`. `{"key1=value1"}` will parse into `{"KEY1": "value1"}` ## Developer Guide This plugin is compatible with Java 14. See diff --git a/data-prepper-plugins/key-value-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/keyvalue/KeyValueProcessor.java b/data-prepper-plugins/key-value-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/keyvalue/KeyValueProcessor.java index 49cfc323cd..d531c0b57a 100644 --- a/data-prepper-plugins/key-value-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/keyvalue/KeyValueProcessor.java +++ b/data-prepper-plugins/key-value-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/keyvalue/KeyValueProcessor.java @@ -35,6 +35,10 @@ public class KeyValueProcessor extends AbstractProcessor, Record includeKeysSet = new HashSet(); + private final String LOWERCASE_KEY = "lowercase"; + private final String UPPERCASE_KEY = "uppercase"; + private final String CAPITALIZE_KEY = "capitalize"; + private final Set validTransformOptionSet = Set.of("", LOWERCASE_KEY, UPPERCASE_KEY, CAPITALIZE_KEY); @DataPrepperPluginConstructor public KeyValueProcessor(final PluginMetrics pluginMetrics, final KeyValueProcessorConfig keyValueProcessorConfig) { @@ -94,6 +98,10 @@ public KeyValueProcessor(final PluginMetrics pluginMetrics, final KeyValueProces if(keyValueProcessorConfig.getIncludeKeys() != null) { includeKeysSet.addAll(keyValueProcessorConfig.getIncludeKeys()); } + + if(!validTransformOptionSet.contains(keyValueProcessorConfig.getTransformKey())) { + throw new IllegalArgumentException(String.format("The transform_key value: %s is not a valid option", keyValueProcessorConfig.getTransformKey())); + } } private String buildRegexFromCharacters(String s) { @@ -162,6 +170,11 @@ public Collection> doExecute(final Collection> recor value = ((String)value).replaceAll(keyValueProcessorConfig.getDeleteValueRegex(), ""); } + if(keyValueProcessorConfig.getTransformKey() != null + && !keyValueProcessorConfig.getTransformKey().isEmpty()) { + key = transformKey(key); + } + addKeyValueToMap(parsedMap, key, value); } @@ -171,6 +184,17 @@ public Collection> doExecute(final Collection> recor return records; } + private String transformKey(String key) { + if(keyValueProcessorConfig.getTransformKey().equals(LOWERCASE_KEY)) { + key = key.toLowerCase(); + } else if(keyValueProcessorConfig.getTransformKey().equals(UPPERCASE_KEY)) { + key = key.substring(0, 1).toUpperCase() + key.substring(1); + } else if(keyValueProcessorConfig.getTransformKey().equals(CAPITALIZE_KEY)) { + key = key.toUpperCase(); + } + return key; + } + private void addKeyValueToMap(final Map parsedMap, final String key, final Object value) { if(!parsedMap.containsKey(key)) { parsedMap.put(key, value); diff --git a/data-prepper-plugins/key-value-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/keyvalue/KeyValueProcessorConfig.java b/data-prepper-plugins/key-value-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/keyvalue/KeyValueProcessorConfig.java index 42d75d7abe..4e696696f1 100644 --- a/data-prepper-plugins/key-value-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/keyvalue/KeyValueProcessorConfig.java +++ b/data-prepper-plugins/key-value-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/keyvalue/KeyValueProcessorConfig.java @@ -22,6 +22,7 @@ public class KeyValueProcessorConfig { static final String DEFAULT_PREFIX = ""; static final String DEFAULT_DELETE_KEY_REGEX = ""; static final String DEFAULT_DELETE_VALUE_REGEX = ""; + static final String DEFAULT_TRANSFORM_KEY = ""; @NotEmpty private String source = DEFAULT_SOURCE; @@ -60,6 +61,10 @@ public class KeyValueProcessorConfig { @NotNull private String deleteValueRegex = DEFAULT_DELETE_VALUE_REGEX; + @JsonProperty("transform_key") + @NotNull + private String transformKey = DEFAULT_TRANSFORM_KEY; + public String getSource() { return source; } @@ -103,4 +108,8 @@ public String getDeleteKeyRegex() { public String getDeleteValueRegex() { return deleteValueRegex; } + + public String getTransformKey() { + return transformKey; + } } diff --git a/data-prepper-plugins/key-value-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/keyvalue/KeyValueProcessorTests.java b/data-prepper-plugins/key-value-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/keyvalue/KeyValueProcessorTests.java index 1b2ef07971..35af3e0108 100644 --- a/data-prepper-plugins/key-value-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/keyvalue/KeyValueProcessorTests.java +++ b/data-prepper-plugins/key-value-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/keyvalue/KeyValueProcessorTests.java @@ -63,6 +63,7 @@ void setup() { lenient().when(mockConfig.getPrefix()).thenReturn(defaultConfig.getPrefix()); lenient().when(mockConfig.getDeleteKeyRegex()).thenReturn(defaultConfig.getDeleteKeyRegex()); lenient().when(mockConfig.getDeleteValueRegex()).thenReturn(defaultConfig.getDeleteValueRegex()); + lenient().when(mockConfig.getTransformKey()).thenReturn(defaultConfig.getTransformKey()); keyValueProcessor = new KeyValueProcessor(pluginMetrics, mockConfig); } @@ -371,6 +372,42 @@ void testDeleteValueAndKeyRegexKvProcessor() { assertThatKeyEquals(parsed_message, "key2", "value2"); } + @Test + void testLowercaseTransformKvProcessor() { + when(mockConfig.getTransformKey()).thenReturn("lowercase"); + + final Record record = getMessage("Key1=value1"); + final List> editedRecords = (List>) keyValueProcessor.doExecute(Collections.singletonList(record)); + final LinkedHashMap parsed_message = getLinkedHashMap(editedRecords); + + assertThat(parsed_message.size(), equalTo(1)); + assertThatKeyEquals(parsed_message, "key1", "value1"); + } + + @Test + void testUppercaseTransformKvProcessor() { + when(mockConfig.getTransformKey()).thenReturn("uppercase"); + + final Record record = getMessage("key1=value1"); + final List> editedRecords = (List>) keyValueProcessor.doExecute(Collections.singletonList(record)); + final LinkedHashMap parsed_message = getLinkedHashMap(editedRecords); + + assertThat(parsed_message.size(), equalTo(1)); + assertThatKeyEquals(parsed_message, "Key1", "value1"); + } + + @Test + void testCapitalizeTransformKvProcessor() { + when(mockConfig.getTransformKey()).thenReturn("capitalize"); + + final Record record = getMessage("key1=value1"); + final List> editedRecords = (List>) keyValueProcessor.doExecute(Collections.singletonList(record)); + final LinkedHashMap parsed_message = getLinkedHashMap(editedRecords); + + assertThat(parsed_message.size(), equalTo(1)); + assertThatKeyEquals(parsed_message, "KEY1", "value1"); + } + @Test void testShutdownIsReady() { assertThat(keyValueProcessor.isReadyForShutdown(), is(true)); diff --git a/data-prepper-plugins/mutate-event-processors/README.md b/data-prepper-plugins/mutate-event-processors/README.md index 8d81f9eb94..bcc869ba08 100644 --- a/data-prepper-plugins/mutate-event-processors/README.md +++ b/data-prepper-plugins/mutate-event-processors/README.md @@ -261,8 +261,11 @@ and the type conversion processor will change it to the following output, where {"message": "10.10.10.10 [19/Feb/2015:15:50:36 -0500] 200", "clientip":"10.10.10.10", "timestamp": "19/Feb/2015:15:50:36 -0500", "response_status": 200} ``` ### Configuration -* `key` - (required) - keys whose value needs to be converted to a different type +* `key` - keys whose value needs to be converted to a different type. Required if `keys` option is not defined. +* `keys` - list of keys whose value needs to be converted to a different type. Required if `key` option is not defined. * `type` - target type for the value of the key. Possible values are `integer`, `double`, `string`, and `boolean`. Default is `integer`. +* `null_values` - treat any value in the null_values list as null. + * Example: `null_values` is `["-"]` and `key` is `key1`. `{"key1": "-", "key2": "value2"}` will parse into `{"key2": "value2"}` ## List-to-map Processor A processor that converts a list of objects from an event, where each object has a key field, to a map of keys to objects. diff --git a/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/ConvertEntryTypeProcessor.java b/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/ConvertEntryTypeProcessor.java index 17b7074fc1..d8806bde6c 100644 --- a/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/ConvertEntryTypeProcessor.java +++ b/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/ConvertEntryTypeProcessor.java @@ -16,12 +16,13 @@ import org.opensearch.dataprepper.typeconverter.TypeConverter; import java.util.Collection; +import java.util.Collections; import java.util.List; import java.util.Objects; @DataPrepperPlugin(name = "convert_entry_type", pluginType = Processor.class, pluginConfigurationType = ConvertEntryTypeProcessorConfig.class) public class ConvertEntryTypeProcessor extends AbstractProcessor, Record> { - private final String key; + private final List convertEntryKeys; private final TypeConverter converter; private final String convertWhen; private final List nullValues; @@ -33,7 +34,7 @@ public ConvertEntryTypeProcessor(final PluginMetrics pluginMetrics, final ConvertEntryTypeProcessorConfig convertEntryTypeProcessorConfig, final ExpressionEvaluator expressionEvaluator) { super(pluginMetrics); - this.key = convertEntryTypeProcessorConfig.getKey(); + this.convertEntryKeys = getKeysToConvert(convertEntryTypeProcessorConfig); this.converter = convertEntryTypeProcessorConfig.getType().getTargetConverter(); this.convertWhen = convertEntryTypeProcessorConfig.getConvertWhen(); this.nullValues = convertEntryTypeProcessorConfig.getNullValues() @@ -50,11 +51,13 @@ public Collection> doExecute(final Collection> recor continue; } - Object keyVal = recordEvent.get(key, Object.class); - if (keyVal != null) { - recordEvent.delete(key); - if (!nullValues.contains(keyVal.toString())){ - recordEvent.put(key, this.converter.convert(keyVal)); + for(final String key : convertEntryKeys) { + Object keyVal = recordEvent.get(key, Object.class); + if (keyVal != null) { + recordEvent.delete(key); + if (!nullValues.contains(keyVal.toString())) { + recordEvent.put(key, this.converter.convert(keyVal)); + } } } } @@ -73,6 +76,25 @@ public boolean isReadyForShutdown() { @Override public void shutdown() { } + + private List getKeysToConvert(final ConvertEntryTypeProcessorConfig convertEntryTypeProcessorConfig) { + final String key = convertEntryTypeProcessorConfig.getKey(); + final List keys = convertEntryTypeProcessorConfig.getKeys(); + if (key == null && keys == null) { + throw new IllegalArgumentException("key and keys cannot both be null. One must be provided."); + } + if (key != null && keys != null) { + throw new IllegalArgumentException("key and keys cannot both be defined."); + } + if (key != null) { + if (key.isEmpty()) { + throw new IllegalArgumentException("key cannot be empty."); + } else { + return Collections.singletonList(key); + } + } + return keys; + } } diff --git a/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/ConvertEntryTypeProcessorConfig.java b/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/ConvertEntryTypeProcessorConfig.java index 983fe57fcf..16f53b324d 100644 --- a/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/ConvertEntryTypeProcessorConfig.java +++ b/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/ConvertEntryTypeProcessorConfig.java @@ -6,16 +6,17 @@ package org.opensearch.dataprepper.plugins.processor.mutateevent; import com.fasterxml.jackson.annotation.JsonProperty; -import jakarta.validation.constraints.NotEmpty; import java.util.List; import java.util.Optional; public class ConvertEntryTypeProcessorConfig { @JsonProperty("key") - @NotEmpty private String key; + @JsonProperty("keys") + private List keys; + @JsonProperty("type") private TargetType type = TargetType.INTEGER; @@ -29,6 +30,8 @@ public String getKey() { return key; } + public List getKeys() { return keys; } + public TargetType getType() { return type; } diff --git a/data-prepper-plugins/mutate-event-processors/src/test/java/org/opensearch/dataprepper/plugins/processor/mutateevent/ConvertEntryTypeProcessorTests.java b/data-prepper-plugins/mutate-event-processors/src/test/java/org/opensearch/dataprepper/plugins/processor/mutateevent/ConvertEntryTypeProcessorTests.java index 02f8712deb..1bddb03718 100644 --- a/data-prepper-plugins/mutate-event-processors/src/test/java/org/opensearch/dataprepper/plugins/processor/mutateevent/ConvertEntryTypeProcessorTests.java +++ b/data-prepper-plugins/mutate-event-processors/src/test/java/org/opensearch/dataprepper/plugins/processor/mutateevent/ConvertEntryTypeProcessorTests.java @@ -26,6 +26,7 @@ import static org.hamcrest.CoreMatchers.notNullValue; import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.mockito.Mockito.lenient; import static org.mockito.Mockito.when; @ExtendWith(MockitoExtension.class) @@ -51,8 +52,9 @@ static Record buildRecordWithEvent(final Map data) { @BeforeEach private void setup() { - when(mockConfig.getKey()).thenReturn(TEST_KEY); - when(mockConfig.getConvertWhen()).thenReturn(null); + lenient().when(mockConfig.getKey()).thenReturn(TEST_KEY); + lenient().when(mockConfig.getKeys()).thenReturn(null); + lenient().when(mockConfig.getConvertWhen()).thenReturn(null); } private Record getMessage(String message, String key, Object value) { @@ -196,4 +198,42 @@ void testNoConversionWhenConvertWhenIsFalse() { Event event = executeAndGetProcessedEvent(record); assertThat(event.get(TEST_KEY, Integer.class), equalTo(testValue)); } + + @Test + void testMultipleKeysConvertEntryTypeProcessor() { + Integer testValue = 123; + String expectedValue = testValue.toString(); + String testKey1 = UUID.randomUUID().toString(); + String testKey2 = UUID.randomUUID().toString(); + when(mockConfig.getKey()).thenReturn(null); + when(mockConfig.getKeys()).thenReturn(List.of(testKey1, testKey2)); + when(mockConfig.getType()).thenReturn(TargetType.fromOptionValue("string")); + final Map testData = new HashMap(); + testData.put("message", "testMessage"); + testData.put(testKey1, testValue); + testData.put(testKey2, testValue); + Record record = buildRecordWithEvent(testData); + typeConversionProcessor = new ConvertEntryTypeProcessor(pluginMetrics, mockConfig, expressionEvaluator); + Event event = executeAndGetProcessedEvent(record); + assertThat(event.get(testKey1, String.class), equalTo(expectedValue)); + assertThat(event.get(testKey2, String.class), equalTo(expectedValue)); + } + + @Test + void testKeyAndKeysBothNullConvertEntryTypeProcessor() { + when(mockConfig.getKey()).thenReturn(null); + assertThrows(IllegalArgumentException.class, () -> new ConvertEntryTypeProcessor(pluginMetrics, mockConfig, expressionEvaluator)); + } + + @Test + void testKeyAndKeysBothDefinedConvertEntryTypeProcessor() { + when(mockConfig.getKeys()).thenReturn(Collections.singletonList(TEST_KEY)); + assertThrows(IllegalArgumentException.class, () -> new ConvertEntryTypeProcessor(pluginMetrics, mockConfig, expressionEvaluator)); + } + + @Test + void testEmptyKeyConvertEntryTypeProcessor() { + when(mockConfig.getKey()).thenReturn(""); + assertThrows(IllegalArgumentException.class, () -> new ConvertEntryTypeProcessor(pluginMetrics, mockConfig, expressionEvaluator)); + } } diff --git a/data-prepper-plugins/mutate-event-processors/src/test/java/org/opensearch/dataprepper/plugins/processor/mutateevent/ConvertEntryTypeProcessor_NullValueTests.java b/data-prepper-plugins/mutate-event-processors/src/test/java/org/opensearch/dataprepper/plugins/processor/mutateevent/ConvertEntryTypeProcessor_NullValueTests.java index b156b2b3b6..5fd9df9bfb 100644 --- a/data-prepper-plugins/mutate-event-processors/src/test/java/org/opensearch/dataprepper/plugins/processor/mutateevent/ConvertEntryTypeProcessor_NullValueTests.java +++ b/data-prepper-plugins/mutate-event-processors/src/test/java/org/opensearch/dataprepper/plugins/processor/mutateevent/ConvertEntryTypeProcessor_NullValueTests.java @@ -22,6 +22,7 @@ import static org.hamcrest.CoreMatchers.notNullValue; import static org.hamcrest.CoreMatchers.nullValue; import static org.hamcrest.MatcherAssert.assertThat; +import static org.mockito.Mockito.lenient; import static org.mockito.Mockito.when; @ExtendWith(MockitoExtension.class) @@ -42,9 +43,10 @@ public class ConvertEntryTypeProcessor_NullValueTests { @BeforeEach private void setup() { - when(mockConfig.getKey()).thenReturn(TEST_KEY); - when(mockConfig.getType()).thenReturn(TargetType.fromOptionValue("integer")); - when(mockConfig.getConvertWhen()).thenReturn(null); + lenient().when(mockConfig.getKey()).thenReturn(TEST_KEY); + lenient().when(mockConfig.getKeys()).thenReturn(null); + lenient().when(mockConfig.getType()).thenReturn(TargetType.fromOptionValue("integer")); + lenient().when(mockConfig.getConvertWhen()).thenReturn(null); } private Event executeAndGetProcessedEvent(final Object testValue) { @@ -117,4 +119,23 @@ void testMultipleElementNullValues() { assertThat(event.get(TEST_KEY, Integer.class), equalTo(testNumber)); } + @Test + void testMultipleKeysNullValues() { + String testValue = "-"; + String testKey1 = UUID.randomUUID().toString(); + String testKey2 = UUID.randomUUID().toString(); + when(mockConfig.getKey()).thenReturn(null); + when(mockConfig.getKeys()).thenReturn(List.of(testKey1, testKey2)); + when(mockConfig.getNullValues()).thenReturn(Optional.of(List.of("-"))); + final Map testData = new HashMap(); + testData.put("message", "testMessage"); + testData.put(testKey1, testValue); + testData.put(testKey2, testValue); + Record record = buildRecordWithEvent(testData); + nullValuesProcessor = new ConvertEntryTypeProcessor(pluginMetrics, mockConfig, expressionEvaluator); + Event event = executeAndGetProcessedEvent(record); + assertThat(event.get(testKey1, String.class), nullValue()); + assertThat(event.get(testKey2, String.class), nullValue()); + } + } diff --git a/data-prepper-plugins/newline-codecs/README.md b/data-prepper-plugins/newline-codecs/README.md new file mode 100644 index 0000000000..094e57d40f --- /dev/null +++ b/data-prepper-plugins/newline-codecs/README.md @@ -0,0 +1,57 @@ +# Newline Delimited JSON Sink/Output Codec + +This is an implementation of Newline Sink Codec that parses the Dataprepper Events into Newline rows and writes them into the underlying OutputStream. + +## Usages + +Newline Output Codec can be configured with sink plugins (e.g. S3 Sink) in the Pipeline file. + +## Configuration Options + +``` +pipeline: + ... + sink: + - s3: + aws: + region: us-east-1 + sts_role_arn: arn:aws:iam::123456789012:role/Data-Prepper + sts_header_overrides: + max_retries: 5 + bucket: bucket_name + object_key: + path_prefix: my-elb/%{yyyy}/%{MM}/%{dd}/ + threshold: + event_count: 2000 + maximum_size: 50mb + event_collect_timeout: 15s + codec: + newline: + header_destination: header + exclude_keys: + - s3 + buffer_type: in_memory +``` + +## AWS Configuration + +### Codec Configuration: + +1) `header_destination`: The key corresponding to which the header value has to be placed by the codec. +2) `exclude_keys`: Those keys of the events that the user wants to exclude while converting them to newline rows. + + +## Developer Guide + +This plugin is compatible with Java 11. See below + +- [CONTRIBUTING](https://github.com/opensearch-project/data-prepper/blob/main/CONTRIBUTING.md) +- [monitoring](https://github.com/opensearch-project/data-prepper/blob/main/docs/monitoring.md) + +The integration tests for this plugin do not run as part of the Data Prepper build. + +The following command runs the integration tests: + +``` +./gradlew :data-prepper-plugins:s3-sink:integrationTest -Dtests.s3sink.region= -Dtests.s3sink.bucket= +``` diff --git a/data-prepper-plugins/newline-codecs/src/main/java/org/opensearch/dataprepper/plugins/codec/newline/NewlineDelimitedOutputCodec.java b/data-prepper-plugins/newline-codecs/src/main/java/org/opensearch/dataprepper/plugins/codec/newline/NewlineDelimitedOutputCodec.java index 46a0db6789..9e07f60e9b 100644 --- a/data-prepper-plugins/newline-codecs/src/main/java/org/opensearch/dataprepper/plugins/codec/newline/NewlineDelimitedOutputCodec.java +++ b/data-prepper-plugins/newline-codecs/src/main/java/org/opensearch/dataprepper/plugins/codec/newline/NewlineDelimitedOutputCodec.java @@ -4,42 +4,75 @@ */ package org.opensearch.dataprepper.plugins.codec.newline; +import com.fasterxml.jackson.databind.ObjectMapper; import org.opensearch.dataprepper.model.annotations.DataPrepperPlugin; +import org.opensearch.dataprepper.model.annotations.DataPrepperPluginConstructor; import org.opensearch.dataprepper.model.codec.OutputCodec; import org.opensearch.dataprepper.model.event.Event; import java.io.IOException; import java.io.OutputStream; +import java.util.Map; +import java.util.Objects; /** * An implementation of {@link OutputCodec} which deserializes Data-Prepper events * and writes them to Output Stream as ND-JSON data */ -@DataPrepperPlugin(name = "newline", pluginType = OutputCodec.class) +@DataPrepperPlugin(name = "newline", pluginType = OutputCodec.class, pluginConfigurationType = NewlineDelimitedOutputConfig.class) public class NewlineDelimitedOutputCodec implements OutputCodec { + private static final String NDJSON = "ndjson"; + private static final ObjectMapper objectMapper = new ObjectMapper(); + private final NewlineDelimitedOutputConfig config; + + @DataPrepperPluginConstructor + public NewlineDelimitedOutputCodec(final NewlineDelimitedOutputConfig config) { + Objects.requireNonNull(config); + this.config = config; + } @Override public void start(final OutputStream outputStream) throws IOException { - // TODO: implement + Objects.requireNonNull(outputStream); } @Override - public void writeEvent(final Event event, final OutputStream outputStream) throws IOException { - // TODO: get the event data and - // get the header record and message record and write event data to the outputstream + public void writeEvent(final Event event, final OutputStream outputStream, String tagsTargetKey) throws IOException { + Objects.requireNonNull(event); + Map eventMap; + if (tagsTargetKey != null) { + eventMap = addTagsToEvent(event, tagsTargetKey).toMap(); + } else { + eventMap = event.toMap(); + } + writeToOutputStream(outputStream, eventMap); } @Override public void complete(final OutputStream outputStream) throws IOException { - // TODO: Close the output stream + outputStream.close(); } - private void writeByteArrayToOutputStream(final OutputStream outputStream, final Object object) throws IOException { - // TODO: common method to write byte array data to OutputStream + private void writeToOutputStream(final OutputStream outputStream, final Object object) throws IOException { + byte[] byteArr = null; + if (object instanceof Map) { + Map map = objectMapper.convertValue(object, Map.class); + for (String key : config.getExcludeKeys()) { + if (map.containsKey(key)) { + map.remove(key); + } + } + String json = objectMapper.writeValueAsString(map); + byteArr = json.getBytes(); + } else { + byteArr = object.toString().getBytes(); + } + outputStream.write(byteArr); + outputStream.write(System.lineSeparator().getBytes()); } @Override public String getExtension() { - return null; + return NDJSON; } } diff --git a/data-prepper-plugins/newline-codecs/src/main/java/org/opensearch/dataprepper/plugins/codec/newline/NewlineDelimitedOutputConfig.java b/data-prepper-plugins/newline-codecs/src/main/java/org/opensearch/dataprepper/plugins/codec/newline/NewlineDelimitedOutputConfig.java new file mode 100644 index 0000000000..d3e91267e6 --- /dev/null +++ b/data-prepper-plugins/newline-codecs/src/main/java/org/opensearch/dataprepper/plugins/codec/newline/NewlineDelimitedOutputConfig.java @@ -0,0 +1,25 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.codec.newline; + +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.ArrayList; +import java.util.List; + +/** + * Configuration class for the newline delimited codec. + */ +public class NewlineDelimitedOutputConfig { + private static final List DEFAULT_EXCLUDE_KEYS = new ArrayList<>(); + + @JsonProperty("exclude_keys") + private List excludeKeys = DEFAULT_EXCLUDE_KEYS; + + public List getExcludeKeys() { + return excludeKeys; + } +} \ No newline at end of file diff --git a/data-prepper-plugins/newline-codecs/src/test/java/org/opensearch/dataprepper/plugins/codec/newline/NewlineDelimitedOutputCodecTest.java b/data-prepper-plugins/newline-codecs/src/test/java/org/opensearch/dataprepper/plugins/codec/newline/NewlineDelimitedOutputCodecTest.java new file mode 100644 index 0000000000..b31ebaf6f0 --- /dev/null +++ b/data-prepper-plugins/newline-codecs/src/test/java/org/opensearch/dataprepper/plugins/codec/newline/NewlineDelimitedOutputCodecTest.java @@ -0,0 +1,87 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.codec.newline; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.hamcrest.Matchers; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.log.JacksonLog; +import org.opensearch.dataprepper.model.record.Record; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.hamcrest.MatcherAssert.assertThat; + + +public class NewlineDelimitedOutputCodecTest { + private ByteArrayOutputStream outputStream; + + private static NewlineDelimitedOutputConfig config; + + private static int numberOfRecords; + private static final String REGEX = "\\r?\\n"; + private static ObjectMapper objectMapper = new ObjectMapper(); + + private NewlineDelimitedOutputCodec createObjectUnderTest() { + config = new NewlineDelimitedOutputConfig(); + return new NewlineDelimitedOutputCodec(config); + } + + @ParameterizedTest + @ValueSource(ints = {1, 3, 10, 100}) + void test_happy_case(final int numberOfRecords) throws IOException { + this.numberOfRecords = numberOfRecords; + NewlineDelimitedOutputCodec newlineDelimitedOutputCodec = createObjectUnderTest(); + outputStream = new ByteArrayOutputStream(); + newlineDelimitedOutputCodec.start(outputStream); + for (int index = 0; index < numberOfRecords; index++) { + final Event event = (Event) getRecord(index).getData(); + newlineDelimitedOutputCodec.writeEvent(event, outputStream, null); + } + newlineDelimitedOutputCodec.complete(outputStream); + byte[] byteArray = outputStream.toByteArray(); + String jsonString = null; + try { + jsonString = new String(byteArray, StandardCharsets.UTF_8); + } catch (Exception e) { + e.printStackTrace(); + } + int index = 0; + List expectedRecords = generateRecords(numberOfRecords); + String[] jsonObjects = jsonString.split(REGEX); + for (String jsonObject : jsonObjects) { + Object expectedMap = expectedRecords.get(index); + Object actualMap = objectMapper.readValue(jsonObject, Map.class); + assertThat(expectedMap, Matchers.equalTo(actualMap)); + index++; + } + } + + private static Record getRecord(int index) { + List recordList = generateRecords(numberOfRecords); + final Event event = JacksonLog.builder().withData(recordList.get(index)).build(); + return new Record<>(event); + } + + private static List generateRecords(int numberOfRecords) { + + List recordList = new ArrayList<>(); + for (int rows = 0; rows < numberOfRecords; rows++) { + HashMap eventData = new HashMap<>(); + eventData.put("name", "Person" + rows); + eventData.put("age", rows); + recordList.add(eventData); + } + return recordList; + } +} \ No newline at end of file diff --git a/data-prepper-plugins/opensearch-source/README.md b/data-prepper-plugins/opensearch-source/README.md new file mode 100644 index 0000000000..b904d67378 --- /dev/null +++ b/data-prepper-plugins/opensearch-source/README.md @@ -0,0 +1,202 @@ +# OpenSearch Source + +This is the Date Prepper OpenSearch source plugin that processes indices for either OpenSearch, Elasticsearch, +or Amazon OpenSearch Service clusters. It is ideal for migrating index data from a cluster. + +Note: Only fully tested versions will be listed below. It is likely many more versions are supported already, but it is untested. + +The OpenSearch source is compatible with the following OpenSearch versions: +* 2.5 + +And is compatible with the following Elasticsearch versions: +* 7.10 + +# Usages + +### Minimum required config with username and password + +```yaml +opensearch-source-pipeline: + source: + opensearch: + connection: + insecure: true + hosts: [ "https://localhost:9200" ] + username: "username" + password: "password" +``` + +### Full config example + +```yaml +opensearch-source-pipeline: + source: + opensearch: + indices: + include: + - index_name_regex: "test-index-.*" + exclude: + - index_name_regex: "test-index-[1-9].*" + scheduling: + rate: "PT1H" + start_time: "2023-06-02T22:01:30.00Z" + job_count: 2 + search_options: + search_context_type: "none" + batch_size: 1000 + connection: + insecure: false + cert: "/path/to/cert.crt" + socket_timeout: "100ms" + connection_timeout: "100ms" + hosts: [ "https://localhost:9200" ] + username: "username" + password: "password" +``` + +### Amazon OpenSearch Service + +The OpenSearch source can also be configured for an Amazon OpenSearch Service domain. + +```yaml +opensearch-source-pipeline: + source: + opensearch: + connection: + insecure: true + hosts: [ "https://search-my-domain-soopywaovobopgs8ywurr3utsu.us-east-1.es.amazonaws.com" ] + aws: + region: "us-east-1" + sts_role_arn: "arn:aws:iam::123456789012:role/my-domain-role" +``` + +### Using Metadata + +When the OpenSearch source constructs Data Prepper Events from documents in the cluster, the +document index is stored in the `EventMetadata` with an `index` key, and the document_id is +stored in the `EventMetadata` with a `document_id` key. This allows conditional routing based on the index or document_id, +among other things. For example, one could send to an OpenSearch sink and use the same index and document_id from the source cluster in +the destination cluster. A full config example for this use case is below + +```yaml +opensearch-source-pipeline: + source: + opensearch: + connection: + insecure: true + hosts: [ "https://source-cluster:9200" ] + username: "username" + password: "password" + processor: + - add_entries: + entries: + - key: "document_id" + value_expression: "getMetadata(\"document_id\")" + - key: "index" + value_expression: "getMetadata(\"index\")" + sink: + - opensearch: + hosts: [ "https://sink-cluster:9200" ] + username: "username" + password: "password" + document_id_field: "document_id" + index: "copied-${index}" +``` + +## Configuration + +- `hosts` (Required) : A list of IP addresses of OpenSearch or Elasticsearch nodes. + + +- `username` (Optional) : A String of username used in the internal users of OpenSearch cluster. Default is null. + + +- `password` (Optional) : A String of password used in the internal users of OpenSearch cluster. Default is null. + + +- `disable_authentication` (Optional) : A boolean that can disable authentication if the cluster supports it. Defaults to false. + + +- `aws` (Optional) : AWS configurations. See [AWS Configuration](#aws_configuration) for details. SigV4 is enabled by default when this option is used. + + +- `search_options` (Optional) : See [Search Configuration](#search_configuration) for details + + +- `indices` (Optional): See [Indices Configurations](#indices_configuration) for filtering options. + + +- `scheduling` (Optional): See [Scheduling Configuration](#scheduling_configuration) for details + + +- `connection` (Optional): See [Connection Configuration](#connection_configuration) + +### AWS Configuration + +* `region` (Optional) : The AWS region to use for credentials. Defaults to [standard SDK behavior to determine the region](https://docs.aws.amazon.com/sdk-for-java/latest/developer-guide/region-selection.html). + + +* `sts_role_arn` (Optional) : The STS role to assume for requests to AWS. Defaults to null, which will use the [standard SDK behavior for credentials](https://docs.aws.amazon.com/sdk-for-java/latest/developer-guide/credentials.html). + + +* `sts_header_overrides` (Optional): A map of header overrides to make when assuming the IAM role for the source plugin. + +### Search Configuration + +* `search_context_type` (Optional) : A direct override for which type of search context should be used to search documents. + Options include `point_in_time`, `scroll`, or `none` (just search after). + By default, the OpenSearch source will attempt to use `point_in_time` on a cluster by auto-detecting that the cluster version and distribution +supports Point in Time. If the cluster does not support `point_in_time`, then `scroll` is the default behavior. + + +* `batch_size` (Optional) : The amount of documents to read in at once while searching. +This size is passed to the search requests for all search context types (`none` (search_after), `point_in_time`, or `scroll`). +Defaults to 1,000. + +### Scheduling Configuration + +Schedule the start time and amount of times an index should be processed. For example, +a `rate` of `PT1H` and a `job_count` of 3 would result in each index getting processed 3 times, starting at `start_time` +and then every hour after the first time the index is processed. + +* `rate` (Optional) : A String that indicates the rate to process an index based on the `job_count`. +Supports ISO_8601 notation Strings ("PT20.345S", "PT15M", etc.) as well as simple notation Strings for seconds ("60s") and milliseconds ("1500ms"). +Defaults to 8 hours, and is only applicable when `job_count` is greater than 1. + + + +* `job_count` (Optional) : An Integer that specifies how many times each index should be processed. Defaults to 1. + + + +* `start_time` (Optional) : A String in the format of a timestamp that is compatible with Java Instant (i.e. `2023-06-02T22:01:30.00Z`). +Processing will be delayed until this timestamp is reached. The default start time is to start immediately. + +### Connection Configuration + +* `insecure` (Optional): A boolean flag to turn off SSL certificate verification. If set to true, CA certificate verification will be turned off and insecure HTTP requests will be sent. Default to false. + + +* `cert` (Optional) : CA certificate that is pem encoded. Accepts both .pem or .crt. This enables the client to trust the CA that has signed the certificate that the OpenSearch cluster is using. Default is null. + + +* `socket_timeout` (Optional) : A String that indicates the timeout duration for waiting for data. Supports ISO_8601 notation Strings ("PT20.345S", "PT15M", etc.) as well as simple notation Strings for seconds ("60s") and milliseconds ("1500ms"). If this timeout value not set, the underlying Apache HttpClient would rely on operating system settings for managing socket timeouts. + + +* `connection_timeout` (Optional) : A String that indicates the timeout duration used when requesting a connection from the connection manager. Supports ISO_8601 notation Strings ("PT20.345S", "PT15M", etc.) as well as simple notation Strings for seconds ("60s") and milliseconds ("1500ms"). If this timeout value is either negative or not set, the underlying Apache HttpClient would rely on operating system settings for managing connection timeouts. + +### Indices Configuration + +Can be used to filter which indices should be processed. +An index will be processed if its name matches one of the `index_name_regex` +patterns in the `include` list, and does not match any of the pattern in the `exclude` list. +The default behavior is to process all indices. + +* `include` (Optional) : A List of [Index Configuration](#index_configuration) that defines which indices should be processed. Defaults to an empty list. + + +* `exclude` (Optional) : A List of [Index Configuration](#index_configuration) that defines which indices should not be processed. + +#### Index Configuration + +* `index_name_regex`: A regex pattern to represent the index names for filtering diff --git a/data-prepper-plugins/opensearch-source/build.gradle b/data-prepper-plugins/opensearch-source/build.gradle index 22a02beeda..cd88114792 100644 --- a/data-prepper-plugins/opensearch-source/build.gradle +++ b/data-prepper-plugins/opensearch-source/build.gradle @@ -8,7 +8,7 @@ dependencies { implementation project(':data-prepper-plugins:aws-plugin-api') implementation 'software.amazon.awssdk:apache-client' implementation 'com.fasterxml.jackson.core:jackson-databind' - implementation 'com.fasterxml.jackson.datatype:jackson-datatype-jsr310:2.14.2' + implementation 'com.fasterxml.jackson.datatype:jackson-datatype-jsr310:2.15.2' implementation 'software.amazon.awssdk:s3' implementation 'software.amazon.awssdk:sts' testImplementation 'com.fasterxml.jackson.dataformat:jackson-dataformat-yaml' diff --git a/data-prepper-plugins/opensearch-source/src/main/java/org/opensearch/dataprepper/plugins/source/opensearch/OpenSearchSource.java b/data-prepper-plugins/opensearch-source/src/main/java/org/opensearch/dataprepper/plugins/source/opensearch/OpenSearchSource.java index 9a91db3ac1..22455dec3f 100644 --- a/data-prepper-plugins/opensearch-source/src/main/java/org/opensearch/dataprepper/plugins/source/opensearch/OpenSearchSource.java +++ b/data-prepper-plugins/opensearch-source/src/main/java/org/opensearch/dataprepper/plugins/source/opensearch/OpenSearchSource.java @@ -31,6 +31,8 @@ public OpenSearchSource(final OpenSearchSourceConfiguration openSearchSourceConf final AwsCredentialsSupplier awsCredentialsSupplier) { this.openSearchSourceConfiguration = openSearchSourceConfiguration; this.awsCredentialsSupplier = awsCredentialsSupplier; + + openSearchSourceConfiguration.validateAwsConfigWithUsernameAndPassword(); } @Override diff --git a/data-prepper-plugins/opensearch-source/src/main/java/org/opensearch/dataprepper/plugins/source/opensearch/OpenSearchSourceConfiguration.java b/data-prepper-plugins/opensearch-source/src/main/java/org/opensearch/dataprepper/plugins/source/opensearch/OpenSearchSourceConfiguration.java index 8e2fd42384..a100e17b20 100644 --- a/data-prepper-plugins/opensearch-source/src/main/java/org/opensearch/dataprepper/plugins/source/opensearch/OpenSearchSourceConfiguration.java +++ b/data-prepper-plugins/opensearch-source/src/main/java/org/opensearch/dataprepper/plugins/source/opensearch/OpenSearchSourceConfiguration.java @@ -6,9 +6,8 @@ import com.fasterxml.jackson.annotation.JsonProperty; import jakarta.validation.Valid; -import jakarta.validation.constraints.AssertTrue; -import jakarta.validation.constraints.Min; import jakarta.validation.constraints.NotNull; +import org.opensearch.dataprepper.model.plugin.InvalidPluginConfigurationException; import org.opensearch.dataprepper.plugins.source.opensearch.configuration.AwsAuthenticationConfiguration; import org.opensearch.dataprepper.plugins.source.opensearch.configuration.ConnectionConfiguration; import org.opensearch.dataprepper.plugins.source.opensearch.configuration.IndexParametersConfiguration; @@ -20,13 +19,6 @@ public class OpenSearchSourceConfiguration { - /** - * 0 indicates infinite retries - */ - @JsonProperty("max_retries") - @Min(0) - private Integer maxRetries = 0; - @NotNull @JsonProperty("hosts") private List hosts; @@ -37,6 +29,9 @@ public class OpenSearchSourceConfiguration { @JsonProperty("password") private String password; + @JsonProperty("disable_authentication") + private Boolean disableAuthentication = false; + @JsonProperty("connection") @Valid private ConnectionConfiguration connectionConfiguration = new ConnectionConfiguration(); @@ -57,10 +52,6 @@ public class OpenSearchSourceConfiguration { @Valid private SearchConfiguration searchConfiguration = new SearchConfiguration(); - public Integer getMaxRetries() { - return maxRetries; - } - public List getHosts() { return hosts; } @@ -73,6 +64,8 @@ public String getPassword() { return password; } + public Boolean isAuthenticationDisabled() { return disableAuthentication; } + public ConnectionConfiguration getConnectionConfiguration() { return connectionConfiguration; } @@ -93,10 +86,13 @@ public SearchConfiguration getSearchConfiguration() { return searchConfiguration; } - @AssertTrue(message = "Either username and password, or aws options must be specified. Both cannot be set at once.") - boolean validateAwsConfigWithUsernameAndPassword() { - return !((Objects.nonNull(awsAuthenticationOptions) && (Objects.nonNull(username) || Objects.nonNull(password))) || - (Objects.isNull(awsAuthenticationOptions) && (Objects.isNull(username) || Objects.isNull(password)))); + void validateAwsConfigWithUsernameAndPassword() { + + if (((Objects.nonNull(awsAuthenticationOptions) && ((Objects.nonNull(username) || Objects.nonNull(password)) || disableAuthentication)) || + (Objects.nonNull(username) || Objects.nonNull(password)) && disableAuthentication) || + (Objects.isNull(awsAuthenticationOptions) && (Objects.isNull(username) || Objects.isNull(password)) && !disableAuthentication)) { + throw new InvalidPluginConfigurationException("Either username and password, or aws options must be specified. Both cannot be set at once. Authentication can be disabled by setting the disable_authentication flag to true."); + } } } diff --git a/data-prepper-plugins/opensearch-source/src/main/java/org/opensearch/dataprepper/plugins/source/opensearch/worker/client/OpenSearchClientFactory.java b/data-prepper-plugins/opensearch-source/src/main/java/org/opensearch/dataprepper/plugins/source/opensearch/worker/client/OpenSearchClientFactory.java index d9e3a2f739..b32f4bb3a5 100644 --- a/data-prepper-plugins/opensearch-source/src/main/java/org/opensearch/dataprepper/plugins/source/opensearch/worker/client/OpenSearchClientFactory.java +++ b/data-prepper-plugins/opensearch-source/src/main/java/org/opensearch/dataprepper/plugins/source/opensearch/worker/client/OpenSearchClientFactory.java @@ -133,8 +133,7 @@ private RestClient createOpenSearchRestClient(final OpenSearchSourceConfiguratio final RestClientBuilder restClientBuilder = RestClient.builder(httpHosts); - LOG.info("Using username and password for auth for the OpenSearch source"); - attachUsernamePassword(restClientBuilder, openSearchSourceConfiguration); + attachBasicAuth(restClientBuilder, openSearchSourceConfiguration); setConnectAndSocketTimeout(restClientBuilder, openSearchSourceConfiguration); @@ -161,33 +160,35 @@ private org.elasticsearch.client.RestClient createElasticSearchRestClient(final new BasicHeader("Content-type", "application/json") }); - LOG.info("Using username and password for auth for the OpenSearch source"); - attachUsernamePassword(restClientBuilder, openSearchSourceConfiguration); - + attachBasicAuth(restClientBuilder, openSearchSourceConfiguration); setConnectAndSocketTimeout(restClientBuilder, openSearchSourceConfiguration); return restClientBuilder.build(); } - private void attachUsernamePassword(final RestClientBuilder restClientBuilder, final OpenSearchSourceConfiguration openSearchSourceConfiguration) { - final CredentialsProvider credentialsProvider = new BasicCredentialsProvider(); - credentialsProvider.setCredentials(AuthScope.ANY, - new UsernamePasswordCredentials(openSearchSourceConfiguration.getUsername(), openSearchSourceConfiguration.getPassword())); + private void attachBasicAuth(final RestClientBuilder restClientBuilder, final OpenSearchSourceConfiguration openSearchSourceConfiguration) { restClientBuilder.setHttpClientConfigCallback(httpClientBuilder -> { - httpClientBuilder.setDefaultCredentialsProvider(credentialsProvider); + if (!openSearchSourceConfiguration.isAuthenticationDisabled()) { + attachUsernameAndPassword(httpClientBuilder, openSearchSourceConfiguration); + } else { + LOG.warn("Authentication was explicitly disabled for the OpenSearch source"); + } + attachSSLContext(httpClientBuilder, openSearchSourceConfiguration); return httpClientBuilder; }); } - private void attachUsernamePassword(final org.elasticsearch.client.RestClientBuilder restClientBuilder, final OpenSearchSourceConfiguration openSearchSourceConfiguration) { - final CredentialsProvider credentialsProvider = new BasicCredentialsProvider(); - credentialsProvider.setCredentials(AuthScope.ANY, - new UsernamePasswordCredentials(openSearchSourceConfiguration.getUsername(), openSearchSourceConfiguration.getPassword())); + private void attachBasicAuth(final org.elasticsearch.client.RestClientBuilder restClientBuilder, final OpenSearchSourceConfiguration openSearchSourceConfiguration) { restClientBuilder.setHttpClientConfigCallback(httpClientBuilder -> { - httpClientBuilder.setDefaultCredentialsProvider(credentialsProvider); + + if (!openSearchSourceConfiguration.isAuthenticationDisabled()) { + attachUsernameAndPassword(httpClientBuilder, openSearchSourceConfiguration); + } else { + LOG.warn("Authentication was explicitly disabled for the OpenSearch source"); + } attachSSLContext(httpClientBuilder, openSearchSourceConfiguration); httpClientBuilder.addInterceptorLast( (HttpResponseInterceptor) @@ -211,6 +212,15 @@ private void setConnectAndSocketTimeout(final RestClientBuilder restClientBuilde }); } + private void attachUsernameAndPassword(final HttpAsyncClientBuilder httpClientBuilder, final OpenSearchSourceConfiguration openSearchSourceConfiguration) { + LOG.info("Using username and password for auth for the OpenSearch source"); + + final CredentialsProvider credentialsProvider = new BasicCredentialsProvider(); + credentialsProvider.setCredentials(AuthScope.ANY, + new UsernamePasswordCredentials(openSearchSourceConfiguration.getUsername(), openSearchSourceConfiguration.getPassword())); + httpClientBuilder.setDefaultCredentialsProvider(credentialsProvider); + } + private void setConnectAndSocketTimeout(final org.elasticsearch.client.RestClientBuilder restClientBuilder, final OpenSearchSourceConfiguration openSearchSourceConfiguration) { restClientBuilder.setRequestConfigCallback(requestConfigBuilder -> { if (Objects.nonNull(openSearchSourceConfiguration.getConnectionConfiguration().getConnectTimeout())) { diff --git a/data-prepper-plugins/opensearch-source/src/test/java/org/opensearch/dataprepper/plugins/source/opensearch/OpenSearchSourceConfigurationTest.java b/data-prepper-plugins/opensearch-source/src/test/java/org/opensearch/dataprepper/plugins/source/opensearch/OpenSearchSourceConfigurationTest.java index 7c9f8dbd19..950533c145 100644 --- a/data-prepper-plugins/opensearch-source/src/test/java/org/opensearch/dataprepper/plugins/source/opensearch/OpenSearchSourceConfigurationTest.java +++ b/data-prepper-plugins/opensearch-source/src/test/java/org/opensearch/dataprepper/plugins/source/opensearch/OpenSearchSourceConfigurationTest.java @@ -9,10 +9,12 @@ import com.fasterxml.jackson.dataformat.yaml.YAMLFactory; import com.fasterxml.jackson.dataformat.yaml.YAMLGenerator; import org.junit.jupiter.api.Test; +import org.opensearch.dataprepper.model.plugin.InvalidPluginConfigurationException; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.notNullValue; +import static org.junit.jupiter.api.Assertions.assertThrows; public class OpenSearchSourceConfigurationTest { @@ -21,7 +23,7 @@ public class OpenSearchSourceConfigurationTest { @Test void open_search_source_username_password_only() throws JsonProcessingException { - final String sourceConfigurationYaml = "max_retries: 5\n" + + final String sourceConfigurationYaml = "hosts: [\"http://localhost:9200\"]\n" + "username: test\n" + "password: test\n" + @@ -44,18 +46,49 @@ void open_search_source_username_password_only() throws JsonProcessingException assertThat(sourceConfiguration.getIndexParametersConfiguration(), notNullValue()); assertThat(sourceConfiguration.getSchedulingParameterConfiguration(), notNullValue()); assertThat(sourceConfiguration.getHosts(), notNullValue()); - assertThat(sourceConfiguration.getMaxRetries(), equalTo(5)); - assertThat(sourceConfiguration.validateAwsConfigWithUsernameAndPassword(), equalTo(true)); + sourceConfiguration.validateAwsConfigWithUsernameAndPassword(); assertThat(sourceConfiguration.getPassword(), equalTo("test")); assertThat(sourceConfiguration.getUsername(), equalTo("test")); assertThat(sourceConfiguration.getAwsAuthenticationOptions(), equalTo(null)); } @Test - void opensearch_source_aws_only() throws JsonProcessingException { - final String sourceConfigurationYaml = "max_retries: 5\n" + + void open_search_disabled_authentication() throws JsonProcessingException { + + final String sourceConfigurationYaml = "hosts: [\"http://localhost:9200\"]\n" + + "disable_authentication: true\n" + + "connection:\n" + + " insecure: true\n" + + " cert: \"cert\"\n" + + "indices:\n" + + " include:\n" + + " - index_name_regex: \"regex\"\n" + + " - index_name_regex: \"regex-two\"\n" + + "scheduling:\n" + + " job_count: 3\n" + + "search_options:\n" + + " batch_size: 1000\n" + + " query: \"test\"\n"; + final OpenSearchSourceConfiguration sourceConfiguration = objectMapper.readValue(sourceConfigurationYaml, OpenSearchSourceConfiguration.class); + + assertThat(sourceConfiguration.getSearchConfiguration(), notNullValue()); + assertThat(sourceConfiguration.getConnectionConfiguration(), notNullValue()); + assertThat(sourceConfiguration.getIndexParametersConfiguration(), notNullValue()); + assertThat(sourceConfiguration.getSchedulingParameterConfiguration(), notNullValue()); + assertThat(sourceConfiguration.getHosts(), notNullValue()); + + sourceConfiguration.validateAwsConfigWithUsernameAndPassword(); + assertThat(sourceConfiguration.isAuthenticationDisabled(), equalTo(true)); + assertThat(sourceConfiguration.getPassword(), equalTo(null)); + assertThat(sourceConfiguration.getUsername(), equalTo(null)); + assertThat(sourceConfiguration.getAwsAuthenticationOptions(), equalTo(null)); + } + + @Test + void opensearch_source_aws_only() throws JsonProcessingException { + final String sourceConfigurationYaml = "hosts: [\"http://localhost:9200\"]\n" + "connection:\n" + " insecure: true\n" + " cert: \"cert\"\n" + @@ -74,7 +107,7 @@ void opensearch_source_aws_only() throws JsonProcessingException { final OpenSearchSourceConfiguration sourceConfiguration = objectMapper.readValue(sourceConfigurationYaml, OpenSearchSourceConfiguration.class); - assertThat(sourceConfiguration.validateAwsConfigWithUsernameAndPassword(), equalTo(true)); + sourceConfiguration.validateAwsConfigWithUsernameAndPassword(); assertThat(sourceConfiguration.getPassword(), equalTo(null)); assertThat(sourceConfiguration.getUsername(), equalTo(null)); assertThat(sourceConfiguration.getAwsAuthenticationOptions(), notNullValue()); @@ -85,8 +118,7 @@ void opensearch_source_aws_only() throws JsonProcessingException { @Test void opensearch_source_aws_sts_external_id() throws JsonProcessingException { - final String sourceConfigurationYaml = "max_retries: 5\n" + - "hosts: [\"http://localhost:9200\"]\n" + + final String sourceConfigurationYaml = "hosts: [\"http://localhost:9200\"]\n" + "connection:\n" + " insecure: true\n" + " cert: \"cert\"\n" + @@ -106,7 +138,7 @@ void opensearch_source_aws_sts_external_id() throws JsonProcessingException { final OpenSearchSourceConfiguration sourceConfiguration = objectMapper.readValue(sourceConfigurationYaml, OpenSearchSourceConfiguration.class); - assertThat(sourceConfiguration.validateAwsConfigWithUsernameAndPassword(), equalTo(true)); + sourceConfiguration.validateAwsConfigWithUsernameAndPassword(); assertThat(sourceConfiguration.getPassword(), equalTo(null)); assertThat(sourceConfiguration.getUsername(), equalTo(null)); assertThat(sourceConfiguration.getAwsAuthenticationOptions(), notNullValue()); @@ -141,14 +173,12 @@ void using_both_aws_config_and_username_password_is_invalid() throws JsonProcess final OpenSearchSourceConfiguration sourceConfiguration = objectMapper.readValue(sourceConfigurationYaml, OpenSearchSourceConfiguration.class); - assertThat(sourceConfiguration.validateAwsConfigWithUsernameAndPassword(), equalTo(false)); - assertThat(sourceConfiguration.getMaxRetries(), equalTo(0)); + assertThrows(InvalidPluginConfigurationException.class, sourceConfiguration::validateAwsConfigWithUsernameAndPassword); } @Test - void one_of_username_password_or_aws_config_is_required() throws JsonProcessingException { + void one_of_username_password_or_aws_config_or_authDisabled_is_required() throws JsonProcessingException { final String sourceConfigurationYaml = - "max_retries: 5\n" + "hosts: [\"http://localhost:9200\"]\n" + "connection:\n" + " insecure: true\n" + @@ -165,6 +195,6 @@ void one_of_username_password_or_aws_config_is_required() throws JsonProcessingE final OpenSearchSourceConfiguration sourceConfiguration = objectMapper.readValue(sourceConfigurationYaml, OpenSearchSourceConfiguration.class); - assertThat(sourceConfiguration.validateAwsConfigWithUsernameAndPassword(), equalTo(false)); + assertThrows(InvalidPluginConfigurationException.class, sourceConfiguration::validateAwsConfigWithUsernameAndPassword); } } diff --git a/data-prepper-plugins/opensearch-source/src/test/java/org/opensearch/dataprepper/plugins/source/opensearch/worker/client/OpenSearchClientFactoryTest.java b/data-prepper-plugins/opensearch-source/src/test/java/org/opensearch/dataprepper/plugins/source/opensearch/worker/client/OpenSearchClientFactoryTest.java index cc811625d1..1cd2ad551c 100644 --- a/data-prepper-plugins/opensearch-source/src/test/java/org/opensearch/dataprepper/plugins/source/opensearch/worker/client/OpenSearchClientFactoryTest.java +++ b/data-prepper-plugins/opensearch-source/src/test/java/org/opensearch/dataprepper/plugins/source/opensearch/worker/client/OpenSearchClientFactoryTest.java @@ -29,6 +29,8 @@ import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.notNullValue; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.verify; import static org.mockito.Mockito.verifyNoInteractions; import static org.mockito.Mockito.when; @@ -119,4 +121,38 @@ void provideOpenSearchClient_with_aws_auth() { assertThat(awsCredentialsOptions.getStsHeaderOverrides(), equalTo(Collections.emptyMap())); assertThat(awsCredentialsOptions.getStsRoleArn(), equalTo(stsRoleArn)); } + + @Test + void provideElasticSearchClient_with_auth_disabled() { + when(openSearchSourceConfiguration.isAuthenticationDisabled()).thenReturn(true); + + when(connectionConfiguration.getCertPath()).thenReturn(null); + when(connectionConfiguration.getSocketTimeout()).thenReturn(null); + when(connectionConfiguration.getConnectTimeout()).thenReturn(null); + when(connectionConfiguration.isInsecure()).thenReturn(true); + + final ElasticsearchClient elasticsearchClient = createObjectUnderTest().provideElasticSearchClient(openSearchSourceConfiguration); + assertThat(elasticsearchClient, notNullValue()); + + verifyNoInteractions(awsCredentialsSupplier); + verify(openSearchSourceConfiguration, never()).getUsername(); + verify(openSearchSourceConfiguration, never()).getPassword(); + } + + @Test + void provideOpenSearchClient_with_auth_disabled() { + when(openSearchSourceConfiguration.isAuthenticationDisabled()).thenReturn(true); + + when(connectionConfiguration.getCertPath()).thenReturn(null); + when(connectionConfiguration.getSocketTimeout()).thenReturn(null); + when(connectionConfiguration.getConnectTimeout()).thenReturn(null); + when(connectionConfiguration.isInsecure()).thenReturn(true); + + final OpenSearchClient openSearchClient = createObjectUnderTest().provideOpenSearchClient(openSearchSourceConfiguration); + assertThat(openSearchClient, notNullValue()); + + verifyNoInteractions(awsCredentialsSupplier); + verify(openSearchSourceConfiguration, never()).getUsername(); + verify(openSearchSourceConfiguration, never()).getPassword(); + } } diff --git a/data-prepper-plugins/parquet-codecs/src/main/java/org/opensearch/dataprepper/plugins/codec/parquet/ParquetOutputCodec.java b/data-prepper-plugins/parquet-codecs/src/main/java/org/opensearch/dataprepper/plugins/codec/parquet/ParquetOutputCodec.java index 2cbce41c7f..c8be600499 100644 --- a/data-prepper-plugins/parquet-codecs/src/main/java/org/opensearch/dataprepper/plugins/codec/parquet/ParquetOutputCodec.java +++ b/data-prepper-plugins/parquet-codecs/src/main/java/org/opensearch/dataprepper/plugins/codec/parquet/ParquetOutputCodec.java @@ -35,7 +35,7 @@ public void complete(final OutputStream outputStream) throws IOException { } @Override - public void writeEvent(final Event event, final OutputStream outputStream) throws IOException { + public void writeEvent(final Event event, final OutputStream outputStream, String tagsTargetKey) throws IOException { // TODO: get the event data and write in output stream } diff --git a/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/codec/json/JsonOutputCodec.java b/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/codec/json/JsonOutputCodec.java index e37be538c0..bc68761f07 100644 --- a/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/codec/json/JsonOutputCodec.java +++ b/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/codec/json/JsonOutputCodec.java @@ -24,7 +24,7 @@ public void start(final OutputStream outputStream) throws IOException { } @Override - public void writeEvent(final Event event, final OutputStream outputStream) throws IOException { + public void writeEvent(final Event event, final OutputStream outputStream, String tagsTargetKey) throws IOException { // TODO: get the event data and write event data to the outputstream } diff --git a/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parsejson/ParseJsonProcessor.java b/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parsejson/ParseJsonProcessor.java index 99f04d3e33..e076b7de10 100644 --- a/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parsejson/ParseJsonProcessor.java +++ b/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parsejson/ParseJsonProcessor.java @@ -64,34 +64,37 @@ public Collection> doExecute(final Collection> recor final boolean doUsePointer = Objects.nonNull(pointer); for (final Record record : records) { - final Event event = record.getData(); - if (Objects.nonNull(parseWhen) && !expressionEvaluator.evaluateConditional(parseWhen, event)) { - continue; - } - - final String message = event.get(source, String.class); - if (Objects.isNull(message)) { - continue; - } - - try { - final TypeReference> hashMapTypeReference = new TypeReference>() {}; - Map parsedJson = objectMapper.readValue(message, hashMapTypeReference); - - if (doUsePointer) { - parsedJson = parseUsingPointer(event, parsedJson, pointer, doWriteToRoot); - } - - if (doWriteToRoot) { - writeToRoot(event, parsedJson); - } else { - event.put(destination, parsedJson); + final Event event = record.getData(); + try { + if (Objects.nonNull(parseWhen) && !expressionEvaluator.evaluateConditional(parseWhen, event)) { + continue; + } + + final String message = event.get(source, String.class); + if (Objects.isNull(message)) { + continue; + } + final TypeReference> hashMapTypeReference = new TypeReference>() { + }; + Map parsedJson = objectMapper.readValue(message, hashMapTypeReference); + + if (doUsePointer) { + parsedJson = parseUsingPointer(event, parsedJson, pointer, doWriteToRoot); + } + + if (doWriteToRoot) { + writeToRoot(event, parsedJson); + } else { + event.put(destination, parsedJson); + } + } catch (final JsonProcessingException jsonException) { + event.getMetadata().addTags(tagsOnFailure); + LOG.error(EVENT, "An exception occurred due to invalid JSON while reading event [{}]", event, jsonException); + } catch (final Exception e) { + event.getMetadata().addTags(tagsOnFailure); + LOG.error(EVENT, "An exception occurred while using the parse_json processor on Event [{}]", event, e); } - } catch (final JsonProcessingException jsonException) { - event.getMetadata().addTags(tagsOnFailure); - LOG.error(EVENT, "An exception occurred due to invalid JSON while reading event [{}]", event, jsonException); - } } return records; } diff --git a/data-prepper-plugins/parse-json-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/parsejson/ParseJsonProcessorTest.java b/data-prepper-plugins/parse-json-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/parsejson/ParseJsonProcessorTest.java index d60c877f95..7fce6ecbe5 100644 --- a/data-prepper-plugins/parse-json-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/parsejson/ParseJsonProcessorTest.java +++ b/data-prepper-plugins/parse-json-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/parsejson/ParseJsonProcessorTest.java @@ -306,6 +306,25 @@ void test_tags_when_json_parse_fails() { assertTrue(parsedEvent.getMetadata().hasTags(testTags)); } + @Test + void when_evaluate_conditional_throws_RuntimeException_events_are_not_dropped() { + final String source = "different_source"; + final String destination = "destination_key"; + when(processorConfig.getSource()).thenReturn(source); + when(processorConfig.getDestination()).thenReturn(destination); + final String whenCondition = UUID.randomUUID().toString(); + when(processorConfig.getParseWhen()).thenReturn(whenCondition); + final Map data = Collections.singletonMap("key", "value"); + final String serializedMessage = convertMapToJSONString(data); + final Record testEvent = createMessageEvent(serializedMessage); + when(expressionEvaluator.evaluateConditional(whenCondition, testEvent.getData())).thenThrow(RuntimeException.class); + parseJsonProcessor = createObjectUnderTest(); + + final Event parsedEvent = createAndParseMessageEvent(testEvent); + + assertThat(parsedEvent.toMap(), equalTo(testEvent.getData().toMap())); + } + private String constructDeeplyNestedJsonPointer(final int numberOfLayers) { String pointer = "/" + DEEPLY_NESTED_KEY_NAME; for (int layer = 0; layer < numberOfLayers; layer++) { diff --git a/data-prepper-plugins/s3-sink/build.gradle b/data-prepper-plugins/s3-sink/build.gradle index 6d8b44cdb8..4827a75b02 100644 --- a/data-prepper-plugins/s3-sink/build.gradle +++ b/data-prepper-plugins/s3-sink/build.gradle @@ -20,6 +20,8 @@ dependencies { implementation 'org.jetbrains.kotlin:kotlin-stdlib-common:1.8.21' implementation 'org.apache.commons:commons-lang3:3.12.0' testImplementation project(':data-prepper-test-common') + + implementation project(':data-prepper-plugins:newline-codecs') } test { diff --git a/data-prepper-plugins/s3-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/S3SinkServiceIT.java b/data-prepper-plugins/s3-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/S3SinkServiceIT.java index cfa9d3657b..7468e6bf44 100644 --- a/data-prepper-plugins/s3-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/S3SinkServiceIT.java +++ b/data-prepper-plugins/s3-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/S3SinkServiceIT.java @@ -14,16 +14,21 @@ import org.mockito.Mock; import org.mockito.junit.jupiter.MockitoExtension; import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.codec.OutputCodec; +import org.opensearch.dataprepper.model.event.DefaultEventMetadata; import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.model.event.EventHandle; +import org.opensearch.dataprepper.model.event.EventMetadata; +import org.opensearch.dataprepper.model.event.EventType; import org.opensearch.dataprepper.model.event.JacksonEvent; import org.opensearch.dataprepper.model.log.JacksonLog; import org.opensearch.dataprepper.model.record.Record; import org.opensearch.dataprepper.model.types.ByteCount; +import org.opensearch.dataprepper.plugins.codec.newline.NewlineDelimitedOutputCodec; +import org.opensearch.dataprepper.plugins.codec.newline.NewlineDelimitedOutputConfig; import org.opensearch.dataprepper.plugins.sink.accumulator.BufferFactory; import org.opensearch.dataprepper.plugins.sink.accumulator.InMemoryBufferFactory; import org.opensearch.dataprepper.plugins.sink.accumulator.ObjectKey; -import org.opensearch.dataprepper.plugins.sink.codec.JsonCodec; import org.opensearch.dataprepper.plugins.sink.configuration.AwsAuthenticationOptions; import org.opensearch.dataprepper.plugins.sink.configuration.ObjectKeyOptions; import org.opensearch.dataprepper.plugins.sink.configuration.ThresholdOptions; @@ -37,12 +42,14 @@ import software.amazon.awssdk.services.s3.model.S3Object; import java.time.Duration; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; import java.util.LinkedHashMap; import java.util.LinkedList; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.UUID; import static org.hamcrest.MatcherAssert.assertThat; @@ -57,7 +64,7 @@ class S3SinkServiceIT { private static final String PATH_PREFIX = UUID.randomUUID().toString() + "/%{yyyy}/%{MM}/%{dd}/"; private S3Client s3Client; private String bucketName; - private JsonCodec codec; + private BufferFactory bufferFactory; @Mock private S3SinkConfig s3SinkConfig; @@ -80,6 +87,11 @@ class S3SinkServiceIT { @Mock private DistributionSummary s3ObjectSizeSummary; + private OutputCodec codec; + + @Mock + NewlineDelimitedOutputConfig newlineDelimitedOutputConfig; + @BeforeEach public void setUp() { @@ -88,7 +100,6 @@ public void setUp() { s3Client = S3Client.builder().region(Region.of(s3region)).build(); bucketName = System.getProperty("tests.s3sink.bucket"); - codec = new JsonCodec(); bufferFactory = new InMemoryBufferFactory(); when(objectKeyOptions.getNamePattern()).thenReturn("elb-log-%{yyyy-MM-dd'T'hh-mm-ss}"); @@ -111,6 +122,7 @@ public void setUp() { @Test void verify_flushed_object_count_into_s3_bucket() { + configureNewLineCodec(); int s3ObjectCountBeforeIngest = gets3ObjectCount(); S3SinkService s3SinkService = createObjectUnderTest(); s3SinkService.output(setEventQueue()); @@ -118,23 +130,31 @@ void verify_flushed_object_count_into_s3_bucket() { assertThat(s3ObjectCountAfterIngest, equalTo(s3ObjectCountBeforeIngest + 1)); } - @Test - void verify_flushed_records_into_s3_bucket() { + void configureNewLineCodec() { + codec = new NewlineDelimitedOutputCodec(newlineDelimitedOutputConfig); + when(newlineDelimitedOutputConfig.getExcludeKeys()).thenReturn(new ArrayList<>()); + } + @Test + void verify_flushed_records_into_s3_bucketNewLine() { + configureNewLineCodec(); S3SinkService s3SinkService = createObjectUnderTest(); Collection> recordsData = setEventQueue(); s3SinkService.output(recordsData); String objectData = getS3Object(); + int count = 0; + String[] objectDataArr = objectData.split("\r\n"); for (Record recordData : recordsData) { String objectRecord = recordData.getData().toJsonString(); - assertThat(objectData, CoreMatchers.containsString(objectRecord)); + assertThat(objectDataArr[count], CoreMatchers.containsString(objectRecord)); + count++; } } private S3SinkService createObjectUnderTest() { - return new S3SinkService(s3SinkConfig, bufferFactory, codec, s3Client, null, pluginMetrics); + return new S3SinkService(s3SinkConfig, bufferFactory, codec, s3Client, "Tag", pluginMetrics); } private int gets3ObjectCount() { @@ -183,19 +203,24 @@ private Collection> setEventQueue() { } private static Record createRecord() { - Map json = generateJson(); - final JacksonEvent event = JacksonLog.builder().withData(json).build(); + final Set testTags = Set.of("tag1"); + final EventMetadata defaultEventMetadata = DefaultEventMetadata.builder(). + withEventType(EventType.LOG.toString()). + withTags(testTags).build(); + Map json = generateJson(testTags); + final JacksonEvent event = JacksonLog.builder().withData(json).withEventMetadata(defaultEventMetadata).build(); event.setEventHandle(mock(EventHandle.class)); return new Record<>(event); } - private static Map generateJson() { + private static Map generateJson(Set testTags) { final Map jsonObject = new LinkedHashMap<>(); for (int i = 0; i < 2; i++) { jsonObject.put(UUID.randomUUID().toString(), UUID.randomUUID().toString()); } jsonObject.put(UUID.randomUUID().toString(), Arrays.asList(UUID.randomUUID().toString(), UUID.randomUUID().toString(), UUID.randomUUID().toString())); + jsonObject.put("Tag", testTags.toArray()); return jsonObject; } } diff --git a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/S3Sink.java b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/S3Sink.java index a4baa538ae..828fd2e414 100644 --- a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/S3Sink.java +++ b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/S3Sink.java @@ -8,6 +8,7 @@ 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.codec.OutputCodec; import org.opensearch.dataprepper.model.configuration.PluginModel; import org.opensearch.dataprepper.model.configuration.PluginSetting; import org.opensearch.dataprepper.model.event.Event; @@ -21,7 +22,6 @@ import org.opensearch.dataprepper.plugins.sink.accumulator.BufferTypeOptions; import org.opensearch.dataprepper.plugins.sink.accumulator.InMemoryBufferFactory; import org.opensearch.dataprepper.plugins.sink.accumulator.LocalFileBufferFactory; -import org.opensearch.dataprepper.plugins.sink.codec.Codec; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import software.amazon.awssdk.services.s3.S3Client; @@ -38,7 +38,7 @@ public class S3Sink extends AbstractSink> { private static final Logger LOG = LoggerFactory.getLogger(S3Sink.class); private final S3SinkConfig s3SinkConfig; - private final Codec codec; + private final OutputCodec codec; private volatile boolean sinkInitialized; private final S3SinkService s3SinkService; private final BufferFactory bufferFactory; @@ -61,7 +61,7 @@ public S3Sink(final PluginSetting pluginSetting, final PluginModel codecConfiguration = s3SinkConfig.getCodec(); final PluginSetting codecPluginSettings = new PluginSetting(codecConfiguration.getPluginName(), codecConfiguration.getPluginSettings()); - codec = pluginFactory.loadPlugin(Codec.class, codecPluginSettings); + codec = pluginFactory.loadPlugin(OutputCodec.class, codecPluginSettings); sinkInitialized = Boolean.FALSE; if (s3SinkConfig.getBufferType().equals(BufferTypeOptions.LOCALFILE)) { diff --git a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/S3SinkService.java b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/S3SinkService.java index 34c49a9b25..0ed2e8bb79 100644 --- a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/S3SinkService.java +++ b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/S3SinkService.java @@ -8,6 +8,7 @@ import io.micrometer.core.instrument.Counter; import io.micrometer.core.instrument.DistributionSummary; import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.codec.OutputCodec; import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.model.event.EventHandle; import org.opensearch.dataprepper.model.record.Record; @@ -15,7 +16,6 @@ import org.opensearch.dataprepper.plugins.sink.accumulator.Buffer; import org.opensearch.dataprepper.plugins.sink.accumulator.BufferFactory; import org.opensearch.dataprepper.plugins.sink.accumulator.ObjectKey; -import org.opensearch.dataprepper.plugins.sink.codec.Codec; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import software.amazon.awssdk.awscore.exception.AwsServiceException; @@ -23,6 +23,7 @@ import software.amazon.awssdk.services.s3.S3Client; import java.io.IOException; +import java.io.OutputStream; import java.util.Collection; import java.util.LinkedList; import java.util.concurrent.locks.Lock; @@ -44,7 +45,7 @@ public class S3SinkService { private final Lock reentrantLock; private final BufferFactory bufferFactory; private final Collection bufferedEventHandles; - private final Codec codec; + private final OutputCodec codec; private final S3Client s3Client; private Buffer currentBuffer; private final int maxEvents; @@ -67,7 +68,7 @@ public class S3SinkService { * @param pluginMetrics metrics. */ public S3SinkService(final S3SinkConfig s3SinkConfig, final BufferFactory bufferFactory, - final Codec codec, final S3Client s3Client, final String tagsTargetKey, final PluginMetrics pluginMetrics) { + final OutputCodec codec, final S3Client s3Client, final String tagsTargetKey, final PluginMetrics pluginMetrics) { this.s3SinkConfig = s3SinkConfig; this.bufferFactory = bufferFactory; this.codec = codec; @@ -100,19 +101,25 @@ void output(Collection> records) { currentBuffer = bufferFactory.getBuffer(); } try { + OutputStream outputStream = currentBuffer.getOutputStream(); + for (Record record : records) { + if(currentBuffer.getEventCount() == 0) { + codec.start(outputStream); + } + final Event event = record.getData(); - final String encodedEvent; - encodedEvent = codec.parse(event, tagsTargetKey); - final byte[] encodedBytes = encodedEvent.getBytes(); + codec.writeEvent(event, outputStream, tagsTargetKey); + int count = currentBuffer.getEventCount() +1; + currentBuffer.setEventCount(count); - currentBuffer.writeEvent(encodedBytes); if(event.getEventHandle() != null) { bufferedEventHandles.add(event.getEventHandle()); } if (ThresholdCheck.checkThresholdExceed(currentBuffer, maxEvents, maxBytes, maxCollectionDuration)) { - final String s3Key = generateKey(); + codec.complete(outputStream); + final String s3Key = generateKey(codec); LOG.info("Writing {} to S3 with {} events and size of {} bytes.", s3Key, currentBuffer.getEventCount(), currentBuffer.getSize()); final boolean isFlushToS3 = retryFlushToS3(currentBuffer, s3Key); @@ -178,9 +185,9 @@ protected boolean retryFlushToS3(final Buffer currentBuffer, final String s3Key) * Generate the s3 object path prefix and object file name. * @return object key path. */ - protected String generateKey() { + protected String generateKey(OutputCodec codec) { final String pathPrefix = ObjectKey.buildingPathPrefix(s3SinkConfig); - final String namePattern = ObjectKey.objectFileName(s3SinkConfig); + final String namePattern = ObjectKey.objectFileName(s3SinkConfig, codec.getExtension()); return (!pathPrefix.isEmpty()) ? pathPrefix + namePattern : namePattern; } } diff --git a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/accumulator/Buffer.java b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/accumulator/Buffer.java index ebb3886eac..df09ea42e1 100644 --- a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/accumulator/Buffer.java +++ b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/accumulator/Buffer.java @@ -6,7 +6,7 @@ package org.opensearch.dataprepper.plugins.sink.accumulator; import software.amazon.awssdk.services.s3.S3Client; -import java.io.IOException; +import java.io.OutputStream; /** * A buffer can hold data before flushing it to S3. @@ -23,5 +23,8 @@ public interface Buffer { long getDuration(); void flushToS3(S3Client s3Client, String bucket, String key) ; - void writeEvent(byte[] bytes) throws IOException; + + OutputStream getOutputStream(); + + void setEventCount(int eventCount); } diff --git a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/accumulator/InMemoryBuffer.java b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/accumulator/InMemoryBuffer.java index 370b1c1882..4a892233de 100644 --- a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/accumulator/InMemoryBuffer.java +++ b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/accumulator/InMemoryBuffer.java @@ -9,8 +9,9 @@ import software.amazon.awssdk.core.sync.RequestBody; import software.amazon.awssdk.services.s3.S3Client; import software.amazon.awssdk.services.s3.model.PutObjectRequest; + import java.io.ByteArrayOutputStream; -import java.io.IOException; +import java.io.OutputStream; import java.util.concurrent.TimeUnit; /** @@ -39,6 +40,15 @@ public int getEventCount() { return eventCount; } + @Override + public void setEventCount(int eventCount) { + this.eventCount = eventCount; + } + @Override + public OutputStream getOutputStream() { + return byteArrayOutputStream; + } + public long getDuration() { return watch.getTime(TimeUnit.SECONDS); } @@ -58,16 +68,5 @@ public void flushToS3(S3Client s3Client, String bucket, String key) { RequestBody.fromBytes(byteArray)); } - /** - * write byte array to output stream. - * - * @param bytes byte array. - * @throws IOException while writing to output stream fails. - */ - @Override - public void writeEvent(byte[] bytes) throws IOException { - byteArrayOutputStream.write(bytes); - byteArrayOutputStream.write(System.lineSeparator().getBytes()); - eventCount++; - } + } \ No newline at end of file diff --git a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/accumulator/LocalFileBuffer.java b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/accumulator/LocalFileBuffer.java index da5c9faa1a..843be56090 100644 --- a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/accumulator/LocalFileBuffer.java +++ b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/accumulator/LocalFileBuffer.java @@ -75,18 +75,6 @@ public void flushToS3(S3Client s3Client, String bucket, String key) { removeTemporaryFile(); } - /** - * write byte array to output stream. - * @param bytes byte array. - * @throws IOException while writing to output stream fails. - */ - @Override - public void writeEvent(byte[] bytes) throws IOException { - outputStream.write(bytes); - outputStream.write(System.lineSeparator().getBytes()); - eventCount++; - } - /** * Flushing the buffered data into the output stream. */ @@ -111,4 +99,15 @@ protected void removeTemporaryFile() { } } } + + + @Override + public void setEventCount(int eventCount) { + this.eventCount = eventCount; + } + + @Override + public OutputStream getOutputStream() { + return outputStream; + } } \ No newline at end of file diff --git a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/accumulator/ObjectKey.java b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/accumulator/ObjectKey.java index b3e85d8381..4eb0a12923 100644 --- a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/accumulator/ObjectKey.java +++ b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/accumulator/ObjectKey.java @@ -50,16 +50,18 @@ public static String buildingPathPrefix(final S3SinkConfig s3SinkConfig) { *Get the object file name with the extension * * @param s3SinkConfig s3 sink configuration + * @param codecExtension extension * @return s3 object name with prefix */ - public static String objectFileName(S3SinkConfig s3SinkConfig) { + public static String objectFileName(S3SinkConfig s3SinkConfig, String codecExtension) { String configNamePattern = s3SinkConfig.getObjectKeyOptions().getNamePattern(); int extensionIndex = configNamePattern.lastIndexOf('.'); if (extensionIndex > 0) { return S3ObjectIndexUtility.getObjectNameWithDateTimeId(configNamePattern.substring(0, extensionIndex)) + "." - + configNamePattern.substring(extensionIndex + 1); + + (codecExtension!=null? codecExtension :configNamePattern.substring(extensionIndex + 1)); } else { - return S3ObjectIndexUtility.getObjectNameWithDateTimeId(configNamePattern) + "." + DEFAULT_CODEC_FILE_EXTENSION; + return S3ObjectIndexUtility.getObjectNameWithDateTimeId(configNamePattern) + "." + + (codecExtension!=null? codecExtension : DEFAULT_CODEC_FILE_EXTENSION); } } } \ No newline at end of file diff --git a/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/S3SinkServiceTest.java b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/S3SinkServiceTest.java index 36302133a6..9c8a4e30ab 100644 --- a/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/S3SinkServiceTest.java +++ b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/S3SinkServiceTest.java @@ -10,6 +10,7 @@ import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.codec.OutputCodec; import org.opensearch.dataprepper.model.configuration.PluginModel; import org.opensearch.dataprepper.model.configuration.PluginSetting; import org.opensearch.dataprepper.model.event.Event; @@ -23,8 +24,6 @@ import org.opensearch.dataprepper.plugins.sink.accumulator.BufferTypeOptions; import org.opensearch.dataprepper.plugins.sink.accumulator.InMemoryBuffer; import org.opensearch.dataprepper.plugins.sink.accumulator.InMemoryBufferFactory; -import org.opensearch.dataprepper.plugins.sink.codec.Codec; -import org.opensearch.dataprepper.plugins.sink.codec.JsonCodec; import org.opensearch.dataprepper.plugins.sink.configuration.AwsAuthenticationOptions; import org.opensearch.dataprepper.plugins.sink.configuration.ObjectKeyOptions; import org.opensearch.dataprepper.plugins.sink.configuration.ThresholdOptions; @@ -34,6 +33,7 @@ import org.apache.commons.lang3.RandomStringUtils; import java.io.IOException; +import java.io.OutputStream; import java.time.Duration; import java.time.LocalDateTime; import java.time.ZoneId; @@ -78,7 +78,7 @@ class S3SinkServiceTest { public static final String PATH_PREFIX = "logdata/"; private S3SinkConfig s3SinkConfig; private S3Client s3Client; - private JsonCodec codec; + private OutputCodec codec; private PluginMetrics pluginMetrics; private BufferFactory bufferFactory; private Counter snapshotSuccessCounter; @@ -100,7 +100,7 @@ void setUp() { pluginMetrics = mock(PluginMetrics.class); PluginModel pluginModel = mock(PluginModel.class); PluginFactory pluginFactory = mock(PluginFactory.class); - codec = mock(JsonCodec.class); + codec = mock(OutputCodec.class); snapshotSuccessCounter = mock(Counter.class); Counter snapshotFailedCounter = mock(Counter.class); Counter numberOfRecordsSuccessCounter = mock(Counter.class); @@ -123,7 +123,7 @@ void setUp() { when(awsAuthenticationOptions.getAwsRegion()).thenReturn(Region.of(S3_REGION)); when(s3SinkConfig.getCodec()).thenReturn(pluginModel); when(pluginModel.getPluginName()).thenReturn(CODEC_PLUGIN_NAME); - when(pluginFactory.loadPlugin(Codec.class, pluginSetting)).thenReturn(codec); + when(pluginFactory.loadPlugin(OutputCodec.class, pluginSetting)).thenReturn(codec); lenient().when(pluginMetrics.counter(S3SinkService.OBJECTS_SUCCEEDED)).thenReturn(snapshotSuccessCounter); lenient().when(pluginMetrics.counter(S3SinkService.OBJECTS_FAILED)).thenReturn(snapshotFailedCounter); @@ -150,7 +150,7 @@ void test_generateKey_with_general_prefix() { String pathPrefix = "events/"; when(s3SinkConfig.getObjectKeyOptions().getPathPrefix()).thenReturn(pathPrefix); S3SinkService s3SinkService = createObjectUnderTest(); - String key = s3SinkService.generateKey(); + String key = s3SinkService.generateKey(codec); assertNotNull(key); assertThat(key, true); assertThat(key, key.contains(pathPrefix)); @@ -169,7 +169,7 @@ void test_generateKey_with_date_prefix() { when(s3SinkConfig.getObjectKeyOptions() .getPathPrefix()).thenReturn(pathPrefix + datePattern); S3SinkService s3SinkService = createObjectUnderTest(); - String key = s3SinkService.generateKey(); + String key = s3SinkService.generateKey(codec); assertNotNull(key); assertThat(key, true); assertThat(key, key.contains(pathPrefix + dateString)); @@ -184,7 +184,9 @@ void test_output_with_threshold_set_as_more_then_zero_event_count() throws IOExc when(bufferFactory.getBuffer()).thenReturn(buffer); when(s3SinkConfig.getThresholdOptions().getEventCount()).thenReturn(5); - when(codec.parse(any(), anyString())).thenReturn("{\"message\":\"31824252-adba-4c47-a2ac-05d16c5b8140\"}"); + final OutputStream outputStream = mock(OutputStream.class); + final Event event = JacksonEvent.fromMessage(UUID.randomUUID().toString()); + doNothing().when(codec).writeEvent(event, outputStream, null); S3SinkService s3SinkService = createObjectUnderTest(); assertNotNull(s3SinkService); s3SinkService.output(generateRandomStringEventRecord()); @@ -205,7 +207,9 @@ void test_output_with_threshold_set_as_zero_event_count() throws IOException { when(s3SinkConfig.getThresholdOptions().getEventCount()).thenReturn(0); when(s3SinkConfig.getThresholdOptions().getMaximumSize()).thenReturn(ByteCount.parse("2kb")); - when(codec.parse(any(), anyString())).thenReturn("{\"message\":\"31824252-adba-4c47-a2ac-05d16c5b8140\"}"); + final OutputStream outputStream = mock(OutputStream.class); + final Event event = JacksonEvent.fromMessage(UUID.randomUUID().toString()); + doNothing().when(codec).writeEvent(event, outputStream, null); S3SinkService s3SinkService = createObjectUnderTest(); assertNotNull(s3SinkService); s3SinkService.output(generateRandomStringEventRecord()); @@ -221,7 +225,9 @@ void test_output_with_uploadedToS3_success() throws IOException { doNothing().when(buffer).flushToS3(any(S3Client.class), anyString(), any(String.class)); when(bufferFactory.getBuffer()).thenReturn(buffer); - when(codec.parse(any(), anyString())).thenReturn("{\"message\":\"31824252-adba-4c47-a2ac-05d16c5b8140\"}"); + final OutputStream outputStream = mock(OutputStream.class); + final Event event = JacksonEvent.fromMessage(UUID.randomUUID().toString()); + doNothing().when(codec).writeEvent(event, outputStream, null); S3SinkService s3SinkService = createObjectUnderTest(); assertNotNull(s3SinkService); assertThat(s3SinkService, instanceOf(S3SinkService.class)); @@ -239,7 +245,9 @@ void test_output_with_uploadedToS3_success_records_byte_count() throws IOExcepti final long objectSize = random.nextInt(1_000_000) + 10_000; when(buffer.getSize()).thenReturn(objectSize); - when(codec.parse(any(), anyString())).thenReturn(UUID.randomUUID().toString()); + final OutputStream outputStream = mock(OutputStream.class); + final Event event = JacksonEvent.fromMessage(UUID.randomUUID().toString()); + doNothing().when(codec).writeEvent(event, outputStream, null); final S3SinkService s3SinkService = createObjectUnderTest(); s3SinkService.output(generateRandomStringEventRecord()); @@ -250,7 +258,9 @@ void test_output_with_uploadedToS3_success_records_byte_count() throws IOExcepti void test_output_with_uploadedToS3_failed() throws IOException { when(s3SinkConfig.getBucketName()).thenReturn(UUID.randomUUID().toString()); when(s3SinkConfig.getMaxUploadRetries()).thenReturn(3); - when(codec.parse(any(), anyString())).thenReturn("{\"message\":\"31824252-adba-4c47-a2ac-05d16c5b8140\"}"); + final OutputStream outputStream = mock(OutputStream.class); + final Event event = JacksonEvent.fromMessage(UUID.randomUUID().toString()); + doNothing().when(codec).writeEvent(event, outputStream, null); S3SinkService s3SinkService = createObjectUnderTest(); assertNotNull(s3SinkService); assertThat(s3SinkService, instanceOf(S3SinkService.class)); @@ -270,9 +280,10 @@ void test_output_with_uploadedToS3_failure_does_not_record_byte_count() throws I final long objectSize = random.nextInt(1_000_000) + 10_000; when(buffer.getSize()).thenReturn(objectSize); - when(codec.parse(any(), anyString())).thenReturn(UUID.randomUUID().toString()); final S3SinkService s3SinkService = createObjectUnderTest(); final Event event = JacksonEvent.fromMessage(UUID.randomUUID().toString()); + final OutputStream outputStream = mock(OutputStream.class); + doNothing().when(codec).writeEvent(event, outputStream, null); s3SinkService.output(Collections.singletonList(new Record<>(event))); verify(s3ObjectSizeSummary, never()).record(anyLong()); @@ -290,7 +301,9 @@ void test_retryFlushToS3_positive() throws InterruptedException, IOException { S3SinkService s3SinkService = createObjectUnderTest(); assertNotNull(s3SinkService); assertNotNull(buffer); - buffer.writeEvent(generateByteArray()); + OutputStream outputStream = buffer.getOutputStream(); + final Event event = JacksonEvent.fromMessage(UUID.randomUUID().toString()); + codec.writeEvent(event, outputStream, null); final String s3Key = UUID.randomUUID().toString(); boolean isUploadedToS3 = s3SinkService.retryFlushToS3(buffer, s3Key); assertTrue(isUploadedToS3); @@ -304,7 +317,9 @@ void test_retryFlushToS3_negative() throws InterruptedException, IOException { when(s3SinkConfig.getBucketName()).thenReturn(""); S3SinkService s3SinkService = createObjectUnderTest(); assertNotNull(s3SinkService); - buffer.writeEvent(generateByteArray()); + OutputStream outputStream = buffer.getOutputStream(); + final Event event = JacksonEvent.fromMessage(UUID.randomUUID().toString()); + codec.writeEvent(event, outputStream, null); final String s3Key = UUID.randomUUID().toString(); doThrow(AwsServiceException.class).when(buffer).flushToS3(eq(s3Client), anyString(), anyString()); boolean isUploadedToS3 = s3SinkService.retryFlushToS3(buffer, s3Key); @@ -321,7 +336,9 @@ void output_will_release_all_handles_since_a_flush() throws IOException { final long objectSize = random.nextInt(1_000_000) + 10_000; when(buffer.getSize()).thenReturn(objectSize); - when(codec.parse(any(), anyString())).thenReturn(UUID.randomUUID().toString()); + final OutputStream outputStream = mock(OutputStream.class); + final Event event = JacksonEvent.fromMessage(UUID.randomUUID().toString()); + doNothing().when(codec).writeEvent(event, outputStream, null); final S3SinkService s3SinkService = createObjectUnderTest(); final Collection> records = generateRandomStringEventRecord(); s3SinkService.output(records); @@ -342,7 +359,9 @@ void output_will_skip_releasing_events_without_EventHandle_objects() throws IOEx final long objectSize = random.nextInt(1_000_000) + 10_000; when(buffer.getSize()).thenReturn(objectSize); - when(codec.parse(any(), anyString())).thenReturn(UUID.randomUUID().toString()); + final OutputStream outputStream = mock(OutputStream.class); + final Event event1 = JacksonEvent.fromMessage(UUID.randomUUID().toString()); + doNothing().when(codec).writeEvent(event1, outputStream, null); final S3SinkService s3SinkService = createObjectUnderTest(); final Collection> records = generateRandomStringEventRecord(); records.stream() @@ -373,7 +392,9 @@ void output_will_release_all_handles_since_a_flush_when_S3_fails() throws IOExce final long objectSize = random.nextInt(1_000_000) + 10_000; when(buffer.getSize()).thenReturn(objectSize); - when(codec.parse(any(), anyString())).thenReturn(UUID.randomUUID().toString()); + final OutputStream outputStream = mock(OutputStream.class); + final Event event = JacksonEvent.fromMessage(UUID.randomUUID().toString()); + doNothing().when(codec).writeEvent(event, outputStream, null); final S3SinkService s3SinkService = createObjectUnderTest(); final List> records = generateEventRecords(1); s3SinkService.output(records); @@ -394,7 +415,9 @@ void output_will_release_only_new_handles_since_a_flush() throws IOException { final long objectSize = random.nextInt(1_000_000) + 10_000; when(buffer.getSize()).thenReturn(objectSize); - when(codec.parse(any(), anyString())).thenReturn(UUID.randomUUID().toString()); + final OutputStream outputStream = mock(OutputStream.class); + final Event event = JacksonEvent.fromMessage(UUID.randomUUID().toString()); + doNothing().when(codec).writeEvent(event, outputStream, null); final S3SinkService s3SinkService = createObjectUnderTest(); final Collection> records = generateRandomStringEventRecord(); s3SinkService.output(records); @@ -425,7 +448,9 @@ void output_will_release_only_new_handles_since_a_flush_when_S3_fails() throws I final long objectSize = random.nextInt(1_000_000) + 10_000; when(buffer.getSize()).thenReturn(objectSize); - when(codec.parse(any(), anyString())).thenReturn(UUID.randomUUID().toString()); + final OutputStream outputStream = mock(OutputStream.class); + final Event event = JacksonEvent.fromMessage(UUID.randomUUID().toString()); + doNothing().when(codec).writeEvent(event, outputStream, null); final S3SinkService s3SinkService = createObjectUnderTest(); final List> records = generateEventRecords(1); s3SinkService.output(records); @@ -470,4 +495,4 @@ private byte[] generateByteArray() { } return bytes; } -} +} \ No newline at end of file diff --git a/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/S3SinkTest.java b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/S3SinkTest.java index b92da61bc6..f7f8ceb0fb 100644 --- a/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/S3SinkTest.java +++ b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/S3SinkTest.java @@ -9,6 +9,7 @@ import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.opensearch.dataprepper.aws.api.AwsCredentialsSupplier; +import org.opensearch.dataprepper.model.codec.OutputCodec; import org.opensearch.dataprepper.model.configuration.PluginModel; import org.opensearch.dataprepper.model.configuration.PluginSetting; import org.opensearch.dataprepper.model.event.Event; @@ -17,8 +18,6 @@ import org.opensearch.dataprepper.model.sink.SinkContext; import org.opensearch.dataprepper.model.types.ByteCount; import org.opensearch.dataprepper.plugins.sink.accumulator.BufferTypeOptions; -import org.opensearch.dataprepper.plugins.sink.codec.Codec; -import org.opensearch.dataprepper.plugins.sink.codec.JsonCodec; import org.opensearch.dataprepper.plugins.sink.configuration.AwsAuthenticationOptions; import org.opensearch.dataprepper.plugins.sink.configuration.ObjectKeyOptions; import org.opensearch.dataprepper.plugins.sink.configuration.ThresholdOptions; @@ -59,7 +58,7 @@ void setUp() { sinkContext = mock(SinkContext.class); ThresholdOptions thresholdOptions = mock(ThresholdOptions.class); AwsAuthenticationOptions awsAuthenticationOptions = mock(AwsAuthenticationOptions.class); - Codec codec = mock(JsonCodec.class); + OutputCodec codec = mock(OutputCodec.class); ObjectKeyOptions objectKeyOptions = mock(ObjectKeyOptions.class); pluginSetting = mock(PluginSetting.class); PluginModel pluginModel = mock(PluginModel.class); diff --git a/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/ThresholdCheckTest.java b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/ThresholdCheckTest.java index 992e17368f..18d6672b0d 100644 --- a/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/ThresholdCheckTest.java +++ b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/ThresholdCheckTest.java @@ -11,6 +11,8 @@ import org.opensearch.dataprepper.plugins.sink.accumulator.Buffer; import org.opensearch.dataprepper.plugins.sink.accumulator.InMemoryBufferFactory; import java.io.IOException; +import java.io.OutputStream; + import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -23,7 +25,10 @@ void setUp() throws IOException { inMemoryBuffer = new InMemoryBufferFactory().getBuffer(); while (inMemoryBuffer.getEventCount() < 100) { - inMemoryBuffer.writeEvent(generateByteArray()); + OutputStream outputStream = inMemoryBuffer.getOutputStream(); + outputStream.write(generateByteArray()); + int eventCount = inMemoryBuffer.getEventCount() +1; + inMemoryBuffer.setEventCount(eventCount); } } @@ -78,7 +83,10 @@ void test_exceedThreshold_ture_dueTo_maxCollectionDuration_is_less_than_buffered boolean isThresholdExceed = Boolean.FALSE; synchronized (this) { while (inMemoryBuffer.getEventCount() < 100) { - inMemoryBuffer.writeEvent(generateByteArray()); + OutputStream outputStream = inMemoryBuffer.getOutputStream(); + outputStream.write(generateByteArray()); + int eventCount = inMemoryBuffer.getEventCount() +1; + inMemoryBuffer.setEventCount(eventCount); isThresholdExceed = ThresholdCheck.checkThresholdExceed(inMemoryBuffer, maxEvents, maxBytes, maxCollectionDuration); if (isThresholdExceed) { @@ -102,7 +110,10 @@ void test_exceedThreshold_ture_dueTo_maxCollectionDuration_is_greater_than_buffe boolean isThresholdExceed = Boolean.FALSE; synchronized (this) { while (inMemoryBuffer.getEventCount() < 100) { - inMemoryBuffer.writeEvent(generateByteArray()); + OutputStream outputStream = inMemoryBuffer.getOutputStream(); + outputStream.write(generateByteArray()); + int eventCount = inMemoryBuffer.getEventCount() +1; + inMemoryBuffer.setEventCount(eventCount); isThresholdExceed = ThresholdCheck.checkThresholdExceed(inMemoryBuffer, maxEvents, maxBytes, maxCollectionDuration); if (isThresholdExceed) { diff --git a/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/accumulator/InMemoryBufferTest.java b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/accumulator/InMemoryBufferTest.java index d97411e37c..254b32278c 100644 --- a/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/accumulator/InMemoryBufferTest.java +++ b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/accumulator/InMemoryBufferTest.java @@ -14,6 +14,8 @@ import org.mockito.junit.jupiter.MockitoExtension; import software.amazon.awssdk.services.s3.S3Client; import java.io.IOException; +import java.io.OutputStream; + import static org.hamcrest.Matchers.greaterThanOrEqualTo; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -31,7 +33,10 @@ void test_with_write_event_into_buffer() throws IOException { inMemoryBuffer = new InMemoryBuffer(); while (inMemoryBuffer.getEventCount() < MAX_EVENTS) { - inMemoryBuffer.writeEvent(generateByteArray()); + OutputStream outputStream = inMemoryBuffer.getOutputStream(); + outputStream.write(generateByteArray()); + int eventCount = inMemoryBuffer.getEventCount() +1; + inMemoryBuffer.setEventCount(eventCount); } assertThat(inMemoryBuffer.getSize(), greaterThanOrEqualTo(54110L)); assertThat(inMemoryBuffer.getEventCount(), equalTo(MAX_EVENTS)); @@ -44,7 +49,10 @@ void test_with_write_event_into_buffer_and_flush_toS3() throws IOException { inMemoryBuffer = new InMemoryBuffer(); while (inMemoryBuffer.getEventCount() < MAX_EVENTS) { - inMemoryBuffer.writeEvent(generateByteArray()); + OutputStream outputStream = inMemoryBuffer.getOutputStream(); + outputStream.write(generateByteArray()); + int eventCount = inMemoryBuffer.getEventCount() +1; + inMemoryBuffer.setEventCount(eventCount); } assertDoesNotThrow(() -> { inMemoryBuffer.flushToS3(s3Client, "data-prepper", "log.txt"); diff --git a/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/accumulator/LocalFileBufferTest.java b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/accumulator/LocalFileBufferTest.java index 65ad70963d..a7c30f7b13 100644 --- a/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/accumulator/LocalFileBufferTest.java +++ b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/accumulator/LocalFileBufferTest.java @@ -13,6 +13,7 @@ import software.amazon.awssdk.services.s3.model.PutObjectRequest; import java.io.File; import java.io.IOException; +import java.io.OutputStream; import java.util.UUID; import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.notNullValue; @@ -46,7 +47,10 @@ void setUp() throws IOException { @Test void test_with_write_events_into_buffer() throws IOException { while (localFileBuffer.getEventCount() < 55) { - localFileBuffer.writeEvent(generateByteArray()); + OutputStream outputStream = localFileBuffer.getOutputStream(); + outputStream.write(generateByteArray()); + int eventCount = localFileBuffer.getEventCount() +1; + localFileBuffer.setEventCount(eventCount); } assertThat(localFileBuffer.getSize(), greaterThan(1l)); assertThat(localFileBuffer.getEventCount(), equalTo(55)); @@ -69,7 +73,10 @@ void test_without_write_events_into_buffer() { @Test void test_with_write_events_into_buffer_and_flush_toS3() throws IOException { while (localFileBuffer.getEventCount() < 55) { - localFileBuffer.writeEvent(generateByteArray()); + OutputStream outputStream = localFileBuffer.getOutputStream(); + outputStream.write(generateByteArray()); + int eventCount = localFileBuffer.getEventCount() +1; + localFileBuffer.setEventCount(eventCount); } assertThat(localFileBuffer.getSize(), greaterThan(1l)); assertThat(localFileBuffer.getEventCount(), equalTo(55)); diff --git a/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/accumulator/ObjectKeyTest.java b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/accumulator/ObjectKeyTest.java index c2ee676e5d..f70cba4d3a 100644 --- a/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/accumulator/ObjectKeyTest.java +++ b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/accumulator/ObjectKeyTest.java @@ -15,7 +15,6 @@ import org.opensearch.dataprepper.model.configuration.PluginSetting; import org.opensearch.dataprepper.model.plugin.PluginFactory; import org.opensearch.dataprepper.plugins.sink.S3SinkConfig; -import org.opensearch.dataprepper.plugins.sink.codec.JsonCodec; import org.opensearch.dataprepper.plugins.sink.configuration.ObjectKeyOptions; import static org.hamcrest.CoreMatchers.startsWith; @@ -36,8 +35,6 @@ class ObjectKeyTest { @Mock private PluginFactory pluginFactory; @Mock - private JsonCodec codec; - @Mock private ObjectKeyOptions objectKeyOptions; @BeforeEach @@ -58,7 +55,7 @@ void test_buildingPathPrefix() { void test_objectFileName() { when(objectKeyOptions.getNamePattern()).thenReturn("my-elb-%{yyyy-MM-dd'T'hh-mm-ss}"); - String objectFileName = ObjectKey.objectFileName(s3SinkConfig); + String objectFileName = ObjectKey.objectFileName(s3SinkConfig, null); Assertions.assertNotNull(objectFileName); assertThat(objectFileName, startsWith("my-elb")); } @@ -68,7 +65,7 @@ void test_objectFileName_with_fileExtension() { when(s3SinkConfig.getObjectKeyOptions().getNamePattern()) .thenReturn("events-%{yyyy-MM-dd'T'hh-mm-ss}.pdf"); - String objectFileName = ObjectKey.objectFileName(s3SinkConfig); + String objectFileName = ObjectKey.objectFileName(s3SinkConfig, null); Assertions.assertNotNull(objectFileName); Assertions.assertTrue(objectFileName.contains(".pdf")); } @@ -78,7 +75,7 @@ void test_objectFileName_default_fileExtension() { when(s3SinkConfig.getObjectKeyOptions().getNamePattern()) .thenReturn("events-%{yyyy-MM-dd'T'hh-mm-ss}"); - String objectFileName = ObjectKey.objectFileName(s3SinkConfig); + String objectFileName = ObjectKey.objectFileName(s3SinkConfig, null); Assertions.assertNotNull(objectFileName); Assertions.assertTrue(objectFileName.contains(".json")); } diff --git a/data-prepper-plugins/s3-source/build.gradle b/data-prepper-plugins/s3-source/build.gradle index 343f578b07..d2fb01a822 100644 --- a/data-prepper-plugins/s3-source/build.gradle +++ b/data-prepper-plugins/s3-source/build.gradle @@ -29,8 +29,8 @@ dependencies { implementation 'joda-time:joda-time:2.11.1' implementation 'org.hibernate.validator:hibernate-validator:7.0.5.Final' implementation 'com.fasterxml.jackson.dataformat:jackson-dataformat-csv' - implementation 'com.fasterxml.jackson.datatype:jackson-datatype-jsr310:2.14.2' - implementation 'org.xerial.snappy:snappy-java:1.1.9.1' + implementation 'com.fasterxml.jackson.datatype:jackson-datatype-jsr310:2.15.2' + implementation 'org.xerial.snappy:snappy-java:1.1.10.1' implementation 'org.apache.parquet:parquet-common:1.12.3' testImplementation 'org.apache.commons:commons-lang3:3.12.0' testImplementation 'com.github.tomakehurst:wiremock:3.0.0-beta-8' diff --git a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/S3InputStream.java b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/S3InputStream.java index 0629db6831..14f8a509ff 100644 --- a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/S3InputStream.java +++ b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/S3InputStream.java @@ -129,11 +129,15 @@ public int read() throws IOException { Preconditions.checkState(!closed, "Cannot read: already closed"); positionStream(); - pos += 1; - next += 1; - bytesCounter.increment(); + final int byteRead = stream.read(); - return stream.read(); + if (byteRead != -1) { + pos += 1; + next += 1; + bytesCounter.increment(); + } + + return byteRead; } /** @@ -161,10 +165,13 @@ public int read(byte[] b, int off, int len) throws IOException { Preconditions.checkState(!closed, "Cannot read: already closed"); positionStream(); - int bytesRead = stream.read(b, off, len); - pos += bytesRead; - next += bytesRead; - bytesCounter.add(bytesRead); + final int bytesRead = stream.read(b, off, len); + + if (bytesRead > 0) { + pos += bytesRead; + next += bytesRead; + bytesCounter.add(bytesRead); + } return bytesRead; } @@ -203,9 +210,11 @@ public int readNBytes(byte[] b, int off, int len) throws IOException { final int bytesRead = stream.readNBytes(b, off, len); - pos += bytesRead; - next += bytesRead; - bytesCounter.add(bytesRead); + if (bytesRead > 0) { + pos += bytesRead; + next += bytesRead; + bytesCounter.add(bytesRead); + } return bytesRead; } @@ -325,9 +334,11 @@ public void readFully(byte[] bytes, int start, int len) throws IOException { int bytesRead = readFully(stream, bytes, start, len); - this.pos += bytesRead; - this.next += bytesRead; - this.bytesCounter.add(bytesRead); + if (bytesRead > 0) { + this.pos += bytesRead; + this.next += bytesRead; + this.bytesCounter.add(bytesRead); + } } /** @@ -354,9 +365,11 @@ public int read(ByteBuffer buf) throws IOException { bytesRead = readDirectBuffer(stream, buf, temp); } - this.pos += bytesRead; - this.next += bytesRead; - this.bytesCounter.add(bytesRead); + if (bytesRead > 0) { + this.pos += bytesRead; + this.next += bytesRead; + this.bytesCounter.add(bytesRead); + } return bytesRead; } @@ -385,9 +398,11 @@ public void readFully(ByteBuffer buf) throws IOException { bytesRead = readFullyDirectBuffer(stream, buf, temp); } - this.pos += bytesRead; - this.next += bytesRead; - this.bytesCounter.add(bytesRead); + if (bytesRead > 0) { + this.pos += bytesRead; + this.next += bytesRead; + this.bytesCounter.add(bytesRead); + } } /** @@ -478,7 +493,7 @@ private void closeStream() throws IOException { */ private void abortStream() { try { - if (stream instanceof Abortable && stream.read() != -1) { + if (stream instanceof Abortable) { ((Abortable) stream).abort(); } } catch (Exception e) { diff --git a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/SqsWorker.java b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/SqsWorker.java index 9ee19294a9..4a1beecf61 100644 --- a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/SqsWorker.java +++ b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/SqsWorker.java @@ -269,6 +269,9 @@ private Optional processS3Object( } private void deleteSqsMessages(final List deleteMessageBatchRequestEntryCollection) { + if (deleteMessageBatchRequestEntryCollection.size() == 0) { + return; + } final DeleteMessageBatchRequest deleteMessageBatchRequest = buildDeleteMessageBatchRequest(deleteMessageBatchRequestEntryCollection); try { final DeleteMessageBatchResponse deleteMessageBatchResponse = sqsClient.deleteMessageBatch(deleteMessageBatchRequest); @@ -288,7 +291,7 @@ private void deleteSqsMessages(final List delete if(LOG.isErrorEnabled()) { final String failedMessages = deleteMessageBatchResponse.failed().stream() - .map(failed -> toString()) + .map(failed -> failed.toString()) .collect(Collectors.joining(", ")); LOG.error("Failed to delete {} messages from SQS with errors: [{}].", failedDeleteCount, failedMessages); } diff --git a/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/S3InputStreamTest.java b/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/S3InputStreamTest.java index 1ac31890ee..d75254189d 100644 --- a/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/S3InputStreamTest.java +++ b/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/S3InputStreamTest.java @@ -15,6 +15,7 @@ import software.amazon.awssdk.services.s3.model.S3Exception; import java.io.ByteArrayInputStream; +import java.io.EOFException; import java.io.IOException; import java.io.InputStream; import java.nio.ByteBuffer; @@ -103,6 +104,19 @@ void testRead() throws IOException { verify(s3ObjectSizeProcessedSummary).record(1.0); } + @Test + void testReadEndOfFile() throws IOException { + InputStream inputStream = new ByteArrayInputStream("".getBytes()); + when(s3Client.getObject(any(GetObjectRequest.class), any(ResponseTransformer.class))).thenReturn(inputStream); + + int firstByte = s3InputStream.read(); + assertEquals(-1, firstByte); + + s3InputStream.close(); + + verify(s3ObjectSizeProcessedSummary).record(0.0); + } + @Test void testReadByteArray() throws IOException { InputStream inputStream = new ByteArrayInputStream("Test data".getBytes()); @@ -150,6 +164,20 @@ void testReadNBytes_intoArray() throws Exception { verify(s3ObjectSizeProcessedSummary).record(4.0); } + @Test + void testReadNBytes_endOfFile() throws Exception { + InputStream inputStream = new ByteArrayInputStream("".getBytes()); + when(s3Client.getObject(any(GetObjectRequest.class), any(ResponseTransformer.class))).thenReturn(inputStream); + + byte[] buffer = new byte[9]; + int bytesRead = s3InputStream.readNBytes(buffer, 0, 4); + + assertEquals(0, bytesRead); + + s3InputStream.close(); + verify(s3ObjectSizeProcessedSummary).record(0.0); + } + @Test void testReadNBytes_getArray() throws Exception { InputStream inputStream = new ByteArrayInputStream("Test data".getBytes()); @@ -242,6 +270,19 @@ void testReadFullyByteBuffer() throws IOException { verify(s3ObjectSizeProcessedSummary).record(4.0); } + @Test + void testReadFullyByteBuffer_endOfFile() throws IOException { + InputStream inputStream = new ByteArrayInputStream("".getBytes()); + when(s3Client.getObject(any(GetObjectRequest.class), any(ResponseTransformer.class))).thenReturn(inputStream); + s3InputStream.seek(0); // Force opening the stream + + ByteBuffer buffer = ByteBuffer.allocate(4); + assertThrows(EOFException.class, () -> s3InputStream.readFully(buffer)); + + s3InputStream.close(); + verify(s3ObjectSizeProcessedSummary).record(0.0); + } + @Test void testReadFullyHeapBuffer() throws IOException { InputStream inputStream = new ByteArrayInputStream("Test data".getBytes()); diff --git a/data-prepper-plugins/sqs-source/build.gradle b/data-prepper-plugins/sqs-source/build.gradle index a7c4f005fa..199271317f 100644 --- a/data-prepper-plugins/sqs-source/build.gradle +++ b/data-prepper-plugins/sqs-source/build.gradle @@ -9,10 +9,41 @@ dependencies { implementation 'software.amazon.awssdk:sts' implementation 'io.micrometer:micrometer-core' implementation 'com.fasterxml.jackson.core:jackson-annotations' - implementation 'com.fasterxml.jackson.datatype:jackson-datatype-jsr310:2.14.2' + implementation 'com.fasterxml.jackson.datatype:jackson-datatype-jsr310:2.15.2' testImplementation 'com.fasterxml.jackson.dataformat:jackson-dataformat-yaml' testImplementation project(':data-prepper-plugins:blocking-buffer') } test { useJUnitPlatform() +} + +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 + systemProperty 'tests.sqs.source.aws.region', System.getProperty('tests.sqs.source.aws.region') + systemProperty 'tests.sqs.source.queue.url', System.getProperty('tests.sqs.source.queue.url') + + filter { + includeTestsMatching '*IT' + } } \ No newline at end of file diff --git a/data-prepper-plugins/sqs-source/src/integrationTest/java/org/opensearch/dataprepper/plugins/source/sqssource/RecordsGenerator.java b/data-prepper-plugins/sqs-source/src/integrationTest/java/org/opensearch/dataprepper/plugins/source/sqssource/RecordsGenerator.java new file mode 100644 index 0000000000..a0d4a24d68 --- /dev/null +++ b/data-prepper-plugins/sqs-source/src/integrationTest/java/org/opensearch/dataprepper/plugins/source/sqssource/RecordsGenerator.java @@ -0,0 +1,11 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.source.sqssource; + +import java.util.List; + +public interface RecordsGenerator { + void pushMessages(final List messages, final String queueUrl); +} diff --git a/data-prepper-plugins/sqs-source/src/integrationTest/java/org/opensearch/dataprepper/plugins/source/sqssource/SqsRecordsGenerator.java b/data-prepper-plugins/sqs-source/src/integrationTest/java/org/opensearch/dataprepper/plugins/source/sqssource/SqsRecordsGenerator.java new file mode 100644 index 0000000000..18e4ea2980 --- /dev/null +++ b/data-prepper-plugins/sqs-source/src/integrationTest/java/org/opensearch/dataprepper/plugins/source/sqssource/SqsRecordsGenerator.java @@ -0,0 +1,47 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.source.sqssource; + +import software.amazon.awssdk.services.sqs.SqsClient; +import software.amazon.awssdk.services.sqs.model.SendMessageBatchRequest; +import software.amazon.awssdk.services.sqs.model.SendMessageBatchRequestEntry; + +import java.util.ArrayList; +import java.util.List; +import java.util.UUID; + +public class SqsRecordsGenerator implements RecordsGenerator { + + private final SqsClient sqsClient; + + public SqsRecordsGenerator(final SqsClient sqsClient){ + this.sqsClient = sqsClient; + } + + @Override + public void pushMessages(final List messages, String queueUrl) { + final List> batches = splitIntoBatches(messages, 10); + batches.forEach(batch -> { + List entries = new ArrayList<>(); + batch.forEach(msg -> entries.add(SendMessageBatchRequestEntry.builder() + .id(UUID.randomUUID() + "-" + UUID.randomUUID()).messageBody(msg).build())); + sqsClient.sendMessageBatch(SendMessageBatchRequest.builder().queueUrl(queueUrl).entries(entries).build()); + }); + } + + private static List> splitIntoBatches(List messages, int batchSize) { + List> batches = new ArrayList<>(); + int totalRecords = messages.size(); + int numBatches = (int) Math.ceil((double) totalRecords / batchSize); + + for (int i = 0; i < numBatches; i++) { + int startIndex = i * batchSize; + int endIndex = Math.min(startIndex + batchSize, totalRecords); + List batch = messages.subList(startIndex, endIndex); + batches.add(batch); + } + return batches; + } +} diff --git a/data-prepper-plugins/sqs-source/src/integrationTest/java/org/opensearch/dataprepper/plugins/source/sqssource/SqsSourceTaskIT.java b/data-prepper-plugins/sqs-source/src/integrationTest/java/org/opensearch/dataprepper/plugins/source/sqssource/SqsSourceTaskIT.java new file mode 100644 index 0000000000..31e12db422 --- /dev/null +++ b/data-prepper-plugins/sqs-source/src/integrationTest/java/org/opensearch/dataprepper/plugins/source/sqssource/SqsSourceTaskIT.java @@ -0,0 +1,147 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.source.sqssource; + +import com.linecorp.armeria.client.retry.Backoff; +import io.micrometer.core.instrument.Counter; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.CsvSource; +import org.mockito.Mock; +import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSetManager; +import org.opensearch.dataprepper.model.configuration.PluginSetting; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.plugins.aws.sqs.common.SqsService; +import org.opensearch.dataprepper.plugins.aws.sqs.common.handler.SqsMessageHandler; +import org.opensearch.dataprepper.plugins.aws.sqs.common.metrics.SqsMetrics; +import org.opensearch.dataprepper.plugins.aws.sqs.common.model.SqsOptions; +import org.opensearch.dataprepper.plugins.buffer.blockingbuffer.BlockingBuffer; +import org.opensearch.dataprepper.plugins.source.sqssource.handler.RawSqsMessageHandler; +import software.amazon.awssdk.regions.Region; +import software.amazon.awssdk.services.sqs.SqsClient; + +import java.time.Duration; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.collection.IsIterableContainingInAnyOrder.containsInAnyOrder; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class SqsSourceTaskIT { + + static final long INITIAL_DELAY = Duration.ofSeconds(20).toMillis(); + + static final long MAXIMUM_DELAY = Duration.ofMinutes(5).toMillis(); + + static final double JITTER_RATE = 0.20; + + private static final String TEST_PIPELINE_NAME = "pipeline"; + + private static final String MESSAGE = "message"; + + private static final String JSON_MESSAGE = "{\"array\":[{\"name\":\"abc\",\"test\":[{\"company\":\"xyz\"}]},{\"number\":1}]}"; + + private static final String LOG_MESSAGE = "2023-06-14T11:59:54,350 [main] INFO Test - Application started Successfully\n"; + + private static final String AWS_SQS_QUEUE_URL = "tests.sqs.source.queue.url"; + + private static final String AWS_REGION = "tests.sqs.source.aws.region"; + public static final Duration BUFFER_TIMEOUT = Duration.ofSeconds(10); + public static final int RECORDS_TO_ACCUMULATE = 100; + + @Mock + private AcknowledgementSetManager acknowledgementSetManager; + + private BlockingBuffer> buffer; + + private Counter messageReceivedCounter; + + private Counter messageDeletedCounter; + + private Backoff backoff; + + private SqsClient sqsClient; + + private SqsMetrics sqsMetrics; + + private ScheduledExecutorService executorService; + + @ParameterizedTest + @CsvSource({"2,1","10,2","50,4","100,5","200,7","500,10","1000,15","2000,24"}) + public void process_sqs_messages(int messageLoad,int threadSleepTime){ + final SqsRecordsGenerator sqsRecordsGenerator = new SqsRecordsGenerator(sqsClient); + final String queueUrl = System.getProperty(AWS_SQS_QUEUE_URL); + + List inputDataList = pushMessagesToQueue(sqsRecordsGenerator, queueUrl,messageLoad); + this.buffer = getBuffer(inputDataList.size()); + + SqsOptions sqsOptions = new SqsOptions.Builder().setSqsUrl(queueUrl).setMaximumMessages(10).build(); + executorService.scheduleAtFixedRate(createObjectUnderTest(sqsOptions),0,1, TimeUnit.MILLISECONDS); + try { + Thread.sleep(Duration.ofSeconds(threadSleepTime).toMillis()); + } catch (InterruptedException e) { + } + executorService.shutdown(); + final List> bufferEvents = new ArrayList<>(buffer.read((int) Duration.ofSeconds(10).toMillis()).getKey()); + final List bufferData = bufferEvents.stream().map(obj -> obj.getData().get(MESSAGE, String.class)).collect(Collectors.toList()); + assertThat(bufferData, containsInAnyOrder(inputDataList.toArray())); + assertThat(bufferData.size(),equalTo(inputDataList.size())); + } + + @BeforeEach + public void setup(){ + this.acknowledgementSetManager = mock(AcknowledgementSetManager.class); + this.messageReceivedCounter = mock(Counter.class); + this.messageDeletedCounter = mock(Counter.class); + this.sqsMetrics = mock(SqsMetrics.class); + when(sqsMetrics.getSqsMessagesReceivedCounter()).thenReturn(messageReceivedCounter); + when(sqsMetrics.getSqsMessagesDeletedCounter()).thenReturn(messageDeletedCounter); + this.backoff = Backoff.exponential(INITIAL_DELAY, MAXIMUM_DELAY).withJitter(JITTER_RATE) + .withMaxAttempts(Integer.MAX_VALUE); + this.sqsClient = SqsClient.builder().region(Region.of(System.getProperty(AWS_REGION))).build(); + executorService = Executors.newSingleThreadScheduledExecutor(); + } + + public SqsSourceTask createObjectUnderTest(final SqsOptions sqsOptions){ + SqsService sqsService = new SqsService(sqsMetrics,sqsClient,backoff); + SqsMessageHandler sqsHandler = new RawSqsMessageHandler(sqsService); + return new SqsSourceTask(buffer, RECORDS_TO_ACCUMULATE, BUFFER_TIMEOUT + ,sqsService,sqsOptions,sqsMetrics, + acknowledgementSetManager,Boolean.FALSE,sqsHandler); + } + + private static List pushMessagesToQueue(SqsRecordsGenerator sqsRecordsGenerator, String queueUrl,final int load) { + List inputDataList = new ArrayList<>(); + for(int msgCount = 0; msgCount < load/2; msgCount++) + generateMessagesForSqsPush().forEach(obj -> inputDataList.add(obj)); + sqsRecordsGenerator.pushMessages(inputDataList, queueUrl); + return inputDataList; + } + + private static List generateMessagesForSqsPush(){ + List messages = new ArrayList<>(2); + messages.add(JSON_MESSAGE); + messages.add(LOG_MESSAGE); + return messages; + } + + private BlockingBuffer> getBuffer(final int bufferSize) { + final HashMap integerHashMap = new HashMap<>(); + integerHashMap.put("buffer_size", bufferSize); + integerHashMap.put("batch_size", bufferSize); + final PluginSetting pluginSetting = new PluginSetting("blocking_buffer", integerHashMap); + pluginSetting.setPipelineName(TEST_PIPELINE_NAME); + return new BlockingBuffer<>(pluginSetting); + } +} diff --git a/data-prepper-plugins/translate-processor/build.gradle b/data-prepper-plugins/translate-processor/build.gradle new file mode 100644 index 0000000000..5139ebbb07 --- /dev/null +++ b/data-prepper-plugins/translate-processor/build.gradle @@ -0,0 +1,24 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +plugins { + id 'java' +} + + +dependencies { + implementation project(':data-prepper-api') + implementation 'com.fasterxml.jackson.dataformat:jackson-dataformat-yaml:2.15.0' + implementation 'io.micrometer:micrometer-core' + implementation project(path: ':data-prepper-api') + implementation project(path: ':data-prepper-plugins:mutate-event-processors') + testImplementation project(':data-prepper-plugins:log-generator-source') + testImplementation project(':data-prepper-test-common') + implementation 'org.apache.commons:commons-lang3:3.12.0' +} + +test { + useJUnitPlatform() +} \ No newline at end of file diff --git a/data-prepper-plugins/translate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/translate/RegexParameterConfiguration.java b/data-prepper-plugins/translate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/translate/RegexParameterConfiguration.java new file mode 100644 index 0000000000..a7e3d494da --- /dev/null +++ b/data-prepper-plugins/translate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/translate/RegexParameterConfiguration.java @@ -0,0 +1,25 @@ +package org.opensearch.dataprepper.plugins.processor.translate; + +import com.fasterxml.jackson.annotation.JsonProperty; +import jakarta.validation.constraints.NotNull; + +import java.util.Map; + + +public class RegexParameterConfiguration { + + final boolean DEFAULT_EXACT = true; + @NotNull + @JsonProperty("patterns") + private Map patterns; + + @JsonProperty("exact") + private Boolean exact = DEFAULT_EXACT; + + public Map getPatterns() { + return patterns; + } + + public Boolean getExact() { return exact; } + +} diff --git a/data-prepper-plugins/translate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/translate/TranslateProcessor.java b/data-prepper-plugins/translate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/translate/TranslateProcessor.java new file mode 100644 index 0000000000..476bfa807b --- /dev/null +++ b/data-prepper-plugins/translate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/translate/TranslateProcessor.java @@ -0,0 +1,281 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.processor.translate; + +import org.apache.commons.lang3.Range; +import org.apache.commons.lang3.StringUtils; +import org.apache.commons.lang3.math.NumberUtils; +import org.opensearch.dataprepper.expression.ExpressionEvaluator; +import org.opensearch.dataprepper.metrics.PluginMetrics; +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.plugin.InvalidPluginConfigurationException; +import org.opensearch.dataprepper.model.processor.AbstractProcessor; +import org.opensearch.dataprepper.model.processor.Processor; +import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.typeconverter.TypeConverter; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import java.util.stream.Collectors; + +import static org.opensearch.dataprepper.logging.DataPrepperMarkers.EVENT; + + +@DataPrepperPlugin(name = "translate", pluginType = Processor.class, pluginConfigurationType = TranslateProcessorConfig.class) +public class TranslateProcessor extends AbstractProcessor, Record> { + + private static final Logger LOG = LoggerFactory.getLogger(TranslateProcessor.class); + private final ExpressionEvaluator expressionEvaluator; + private final TranslateProcessorConfig translateProcessorConfig; + private final LinkedHashMap, Object> rangeMappings; + private final Map individualMappings; + private final Map compiledPatterns; + private final TypeConverter converter; + + @DataPrepperPluginConstructor + public TranslateProcessor(PluginMetrics pluginMetrics, final TranslateProcessorConfig translateProcessorConfig, final ExpressionEvaluator expressionEvaluator) { + super(pluginMetrics); + this.translateProcessorConfig = translateProcessorConfig; + this.expressionEvaluator = expressionEvaluator; + this.converter = translateProcessorConfig.getTargetType().getTargetConverter(); + individualMappings = new HashMap<>(); + rangeMappings = new LinkedHashMap<>(); + compiledPatterns = new HashMap<>(); + if (Objects.nonNull(this.translateProcessorConfig.getRegexParameterConfiguration())) { + compilePatterns(translateProcessorConfig + .getRegexParameterConfiguration() + .getPatterns()); + } + processMapField(translateProcessorConfig.getMap()); + parseFile(translateProcessorConfig.getFilePath()); + checkOverlappingKeys(); + } + + private void compilePatterns(Map mappings) { + for (String pattern : mappings.keySet()) { + Pattern compiledPattern = Pattern.compile(pattern); + compiledPatterns.put(compiledPattern, mappings.get(pattern)); + } + } + + private void processMapField(Map map) { + if (Objects.nonNull(map)) { + for (Map.Entry mapEntry : map.entrySet()) { + parseIndividualKeys(mapEntry); + } + } + } + + private void parseIndividualKeys(Map.Entry mapEntry){ + String[] commaSeparatedKeys = mapEntry.getKey().split(","); + for(String individualKey : commaSeparatedKeys){ + if(individualKey.contains("-")){ + addRangeMapping(Map.entry(individualKey, mapEntry.getValue())); + } else { + addIndividualMapping(individualKey, mapEntry.getValue()); + } + } + } + + private void addRangeMapping(Map.Entry mapEntry){ + String[] rangeKeys = mapEntry.getKey().split("-"); + if(rangeKeys.length!=2 || !StringUtils.isNumericSpace(rangeKeys[0]) || !StringUtils.isNumericSpace(rangeKeys[1])){ + addIndividualMapping(mapEntry.getKey(), mapEntry.getValue()); + } else { + Float lowKey = Float.parseFloat(rangeKeys[0]); + Float highKey = Float.parseFloat(rangeKeys[1]); + Range rangeEntry = Range.between(lowKey, highKey); + if (isRangeOverlapping(rangeEntry)) { + String exceptionMsg = "map option contains key "+mapEntry.getKey()+" that overlaps with other range entries"; + throw new InvalidPluginConfigurationException(exceptionMsg); + } else { + rangeMappings.put(Range.between(lowKey, highKey), mapEntry.getValue()); + } + } + } + + private void addIndividualMapping(final String key, final Object value){ + if(individualMappings.containsKey(key)){ + String exceptionMsg = "map option contains duplicate entries of "+key; + throw new InvalidPluginConfigurationException(exceptionMsg); + } else { + individualMappings.put(key.strip(), value); + } + } + + private boolean isRangeOverlapping(Range rangeEntry) { + for (Range range : rangeMappings.keySet()) { + if (range.isOverlappedBy(rangeEntry)) { + return true; + } + } + return false; + } + + private void checkOverlappingKeys() { + for (String individualKey : individualMappings.keySet()) { + if (NumberUtils.isParsable(individualKey)) { + Float floatKey = Float.parseFloat(individualKey); + Range range = Range.between(floatKey, floatKey); + if (isRangeOverlapping(range)) { + String exceptionMsg = "map option contains key " + individualKey + " that overlaps with other range entries"; + throw new InvalidPluginConfigurationException(exceptionMsg); + } + } + } + } + + private void parseFile(String filePath){ + //todo + } + + @Override + public Collection> doExecute(Collection> records) { + for (final Record record : records) { + final Event recordEvent = record.getData(); + if (Objects.nonNull(translateProcessorConfig.getTranslateWhen()) && !expressionEvaluator.evaluateConditional(translateProcessorConfig.getTranslateWhen(), recordEvent)) { + continue; + } + try { + String iterateOn = translateProcessorConfig.getIterateOn(); + if (Objects.nonNull(iterateOn)) { + List> objectsToIterate = recordEvent.get(iterateOn, List.class); + for (Map recordObject : objectsToIterate) { + performMappings(recordObject); + } + recordEvent.put(iterateOn, objectsToIterate); + } else { + performMappings(recordEvent); + } + } catch (Exception ex) { + LOG.error(EVENT, "Error mapping the source [{}] of entry [{}]", translateProcessorConfig.getSource(), + record.getData(), ex); + } + } + return records; + } + + private String getSourceValue(Object recordObject, String sourceKey) { + if (recordObject instanceof Map) { + return (String) ((Map) recordObject).get(sourceKey); + } else { + return ((Event) recordObject).get(sourceKey, String.class); + } + } + + private Object getTargetValue(Object sourceObject, List targetValues){ + if(sourceObject instanceof String) { + return converter.convert(targetValues.get(0)); + } + return targetValues.stream().map(converter::convert).collect(Collectors.toList()); + } + + private void performMappings(Object recordObject) { + List targetValues = new ArrayList<>(); + Object sourceObject = translateProcessorConfig.getSource(); + List sourceKeys; + if (sourceObject instanceof List) { + sourceKeys = (ArrayList) sourceObject; + } else if (sourceObject instanceof String) { + sourceKeys = List.of((String) sourceObject); + } else { + String exceptionMsg = "source option configured incorrectly. source can only be a String or list of Strings"; + throw new InvalidPluginConfigurationException(exceptionMsg); + } + for (String sourceKey : sourceKeys) { + String sourceValue = getSourceValue(recordObject, sourceKey); + Optional targetValue = getTargetValueForSource(sourceValue); + targetValue.ifPresent(targetValues::add); + } + addTargetToRecords(sourceObject, targetValues, recordObject); + } + + private Optional getTargetValueForSource(final String sourceValue) { + Optional targetValue = Optional.empty(); + targetValue = targetValue + .or(() -> matchesIndividualEntry(sourceValue)) + .or(() -> matchesRangeEntry(sourceValue)) + .or(() -> matchesPatternEntry(sourceValue)) + .or(() -> Optional.ofNullable(translateProcessorConfig.getDefaultValue())); + return targetValue; + } + + private Optional matchesIndividualEntry(final String sourceValue) { + if (individualMappings.containsKey(sourceValue)) { + return Optional.of(individualMappings.get(sourceValue)); + } + return Optional.empty(); + } + + private Optional matchesRangeEntry(final String sourceValue) { + if (!NumberUtils.isParsable(sourceValue)) { + return Optional.empty(); + } + Float floatKey = Float.parseFloat(sourceValue); + for (Map.Entry, Object> rangeEntry : rangeMappings.entrySet()) { + Range range = rangeEntry.getKey(); + if (range.contains(floatKey)) { + return Optional.of(rangeEntry.getValue()); + } + } + return Optional.empty(); + } + + private Optional matchesPatternEntry(final String sourceValue) { + if (compiledPatterns.isEmpty()) { + return Optional.empty(); + } + final boolean exact = translateProcessorConfig.getRegexParameterConfiguration().getExact(); + for (Pattern pattern : compiledPatterns.keySet()) { + Matcher matcher = pattern.matcher(sourceValue); + if (matcher.matches() || (!exact && matcher.find())) { + return Optional.of(compiledPatterns.get(pattern)); + } + } + return Optional.empty(); + } + + private void addTargetToRecords(Object sourceObject, List targetValues, Object recordObject) { + if (targetValues.isEmpty()) { + return; + } + final String targetField = translateProcessorConfig.getTarget(); + if (recordObject instanceof Map) { + Map recordMap = (Map) recordObject; + recordMap.put(targetField, getTargetValue(sourceObject, targetValues)); + } else if (recordObject instanceof Event) { + Event event = (Event) recordObject; + event.put(targetField, getTargetValue(sourceObject, targetValues)); + } + } + + @Override + public void prepareForShutdown() { + + } + + @Override + public boolean isReadyForShutdown() { + return true; + } + + @Override + public void shutdown() { + + } +} 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 new file mode 100644 index 0000000000..845442bc40 --- /dev/null +++ b/data-prepper-plugins/translate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/translate/TranslateProcessorConfig.java @@ -0,0 +1,123 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.processor.translate; + + +import com.fasterxml.jackson.annotation.JsonProperty; +import jakarta.validation.constraints.AssertTrue; +import jakarta.validation.constraints.NotEmpty; +import jakarta.validation.constraints.NotNull; +import org.opensearch.dataprepper.plugins.processor.mutateevent.TargetType; +import org.opensearch.dataprepper.typeconverter.TypeConverter; + +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.stream.Stream; + + +public class TranslateProcessorConfig { + + + @JsonProperty("source") + @NotNull + private Object source; + + @JsonProperty("target") + @NotNull + @NotEmpty + private String target; + + @JsonProperty("map") + private Map map; + + @JsonProperty("file_path") + private String filePath; + + @JsonProperty("default") + private String defaultValue; + + @JsonProperty("translate_when") + private String translateWhen; + + @JsonProperty("iterate_on") + private String iterateOn; + + @JsonProperty("regex") + private RegexParameterConfiguration regexParameterConfiguration; + + @JsonProperty("target_type") + private TargetType targetType = TargetType.STRING; + + + public Object getSource() { return source; } + + public String getTarget() { return target; } + + public Map getMap() { return map; } + + public String getDefaultValue() { return defaultValue; } + + public String getFilePath() { return filePath; } + + public String getTranslateWhen() { return translateWhen; } + + public String getIterateOn() { return iterateOn; } + + public TargetType getTargetType() { return targetType; } + + public RegexParameterConfiguration getRegexParameterConfiguration(){ return regexParameterConfiguration; } + + + @AssertTrue(message = "source field must be a string or list of strings") + public boolean isSourceFieldValid(){ + if(source instanceof String){ + return true; + } + if(source instanceof List){ + List sourceList = (List) source; + return sourceList.stream().allMatch(sourceItem -> sourceItem instanceof String); + } + return false; + } + + @AssertTrue(message = "Either of map or patterns or file_path options need to be configured.") + public boolean hasMappings() { + return Stream.of(map, filePath, regexParameterConfiguration).filter(n -> n!=null).count() != 0; + } + + @AssertTrue(message = "pattern option is mandatory while configuring regex option") + public boolean isPatternPresent(){ + return regexParameterConfiguration == null || regexParameterConfiguration.getPatterns() != null; + } + + @AssertTrue(message = "The mapped values do not match the target type provided") + public boolean isMapTypeValid() { + return map.keySet().stream().allMatch(key -> checkTargetValueType(map.get(key))); + } + + @AssertTrue(message = "The pattern values do not match the target type provided") + public boolean isPatternTypeValid() { + if (Objects.isNull(regexParameterConfiguration) || Objects.isNull(regexParameterConfiguration.getPatterns())) { + return true; + } + Map patterns = regexParameterConfiguration.getPatterns(); + return patterns.keySet().stream().allMatch(key -> checkTargetValueType(patterns.get(key))); + } + + private boolean checkTargetValueType(Object val) throws NumberFormatException { + if (Objects.isNull(targetType)) { + return true; + } + try { + final TypeConverter converter = targetType.getTargetConverter(); + converter.convert(val); + } catch (Exception ex) { + return false; + } + return true; + } +} diff --git a/data-prepper-plugins/translate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/translate/RegexParameterConfigurationTest.java b/data-prepper-plugins/translate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/translate/RegexParameterConfigurationTest.java new file mode 100644 index 0000000000..4be2f2a34f --- /dev/null +++ b/data-prepper-plugins/translate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/translate/RegexParameterConfigurationTest.java @@ -0,0 +1,46 @@ +package org.opensearch.dataprepper.plugins.processor.translate; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.util.Collections; +import java.util.Map; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.opensearch.dataprepper.test.helper.ReflectivelySetField.setField; +import static org.hamcrest.CoreMatchers.is; + +class RegexParameterConfigurationTest { + + private RegexParameterConfiguration regexParameterConfiguration; + + @BeforeEach + void setup(){ + regexParameterConfiguration = createObjectUnderTest(); + } + + @Test + void test_get_patterns() throws NoSuchFieldException, IllegalAccessException{ + final Map patternMap = Collections.singletonMap("key1", "val1"); + setField(RegexParameterConfiguration.class, regexParameterConfiguration, "patterns", patternMap); + assertThat(regexParameterConfiguration.getPatterns(), is(patternMap)); + } + + @Test + void test_get_exact() throws NoSuchFieldException, IllegalAccessException{ + setField(RegexParameterConfiguration.class, regexParameterConfiguration, "exact", false); + assertFalse(regexParameterConfiguration.getExact()); + } + + @Test + void test_default_exact_option(){ + assertTrue(regexParameterConfiguration.getExact()); + } + + private RegexParameterConfiguration createObjectUnderTest() { + return new RegexParameterConfiguration(); + } + +} \ No newline at end of file diff --git a/data-prepper-plugins/translate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/translate/TranslateProcessorConfigTest.java b/data-prepper-plugins/translate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/translate/TranslateProcessorConfigTest.java new file mode 100644 index 0000000000..6af0f325eb --- /dev/null +++ b/data-prepper-plugins/translate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/translate/TranslateProcessorConfigTest.java @@ -0,0 +1,112 @@ +package org.opensearch.dataprepper.plugins.processor.translate; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.opensearch.dataprepper.plugins.processor.mutateevent.TargetType; + +import java.util.Collections; +import java.util.List; + +import static org.hamcrest.core.Is.is; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.opensearch.dataprepper.test.helper.ReflectivelySetField.setField; + + +class TranslateProcessorConfigTest { + private TranslateProcessorConfig translateProcessorConfig; + private RegexParameterConfiguration regexParameterConfiguration; + private TranslateProcessorConfig createObjectUnderTest() { + return new TranslateProcessorConfig(); + } + + @BeforeEach + void setup() throws NoSuchFieldException, IllegalAccessException{ + translateProcessorConfig = createObjectUnderTest(); + setField(TranslateProcessorConfig.class, translateProcessorConfig, "source", "sourceKey"); + setField(TranslateProcessorConfig.class, translateProcessorConfig, "target", "targetKey"); + } + + @Test + void test_no_map_patterns_filepath_options_present(){ + assertFalse(translateProcessorConfig.hasMappings()); + } + + @Test + void test_only_map_option_present() throws NoSuchFieldException, IllegalAccessException{ + setField(TranslateProcessorConfig.class, translateProcessorConfig, "map", Collections.singletonMap("key1", "val1")); + assertTrue(translateProcessorConfig.hasMappings()); + } + + @Test + void test_only_filepath_option_present() throws NoSuchFieldException, IllegalAccessException{ + setField(TranslateProcessorConfig.class, translateProcessorConfig, "filePath", "/path/to/file.yaml"); + assertTrue(translateProcessorConfig.hasMappings()); + } + + @Test + void test_only_patterns_option_present() throws NoSuchFieldException, IllegalAccessException{ + regexParameterConfiguration = new RegexParameterConfiguration(); + setField(RegexParameterConfiguration.class, regexParameterConfiguration, "patterns", Collections.singletonMap("patternKey1", "patternVal1")); + setField(TranslateProcessorConfig.class, translateProcessorConfig, "regexParameterConfiguration", regexParameterConfiguration); + assertTrue(translateProcessorConfig.hasMappings()); + } + + @Test + void test_no_patterns_under_regex() throws NoSuchFieldException, IllegalAccessException{ + regexParameterConfiguration = new RegexParameterConfiguration(); + setField(RegexParameterConfiguration.class, regexParameterConfiguration, "exact", true); + setField(TranslateProcessorConfig.class, translateProcessorConfig, "map", Collections.singletonMap("key1", "val1")); + setField(TranslateProcessorConfig.class, translateProcessorConfig, "regexParameterConfiguration", regexParameterConfiguration); + assertFalse(translateProcessorConfig.isPatternPresent()); + } + + @Test + void test_source_field_valid_types() throws NoSuchFieldException, IllegalAccessException{ + setField(TranslateProcessorConfig.class, translateProcessorConfig, "source", "key1"); + assertTrue(translateProcessorConfig.isSourceFieldValid()); + assertThat(translateProcessorConfig.getSource(), is("key1")); + setField(TranslateProcessorConfig.class, translateProcessorConfig, "source", List.of("key1", "key2", "key3")); + assertTrue(translateProcessorConfig.isSourceFieldValid()); + assertThat(translateProcessorConfig.getSource(), is(List.of("key1", "key2", "key3"))); + } + + @Test + void test_source_field_invalid_types() throws NoSuchFieldException, IllegalAccessException{ + setField(TranslateProcessorConfig.class, translateProcessorConfig, "source", 200); + assertFalse(translateProcessorConfig.isSourceFieldValid()); + setField(TranslateProcessorConfig.class, translateProcessorConfig, "source", false); + assertFalse(translateProcessorConfig.isSourceFieldValid()); + setField(TranslateProcessorConfig.class, translateProcessorConfig, "source", 20.1); + assertFalse(translateProcessorConfig.isSourceFieldValid()); + setField(TranslateProcessorConfig.class, translateProcessorConfig, "source", List.of("key1", 200)); + assertFalse(translateProcessorConfig.isSourceFieldValid()); + } + + @Test + void test_get_default() throws NoSuchFieldException, IllegalAccessException{ + assertNull(translateProcessorConfig.getDefaultValue()); + setField(TranslateProcessorConfig.class, translateProcessorConfig, "defaultValue", "No match"); + assertThat(translateProcessorConfig.getDefaultValue(),is("No match")); + } + + @Test + void test_get_iterate_on() throws NoSuchFieldException, IllegalAccessException{ + assertNull(translateProcessorConfig.getIterateOn()); + setField(TranslateProcessorConfig.class, translateProcessorConfig, "iterateOn", "iteratorField"); + assertThat(translateProcessorConfig.getIterateOn(),is("iteratorField")); + } + + @Test + void test_target_type_default(){ + assertThat(translateProcessorConfig.getTargetType(), is(TargetType.STRING)); + } + + @Test + void test_get_target_type() throws NoSuchFieldException, IllegalAccessException{ + setField(TranslateProcessorConfig.class, translateProcessorConfig, "targetType", TargetType.INTEGER); + assertThat(translateProcessorConfig.getTargetType(), is(TargetType.INTEGER)); + } +} \ No newline at end of file diff --git a/data-prepper-plugins/translate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/translate/TranslateProcessorTest.java b/data-prepper-plugins/translate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/translate/TranslateProcessorTest.java new file mode 100644 index 0000000000..394cb11ad8 --- /dev/null +++ b/data-prepper-plugins/translate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/translate/TranslateProcessorTest.java @@ -0,0 +1,501 @@ +package org.opensearch.dataprepper.plugins.processor.translate; + +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.expression.ExpressionEvaluator; +import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.event.JacksonEvent; +import org.opensearch.dataprepper.model.plugin.InvalidPluginConfigurationException; +import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.plugins.processor.mutateevent.TargetType; + +import java.util.AbstractMap; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.mockito.Mockito.lenient; +import static org.mockito.Mockito.when; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.CoreMatchers.is; + +@ExtendWith(MockitoExtension.class) +class TranslateProcessorTest { + + @Mock + private PluginMetrics pluginMetrics; + @Mock + private TranslateProcessorConfig mockConfig; + + @Mock + private RegexParameterConfiguration mockRegexConfig; + + @Mock + private ExpressionEvaluator expressionEvaluator; + + @BeforeEach + void setup() { + lenient().when(mockConfig.getSource()).thenReturn("sourceField"); + lenient().when(mockConfig.getTarget()).thenReturn("targetField"); + lenient().when(mockConfig.getTargetType()).thenReturn(TargetType.STRING); + lenient().when(mockRegexConfig.getExact()).thenReturn(mockRegexConfig.DEFAULT_EXACT); + } + + @Test + void test_string_keys_in_map(){ + when(mockConfig.getMap()).thenReturn(createMapEntries(createMapping("key1","mappedValue1"))); + final TranslateProcessor processor = createObjectUnderTest(); + final Record record = getEvent("key1"); + final List> translatedRecords = (List>) processor.doExecute(Collections.singletonList(record)); + + assertTrue(translatedRecords.get(0).getData().containsKey("targetField")); + assertThat(translatedRecords.get(0).getData().get("targetField", String.class), is("mappedValue1")); + } + + @Test + void test_integer_keys_in_map() { + when(mockConfig.getMap()).thenReturn(createMapEntries(createMapping("123", "mappedValue1"))); + final TranslateProcessor processor = createObjectUnderTest(); + final Record record = getEvent("123"); + final List> translatedRecords = (List>) processor.doExecute(Collections.singletonList(record)); + + assertTrue(translatedRecords.get(0).getData().containsKey("targetField")); + assertThat(translatedRecords.get(0).getData().get("targetField", String.class), is("mappedValue1")); + } + + @Test + void test_integer_range_keys_in_map() { + when(mockConfig.getMap()).thenReturn(createMapEntries(createMapping("1-10", "mappedValue1"))); + final TranslateProcessor processor = createObjectUnderTest(); + final Record record = getEvent("5"); + final List> translatedRecords = (List>) processor.doExecute(Collections.singletonList(record)); + + assertTrue(translatedRecords.get(0).getData().containsKey("targetField")); + assertThat(translatedRecords.get(0).getData().get("targetField", String.class), is("mappedValue1")); + + } + + @Test + void test_comma_separated_keys_in_map() { + when(mockConfig.getMap()).thenReturn(createMapEntries(createMapping("key1,key2, key3", "mappedValue1"))); + final TranslateProcessor processor = createObjectUnderTest(); + + for (String key : Arrays.asList("key1", "key2", "key3")) { + final Record record = getEvent(key); + final List> translatedRecords = (List>) processor.doExecute(Collections.singletonList(record)); + + assertTrue(translatedRecords.get(0).getData().containsKey("targetField")); + assertThat(translatedRecords.get(0).getData().get("targetField", String.class), is("mappedValue1")); + } + + final Record failureRecord = getEvent("key4"); + final List> failingTranslatedRecords = (List>) processor.doExecute(Collections.singletonList(failureRecord)); + + assertFalse(failingTranslatedRecords.get(0).getData().containsKey("targetField")); + } + + @Test + void test_comma_separated_range_keys_in_map() { + when(mockConfig.getMap()).thenReturn(createMapEntries(createMapping("1-10,11-20, 21-30", "mappedValue1"))); + final TranslateProcessor processor = createObjectUnderTest(); + + for (String key : Arrays.asList("5", "15", "25")) { + final Record record = getEvent(key); + final List> translatedRecords = (List>) processor.doExecute(Collections.singletonList(record)); + + assertTrue(translatedRecords.get(0).getData().containsKey("targetField")); + assertThat(translatedRecords.get(0).getData().get("targetField", String.class), is("mappedValue1")); + } + + final Record failureRecord = getEvent("35"); + final List> failingTranslatedRecords = (List>) processor.doExecute(Collections.singletonList(failureRecord)); + + assertFalse(failingTranslatedRecords.get(0).getData().containsKey("targetField")); + } + + @Test + void test_float_source() { + when(mockConfig.getMap()).thenReturn(createMapEntries(createMapping("1-10,11-20, 21-30", "mappedValue1"))); + final TranslateProcessor processor = createObjectUnderTest(); + final Record record = getEvent("11.1"); + final List> translatedRecords = (List>) processor.doExecute(Collections.singletonList(record)); + + assertTrue(translatedRecords.get(0).getData().containsKey("targetField")); + assertThat(translatedRecords.get(0).getData().get("targetField", String.class), is("mappedValue1")); + + final Record failureRecord = getEvent("20.5"); + final List> failingTranslatedRecords = (List>) processor.doExecute(Collections.singletonList(failureRecord)); + + assertFalse(failingTranslatedRecords.get(0).getData().containsKey("targetField")); + } + + @Test + void test_comma_separated_integer_ranges_and_string_keys() { + when(mockConfig.getMap()).thenReturn(createMapEntries(createMapping("1-10,key1", "mappedValue1"))); + final TranslateProcessor processor = createObjectUnderTest(); + final Record record = getEvent("5.2"); + final List> translatedRecords = (List>) processor.doExecute(Collections.singletonList(record)); + + assertTrue(translatedRecords.get(0).getData().containsKey("targetField")); + assertThat(translatedRecords.get(0).getData().get("targetField", String.class), is("mappedValue1")); + + final Record recordStringKey = getEvent("key1"); + final List> translatedStringKeyRecords = (List>) processor.doExecute(Collections.singletonList(recordStringKey)); + + assertTrue(translatedStringKeyRecords.get(0).getData().containsKey("targetField")); + assertThat(translatedStringKeyRecords.get(0).getData().get("targetField", String.class), is("mappedValue1")); + } + + @Test + void test_multiple_dashes_in_keys_should_be_treated_as_string_literal() { + when(mockConfig.getMap()).thenReturn(createMapEntries(createMapping("1-10-20", "mappedValue1"))); + final TranslateProcessor processor = createObjectUnderTest(); + final Record failureRecord = getEvent("1-10-20"); + final List> failingTranslatedRecords = (List>) processor.doExecute(Collections.singletonList(failureRecord)); + + assertTrue(failingTranslatedRecords.get(0).getData().containsKey("targetField")); + + final Record record = getEvent("10"); + final List> translatedRecords = (List>) processor.doExecute(Collections.singletonList(record)); + + assertFalse(translatedRecords.get(0).getData().containsKey("targetField")); + + } + + @Test + void test_overlapping_ranges_should_fail_when_overlapping() { + when(mockConfig.getMap()).thenReturn(createMapEntries(createMapping("1-10", "mappedValue1"), createMapping("10-20", "mappedValue2"))); + + assertThrows(InvalidPluginConfigurationException.class, () -> createObjectUnderTest()); + } + + @Test + void test_overlapping_key_and_range_in_map_option() { + when(mockConfig.getMap()).thenReturn(createMapEntries(createMapping("1-10", "mappedValue1"), createMapping("5.3", "mappedValue2"))); + + assertThrows(InvalidPluginConfigurationException.class, () -> createObjectUnderTest()); + } + + @Test + void test_string_literal_in_pattern_option() { + when(mockConfig.getRegexParameterConfiguration()).thenReturn(mockRegexConfig); + when(mockRegexConfig.getPatterns()).thenReturn(createMapEntries(createMapping("key1", "mappedValue1"))); + + final TranslateProcessor processor = createObjectUnderTest(); + final Record record = getEvent("key1"); + final List> translatedRecords = (List>) processor.doExecute(Collections.singletonList(record)); + + assertTrue(translatedRecords.get(0).getData().containsKey("targetField")); + assertThat(translatedRecords.get(0).getData().get("targetField", String.class), is("mappedValue1")); + + final Record failureRecord = getEvent("key2"); + final List> failingTranslatedRecords = (List>) processor.doExecute(Collections.singletonList(failureRecord)); + + assertFalse(failingTranslatedRecords.get(0).getData().containsKey("targetField")); + } + + @Test + void test_matching_of_regex_pattern_in_pattern_option() { + when(mockConfig.getRegexParameterConfiguration()).thenReturn(mockRegexConfig); + when(mockRegexConfig.getPatterns()).thenReturn(createMapEntries(createMapping("^(1[0-9]|20)$", "patternValue1"))); //Range between 10-20 + final TranslateProcessor processor = createObjectUnderTest(); + final Record record = getEvent("15"); + final List> translatedRecords = (List>) processor.doExecute(Collections.singletonList(record)); + + assertTrue(translatedRecords.get(0).getData().containsKey("targetField")); + assertThat(translatedRecords.get(0).getData().get("targetField", String.class), is("patternValue1")); + + final Record failureRecord = getEvent("1"); + final List> failingTranslatedRecords = (List>) processor.doExecute(Collections.singletonList(failureRecord)); + + assertFalse(failingTranslatedRecords.get(0).getData().containsKey("targetField")); + } + + @Test + void test_pattern_matching_when_no_match_in_map() { + when(mockConfig.getRegexParameterConfiguration()).thenReturn(mockRegexConfig); + when(mockConfig.getMap()).thenReturn((createMapEntries(createMapping("key1", "mappedValue1"), createMapping("key2", "mappedValue2")))); + when(mockRegexConfig.getPatterns()).thenReturn(createMapEntries(createMapping("patternKey1", "patternValue1"))); + + final TranslateProcessor processor = createObjectUnderTest(); + final Record record = getEvent("patternKey1"); + final List> translatedRecords = (List>) processor.doExecute(Collections.singletonList(record)); + + assertTrue(translatedRecords.get(0).getData().containsKey("targetField")); + assertThat(translatedRecords.get(0).getData().get("targetField", String.class), is("patternValue1")); + + final Record recordMapKey = getEvent("key1"); + final List> translatedMapKeyRecords = (List>) processor.doExecute(Collections.singletonList(recordMapKey)); + + assertTrue(translatedMapKeyRecords.get(0).getData().containsKey("targetField")); + assertThat(translatedMapKeyRecords.get(0).getData().get("targetField", String.class), is("mappedValue1")); + } + + @Test + void test_map_matching_when_overlapping_ranges_in_map_and_pattern() { + when(mockConfig.getRegexParameterConfiguration()).thenReturn(mockRegexConfig); + when(mockConfig.getMap()).thenReturn((createMapEntries(createMapping("400", "mappedValue1")))); + when(mockRegexConfig.getPatterns()).thenReturn(createMapEntries(createMapping("^(400|404)$", "patternValue1"))); // Matches 400 or 404 + + final TranslateProcessor processor = createObjectUnderTest(); + final Record record = getEvent("400"); + final List> translatedRecords = (List>) processor.doExecute(Collections.singletonList(record)); + + assertTrue(translatedRecords.get(0).getData().containsKey("targetField")); + assertThat(translatedRecords.get(0).getData().get("targetField", String.class), is("mappedValue1")); + + final Record recordPatternKey = getEvent("404"); + final List> translatedPatternKeyRecords = (List>) processor.doExecute(Collections.singletonList(recordPatternKey)); + + assertTrue(translatedPatternKeyRecords.get(0).getData().containsKey("targetField")); + assertThat(translatedPatternKeyRecords.get(0).getData().get("targetField", String.class), is("patternValue1")); + } + + @Test + void test_source_array_single_key() { + when(mockConfig.getSource()).thenReturn(new ArrayList(List.of("sourceField"))); + when(mockConfig.getMap()).thenReturn((createMapEntries(createMapping("400", "mappedValue1")))); + + final TranslateProcessor processor = createObjectUnderTest(); + final Record record = getEvent("400"); + final List> translatedRecords = (List>) processor.doExecute(Collections.singletonList(record)); + + assertTrue(translatedRecords.get(0).getData().containsKey("targetField")); + assertThat(translatedRecords.get(0).getData().get("targetField", ArrayList.class), is(new ArrayList(List.of("mappedValue1")))); + } + + @Test + void test_source_array_multiple_keys() { + when(mockConfig.getSource()).thenReturn(new ArrayList(List.of("sourceField1", "sourceField2"))); + when(mockConfig.getMap()).thenReturn((createMapEntries(createMapping("key1", "mappedValue1"), createMapping("key2", "mappedValue2"), createMapping("key3", "mappedValue3")))); + + final TranslateProcessor processor = createObjectUnderTest(); + final Record record = buildRecordWithEvent(Map.of("sourceField1", "key1", "sourceField2", "key3")); + final List> translatedRecords = (List>) processor.doExecute(Collections.singletonList(record)); + + assertTrue(translatedRecords.get(0).getData().containsKey("targetField")); + assertThat(translatedRecords.get(0).getData().get("targetField", ArrayList.class), is(new ArrayList(List.of("mappedValue1", "mappedValue3")))); + } + + @Test + void test_source_array_with_partial_match_without_default() { + when(mockConfig.getSource()).thenReturn(new ArrayList(List.of("sourceField1", "sourceField2"))); + when(mockConfig.getMap()).thenReturn((createMapEntries(createMapping("key1", "mappedValue1"), createMapping("key2", "mappedValue2"), createMapping("key3", "mappedValue3")))); + + final TranslateProcessor processor = createObjectUnderTest(); + final Record record = buildRecordWithEvent(Map.of("sourceField1", "key1", "sourceField2", "key4")); + final List> translatedRecords = (List>) processor.doExecute(Collections.singletonList(record)); + + assertTrue(translatedRecords.get(0).getData().containsKey("targetField")); + assertThat(translatedRecords.get(0).getData().get("targetField", ArrayList.class), is(new ArrayList(List.of("mappedValue1")))); + } + + @Test + void test_source_array_with_partial_match_with_default() { + final String defaultValue = "No Match Found"; + when(mockConfig.getSource()).thenReturn(new ArrayList(List.of("sourceField1", "sourceField2"))); + when(mockConfig.getDefaultValue()).thenReturn(defaultValue); + when(mockConfig.getMap()).thenReturn((createMapEntries(createMapping("key1", "mappedValue1"), createMapping("key2", "mappedValue2"), createMapping("key3", "mappedValue3")))); + + final TranslateProcessor processor = createObjectUnderTest(); + final Record record = buildRecordWithEvent(Map.of("sourceField1", "key1", "sourceField2", "key4")); + final List> translatedRecords = (List>) processor.doExecute(Collections.singletonList(record)); + + assertTrue(translatedRecords.get(0).getData().containsKey("targetField")); + assertThat(translatedRecords.get(0).getData().get("targetField", ArrayList.class), is(new ArrayList(List.of("mappedValue1", defaultValue)))); + } + + @Test + void test_non_exact_matching() { + when(mockConfig.getRegexParameterConfiguration()).thenReturn(mockRegexConfig); + when(mockRegexConfig.getPatterns()).thenReturn(createMapEntries( + createMapping("^(1[0-9]|20)$", "patternValue1"), + createMapping("foo", "bar2"))); + when(mockRegexConfig.getExact()).thenReturn(false); + + final TranslateProcessor processor = createObjectUnderTest(); + final Record record = getEvent("footer"); + final List> translatedRecords = (List>) processor.doExecute(Collections.singletonList(record)); + + assertTrue(translatedRecords.get(0).getData().containsKey("targetField")); + assertThat(translatedRecords.get(0).getData().get("targetField", String.class), is("bar2")); + + final Record regexRecord = getEvent("15"); + final List> translatedRegexRecords = (List>) processor.doExecute(Collections.singletonList(regexRecord)); + + assertTrue(translatedRegexRecords.get(0).getData().containsKey("targetField")); + assertThat(translatedRegexRecords.get(0).getData().get("targetField", String.class), is("patternValue1")); + + final Record negativeRecord = getEvent("fo"); + final List> translatedNegativeRecords = (List>) processor.doExecute(Collections.singletonList(negativeRecord)); + + assertFalse(translatedNegativeRecords.get(0).getData().containsKey("targetField")); + } + + @Test + void test_nested_records_with_default_value() { + final Map testJson = Map.of("collection", List.of( + Map.of("sourceField", "key1"), + Map.of("sourceField", "key2"), + Map.of("sourceField", "key3"))); + final List> outputJson = List.of( + Map.of("sourceField", "key1", "targetField", "mappedValue1"), + Map.of("sourceField", "key2", "targetField", "mappedValue2"), + Map.of("sourceField", "key3", "targetField", "No Match")); + + when(mockConfig.getRegexParameterConfiguration()).thenReturn(mockRegexConfig); + when(mockRegexConfig.getPatterns()).thenReturn(createMapEntries( + createMapping("key1", "mappedValue1"), + createMapping("key2", "mappedValue2"))); + when(mockConfig.getDefaultValue()).thenReturn("No Match"); + when(mockConfig.getIterateOn()).thenReturn("collection"); + + final TranslateProcessor processor = createObjectUnderTest(); + final Record record = buildRecordWithEvent(testJson); + final List> translatedRecords = (List>) processor.doExecute(Collections.singletonList(record)); + + assertThat(translatedRecords.get(0).getData().get("collection", ArrayList.class), is(outputJson)); + } + + @Test + void test_nested_records_without_default_value() { + final Map testJson = Map.of("collection", List.of( + Map.of("sourceField", "key1"), + Map.of("sourceField", "key2"), + Map.of("sourceField", "key3"))); + final List> outputJson = List.of( + Map.of("sourceField", "key1", "targetField", "mappedValue1"), + Map.of("sourceField", "key2", "targetField", "mappedValue2"), + Map.of("sourceField", "key3")); + + when(mockConfig.getRegexParameterConfiguration()).thenReturn(mockRegexConfig); + when(mockRegexConfig.getPatterns()).thenReturn(createMapEntries( + createMapping("key1", "mappedValue1"), + createMapping("key2", "mappedValue2"))); + when(mockConfig.getIterateOn()).thenReturn("collection"); + + final TranslateProcessor processor = createObjectUnderTest(); + final Record record = buildRecordWithEvent(testJson); + final List> translatedRecords = (List>) processor.doExecute(Collections.singletonList(record)); + + assertThat(translatedRecords.get(0).getData().get("collection", ArrayList.class), is(outputJson)); + } + + @Test + void test_nested_records_no_match() { + final Map testJson = Map.of("collection", List.of( + Map.of("sourceField", "key1"), + Map.of("sourceField", "key2"), + Map.of("sourceField", "key3"))); + final List> outputJson = List.of( + Map.of("sourceField", "key1"), + Map.of("sourceField", "key2"), + Map.of("sourceField", "key3")); + + when(mockConfig.getRegexParameterConfiguration()).thenReturn(mockRegexConfig); + when(mockRegexConfig.getPatterns()).thenReturn(createMapEntries(createMapping("key4", "mappedValue1"))); + when(mockConfig.getIterateOn()).thenReturn("collection"); + + final TranslateProcessor processor = createObjectUnderTest(); + final Record record = buildRecordWithEvent(testJson); + final List> translatedRecords = (List>) processor.doExecute(Collections.singletonList(record)); + + assertThat(translatedRecords.get(0).getData().get("collection", ArrayList.class), is(outputJson)); + } + + @Test + void test_target_type_default(){ + when(mockConfig.getMap()).thenReturn(createMapEntries(createMapping("key1", "200"))); + final TranslateProcessor processor = createObjectUnderTest(); + final Record record = getEvent("key1"); + final List> translatedRecords = (List>) processor.doExecute(Collections.singletonList(record)); + + assertTrue(translatedRecords.get(0).getData().containsKey("targetField")); + assertThat(translatedRecords.get(0).getData().get("targetField", String.class), is("200")); + } + + @Test + void test_target_type_integer(){ + when(mockConfig.getMap()).thenReturn(createMapEntries(createMapping("key1", "200"))); + when(mockConfig.getTargetType()).thenReturn(TargetType.INTEGER); + final TranslateProcessor processor = createObjectUnderTest(); + final Record record = getEvent("key1"); + final List> translatedRecords = (List>) processor.doExecute(Collections.singletonList(record)); + + assertTrue(translatedRecords.get(0).getData().containsKey("targetField")); + assertThat(translatedRecords.get(0).getData().get("targetField", Integer.class), is(200)); + } + + @Test + void test_target_type_boolean(){ + when(mockConfig.getMap()).thenReturn(createMapEntries(createMapping("key1", "false"))); + when(mockConfig.getTargetType()).thenReturn(TargetType.BOOLEAN); + final TranslateProcessor processor = createObjectUnderTest(); + final Record record = getEvent("key1"); + final List> translatedRecords = (List>) processor.doExecute(Collections.singletonList(record)); + + assertTrue(translatedRecords.get(0).getData().containsKey("targetField")); + assertThat(translatedRecords.get(0).getData().get("targetField", Boolean.class), is(false)); + } + + @Test + void test_target_type_double(){ + when(mockConfig.getMap()).thenReturn(createMapEntries(createMapping("key1", "20.3"))); + when(mockConfig.getTargetType()).thenReturn(TargetType.DOUBLE); + final TranslateProcessor processor = createObjectUnderTest(); + final Record record = getEvent("key1"); + final List> translatedRecords = (List>) processor.doExecute(Collections.singletonList(record)); + + assertTrue(translatedRecords.get(0).getData().containsKey("targetField")); + assertThat(translatedRecords.get(0).getData().get("targetField", Double.class), is(20.3)); + } + + + private TranslateProcessor createObjectUnderTest() { + return new TranslateProcessor(pluginMetrics, mockConfig, expressionEvaluator); + } + + private Record sourceAndTargetFields(Object sourceValue, Object targetValue) { + final Map testData = new HashMap<>(); + testData.put("sourceField", sourceValue); + testData.put("targetField", targetValue); + return buildRecordWithEvent(testData); + } + + private Record getEvent(Object sourceField) { + final Map testData = new HashMap<>(); + testData.put("sourceField", sourceField); + return buildRecordWithEvent(testData); + } + + private static Record buildRecordWithEvent(final Map data) { + return new Record<>(JacksonEvent.builder() + .withData(data) + .withEventType("event") + .build()); + } + + private Map.Entry createMapping(String key, String value) { + return new AbstractMap.SimpleEntry<>(key, value); + } + + private Map createMapEntries(Map.Entry... mappings) { + final Map finalMap = new HashMap<>(); + for (Map.Entry mapping : mappings) { + finalMap.put(mapping.getKey(), mapping.getValue()); + } + + return finalMap; + } +} \ No newline at end of file diff --git a/docs/simple_pipelines.md b/docs/simple_pipelines.md index b05953e575..0f6aea94aa 100644 --- a/docs/simple_pipelines.md +++ b/docs/simple_pipelines.md @@ -40,7 +40,7 @@ built from source, you will need to make some modifications to the example comma For Data Prepper 2.0 or above, use this command: ``` -docker run --name data-prepper -p 4900:4900 -v ${PWD}/pipelines.yaml:/usr/share/data-prepper/pipelines/pipelines.yaml -v ${PWD} /data-prepper-config.yaml:/usr/share/data-prepper/config/data-prepper-config.yaml opensearchproject/data-prepper:latest +docker run --name data-prepper -p 4900:4900 -v ${PWD}/pipelines.yaml:/usr/share/data-prepper/pipelines/pipelines.yaml -v ${PWD}/data-prepper-config.yaml:/usr/share/data-prepper/config/data-prepper-config.yaml opensearchproject/data-prepper:latest ``` For Data Prepper before version 2.0, use this command: diff --git a/release/staging-resources-cdk/package-lock.json b/release/staging-resources-cdk/package-lock.json index a6fdae5930..8b77d7582f 100644 --- a/release/staging-resources-cdk/package-lock.json +++ b/release/staging-resources-cdk/package-lock.json @@ -8,7 +8,7 @@ "name": "staging-resources-cdk", "version": "0.1.0", "dependencies": { - "aws-cdk-lib": "2.13.0", + "aws-cdk-lib": "2.80.0", "constructs": "^10.0.0", "source-map-support": "^0.5.16" }, @@ -41,43 +41,58 @@ "node": ">=6.0.0" } }, + "node_modules/@aws-cdk/asset-awscli-v1": { + "version": "2.2.199", + "resolved": "https://registry.npmjs.org/@aws-cdk/asset-awscli-v1/-/asset-awscli-v1-2.2.199.tgz", + "integrity": "sha512-zNdD2OxALdsdQaRZBpTfMTuudxV+4jLMznJIvVj6O+OqCru4m5UtgVQmyApW1z2H9s4/06ovVt20aXe2G8Ta+w==" + }, + "node_modules/@aws-cdk/asset-kubectl-v20": { + "version": "2.1.2", + "resolved": "https://registry.npmjs.org/@aws-cdk/asset-kubectl-v20/-/asset-kubectl-v20-2.1.2.tgz", + "integrity": "sha512-3M2tELJOxQv0apCIiuKQ4pAbncz9GuLwnKFqxifWfe77wuMxyTRPmxssYHs42ePqzap1LT6GDcPygGs+hHstLg==" + }, + "node_modules/@aws-cdk/asset-node-proxy-agent-v5": { + "version": "2.0.165", + "resolved": "https://registry.npmjs.org/@aws-cdk/asset-node-proxy-agent-v5/-/asset-node-proxy-agent-v5-2.0.165.tgz", + "integrity": "sha512-bsyLQD/vqXQcc9RDmlM1XqiFNO/yewgVFXmkMcQkndJbmE/jgYkzewwYGrBlfL725hGLQipXq19+jwWwdsXQqg==" + }, "node_modules/@babel/code-frame": { - "version": "7.18.6", - "resolved": "https://registry.npmjs.org/@babel/code-frame/-/code-frame-7.18.6.tgz", - "integrity": "sha512-TDCmlK5eOvH+eH7cdAFlNXeVJqWIQ7gW9tY1GJIpUtFb6CmjVyq2VM3u71bOyR8CRihcCgMUYoDNyLXao3+70Q==", + "version": "7.22.5", + "resolved": "https://registry.npmjs.org/@babel/code-frame/-/code-frame-7.22.5.tgz", + "integrity": "sha512-Xmwn266vad+6DAqEB2A6V/CcZVp62BbwVmcOJc2RPuwih1kw02TjQvWVWlcKGbBPd+8/0V5DEkOcizRGYsspYQ==", "dev": true, "dependencies": { - "@babel/highlight": "^7.18.6" + "@babel/highlight": "^7.22.5" }, "engines": { "node": ">=6.9.0" } }, "node_modules/@babel/compat-data": { - "version": "7.21.0", - "resolved": "https://registry.npmjs.org/@babel/compat-data/-/compat-data-7.21.0.tgz", - "integrity": "sha512-gMuZsmsgxk/ENC3O/fRw5QY8A9/uxQbbCEypnLIiYYc/qVJtEV7ouxC3EllIIwNzMqAQee5tanFabWsUOutS7g==", + "version": "7.22.5", + "resolved": "https://registry.npmjs.org/@babel/compat-data/-/compat-data-7.22.5.tgz", + "integrity": "sha512-4Jc/YuIaYqKnDDz892kPIledykKg12Aw1PYX5i/TY28anJtacvM1Rrr8wbieB9GfEJwlzqT0hUEao0CxEebiDA==", "dev": true, "engines": { "node": ">=6.9.0" } }, "node_modules/@babel/core": { - "version": "7.21.0", - "resolved": "https://registry.npmjs.org/@babel/core/-/core-7.21.0.tgz", - "integrity": "sha512-PuxUbxcW6ZYe656yL3EAhpy7qXKq0DmYsrJLpbB8XrsCP9Nm+XCg9XFMb5vIDliPD7+U/+M+QJlH17XOcB7eXA==", + "version": "7.22.5", + "resolved": "https://registry.npmjs.org/@babel/core/-/core-7.22.5.tgz", + "integrity": "sha512-SBuTAjg91A3eKOvD+bPEz3LlhHZRNu1nFOVts9lzDJTXshHTjII0BAtDS3Y2DAkdZdDKWVZGVwkDfc4Clxn1dg==", "dev": true, "dependencies": { "@ampproject/remapping": "^2.2.0", - "@babel/code-frame": "^7.18.6", - "@babel/generator": "^7.21.0", - "@babel/helper-compilation-targets": "^7.20.7", - "@babel/helper-module-transforms": "^7.21.0", - "@babel/helpers": "^7.21.0", - "@babel/parser": "^7.21.0", - "@babel/template": "^7.20.7", - "@babel/traverse": "^7.21.0", - "@babel/types": "^7.21.0", + "@babel/code-frame": "^7.22.5", + "@babel/generator": "^7.22.5", + "@babel/helper-compilation-targets": "^7.22.5", + "@babel/helper-module-transforms": "^7.22.5", + "@babel/helpers": "^7.22.5", + "@babel/parser": "^7.22.5", + "@babel/template": "^7.22.5", + "@babel/traverse": "^7.22.5", + "@babel/types": "^7.22.5", "convert-source-map": "^1.7.0", "debug": "^4.1.0", "gensync": "^1.0.0-beta.2", @@ -102,12 +117,12 @@ } }, "node_modules/@babel/generator": { - "version": "7.21.1", - "resolved": "https://registry.npmjs.org/@babel/generator/-/generator-7.21.1.tgz", - "integrity": "sha512-1lT45bAYlQhFn/BHivJs43AiW2rg3/UbLyShGfF3C0KmHvO5fSghWd5kBJy30kpRRucGzXStvnnCFniCR2kXAA==", + "version": "7.22.5", + "resolved": "https://registry.npmjs.org/@babel/generator/-/generator-7.22.5.tgz", + "integrity": "sha512-+lcUbnTRhd0jOewtFSedLyiPsD5tswKkbgcezOqqWFUVNEwoUTlpPOBmvhG7OXWLR4jMdv0czPGH5XbflnD1EA==", "dev": true, "dependencies": { - "@babel/types": "^7.21.0", + "@babel/types": "^7.22.5", "@jridgewell/gen-mapping": "^0.3.2", "@jridgewell/trace-mapping": "^0.3.17", "jsesc": "^2.5.1" @@ -131,13 +146,13 @@ } }, "node_modules/@babel/helper-compilation-targets": { - "version": "7.20.7", - "resolved": "https://registry.npmjs.org/@babel/helper-compilation-targets/-/helper-compilation-targets-7.20.7.tgz", - "integrity": "sha512-4tGORmfQcrc+bvrjb5y3dG9Mx1IOZjsHqQVUz7XCNHO+iTmqxWnVg3KRygjGmpRLJGdQSKuvFinbIb0CnZwHAQ==", + "version": "7.22.5", + "resolved": "https://registry.npmjs.org/@babel/helper-compilation-targets/-/helper-compilation-targets-7.22.5.tgz", + "integrity": "sha512-Ji+ywpHeuqxB8WDxraCiqR0xfhYjiDE/e6k7FuIaANnoOFxAHskHChz4vA1mJC9Lbm01s1PVAGhQY4FUKSkGZw==", "dev": true, "dependencies": { - "@babel/compat-data": "^7.20.5", - "@babel/helper-validator-option": "^7.18.6", + "@babel/compat-data": "^7.22.5", + "@babel/helper-validator-option": "^7.22.5", "browserslist": "^4.21.3", "lru-cache": "^5.1.1", "semver": "^6.3.0" @@ -159,65 +174,65 @@ } }, "node_modules/@babel/helper-environment-visitor": { - "version": "7.18.9", - "resolved": "https://registry.npmjs.org/@babel/helper-environment-visitor/-/helper-environment-visitor-7.18.9.tgz", - "integrity": "sha512-3r/aACDJ3fhQ/EVgFy0hpj8oHyHpQc+LPtJoY9SzTThAsStm4Ptegq92vqKoE3vD706ZVFWITnMnxucw+S9Ipg==", + "version": "7.22.5", + "resolved": "https://registry.npmjs.org/@babel/helper-environment-visitor/-/helper-environment-visitor-7.22.5.tgz", + "integrity": "sha512-XGmhECfVA/5sAt+H+xpSg0mfrHq6FzNr9Oxh7PSEBBRUb/mL7Kz3NICXb194rCqAEdxkhPT1a88teizAFyvk8Q==", "dev": true, "engines": { "node": ">=6.9.0" } }, "node_modules/@babel/helper-function-name": { - "version": "7.21.0", - "resolved": "https://registry.npmjs.org/@babel/helper-function-name/-/helper-function-name-7.21.0.tgz", - "integrity": "sha512-HfK1aMRanKHpxemaY2gqBmL04iAPOPRj7DxtNbiDOrJK+gdwkiNRVpCpUJYbUT+aZyemKN8brqTOxzCaG6ExRg==", + "version": "7.22.5", + "resolved": "https://registry.npmjs.org/@babel/helper-function-name/-/helper-function-name-7.22.5.tgz", + "integrity": "sha512-wtHSq6jMRE3uF2otvfuD3DIvVhOsSNshQl0Qrd7qC9oQJzHvOL4qQXlQn2916+CXGywIjpGuIkoyZRRxHPiNQQ==", "dev": true, "dependencies": { - "@babel/template": "^7.20.7", - "@babel/types": "^7.21.0" + "@babel/template": "^7.22.5", + "@babel/types": "^7.22.5" }, "engines": { "node": ">=6.9.0" } }, "node_modules/@babel/helper-hoist-variables": { - "version": "7.18.6", - "resolved": "https://registry.npmjs.org/@babel/helper-hoist-variables/-/helper-hoist-variables-7.18.6.tgz", - "integrity": "sha512-UlJQPkFqFULIcyW5sbzgbkxn2FKRgwWiRexcuaR8RNJRy8+LLveqPjwZV/bwrLZCN0eUHD/x8D0heK1ozuoo6Q==", + "version": "7.22.5", + "resolved": "https://registry.npmjs.org/@babel/helper-hoist-variables/-/helper-hoist-variables-7.22.5.tgz", + "integrity": "sha512-wGjk9QZVzvknA6yKIUURb8zY3grXCcOZt+/7Wcy8O2uctxhplmUPkOdlgoNhmdVee2c92JXbf1xpMtVNbfoxRw==", "dev": true, "dependencies": { - "@babel/types": "^7.18.6" + "@babel/types": "^7.22.5" }, "engines": { "node": ">=6.9.0" } }, "node_modules/@babel/helper-module-imports": { - "version": "7.18.6", - "resolved": "https://registry.npmjs.org/@babel/helper-module-imports/-/helper-module-imports-7.18.6.tgz", - "integrity": "sha512-0NFvs3VkuSYbFi1x2Vd6tKrywq+z/cLeYC/RJNFrIX/30Bf5aiGYbtvGXolEktzJH8o5E5KJ3tT+nkxuuZFVlA==", + "version": "7.22.5", + "resolved": "https://registry.npmjs.org/@babel/helper-module-imports/-/helper-module-imports-7.22.5.tgz", + "integrity": "sha512-8Dl6+HD/cKifutF5qGd/8ZJi84QeAKh+CEe1sBzz8UayBBGg1dAIJrdHOcOM5b2MpzWL2yuotJTtGjETq0qjXg==", "dev": true, "dependencies": { - "@babel/types": "^7.18.6" + "@babel/types": "^7.22.5" }, "engines": { "node": ">=6.9.0" } }, "node_modules/@babel/helper-module-transforms": { - "version": "7.21.2", - "resolved": "https://registry.npmjs.org/@babel/helper-module-transforms/-/helper-module-transforms-7.21.2.tgz", - "integrity": "sha512-79yj2AR4U/Oqq/WOV7Lx6hUjau1Zfo4cI+JLAVYeMV5XIlbOhmjEk5ulbTc9fMpmlojzZHkUUxAiK+UKn+hNQQ==", + "version": "7.22.5", + "resolved": "https://registry.npmjs.org/@babel/helper-module-transforms/-/helper-module-transforms-7.22.5.tgz", + "integrity": "sha512-+hGKDt/Ze8GFExiVHno/2dvG5IdstpzCq0y4Qc9OJ25D4q3pKfiIP/4Vp3/JvhDkLKsDK2api3q3fpIgiIF5bw==", "dev": true, "dependencies": { - "@babel/helper-environment-visitor": "^7.18.9", - "@babel/helper-module-imports": "^7.18.6", - "@babel/helper-simple-access": "^7.20.2", - "@babel/helper-split-export-declaration": "^7.18.6", - "@babel/helper-validator-identifier": "^7.19.1", - "@babel/template": "^7.20.7", - "@babel/traverse": "^7.21.2", - "@babel/types": "^7.21.2" + "@babel/helper-environment-visitor": "^7.22.5", + "@babel/helper-module-imports": "^7.22.5", + "@babel/helper-simple-access": "^7.22.5", + "@babel/helper-split-export-declaration": "^7.22.5", + "@babel/helper-validator-identifier": "^7.22.5", + "@babel/template": "^7.22.5", + "@babel/traverse": "^7.22.5", + "@babel/types": "^7.22.5" }, "engines": { "node": ">=6.9.0" @@ -233,77 +248,77 @@ } }, "node_modules/@babel/helper-simple-access": { - "version": "7.20.2", - "resolved": "https://registry.npmjs.org/@babel/helper-simple-access/-/helper-simple-access-7.20.2.tgz", - "integrity": "sha512-+0woI/WPq59IrqDYbVGfshjT5Dmk/nnbdpcF8SnMhhXObpTq2KNBdLFRFrkVdbDOyUmHBCxzm5FHV1rACIkIbA==", + "version": "7.22.5", + "resolved": "https://registry.npmjs.org/@babel/helper-simple-access/-/helper-simple-access-7.22.5.tgz", + "integrity": "sha512-n0H99E/K+Bika3++WNL17POvo4rKWZ7lZEp1Q+fStVbUi8nxPQEBOlTmCOxW/0JsS56SKKQ+ojAe2pHKJHN35w==", "dev": true, "dependencies": { - "@babel/types": "^7.20.2" + "@babel/types": "^7.22.5" }, "engines": { "node": ">=6.9.0" } }, "node_modules/@babel/helper-split-export-declaration": { - "version": "7.18.6", - "resolved": "https://registry.npmjs.org/@babel/helper-split-export-declaration/-/helper-split-export-declaration-7.18.6.tgz", - "integrity": "sha512-bde1etTx6ZyTmobl9LLMMQsaizFVZrquTEHOqKeQESMKo4PlObf+8+JA25ZsIpZhT/WEd39+vOdLXAFG/nELpA==", + "version": "7.22.5", + "resolved": "https://registry.npmjs.org/@babel/helper-split-export-declaration/-/helper-split-export-declaration-7.22.5.tgz", + "integrity": "sha512-thqK5QFghPKWLhAV321lxF95yCg2K3Ob5yw+M3VHWfdia0IkPXUtoLH8x/6Fh486QUvzhb8YOWHChTVen2/PoQ==", "dev": true, "dependencies": { - "@babel/types": "^7.18.6" + "@babel/types": "^7.22.5" }, "engines": { "node": ">=6.9.0" } }, "node_modules/@babel/helper-string-parser": { - "version": "7.19.4", - "resolved": "https://registry.npmjs.org/@babel/helper-string-parser/-/helper-string-parser-7.19.4.tgz", - "integrity": "sha512-nHtDoQcuqFmwYNYPz3Rah5ph2p8PFeFCsZk9A/48dPc/rGocJ5J3hAAZ7pb76VWX3fZKu+uEr/FhH5jLx7umrw==", + "version": "7.22.5", + "resolved": "https://registry.npmjs.org/@babel/helper-string-parser/-/helper-string-parser-7.22.5.tgz", + "integrity": "sha512-mM4COjgZox8U+JcXQwPijIZLElkgEpO5rsERVDJTc2qfCDfERyob6k5WegS14SX18IIjv+XD+GrqNumY5JRCDw==", "dev": true, "engines": { "node": ">=6.9.0" } }, "node_modules/@babel/helper-validator-identifier": { - "version": "7.19.1", - "resolved": "https://registry.npmjs.org/@babel/helper-validator-identifier/-/helper-validator-identifier-7.19.1.tgz", - "integrity": "sha512-awrNfaMtnHUr653GgGEs++LlAvW6w+DcPrOliSMXWCKo597CwL5Acf/wWdNkf/tfEQE3mjkeD1YOVZOUV/od1w==", + "version": "7.22.5", + "resolved": "https://registry.npmjs.org/@babel/helper-validator-identifier/-/helper-validator-identifier-7.22.5.tgz", + "integrity": "sha512-aJXu+6lErq8ltp+JhkJUfk1MTGyuA4v7f3pA+BJ5HLfNC6nAQ0Cpi9uOquUj8Hehg0aUiHzWQbOVJGao6ztBAQ==", "dev": true, "engines": { "node": ">=6.9.0" } }, "node_modules/@babel/helper-validator-option": { - "version": "7.21.0", - "resolved": "https://registry.npmjs.org/@babel/helper-validator-option/-/helper-validator-option-7.21.0.tgz", - "integrity": "sha512-rmL/B8/f0mKS2baE9ZpyTcTavvEuWhTTW8amjzXNvYG4AwBsqTLikfXsEofsJEfKHf+HQVQbFOHy6o+4cnC/fQ==", + "version": "7.22.5", + "resolved": "https://registry.npmjs.org/@babel/helper-validator-option/-/helper-validator-option-7.22.5.tgz", + "integrity": "sha512-R3oB6xlIVKUnxNUxbmgq7pKjxpru24zlimpE8WK47fACIlM0II/Hm1RS8IaOI7NgCr6LNS+jl5l75m20npAziw==", "dev": true, "engines": { "node": ">=6.9.0" } }, "node_modules/@babel/helpers": { - "version": "7.21.0", - "resolved": "https://registry.npmjs.org/@babel/helpers/-/helpers-7.21.0.tgz", - "integrity": "sha512-XXve0CBtOW0pd7MRzzmoyuSj0e3SEzj8pgyFxnTT1NJZL38BD1MK7yYrm8yefRPIDvNNe14xR4FdbHwpInD4rA==", + "version": "7.22.5", + "resolved": "https://registry.npmjs.org/@babel/helpers/-/helpers-7.22.5.tgz", + "integrity": "sha512-pSXRmfE1vzcUIDFQcSGA5Mr+GxBV9oiRKDuDxXvWQQBCh8HoIjs/2DlDB7H8smac1IVrB9/xdXj2N3Wol9Cr+Q==", "dev": true, "dependencies": { - "@babel/template": "^7.20.7", - "@babel/traverse": "^7.21.0", - "@babel/types": "^7.21.0" + "@babel/template": "^7.22.5", + "@babel/traverse": "^7.22.5", + "@babel/types": "^7.22.5" }, "engines": { "node": ">=6.9.0" } }, "node_modules/@babel/highlight": { - "version": "7.18.6", - "resolved": "https://registry.npmjs.org/@babel/highlight/-/highlight-7.18.6.tgz", - "integrity": "sha512-u7stbOuYjaPezCuLj29hNW1v64M2Md2qupEKP1fHc7WdOA3DgLh37suiSrZYY7haUB7iBeQZ9P1uiRF359do3g==", + "version": "7.22.5", + "resolved": "https://registry.npmjs.org/@babel/highlight/-/highlight-7.22.5.tgz", + "integrity": "sha512-BSKlD1hgnedS5XRnGOljZawtag7H1yPfQp0tdNJCHoH6AZ+Pcm9VvkrK59/Yy593Ypg0zMxH2BxD1VPYUQ7UIw==", "dev": true, "dependencies": { - "@babel/helper-validator-identifier": "^7.18.6", + "@babel/helper-validator-identifier": "^7.22.5", "chalk": "^2.0.0", "js-tokens": "^4.0.0" }, @@ -383,9 +398,9 @@ } }, "node_modules/@babel/parser": { - "version": "7.21.2", - "resolved": "https://registry.npmjs.org/@babel/parser/-/parser-7.21.2.tgz", - "integrity": "sha512-URpaIJQwEkEC2T9Kn+Ai6Xe/02iNaVCuT/PtoRz3GPVJVDpPd7mLo+VddTbhCRU9TXqW5mSrQfXZyi8kDKOVpQ==", + "version": "7.22.5", + "resolved": "https://registry.npmjs.org/@babel/parser/-/parser-7.22.5.tgz", + "integrity": "sha512-DFZMC9LJUG9PLOclRC32G63UXwzqS2koQC8dkx+PLdmt1xSePYpbT/NbsrJy8Q/muXz7o/h/d4A7Fuyixm559Q==", "dev": true, "bin": { "parser": "bin/babel-parser.js" @@ -542,33 +557,33 @@ } }, "node_modules/@babel/template": { - "version": "7.20.7", - "resolved": "https://registry.npmjs.org/@babel/template/-/template-7.20.7.tgz", - "integrity": "sha512-8SegXApWe6VoNw0r9JHpSteLKTpTiLZ4rMlGIm9JQ18KiCtyQiAMEazujAHrUS5flrcqYZa75ukev3P6QmUwUw==", + "version": "7.22.5", + "resolved": "https://registry.npmjs.org/@babel/template/-/template-7.22.5.tgz", + "integrity": "sha512-X7yV7eiwAxdj9k94NEylvbVHLiVG1nvzCV2EAowhxLTwODV1jl9UzZ48leOC0sH7OnuHrIkllaBgneUykIcZaw==", "dev": true, "dependencies": { - "@babel/code-frame": "^7.18.6", - "@babel/parser": "^7.20.7", - "@babel/types": "^7.20.7" + "@babel/code-frame": "^7.22.5", + "@babel/parser": "^7.22.5", + "@babel/types": "^7.22.5" }, "engines": { "node": ">=6.9.0" } }, "node_modules/@babel/traverse": { - "version": "7.21.2", - "resolved": "https://registry.npmjs.org/@babel/traverse/-/traverse-7.21.2.tgz", - "integrity": "sha512-ts5FFU/dSUPS13tv8XiEObDu9K+iagEKME9kAbaP7r0Y9KtZJZ+NGndDvWoRAYNpeWafbpFeki3q9QoMD6gxyw==", - "dev": true, - "dependencies": { - "@babel/code-frame": "^7.18.6", - "@babel/generator": "^7.21.1", - "@babel/helper-environment-visitor": "^7.18.9", - "@babel/helper-function-name": "^7.21.0", - "@babel/helper-hoist-variables": "^7.18.6", - "@babel/helper-split-export-declaration": "^7.18.6", - "@babel/parser": "^7.21.2", - "@babel/types": "^7.21.2", + "version": "7.22.5", + "resolved": "https://registry.npmjs.org/@babel/traverse/-/traverse-7.22.5.tgz", + "integrity": "sha512-7DuIjPgERaNo6r+PZwItpjCZEa5vyw4eJGufeLxrPdBXBoLcCJCIasvK6pK/9DVNrLZTLFhUGqaC6X/PA007TQ==", + "dev": true, + "dependencies": { + "@babel/code-frame": "^7.22.5", + "@babel/generator": "^7.22.5", + "@babel/helper-environment-visitor": "^7.22.5", + "@babel/helper-function-name": "^7.22.5", + "@babel/helper-hoist-variables": "^7.22.5", + "@babel/helper-split-export-declaration": "^7.22.5", + "@babel/parser": "^7.22.5", + "@babel/types": "^7.22.5", "debug": "^4.1.0", "globals": "^11.1.0" }, @@ -586,13 +601,13 @@ } }, "node_modules/@babel/types": { - "version": "7.21.2", - "resolved": "https://registry.npmjs.org/@babel/types/-/types-7.21.2.tgz", - "integrity": "sha512-3wRZSs7jiFaB8AjxiiD+VqN5DTG2iRvJGQ+qYFrs/654lg6kGTQWIOFjlBo5RaXuAZjBmP3+OQH4dmhqiiyYxw==", + "version": "7.22.5", + "resolved": "https://registry.npmjs.org/@babel/types/-/types-7.22.5.tgz", + "integrity": "sha512-zo3MIHGOkPOfoRXitsgHLjEXmlDaD/5KU1Uzuc9GNiZPhSqVxVRtxuPaSBZDsYZ9qV88AjtMtWW7ww98loJ9KA==", "dev": true, "dependencies": { - "@babel/helper-string-parser": "^7.19.4", - "@babel/helper-validator-identifier": "^7.19.1", + "@babel/helper-string-parser": "^7.22.5", + "@babel/helper-validator-identifier": "^7.22.5", "to-fast-properties": "^2.0.0" }, "engines": { @@ -1703,9 +1718,9 @@ } }, "node_modules/aws-cdk-lib": { - "version": "2.13.0", - "resolved": "https://registry.npmjs.org/aws-cdk-lib/-/aws-cdk-lib-2.13.0.tgz", - "integrity": "sha512-nKpQk+9H7T128gpzl+7XTu+19Yzj6kmCMrvSwTXLa/qr4/soEpXI68/+19ymEAHOYEL4Dd3eyk490P+y0wzi6A==", + "version": "2.80.0", + "resolved": "https://registry.npmjs.org/aws-cdk-lib/-/aws-cdk-lib-2.80.0.tgz", + "integrity": "sha512-PoqD3Yms5I0ajuTi071nTW/hpkH3XsdyZzn5gYsPv0qD7mqP3h6Qr+6RiGx+yQ1KcVFyxWdX15uK+DsC0KwvcQ==", "bundleDependencies": [ "@balena/dockerignore", "case", @@ -1715,17 +1730,22 @@ "minimatch", "punycode", "semver", + "table", "yaml" ], "dependencies": { + "@aws-cdk/asset-awscli-v1": "^2.2.177", + "@aws-cdk/asset-kubectl-v20": "^2.1.1", + "@aws-cdk/asset-node-proxy-agent-v5": "^2.0.148", "@balena/dockerignore": "^1.0.2", "case": "1.6.3", - "fs-extra": "^9.1.0", - "ignore": "^5.2.0", - "jsonschema": "^1.4.0", + "fs-extra": "^11.1.1", + "ignore": "^5.2.4", + "jsonschema": "^1.4.1", "minimatch": "^3.1.2", - "punycode": "^2.1.1", - "semver": "^7.3.5", + "punycode": "^2.3.0", + "semver": "^7.5.1", + "table": "^6.8.1", "yaml": "1.10.2" }, "engines": { @@ -1740,12 +1760,49 @@ "inBundle": true, "license": "Apache-2.0" }, - "node_modules/aws-cdk-lib/node_modules/at-least-node": { - "version": "1.0.0", + "node_modules/aws-cdk-lib/node_modules/ajv": { + "version": "8.12.0", "inBundle": true, - "license": "ISC", + "license": "MIT", + "dependencies": { + "fast-deep-equal": "^3.1.1", + "json-schema-traverse": "^1.0.0", + "require-from-string": "^2.0.2", + "uri-js": "^4.2.2" + }, + "funding": { + "type": "github", + "url": "https://github.com/sponsors/epoberezkin" + } + }, + "node_modules/aws-cdk-lib/node_modules/ansi-regex": { + "version": "5.0.1", + "inBundle": true, + "license": "MIT", "engines": { - "node": ">= 4.0.0" + "node": ">=8" + } + }, + "node_modules/aws-cdk-lib/node_modules/ansi-styles": { + "version": "4.3.0", + "inBundle": true, + "license": "MIT", + "dependencies": { + "color-convert": "^2.0.1" + }, + "engines": { + "node": ">=8" + }, + "funding": { + "url": "https://github.com/chalk/ansi-styles?sponsor=1" + } + }, + "node_modules/aws-cdk-lib/node_modules/astral-regex": { + "version": "2.0.0", + "inBundle": true, + "license": "MIT", + "engines": { + "node": ">=8" } }, "node_modules/aws-cdk-lib/node_modules/balanced-match": { @@ -1770,38 +1827,76 @@ "node": ">= 0.8.0" } }, + "node_modules/aws-cdk-lib/node_modules/color-convert": { + "version": "2.0.1", + "inBundle": true, + "license": "MIT", + "dependencies": { + "color-name": "~1.1.4" + }, + "engines": { + "node": ">=7.0.0" + } + }, + "node_modules/aws-cdk-lib/node_modules/color-name": { + "version": "1.1.4", + "inBundle": true, + "license": "MIT" + }, "node_modules/aws-cdk-lib/node_modules/concat-map": { "version": "0.0.1", "inBundle": true, "license": "MIT" }, + "node_modules/aws-cdk-lib/node_modules/emoji-regex": { + "version": "8.0.0", + "inBundle": true, + "license": "MIT" + }, + "node_modules/aws-cdk-lib/node_modules/fast-deep-equal": { + "version": "3.1.3", + "inBundle": true, + "license": "MIT" + }, "node_modules/aws-cdk-lib/node_modules/fs-extra": { - "version": "9.1.0", + "version": "11.1.1", "inBundle": true, "license": "MIT", "dependencies": { - "at-least-node": "^1.0.0", "graceful-fs": "^4.2.0", "jsonfile": "^6.0.1", "universalify": "^2.0.0" }, "engines": { - "node": ">=10" + "node": ">=14.14" } }, "node_modules/aws-cdk-lib/node_modules/graceful-fs": { - "version": "4.2.9", + "version": "4.2.11", "inBundle": true, "license": "ISC" }, "node_modules/aws-cdk-lib/node_modules/ignore": { - "version": "5.2.0", + "version": "5.2.4", "inBundle": true, "license": "MIT", "engines": { "node": ">= 4" } }, + "node_modules/aws-cdk-lib/node_modules/is-fullwidth-code-point": { + "version": "3.0.0", + "inBundle": true, + "license": "MIT", + "engines": { + "node": ">=8" + } + }, + "node_modules/aws-cdk-lib/node_modules/json-schema-traverse": { + "version": "1.0.0", + "inBundle": true, + "license": "MIT" + }, "node_modules/aws-cdk-lib/node_modules/jsonfile": { "version": "6.1.0", "inBundle": true, @@ -1814,13 +1909,18 @@ } }, "node_modules/aws-cdk-lib/node_modules/jsonschema": { - "version": "1.4.0", + "version": "1.4.1", "inBundle": true, "license": "MIT", "engines": { "node": "*" } }, + "node_modules/aws-cdk-lib/node_modules/lodash.truncate": { + "version": "4.4.2", + "inBundle": true, + "license": "MIT" + }, "node_modules/aws-cdk-lib/node_modules/lru-cache": { "version": "6.0.0", "inBundle": true, @@ -1844,15 +1944,23 @@ } }, "node_modules/aws-cdk-lib/node_modules/punycode": { - "version": "2.1.1", + "version": "2.3.0", "inBundle": true, "license": "MIT", "engines": { "node": ">=6" } }, + "node_modules/aws-cdk-lib/node_modules/require-from-string": { + "version": "2.0.2", + "inBundle": true, + "license": "MIT", + "engines": { + "node": ">=0.10.0" + } + }, "node_modules/aws-cdk-lib/node_modules/semver": { - "version": "7.3.5", + "version": "7.5.1", "inBundle": true, "license": "ISC", "dependencies": { @@ -1865,6 +1973,61 @@ "node": ">=10" } }, + "node_modules/aws-cdk-lib/node_modules/slice-ansi": { + "version": "4.0.0", + "inBundle": true, + "license": "MIT", + "dependencies": { + "ansi-styles": "^4.0.0", + "astral-regex": "^2.0.0", + "is-fullwidth-code-point": "^3.0.0" + }, + "engines": { + "node": ">=10" + }, + "funding": { + "url": "https://github.com/chalk/slice-ansi?sponsor=1" + } + }, + "node_modules/aws-cdk-lib/node_modules/string-width": { + "version": "4.2.3", + "inBundle": true, + "license": "MIT", + "dependencies": { + "emoji-regex": "^8.0.0", + "is-fullwidth-code-point": "^3.0.0", + "strip-ansi": "^6.0.1" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/aws-cdk-lib/node_modules/strip-ansi": { + "version": "6.0.1", + "inBundle": true, + "license": "MIT", + "dependencies": { + "ansi-regex": "^5.0.1" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/aws-cdk-lib/node_modules/table": { + "version": "6.8.1", + "inBundle": true, + "license": "BSD-3-Clause", + "dependencies": { + "ajv": "^8.0.1", + "lodash.truncate": "^4.4.2", + "slice-ansi": "^4.0.0", + "string-width": "^4.2.3", + "strip-ansi": "^6.0.1" + }, + "engines": { + "node": ">=10.0.0" + } + }, "node_modules/aws-cdk-lib/node_modules/universalify": { "version": "2.0.0", "inBundle": true, @@ -1873,6 +2036,14 @@ "node": ">= 10.0.0" } }, + "node_modules/aws-cdk-lib/node_modules/uri-js": { + "version": "4.4.1", + "inBundle": true, + "license": "BSD-2-Clause", + "dependencies": { + "punycode": "^2.1.0" + } + }, "node_modules/aws-cdk-lib/node_modules/yallist": { "version": "4.0.0", "inBundle": true, @@ -2006,7 +2177,8 @@ "node_modules/balanced-match": { "version": "1.0.2", "resolved": "https://registry.npmjs.org/balanced-match/-/balanced-match-1.0.2.tgz", - "integrity": "sha512-3oSeUO0TMV67hN1AmbXsK4yaqU7tjiHlbxRDZOpH0KW9+CeX4bRAaX0Anxt0tx2MrpRpWwQaPwIlISEJhYU5Pw==" + "integrity": "sha512-3oSeUO0TMV67hN1AmbXsK4yaqU7tjiHlbxRDZOpH0KW9+CeX4bRAaX0Anxt0tx2MrpRpWwQaPwIlISEJhYU5Pw==", + "dev": true }, "node_modules/base": { "version": "0.11.2", @@ -2042,6 +2214,7 @@ "version": "1.1.11", "resolved": "https://registry.npmjs.org/brace-expansion/-/brace-expansion-1.1.11.tgz", "integrity": "sha512-iCuPHDFgrHX7H2vEI/5xpz07zSHB00TpugqhmYtVmMO6518mCuRMoOYFldEBl0g187ufozdaHgWKcYFb61qGiA==", + "dev": true, "dependencies": { "balanced-match": "^1.0.0", "concat-map": "0.0.1" @@ -2399,7 +2572,8 @@ "node_modules/concat-map": { "version": "0.0.1", "resolved": "https://registry.npmjs.org/concat-map/-/concat-map-0.0.1.tgz", - "integrity": "sha512-/Srv4dswyQNBfohGpz9o6Yb3Gz3SrUDqBH5rTuhGR7ahtlbYKnVxw2bCFMRljaA7EXHaXZ8wsHdodFvbkhKmqg==" + "integrity": "sha512-/Srv4dswyQNBfohGpz9o6Yb3Gz3SrUDqBH5rTuhGR7ahtlbYKnVxw2bCFMRljaA7EXHaXZ8wsHdodFvbkhKmqg==", + "dev": true }, "node_modules/constructs": { "version": "10.1.264", @@ -4977,6 +5151,7 @@ "version": "3.1.2", "resolved": "https://registry.npmjs.org/minimatch/-/minimatch-3.1.2.tgz", "integrity": "sha512-J7p63hRiAjw1NDEww1W7i37+ByIrOWO5XQQAzZ3VOcL0PNybwpfmV/N05zFAzwQ9USyEcX6t3UO+K5aqBQOIHw==", + "dev": true, "dependencies": { "brace-expansion": "^1.1.7" }, @@ -6243,9 +6418,9 @@ } }, "node_modules/semver": { - "version": "7.3.8", - "resolved": "https://registry.npmjs.org/semver/-/semver-7.3.8.tgz", - "integrity": "sha512-NB1ctGL5rlHrPJtFDVIVzTyQylMLu9N9VICA6HSFJo8MCGVTMW6gfpicwKmmK/dAjTOrqu5l63JJOpDSrAis3A==", + "version": "7.5.3", + "resolved": "https://registry.npmjs.org/semver/-/semver-7.5.3.tgz", + "integrity": "sha512-QBlUtyVk/5EeHbi7X0fw6liDZc7BBmEaSYn01fMU1OUYbf6GPsbTtd8WmnqbI20SeycoHSeiybkE/q1Q+qlThQ==", "dependencies": { "lru-cache": "^6.0.0" }, @@ -7641,37 +7816,52 @@ "@jridgewell/trace-mapping": "^0.3.9" } }, + "@aws-cdk/asset-awscli-v1": { + "version": "2.2.199", + "resolved": "https://registry.npmjs.org/@aws-cdk/asset-awscli-v1/-/asset-awscli-v1-2.2.199.tgz", + "integrity": "sha512-zNdD2OxALdsdQaRZBpTfMTuudxV+4jLMznJIvVj6O+OqCru4m5UtgVQmyApW1z2H9s4/06ovVt20aXe2G8Ta+w==" + }, + "@aws-cdk/asset-kubectl-v20": { + "version": "2.1.2", + "resolved": "https://registry.npmjs.org/@aws-cdk/asset-kubectl-v20/-/asset-kubectl-v20-2.1.2.tgz", + "integrity": "sha512-3M2tELJOxQv0apCIiuKQ4pAbncz9GuLwnKFqxifWfe77wuMxyTRPmxssYHs42ePqzap1LT6GDcPygGs+hHstLg==" + }, + "@aws-cdk/asset-node-proxy-agent-v5": { + "version": "2.0.165", + "resolved": "https://registry.npmjs.org/@aws-cdk/asset-node-proxy-agent-v5/-/asset-node-proxy-agent-v5-2.0.165.tgz", + "integrity": "sha512-bsyLQD/vqXQcc9RDmlM1XqiFNO/yewgVFXmkMcQkndJbmE/jgYkzewwYGrBlfL725hGLQipXq19+jwWwdsXQqg==" + }, "@babel/code-frame": { - "version": "7.18.6", - "resolved": "https://registry.npmjs.org/@babel/code-frame/-/code-frame-7.18.6.tgz", - "integrity": "sha512-TDCmlK5eOvH+eH7cdAFlNXeVJqWIQ7gW9tY1GJIpUtFb6CmjVyq2VM3u71bOyR8CRihcCgMUYoDNyLXao3+70Q==", + "version": "7.22.5", + "resolved": "https://registry.npmjs.org/@babel/code-frame/-/code-frame-7.22.5.tgz", + "integrity": "sha512-Xmwn266vad+6DAqEB2A6V/CcZVp62BbwVmcOJc2RPuwih1kw02TjQvWVWlcKGbBPd+8/0V5DEkOcizRGYsspYQ==", "dev": true, "requires": { - "@babel/highlight": "^7.18.6" + "@babel/highlight": "^7.22.5" } }, "@babel/compat-data": { - "version": "7.21.0", - "resolved": "https://registry.npmjs.org/@babel/compat-data/-/compat-data-7.21.0.tgz", - "integrity": "sha512-gMuZsmsgxk/ENC3O/fRw5QY8A9/uxQbbCEypnLIiYYc/qVJtEV7ouxC3EllIIwNzMqAQee5tanFabWsUOutS7g==", + "version": "7.22.5", + "resolved": "https://registry.npmjs.org/@babel/compat-data/-/compat-data-7.22.5.tgz", + "integrity": "sha512-4Jc/YuIaYqKnDDz892kPIledykKg12Aw1PYX5i/TY28anJtacvM1Rrr8wbieB9GfEJwlzqT0hUEao0CxEebiDA==", "dev": true }, "@babel/core": { - "version": "7.21.0", - "resolved": "https://registry.npmjs.org/@babel/core/-/core-7.21.0.tgz", - "integrity": "sha512-PuxUbxcW6ZYe656yL3EAhpy7qXKq0DmYsrJLpbB8XrsCP9Nm+XCg9XFMb5vIDliPD7+U/+M+QJlH17XOcB7eXA==", + "version": "7.22.5", + "resolved": "https://registry.npmjs.org/@babel/core/-/core-7.22.5.tgz", + "integrity": "sha512-SBuTAjg91A3eKOvD+bPEz3LlhHZRNu1nFOVts9lzDJTXshHTjII0BAtDS3Y2DAkdZdDKWVZGVwkDfc4Clxn1dg==", "dev": true, "requires": { "@ampproject/remapping": "^2.2.0", - "@babel/code-frame": "^7.18.6", - "@babel/generator": "^7.21.0", - "@babel/helper-compilation-targets": "^7.20.7", - "@babel/helper-module-transforms": "^7.21.0", - "@babel/helpers": "^7.21.0", - "@babel/parser": "^7.21.0", - "@babel/template": "^7.20.7", - "@babel/traverse": "^7.21.0", - "@babel/types": "^7.21.0", + "@babel/code-frame": "^7.22.5", + "@babel/generator": "^7.22.5", + "@babel/helper-compilation-targets": "^7.22.5", + "@babel/helper-module-transforms": "^7.22.5", + "@babel/helpers": "^7.22.5", + "@babel/parser": "^7.22.5", + "@babel/template": "^7.22.5", + "@babel/traverse": "^7.22.5", + "@babel/types": "^7.22.5", "convert-source-map": "^1.7.0", "debug": "^4.1.0", "gensync": "^1.0.0-beta.2", @@ -7688,12 +7878,12 @@ } }, "@babel/generator": { - "version": "7.21.1", - "resolved": "https://registry.npmjs.org/@babel/generator/-/generator-7.21.1.tgz", - "integrity": "sha512-1lT45bAYlQhFn/BHivJs43AiW2rg3/UbLyShGfF3C0KmHvO5fSghWd5kBJy30kpRRucGzXStvnnCFniCR2kXAA==", + "version": "7.22.5", + "resolved": "https://registry.npmjs.org/@babel/generator/-/generator-7.22.5.tgz", + "integrity": "sha512-+lcUbnTRhd0jOewtFSedLyiPsD5tswKkbgcezOqqWFUVNEwoUTlpPOBmvhG7OXWLR4jMdv0czPGH5XbflnD1EA==", "dev": true, "requires": { - "@babel/types": "^7.21.0", + "@babel/types": "^7.22.5", "@jridgewell/gen-mapping": "^0.3.2", "@jridgewell/trace-mapping": "^0.3.17", "jsesc": "^2.5.1" @@ -7713,13 +7903,13 @@ } }, "@babel/helper-compilation-targets": { - "version": "7.20.7", - "resolved": "https://registry.npmjs.org/@babel/helper-compilation-targets/-/helper-compilation-targets-7.20.7.tgz", - "integrity": "sha512-4tGORmfQcrc+bvrjb5y3dG9Mx1IOZjsHqQVUz7XCNHO+iTmqxWnVg3KRygjGmpRLJGdQSKuvFinbIb0CnZwHAQ==", + "version": "7.22.5", + "resolved": "https://registry.npmjs.org/@babel/helper-compilation-targets/-/helper-compilation-targets-7.22.5.tgz", + "integrity": "sha512-Ji+ywpHeuqxB8WDxraCiqR0xfhYjiDE/e6k7FuIaANnoOFxAHskHChz4vA1mJC9Lbm01s1PVAGhQY4FUKSkGZw==", "dev": true, "requires": { - "@babel/compat-data": "^7.20.5", - "@babel/helper-validator-option": "^7.18.6", + "@babel/compat-data": "^7.22.5", + "@babel/helper-validator-option": "^7.22.5", "browserslist": "^4.21.3", "lru-cache": "^5.1.1", "semver": "^6.3.0" @@ -7734,53 +7924,53 @@ } }, "@babel/helper-environment-visitor": { - "version": "7.18.9", - "resolved": "https://registry.npmjs.org/@babel/helper-environment-visitor/-/helper-environment-visitor-7.18.9.tgz", - "integrity": "sha512-3r/aACDJ3fhQ/EVgFy0hpj8oHyHpQc+LPtJoY9SzTThAsStm4Ptegq92vqKoE3vD706ZVFWITnMnxucw+S9Ipg==", + "version": "7.22.5", + "resolved": "https://registry.npmjs.org/@babel/helper-environment-visitor/-/helper-environment-visitor-7.22.5.tgz", + "integrity": "sha512-XGmhECfVA/5sAt+H+xpSg0mfrHq6FzNr9Oxh7PSEBBRUb/mL7Kz3NICXb194rCqAEdxkhPT1a88teizAFyvk8Q==", "dev": true }, "@babel/helper-function-name": { - "version": "7.21.0", - "resolved": "https://registry.npmjs.org/@babel/helper-function-name/-/helper-function-name-7.21.0.tgz", - "integrity": "sha512-HfK1aMRanKHpxemaY2gqBmL04iAPOPRj7DxtNbiDOrJK+gdwkiNRVpCpUJYbUT+aZyemKN8brqTOxzCaG6ExRg==", + "version": "7.22.5", + "resolved": "https://registry.npmjs.org/@babel/helper-function-name/-/helper-function-name-7.22.5.tgz", + "integrity": "sha512-wtHSq6jMRE3uF2otvfuD3DIvVhOsSNshQl0Qrd7qC9oQJzHvOL4qQXlQn2916+CXGywIjpGuIkoyZRRxHPiNQQ==", "dev": true, "requires": { - "@babel/template": "^7.20.7", - "@babel/types": "^7.21.0" + "@babel/template": "^7.22.5", + "@babel/types": "^7.22.5" } }, "@babel/helper-hoist-variables": { - "version": "7.18.6", - "resolved": "https://registry.npmjs.org/@babel/helper-hoist-variables/-/helper-hoist-variables-7.18.6.tgz", - "integrity": "sha512-UlJQPkFqFULIcyW5sbzgbkxn2FKRgwWiRexcuaR8RNJRy8+LLveqPjwZV/bwrLZCN0eUHD/x8D0heK1ozuoo6Q==", + "version": "7.22.5", + "resolved": "https://registry.npmjs.org/@babel/helper-hoist-variables/-/helper-hoist-variables-7.22.5.tgz", + "integrity": "sha512-wGjk9QZVzvknA6yKIUURb8zY3grXCcOZt+/7Wcy8O2uctxhplmUPkOdlgoNhmdVee2c92JXbf1xpMtVNbfoxRw==", "dev": true, "requires": { - "@babel/types": "^7.18.6" + "@babel/types": "^7.22.5" } }, "@babel/helper-module-imports": { - "version": "7.18.6", - "resolved": "https://registry.npmjs.org/@babel/helper-module-imports/-/helper-module-imports-7.18.6.tgz", - "integrity": "sha512-0NFvs3VkuSYbFi1x2Vd6tKrywq+z/cLeYC/RJNFrIX/30Bf5aiGYbtvGXolEktzJH8o5E5KJ3tT+nkxuuZFVlA==", + "version": "7.22.5", + "resolved": "https://registry.npmjs.org/@babel/helper-module-imports/-/helper-module-imports-7.22.5.tgz", + "integrity": "sha512-8Dl6+HD/cKifutF5qGd/8ZJi84QeAKh+CEe1sBzz8UayBBGg1dAIJrdHOcOM5b2MpzWL2yuotJTtGjETq0qjXg==", "dev": true, "requires": { - "@babel/types": "^7.18.6" + "@babel/types": "^7.22.5" } }, "@babel/helper-module-transforms": { - "version": "7.21.2", - "resolved": "https://registry.npmjs.org/@babel/helper-module-transforms/-/helper-module-transforms-7.21.2.tgz", - "integrity": "sha512-79yj2AR4U/Oqq/WOV7Lx6hUjau1Zfo4cI+JLAVYeMV5XIlbOhmjEk5ulbTc9fMpmlojzZHkUUxAiK+UKn+hNQQ==", + "version": "7.22.5", + "resolved": "https://registry.npmjs.org/@babel/helper-module-transforms/-/helper-module-transforms-7.22.5.tgz", + "integrity": "sha512-+hGKDt/Ze8GFExiVHno/2dvG5IdstpzCq0y4Qc9OJ25D4q3pKfiIP/4Vp3/JvhDkLKsDK2api3q3fpIgiIF5bw==", "dev": true, "requires": { - "@babel/helper-environment-visitor": "^7.18.9", - "@babel/helper-module-imports": "^7.18.6", - "@babel/helper-simple-access": "^7.20.2", - "@babel/helper-split-export-declaration": "^7.18.6", - "@babel/helper-validator-identifier": "^7.19.1", - "@babel/template": "^7.20.7", - "@babel/traverse": "^7.21.2", - "@babel/types": "^7.21.2" + "@babel/helper-environment-visitor": "^7.22.5", + "@babel/helper-module-imports": "^7.22.5", + "@babel/helper-simple-access": "^7.22.5", + "@babel/helper-split-export-declaration": "^7.22.5", + "@babel/helper-validator-identifier": "^7.22.5", + "@babel/template": "^7.22.5", + "@babel/traverse": "^7.22.5", + "@babel/types": "^7.22.5" } }, "@babel/helper-plugin-utils": { @@ -7790,59 +7980,59 @@ "dev": true }, "@babel/helper-simple-access": { - "version": "7.20.2", - "resolved": "https://registry.npmjs.org/@babel/helper-simple-access/-/helper-simple-access-7.20.2.tgz", - "integrity": "sha512-+0woI/WPq59IrqDYbVGfshjT5Dmk/nnbdpcF8SnMhhXObpTq2KNBdLFRFrkVdbDOyUmHBCxzm5FHV1rACIkIbA==", + "version": "7.22.5", + "resolved": "https://registry.npmjs.org/@babel/helper-simple-access/-/helper-simple-access-7.22.5.tgz", + "integrity": "sha512-n0H99E/K+Bika3++WNL17POvo4rKWZ7lZEp1Q+fStVbUi8nxPQEBOlTmCOxW/0JsS56SKKQ+ojAe2pHKJHN35w==", "dev": true, "requires": { - "@babel/types": "^7.20.2" + "@babel/types": "^7.22.5" } }, "@babel/helper-split-export-declaration": { - "version": "7.18.6", - "resolved": "https://registry.npmjs.org/@babel/helper-split-export-declaration/-/helper-split-export-declaration-7.18.6.tgz", - "integrity": "sha512-bde1etTx6ZyTmobl9LLMMQsaizFVZrquTEHOqKeQESMKo4PlObf+8+JA25ZsIpZhT/WEd39+vOdLXAFG/nELpA==", + "version": "7.22.5", + "resolved": "https://registry.npmjs.org/@babel/helper-split-export-declaration/-/helper-split-export-declaration-7.22.5.tgz", + "integrity": "sha512-thqK5QFghPKWLhAV321lxF95yCg2K3Ob5yw+M3VHWfdia0IkPXUtoLH8x/6Fh486QUvzhb8YOWHChTVen2/PoQ==", "dev": true, "requires": { - "@babel/types": "^7.18.6" + "@babel/types": "^7.22.5" } }, "@babel/helper-string-parser": { - "version": "7.19.4", - "resolved": "https://registry.npmjs.org/@babel/helper-string-parser/-/helper-string-parser-7.19.4.tgz", - "integrity": "sha512-nHtDoQcuqFmwYNYPz3Rah5ph2p8PFeFCsZk9A/48dPc/rGocJ5J3hAAZ7pb76VWX3fZKu+uEr/FhH5jLx7umrw==", + "version": "7.22.5", + "resolved": "https://registry.npmjs.org/@babel/helper-string-parser/-/helper-string-parser-7.22.5.tgz", + "integrity": "sha512-mM4COjgZox8U+JcXQwPijIZLElkgEpO5rsERVDJTc2qfCDfERyob6k5WegS14SX18IIjv+XD+GrqNumY5JRCDw==", "dev": true }, "@babel/helper-validator-identifier": { - "version": "7.19.1", - "resolved": "https://registry.npmjs.org/@babel/helper-validator-identifier/-/helper-validator-identifier-7.19.1.tgz", - "integrity": "sha512-awrNfaMtnHUr653GgGEs++LlAvW6w+DcPrOliSMXWCKo597CwL5Acf/wWdNkf/tfEQE3mjkeD1YOVZOUV/od1w==", + "version": "7.22.5", + "resolved": "https://registry.npmjs.org/@babel/helper-validator-identifier/-/helper-validator-identifier-7.22.5.tgz", + "integrity": "sha512-aJXu+6lErq8ltp+JhkJUfk1MTGyuA4v7f3pA+BJ5HLfNC6nAQ0Cpi9uOquUj8Hehg0aUiHzWQbOVJGao6ztBAQ==", "dev": true }, "@babel/helper-validator-option": { - "version": "7.21.0", - "resolved": "https://registry.npmjs.org/@babel/helper-validator-option/-/helper-validator-option-7.21.0.tgz", - "integrity": "sha512-rmL/B8/f0mKS2baE9ZpyTcTavvEuWhTTW8amjzXNvYG4AwBsqTLikfXsEofsJEfKHf+HQVQbFOHy6o+4cnC/fQ==", + "version": "7.22.5", + "resolved": "https://registry.npmjs.org/@babel/helper-validator-option/-/helper-validator-option-7.22.5.tgz", + "integrity": "sha512-R3oB6xlIVKUnxNUxbmgq7pKjxpru24zlimpE8WK47fACIlM0II/Hm1RS8IaOI7NgCr6LNS+jl5l75m20npAziw==", "dev": true }, "@babel/helpers": { - "version": "7.21.0", - "resolved": "https://registry.npmjs.org/@babel/helpers/-/helpers-7.21.0.tgz", - "integrity": "sha512-XXve0CBtOW0pd7MRzzmoyuSj0e3SEzj8pgyFxnTT1NJZL38BD1MK7yYrm8yefRPIDvNNe14xR4FdbHwpInD4rA==", + "version": "7.22.5", + "resolved": "https://registry.npmjs.org/@babel/helpers/-/helpers-7.22.5.tgz", + "integrity": "sha512-pSXRmfE1vzcUIDFQcSGA5Mr+GxBV9oiRKDuDxXvWQQBCh8HoIjs/2DlDB7H8smac1IVrB9/xdXj2N3Wol9Cr+Q==", "dev": true, "requires": { - "@babel/template": "^7.20.7", - "@babel/traverse": "^7.21.0", - "@babel/types": "^7.21.0" + "@babel/template": "^7.22.5", + "@babel/traverse": "^7.22.5", + "@babel/types": "^7.22.5" } }, "@babel/highlight": { - "version": "7.18.6", - "resolved": "https://registry.npmjs.org/@babel/highlight/-/highlight-7.18.6.tgz", - "integrity": "sha512-u7stbOuYjaPezCuLj29hNW1v64M2Md2qupEKP1fHc7WdOA3DgLh37suiSrZYY7haUB7iBeQZ9P1uiRF359do3g==", + "version": "7.22.5", + "resolved": "https://registry.npmjs.org/@babel/highlight/-/highlight-7.22.5.tgz", + "integrity": "sha512-BSKlD1hgnedS5XRnGOljZawtag7H1yPfQp0tdNJCHoH6AZ+Pcm9VvkrK59/Yy593Ypg0zMxH2BxD1VPYUQ7UIw==", "dev": true, "requires": { - "@babel/helper-validator-identifier": "^7.18.6", + "@babel/helper-validator-identifier": "^7.22.5", "chalk": "^2.0.0", "js-tokens": "^4.0.0" }, @@ -7906,9 +8096,9 @@ } }, "@babel/parser": { - "version": "7.21.2", - "resolved": "https://registry.npmjs.org/@babel/parser/-/parser-7.21.2.tgz", - "integrity": "sha512-URpaIJQwEkEC2T9Kn+Ai6Xe/02iNaVCuT/PtoRz3GPVJVDpPd7mLo+VddTbhCRU9TXqW5mSrQfXZyi8kDKOVpQ==", + "version": "7.22.5", + "resolved": "https://registry.npmjs.org/@babel/parser/-/parser-7.22.5.tgz", + "integrity": "sha512-DFZMC9LJUG9PLOclRC32G63UXwzqS2koQC8dkx+PLdmt1xSePYpbT/NbsrJy8Q/muXz7o/h/d4A7Fuyixm559Q==", "dev": true }, "@babel/plugin-syntax-async-generators": { @@ -8020,30 +8210,30 @@ } }, "@babel/template": { - "version": "7.20.7", - "resolved": "https://registry.npmjs.org/@babel/template/-/template-7.20.7.tgz", - "integrity": "sha512-8SegXApWe6VoNw0r9JHpSteLKTpTiLZ4rMlGIm9JQ18KiCtyQiAMEazujAHrUS5flrcqYZa75ukev3P6QmUwUw==", + "version": "7.22.5", + "resolved": "https://registry.npmjs.org/@babel/template/-/template-7.22.5.tgz", + "integrity": "sha512-X7yV7eiwAxdj9k94NEylvbVHLiVG1nvzCV2EAowhxLTwODV1jl9UzZ48leOC0sH7OnuHrIkllaBgneUykIcZaw==", "dev": true, "requires": { - "@babel/code-frame": "^7.18.6", - "@babel/parser": "^7.20.7", - "@babel/types": "^7.20.7" + "@babel/code-frame": "^7.22.5", + "@babel/parser": "^7.22.5", + "@babel/types": "^7.22.5" } }, "@babel/traverse": { - "version": "7.21.2", - "resolved": "https://registry.npmjs.org/@babel/traverse/-/traverse-7.21.2.tgz", - "integrity": "sha512-ts5FFU/dSUPS13tv8XiEObDu9K+iagEKME9kAbaP7r0Y9KtZJZ+NGndDvWoRAYNpeWafbpFeki3q9QoMD6gxyw==", - "dev": true, - "requires": { - "@babel/code-frame": "^7.18.6", - "@babel/generator": "^7.21.1", - "@babel/helper-environment-visitor": "^7.18.9", - "@babel/helper-function-name": "^7.21.0", - "@babel/helper-hoist-variables": "^7.18.6", - "@babel/helper-split-export-declaration": "^7.18.6", - "@babel/parser": "^7.21.2", - "@babel/types": "^7.21.2", + "version": "7.22.5", + "resolved": "https://registry.npmjs.org/@babel/traverse/-/traverse-7.22.5.tgz", + "integrity": "sha512-7DuIjPgERaNo6r+PZwItpjCZEa5vyw4eJGufeLxrPdBXBoLcCJCIasvK6pK/9DVNrLZTLFhUGqaC6X/PA007TQ==", + "dev": true, + "requires": { + "@babel/code-frame": "^7.22.5", + "@babel/generator": "^7.22.5", + "@babel/helper-environment-visitor": "^7.22.5", + "@babel/helper-function-name": "^7.22.5", + "@babel/helper-hoist-variables": "^7.22.5", + "@babel/helper-split-export-declaration": "^7.22.5", + "@babel/parser": "^7.22.5", + "@babel/types": "^7.22.5", "debug": "^4.1.0", "globals": "^11.1.0" }, @@ -8057,13 +8247,13 @@ } }, "@babel/types": { - "version": "7.21.2", - "resolved": "https://registry.npmjs.org/@babel/types/-/types-7.21.2.tgz", - "integrity": "sha512-3wRZSs7jiFaB8AjxiiD+VqN5DTG2iRvJGQ+qYFrs/654lg6kGTQWIOFjlBo5RaXuAZjBmP3+OQH4dmhqiiyYxw==", + "version": "7.22.5", + "resolved": "https://registry.npmjs.org/@babel/types/-/types-7.22.5.tgz", + "integrity": "sha512-zo3MIHGOkPOfoRXitsgHLjEXmlDaD/5KU1Uzuc9GNiZPhSqVxVRtxuPaSBZDsYZ9qV88AjtMtWW7ww98loJ9KA==", "dev": true, "requires": { - "@babel/helper-string-parser": "^7.19.4", - "@babel/helper-validator-identifier": "^7.19.1", + "@babel/helper-string-parser": "^7.22.5", + "@babel/helper-validator-identifier": "^7.22.5", "to-fast-properties": "^2.0.0" } }, @@ -8902,18 +9092,22 @@ } }, "aws-cdk-lib": { - "version": "2.13.0", - "resolved": "https://registry.npmjs.org/aws-cdk-lib/-/aws-cdk-lib-2.13.0.tgz", - "integrity": "sha512-nKpQk+9H7T128gpzl+7XTu+19Yzj6kmCMrvSwTXLa/qr4/soEpXI68/+19ymEAHOYEL4Dd3eyk490P+y0wzi6A==", + "version": "2.80.0", + "resolved": "https://registry.npmjs.org/aws-cdk-lib/-/aws-cdk-lib-2.80.0.tgz", + "integrity": "sha512-PoqD3Yms5I0ajuTi071nTW/hpkH3XsdyZzn5gYsPv0qD7mqP3h6Qr+6RiGx+yQ1KcVFyxWdX15uK+DsC0KwvcQ==", "requires": { + "@aws-cdk/asset-awscli-v1": "^2.2.177", + "@aws-cdk/asset-kubectl-v20": "^2.1.1", + "@aws-cdk/asset-node-proxy-agent-v5": "^2.0.148", "@balena/dockerignore": "^1.0.2", "case": "1.6.3", - "fs-extra": "^9.1.0", - "ignore": "^5.2.0", - "jsonschema": "^1.4.0", + "fs-extra": "^11.1.1", + "ignore": "^5.2.4", + "jsonschema": "^1.4.1", "minimatch": "^3.1.2", - "punycode": "^2.1.1", - "semver": "^7.3.5", + "punycode": "^2.3.0", + "semver": "^7.5.1", + "table": "^6.8.1", "yaml": "1.10.2" }, "dependencies": { @@ -8921,8 +9115,29 @@ "version": "1.0.2", "bundled": true }, - "at-least-node": { - "version": "1.0.0", + "ajv": { + "version": "8.12.0", + "bundled": true, + "requires": { + "fast-deep-equal": "^3.1.1", + "json-schema-traverse": "^1.0.0", + "require-from-string": "^2.0.2", + "uri-js": "^4.2.2" + } + }, + "ansi-regex": { + "version": "5.0.1", + "bundled": true + }, + "ansi-styles": { + "version": "4.3.0", + "bundled": true, + "requires": { + "color-convert": "^2.0.1" + } + }, + "astral-regex": { + "version": "2.0.0", "bundled": true }, "balanced-match": { @@ -8941,26 +9156,52 @@ "version": "1.6.3", "bundled": true }, + "color-convert": { + "version": "2.0.1", + "bundled": true, + "requires": { + "color-name": "~1.1.4" + } + }, + "color-name": { + "version": "1.1.4", + "bundled": true + }, "concat-map": { "version": "0.0.1", "bundled": true }, + "emoji-regex": { + "version": "8.0.0", + "bundled": true + }, + "fast-deep-equal": { + "version": "3.1.3", + "bundled": true + }, "fs-extra": { - "version": "9.1.0", + "version": "11.1.1", "bundled": true, "requires": { - "at-least-node": "^1.0.0", "graceful-fs": "^4.2.0", "jsonfile": "^6.0.1", "universalify": "^2.0.0" } }, "graceful-fs": { - "version": "4.2.9", + "version": "4.2.11", "bundled": true }, "ignore": { - "version": "5.2.0", + "version": "5.2.4", + "bundled": true + }, + "is-fullwidth-code-point": { + "version": "3.0.0", + "bundled": true + }, + "json-schema-traverse": { + "version": "1.0.0", "bundled": true }, "jsonfile": { @@ -8972,7 +9213,11 @@ } }, "jsonschema": { - "version": "1.4.0", + "version": "1.4.1", + "bundled": true + }, + "lodash.truncate": { + "version": "4.4.2", "bundled": true }, "lru-cache": { @@ -8990,20 +9235,67 @@ } }, "punycode": { - "version": "2.1.1", + "version": "2.3.0", + "bundled": true + }, + "require-from-string": { + "version": "2.0.2", "bundled": true }, "semver": { - "version": "7.3.5", + "version": "7.5.1", "bundled": true, "requires": { "lru-cache": "^6.0.0" } }, + "slice-ansi": { + "version": "4.0.0", + "bundled": true, + "requires": { + "ansi-styles": "^4.0.0", + "astral-regex": "^2.0.0", + "is-fullwidth-code-point": "^3.0.0" + } + }, + "string-width": { + "version": "4.2.3", + "bundled": true, + "requires": { + "emoji-regex": "^8.0.0", + "is-fullwidth-code-point": "^3.0.0", + "strip-ansi": "^6.0.1" + } + }, + "strip-ansi": { + "version": "6.0.1", + "bundled": true, + "requires": { + "ansi-regex": "^5.0.1" + } + }, + "table": { + "version": "6.8.1", + "bundled": true, + "requires": { + "ajv": "^8.0.1", + "lodash.truncate": "^4.4.2", + "slice-ansi": "^4.0.0", + "string-width": "^4.2.3", + "strip-ansi": "^6.0.1" + } + }, "universalify": { "version": "2.0.0", "bundled": true }, + "uri-js": { + "version": "4.4.1", + "bundled": true, + "requires": { + "punycode": "^2.1.0" + } + }, "yallist": { "version": "4.0.0", "bundled": true @@ -9109,7 +9401,8 @@ "balanced-match": { "version": "1.0.2", "resolved": "https://registry.npmjs.org/balanced-match/-/balanced-match-1.0.2.tgz", - "integrity": "sha512-3oSeUO0TMV67hN1AmbXsK4yaqU7tjiHlbxRDZOpH0KW9+CeX4bRAaX0Anxt0tx2MrpRpWwQaPwIlISEJhYU5Pw==" + "integrity": "sha512-3oSeUO0TMV67hN1AmbXsK4yaqU7tjiHlbxRDZOpH0KW9+CeX4bRAaX0Anxt0tx2MrpRpWwQaPwIlISEJhYU5Pw==", + "dev": true }, "base": { "version": "0.11.2", @@ -9141,6 +9434,7 @@ "version": "1.1.11", "resolved": "https://registry.npmjs.org/brace-expansion/-/brace-expansion-1.1.11.tgz", "integrity": "sha512-iCuPHDFgrHX7H2vEI/5xpz07zSHB00TpugqhmYtVmMO6518mCuRMoOYFldEBl0g187ufozdaHgWKcYFb61qGiA==", + "dev": true, "requires": { "balanced-match": "^1.0.0", "concat-map": "0.0.1" @@ -9414,7 +9708,8 @@ "concat-map": { "version": "0.0.1", "resolved": "https://registry.npmjs.org/concat-map/-/concat-map-0.0.1.tgz", - "integrity": "sha512-/Srv4dswyQNBfohGpz9o6Yb3Gz3SrUDqBH5rTuhGR7ahtlbYKnVxw2bCFMRljaA7EXHaXZ8wsHdodFvbkhKmqg==" + "integrity": "sha512-/Srv4dswyQNBfohGpz9o6Yb3Gz3SrUDqBH5rTuhGR7ahtlbYKnVxw2bCFMRljaA7EXHaXZ8wsHdodFvbkhKmqg==", + "dev": true }, "constructs": { "version": "10.1.264", @@ -11399,6 +11694,7 @@ "version": "3.1.2", "resolved": "https://registry.npmjs.org/minimatch/-/minimatch-3.1.2.tgz", "integrity": "sha512-J7p63hRiAjw1NDEww1W7i37+ByIrOWO5XQQAzZ3VOcL0PNybwpfmV/N05zFAzwQ9USyEcX6t3UO+K5aqBQOIHw==", + "dev": true, "requires": { "brace-expansion": "^1.1.7" } @@ -12356,9 +12652,9 @@ } }, "semver": { - "version": "7.3.8", - "resolved": "https://registry.npmjs.org/semver/-/semver-7.3.8.tgz", - "integrity": "sha512-NB1ctGL5rlHrPJtFDVIVzTyQylMLu9N9VICA6HSFJo8MCGVTMW6gfpicwKmmK/dAjTOrqu5l63JJOpDSrAis3A==", + "version": "7.5.3", + "resolved": "https://registry.npmjs.org/semver/-/semver-7.5.3.tgz", + "integrity": "sha512-QBlUtyVk/5EeHbi7X0fw6liDZc7BBmEaSYn01fMU1OUYbf6GPsbTtd8WmnqbI20SeycoHSeiybkE/q1Q+qlThQ==", "requires": { "lru-cache": "^6.0.0" }, diff --git a/release/staging-resources-cdk/package.json b/release/staging-resources-cdk/package.json index 9b16af0014..b293856d63 100644 --- a/release/staging-resources-cdk/package.json +++ b/release/staging-resources-cdk/package.json @@ -25,7 +25,7 @@ "typescript": "~3.9.7" }, "dependencies": { - "aws-cdk-lib": "2.13.0", + "aws-cdk-lib": "2.80.0", "constructs": "^10.0.0", "source-map-support": "^0.5.16" } diff --git a/settings.gradle b/settings.gradle index 0760851409..67c85c3a12 100644 --- a/settings.gradle +++ b/settings.gradle @@ -31,10 +31,10 @@ dependencyResolutionManagement { version('opensearch', '1.3.8') library('opensearch-client', 'org.opensearch.client', 'opensearch-rest-client').versionRef('opensearch') library('opensearch-rhlc', 'org.opensearch.client', 'opensearch-rest-high-level-client').versionRef('opensearch') - version('spring', '5.3.27') + version('spring', '5.3.28') library('spring-core', 'org.springframework', 'spring-core').versionRef('spring') library('spring-context', 'org.springframework', 'spring-context').versionRef('spring') - version('guava', '31.1-jre') + version('guava', '32.0.1-jre') library('guava-core', 'com.google.guava', 'guava').versionRef('guava') } testLibs { @@ -42,7 +42,7 @@ dependencyResolutionManagement { version('mockito', '3.11.2') version('hamcrest', '2.2') version('awaitility', '4.2.0') - version('spring', '5.3.26') + version('spring', '5.3.28') version('slf4j', '2.0.6') library('junit-core', 'org.junit.jupiter', 'junit-jupiter').versionRef('junit') library('junit-params', 'org.junit.jupiter', 'junit-jupiter-params').versionRef('junit') @@ -103,6 +103,7 @@ include 'data-prepper-plugins:rss-source' include 'data-prepper-plugins:csv-processor' include 'data-prepper-plugins:parse-json-processor' include 'data-prepper-plugins:trace-peer-forwarder-processor' +include 'data-prepper-plugins:translate-processor' include 'data-prepper-plugins:dynamodb-source-coordination-store' include 'release' include 'release:archives' @@ -125,4 +126,5 @@ include 'data-prepper-plugins:parquet-codecs' include 'data-prepper-plugins:aws-sqs-common' include 'data-prepper-plugins:buffer-common' include 'data-prepper-plugins:sqs-source' +include 'data-prepper-plugins:cloudwatch-logs' include 'data-prepper-plugins:http-sink'