diff --git a/.github/workflows/release.yml b/.github/workflows/release.yml index c1c65fda49..daf820c6fe 100644 --- a/.github/workflows/release.yml +++ b/.github/workflows/release.yml @@ -2,6 +2,14 @@ name: Release Artifacts on: workflow_dispatch: + inputs: + release-major-tag: + description: 'Whether to create major tag of docker image or not. This will create a tag such as 2.3 which points to this version.' + required: true + release-latest-tag: + description: > + 'Whether to create latest tag of docker image or not. This will update the latest tag to point to this version. You should set this when releasing the latest version, but not patches to old versions.' + required: true permissions: id-token: write @@ -95,3 +103,68 @@ jobs: - name: Smoke Test Tarball Files run: ./release/smoke-tests/run-tarball-files-smoke-tests.sh -v ${{ env.version }} -u ${{ secrets.ARCHIVES_PUBLIC_URL }} -n ${{ github.run_number }} -i ${{ matrix.image }} -t ${{ matrix.archive }} + + protomote: + runs-on: ubuntu-latest + if: success() || failure() + needs: [build, validate-docker, validate-archive] + permissions: + contents: write + issues: write + + steps: + - name: Checkout Data Prepper + uses: actions/checkout@v3 + - name: Get Version + run: grep '^version=' gradle.properties >> $GITHUB_ENV + + - name: Get Approvers + id: get_approvers + run: | + echo "approvers=$(cat .github/CODEOWNERS | grep @ | tr -d '* ' | sed 's/@/,/g' | sed 's/,//1')" >> $GITHUB_OUTPUT + - uses: trstringer/manual-approval@v1 + with: + secret: ${{ github.TOKEN }} + approvers: ${{ steps.get_approvers.outputs.approvers }} + minimum-approvals: 2 + issue-title: 'Release Data Prepper : ${{ env.version }}' + issue-body: > + Please approve or deny the release of Data Prepper. + + **VERSION**: ${{ env.version }} + + **BUILD NUMBER**: ${{ github.run_number }} + + **RELEASE MAJOR TAG**: ${{ github.event.inputs.release-major-tag }} + + **RELEASE LATEST TAG**: ${{ github.event.inputs.release-latest-tag }} + + exclude-workflow-initiator-as-approver: false + + - name: Create Release Description + run: | + echo 'version: ${{ env.version }}' > release-description.yaml + echo 'build_number: ${{ github.run_number }}' >> release-description.yaml + echo 'release_major_tag: ${{ github.event.inputs.release-major-tag }}' >> release-description.yaml + echo 'release_latest_tag: ${{ github.event.inputs.release-latest-tag }}' >> release-description.yaml + + - name: Create tag + uses: actions/github-script@v6 + with: + github-token: ${{ github.TOKEN }} + script: | + github.rest.git.createRef({ + owner: context.repo.owner, + repo: context.repo.repo, + ref: 'refs/tags/${{ env.version }}', + sha: context.sha + }) + + - name: Draft release + uses: softprops/action-gh-release@v1 + with: + draft: true + name: '${{ env.version }}' + tag_name: 'refs/tags/${{ env.version }}' + files: | + release-description.yaml diff --git a/build.gradle b/build.gradle index 006e641354..efc059810d 100644 --- a/build.gradle +++ b/build.gradle @@ -196,7 +196,7 @@ subprojects { configure(subprojects.findAll {it.name != 'data-prepper-api'}) { dependencies { - implementation platform('software.amazon.awssdk:bom:2.17.264') + implementation platform('software.amazon.awssdk:bom:2.20.67') implementation 'jakarta.validation:jakarta.validation-api:3.0.2' } } 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 22280b7464..afb9b975fe 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 @@ -58,6 +58,22 @@ public interface OutputCodec { */ String getExtension(); + /** + * Returns true if this codec has an internal compression. That is, the entire + * {@link OutputStream} should not be compressed. + *

+ * When this value is true, sinks should not attempt to encrypt the final {@link OutputStream} + * at all. + *

+ * For example, Parquet compression happens within the file. Each column chunk + * is compressed independently. + * + * @return True if the compression is internal to the codec; false if whole-file compression is ok. + */ + default boolean isCompressionInternal() { + return false; + } + default Event addTagsToEvent(Event event, String tagsTargetKey) throws JsonProcessingException { String eventJsonString = event.jsonBuilder().includeTags(tagsTargetKey).toJsonString(); Map eventData = objectMapper.readValue(eventJsonString, new TypeReference<>() { diff --git a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/configuration/SinkModel.java b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/configuration/SinkModel.java index 328a84e586..936a5445e8 100644 --- a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/configuration/SinkModel.java +++ b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/configuration/SinkModel.java @@ -12,13 +12,11 @@ import com.fasterxml.jackson.databind.annotation.JsonSerialize; import org.opensearch.dataprepper.model.plugin.InvalidPluginConfigurationException; -import java.util.ArrayList; import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.stream.Collectors; /** * Represents an extension of the {@link PluginModel} which is specific to Sink @@ -120,10 +118,11 @@ private SinkInternalJsonModel(@JsonProperty("routes") final List routes, private SinkInternalJsonModel(final List routes, final String tagsTargetKey, final List includeKeys, final List excludeKeys, final Map pluginSettings) { super(pluginSettings); this.routes = routes != null ? routes : Collections.emptyList(); - this.includeKeys = includeKeys != null ? preprocessingKeys(includeKeys) : Collections.emptyList(); - this.excludeKeys = excludeKeys != null ? preprocessingKeys(excludeKeys) : Collections.emptyList(); + this.includeKeys = includeKeys != null ? includeKeys : Collections.emptyList(); + this.excludeKeys = excludeKeys != null ? excludeKeys : Collections.emptyList(); this.tagsTargetKey = tagsTargetKey; validateConfiguration(); + validateKeys(); } void validateConfiguration() { @@ -132,24 +131,18 @@ void validateConfiguration() { } } - /** - * Pre-processes a list of Keys and returns a sorted list - * The keys must start with `/` and not end with `/` - * - * @param keys a list of raw keys - * @return a sorted processed keys + * Validates both include and exclude keys if they contain / */ - private List preprocessingKeys(final List keys) { - if (keys.contains("/")) { - return new ArrayList<>(); - } - List result = keys.stream() - .map(k -> k.startsWith("/") ? k : "/" + k) - .map(k -> k.endsWith("/") ? k.substring(0, k.length() - 1) : k) - .collect(Collectors.toList()); - Collections.sort(result); - return result; + private void validateKeys() { + includeKeys.forEach(key -> { + if(key.contains("/")) + throw new InvalidPluginConfigurationException("include_keys cannot contain /"); + }); + excludeKeys.forEach(key -> { + if(key.contains("/")) + throw new InvalidPluginConfigurationException("exclude_keys cannot contain /"); + }); } } diff --git a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/sink/OutputCodecContext.java b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/sink/OutputCodecContext.java index d9cc83c9cb..f784233d49 100644 --- a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/sink/OutputCodecContext.java +++ b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/sink/OutputCodecContext.java @@ -7,6 +7,7 @@ import java.util.Collections; import java.util.List; +import java.util.function.Predicate; /** * Data Prepper Output Codec Context class. @@ -18,6 +19,7 @@ public class OutputCodecContext { private final List includeKeys; private final List excludeKeys; + private final Predicate inclusionPredicate; public OutputCodecContext() { this(null, Collections.emptyList(), Collections.emptyList()); @@ -28,6 +30,14 @@ public OutputCodecContext(String tagsTargetKey, List includeKeys, List includeKeys.contains(k); + } else if (excludeKeys != null && !excludeKeys.isEmpty()) { + inclusionPredicate = k -> !excludeKeys.contains(k); + } else { + inclusionPredicate = k -> true; + } } @@ -49,4 +59,8 @@ public List getIncludeKeys() { public List getExcludeKeys() { return excludeKeys; } + + public boolean shouldIncludeKey(String key) { + return inclusionPredicate.test(key); + } } 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 index 4c70ef1b18..193f0ff196 100644 --- 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 @@ -1,8 +1,8 @@ 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.mockito.invocation.InvocationOnMock; import org.opensearch.dataprepper.model.event.DefaultEventMetadata; import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.model.event.EventMetadata; @@ -19,12 +19,17 @@ import java.util.Set; import java.util.UUID; +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.Assert.assertNotEquals; +import static org.mockito.Mockito.mock; public class OutputCodecTest { + @Test + void isCompressionInternal_returns_false() { + OutputCodec objectUnderTest = mock(OutputCodec.class, InvocationOnMock::callRealMethod); - @BeforeEach - public void setUp() { + assertThat(objectUnderTest.isCompressionInternal(), equalTo(false)); } @Test diff --git a/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/configuration/SinkModelTest.java b/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/configuration/SinkModelTest.java index a5fc6363cb..9d895ecd32 100644 --- a/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/configuration/SinkModelTest.java +++ b/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/configuration/SinkModelTest.java @@ -144,27 +144,41 @@ void serialize_with_just_pluginModel() throws IOException { } @Test - void sinkModel_with_include_keys() throws IOException { + void sinkModel_with_include_keys() { final Map pluginSettings = new LinkedHashMap<>(); - final SinkModel sinkModel = new SinkModel("customSinkPlugin", Arrays.asList("routeA", "routeB"), null, Arrays.asList("bcd", "/abc", "efg/"), null, pluginSettings); + final SinkModel sinkModel = new SinkModel("customSinkPlugin", Arrays.asList("routeA", "routeB"), null, Arrays.asList("bcd", "abc", "efg"), null, pluginSettings); assertThat(sinkModel.getExcludeKeys(), equalTo(new ArrayList())); - assertThat(sinkModel.getIncludeKeys(), equalTo(Arrays.asList("/abc", "/bcd", "/efg"))); + assertThat(sinkModel.getIncludeKeys(), equalTo(Arrays.asList("bcd", "abc", "efg"))); } @Test - void sinkModel_with_exclude_keys() throws IOException { + void sinkModel_with_invalid_include_keys() { final Map pluginSettings = new LinkedHashMap<>(); - final SinkModel sinkModel = new SinkModel("customSinkPlugin", Arrays.asList("routeA", "routeB"), null, List.of("/"), Arrays.asList("bcd", "/abc", "efg/"), pluginSettings); + assertThrows(InvalidPluginConfigurationException.class, () -> new SinkModel("customSinkPlugin", Arrays.asList("routeA", "routeB"), null, List.of("/bcd"), List.of(), pluginSettings)); + } + + @Test + void sinkModel_with_exclude_keys() { + final Map pluginSettings = new LinkedHashMap<>(); + final SinkModel sinkModel = new SinkModel("customSinkPlugin", Arrays.asList("routeA", "routeB"), null, List.of(), Arrays.asList("abc", "bcd", "efg"), pluginSettings); assertThat(sinkModel.getIncludeKeys(), equalTo(new ArrayList())); - assertThat(sinkModel.getExcludeKeys(), equalTo(Arrays.asList("/abc", "/bcd", "/efg"))); + assertThat(sinkModel.getExcludeKeys(), equalTo(Arrays.asList("abc", "bcd", "efg"))); + + } + @Test + void sinkModel_with_invalid_exclude_keys() { + final Map pluginSettings = new LinkedHashMap<>(); + assertThrows(InvalidPluginConfigurationException.class, () -> new SinkModel("customSinkPlugin", Arrays.asList("routeA", "routeB"), null, List.of(), List.of("/bcd"), pluginSettings)); } + + @Test - void sinkModel_with_both_include_and_exclude_keys() throws IOException { + void sinkModel_with_both_include_and_exclude_keys() { final Map pluginSettings = new LinkedHashMap<>(); assertThrows(InvalidPluginConfigurationException.class, () -> new SinkModel("customSinkPlugin", Arrays.asList("routeA", "routeB"), null, List.of("abc"), List.of("bcd"), pluginSettings)); } diff --git a/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/sink/OutputCodecContextTest.java b/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/sink/OutputCodecContextTest.java index db1ec8cf86..964559afff 100644 --- a/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/sink/OutputCodecContextTest.java +++ b/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/sink/OutputCodecContextTest.java @@ -10,14 +10,13 @@ import java.util.Collections; import java.util.List; +import java.util.UUID; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.equalTo; import static org.junit.jupiter.api.Assertions.assertNull; public class OutputCodecContextTest { - - @Test public void testOutputCodecContextBasic() { final String testTagsTargetKey = RandomStringUtils.randomAlphabetic(6); @@ -32,8 +31,6 @@ public void testOutputCodecContextBasic() { assertNull(emptyContext.getTagsTargetKey()); assertThat(emptyContext.getIncludeKeys(), equalTo(testIncludeKeys)); assertThat(emptyContext.getExcludeKeys(), equalTo(testExcludeKeys)); - - } @Test @@ -53,7 +50,43 @@ public void testOutputCodecContextAdapter() { assertNull(emptyContext.getTagsTargetKey()); assertThat(emptyContext.getIncludeKeys(), equalTo(testIncludeKeys)); assertThat(emptyContext.getExcludeKeys(), equalTo(testExcludeKeys)); + } + + @Test + void shouldIncludeKey_returns_expected_when_no_include_exclude() { + OutputCodecContext objectUnderTest = new OutputCodecContext(null, null, null); + assertThat(objectUnderTest.shouldIncludeKey(UUID.randomUUID().toString()), equalTo(true)); + } + + @Test + void shouldIncludeKey_returns_expected_when_empty_lists_for_include_exclude() { + OutputCodecContext objectUnderTest = new OutputCodecContext(null, Collections.emptyList(), Collections.emptyList()); + assertThat(objectUnderTest.shouldIncludeKey(UUID.randomUUID().toString()), equalTo(true)); + } + + @Test + void shouldIncludeKey_returns_expected_when_includeKey() { + String includeKey1 = UUID.randomUUID().toString(); + String includeKey2 = UUID.randomUUID().toString(); + final List includeKeys = List.of(includeKey1, includeKey2); + + OutputCodecContext objectUnderTest = new OutputCodecContext(null, includeKeys, null); + + assertThat(objectUnderTest.shouldIncludeKey(includeKey1), equalTo(true)); + assertThat(objectUnderTest.shouldIncludeKey(includeKey2), equalTo(true)); + assertThat(objectUnderTest.shouldIncludeKey(UUID.randomUUID().toString()), equalTo(false)); + } + + @Test + void shouldIncludeKey_returns_expected_when_excludeKey() { + String excludeKey1 = UUID.randomUUID().toString(); + String excludeKey2 = UUID.randomUUID().toString(); + final List excludeKeys = List.of(excludeKey1, excludeKey2); + OutputCodecContext objectUnderTest = new OutputCodecContext(null, null, excludeKeys); + assertThat(objectUnderTest.shouldIncludeKey(excludeKey1), equalTo(false)); + assertThat(objectUnderTest.shouldIncludeKey(excludeKey2), equalTo(false)); + assertThat(objectUnderTest.shouldIncludeKey(UUID.randomUUID().toString()), equalTo(true)); } } diff --git a/data-prepper-core/src/main/java/org/opensearch/dataprepper/breaker/HeapCircuitBreaker.java b/data-prepper-core/src/main/java/org/opensearch/dataprepper/breaker/HeapCircuitBreaker.java index 827f3c9567..dc34bd5cd0 100644 --- a/data-prepper-core/src/main/java/org/opensearch/dataprepper/breaker/HeapCircuitBreaker.java +++ b/data-prepper-core/src/main/java/org/opensearch/dataprepper/breaker/HeapCircuitBreaker.java @@ -66,7 +66,7 @@ class HeapCircuitBreaker implements InnerCircuitBreaker, AutoCloseable { scheduledExecutorService .scheduleAtFixedRate(this::checkMemory, 0L, checkInterval.toMillis(), TimeUnit.MILLISECONDS); - LOG.info("Heap circuit breaker with usage of {} bytes.", usageBytes); + LOG.info("Circuit breaker heap limit is set to {} bytes.", usageBytes); } @Override diff --git a/data-prepper-core/src/main/java/org/opensearch/dataprepper/pipeline/Pipeline.java b/data-prepper-core/src/main/java/org/opensearch/dataprepper/pipeline/Pipeline.java index f47a4da2ca..f41ab33d7c 100644 --- a/data-prepper-core/src/main/java/org/opensearch/dataprepper/pipeline/Pipeline.java +++ b/data-prepper-core/src/main/java/org/opensearch/dataprepper/pipeline/Pipeline.java @@ -274,7 +274,7 @@ public synchronized void shutdown() { stopRequested.set(true); } catch (Exception ex) { LOG.error("Pipeline [{}] - Encountered exception while stopping the source, " + - "proceeding with termination of process workers", name); + "proceeding with termination of process workers", name, ex); } shutdownExecutorService(processorExecutorService, processorShutdownTimeout.toMillis(), "processor"); diff --git a/data-prepper-plugins/aggregate-processor/README.md b/data-prepper-plugins/aggregate-processor/README.md index a932c9a28d..2876aaa4dd 100644 --- a/data-prepper-plugins/aggregate-processor/README.md +++ b/data-prepper-plugins/aggregate-processor/README.md @@ -98,7 +98,7 @@ While not necessary, a great way to set up the Aggregate Processor [identificati The values in a list are merely appended, so there can be duplicates. ### -* `count`: Count Events belonging to the same group and generate a new event with values of the identification keys and the count, indicating the number of events. All Events that make up the combined Event will be dropped. +* `count`: Count Events belonging to the same group and generate a new event with values of the identification keys and the count, indicating the number of events. All Events that make up the combined Event will be dropped. One of the events is added as exemplar. If the aggregation is done on traces, then traceId and spanId are included in the exemplar, otherwise, spanId and traceId would be null. * It supports the following config options * `count_key`: key name to use for storing the count, default name is `aggr._count` * `start_time_key`: key name to use for storing the start time, default name is `aggr._start_time` @@ -114,7 +114,7 @@ While not necessary, a great way to set up the Aggregate Processor [identificati ``` The following Event will be created and processed by the rest of the pipeline when the group is concluded: ```json - {"isMonotonic":true,"unit":"1","aggregationTemporality":"AGGREGATION_TEMPORALITY_DELTA","kind":"SUM","name":"count","description":"Number of events","startTime":"2022-12-02T19:29:51.245358486Z","time":"2022-12-02T19:30:15.247799684Z","value":3.0,"sourceIp":"127.0.0.1","destinationIp":"192.168.0.1"} + {"isMonotonic":true,"unit":"1","aggregationTemporality":"AGGREGATION_TEMPORALITY_DELTA","kind":"SUM","name":"count","description":"Number of events","startTime":"2022-12-02T19:29:51.245358486Z","time":"2022-12-02T19:30:15.247799684Z","value":3.0,"sourceIp":"127.0.0.1","destinationIp":"192.168.0.1", "exemplars":[{"time":"2022-12-02T19:29:51.245358486Z", "value": 1.0, "attributes":{"sourceIp":"127.0.0.1","destinationIp": "192.168.0.1", "request" : "/index.html"}, "spanId":null, "traceId":null}]} ``` If raw output format is used, the following Event will be created and processed by the rest of the pipeline when the group is concluded: ```json @@ -130,7 +130,7 @@ While not necessary, a great way to set up the Aggregate Processor [identificati ``` ### -* `histogram`: Aggreates events belonging to the same group and generate a new event with values of the identification keys and histogram of the aggregated events based on a configured `key`. The histogram contains the number of events, sum, buckets, bucket counts, and optionally min and max of the values corresponding to the `key`. All events that make up the combined Event will be dropped. +* `histogram`: Aggreates events belonging to the same group and generate a new event with values of the identification keys and histogram of the aggregated events based on a configured `key`. The histogram contains the number of events, sum, buckets, bucket counts, and optionally min and max of the values corresponding to the `key`. All events that make up the combined Event will be dropped. Events corresponding to min and max values are added as exemplars. If the aggregation is done on traces, then traceId and spanId are included in the exemplar, otherwise, spanId and traceId would be null. * It supports the following config options * `key`: name of the field in the events for which histogram needs to be generated * `generated_key_prefix`: key prefix to be used for all the fields created in the aggregated event. This allows the user to make sure that the names of the histogram event does not conflict with the field names in the event @@ -149,7 +149,7 @@ While not necessary, a great way to set up the Aggregate Processor [identificati ``` The following Event will be created and processed by the rest of the pipeline when the group is concluded: ```json - {"max":0.55,"kind":"HISTOGRAM","buckets":[{"min":-3.4028234663852886E38,"max":0.0,"count":0},{"min":0.0,"max":0.25,"count":2},{"min":0.25,"max":0.50,"count":1},{"min":0.50,"max":3.4028234663852886E38,"count":1}],"count":4,"bucketCountsList":[0,2,1,1],"description":"Histogram of latency in the events","sum":1.15,"unit":"seconds","aggregationTemporality":"AGGREGATION_TEMPORALITY_DELTA","min":0.15,"bucketCounts":4,"name":"histogram","startTime":"2022-12-14T06:43:40.848762215Z","explicitBoundsCount":3,"time":"2022-12-14T06:44:04.852564623Z","explicitBounds":[0.0,0.25,0.5],"request":"/index.html","sourceIp": "127.0.0.1", "destinationIp": "192.168.0.1", "key": "latency"} + {"max":0.55,"kind":"HISTOGRAM","buckets":[{"min":-3.4028234663852886E38,"max":0.0,"count":0},{"min":0.0,"max":0.25,"count":2},{"min":0.25,"max":0.50,"count":1},{"min":0.50,"max":3.4028234663852886E38,"count":1}],"count":4,"bucketCountsList":[0,2,1,1],"description":"Histogram of latency in the events","sum":1.15,"unit":"seconds","aggregationTemporality":"AGGREGATION_TEMPORALITY_DELTA","min":0.15,"bucketCounts":4,"name":"histogram","startTime":"2022-12-14T06:43:40.848762215Z","explicitBoundsCount":3,"time":"2022-12-14T06:44:04.852564623Z","explicitBounds":[0.0,0.25,0.5],"request":"/index.html","sourceIp": "127.0.0.1", "destinationIp": "192.168.0.1", "key": "latency", "exemplars": [{"time":"2023-12-14T06:43:43.840684487Z","value":0.50,"attributes":{"sourceIp":"127.0.0.1","destinationIp": "192.168.0.1", "request" : "/index.html", "exemplar_id":"min"},"spanId":null,"traceId":null},{"time":"2022-12-14T06:43:43.844339030Z","value":0.55,"attributes":{"sourceIp":"127.0.0.1","destinationIp": "192.168.0.1", "request" : "/index.html","exemplar_id":"max"},"spanId":null,"traceId":null}]} ``` If raw output format is used, the following event will be created and processed by the rest of the pipeline when the group is concluded: ```json diff --git a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/CountAggregateAction.java b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/CountAggregateAction.java index 61a4ecce90..382d1cc99b 100644 --- a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/CountAggregateAction.java +++ b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/CountAggregateAction.java @@ -6,11 +6,14 @@ package org.opensearch.dataprepper.plugins.processor.aggregate.actions; import org.opensearch.dataprepper.model.metric.JacksonSum; +import org.opensearch.dataprepper.model.metric.Exemplar; +import org.opensearch.dataprepper.model.metric.DefaultExemplar; import org.opensearch.dataprepper.model.annotations.DataPrepperPlugin; import org.opensearch.dataprepper.model.annotations.DataPrepperPluginConstructor; import org.opensearch.dataprepper.plugins.otel.codec.OTelProtoCodec; import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.model.event.JacksonEvent; +import org.opensearch.dataprepper.model.trace.Span; import org.opensearch.dataprepper.plugins.processor.aggregate.AggregateAction; import org.opensearch.dataprepper.plugins.processor.aggregate.AggregateActionInput; import org.opensearch.dataprepper.plugins.processor.aggregate.AggregateActionOutput; @@ -19,6 +22,7 @@ import io.opentelemetry.proto.metrics.v1.AggregationTemporality; import java.time.Instant; +import java.util.ArrayList; import java.util.List; import java.time.ZoneId; import java.time.format.DateTimeFormatter; @@ -42,12 +46,14 @@ public class CountAggregateAction implements AggregateAction { public final String startTimeKey; public final String outputFormat; private long startTimeNanos; + private List exemplarList; @DataPrepperPluginConstructor public CountAggregateAction(final CountAggregateActionConfig countAggregateActionConfig) { this.countKey = countAggregateActionConfig.getCountKey(); this.startTimeKey = countAggregateActionConfig.getStartTimeKey(); this.outputFormat = countAggregateActionConfig.getOutputFormat(); + this.exemplarList = new ArrayList<>(); } private long getTimeNanos(Instant time) { @@ -56,6 +62,24 @@ private long getTimeNanos(Instant time) { return currentTimeNanos; } + public Exemplar createExemplar(final Event event) { + long curTimeNanos = getTimeNanos(Instant.now()); + Map attributes = event.toMap(); + String spanId = null; + String traceId = null; + if (event instanceof Span) { + Span span = (Span)event; + spanId = span.getSpanId(); + traceId = span.getTraceId(); + } + return new DefaultExemplar( + OTelProtoCodec.convertUnixNanosToISO8601(curTimeNanos), + 1.0, + spanId, + traceId, + attributes); + } + @Override public AggregateActionResponse handleEvent(final Event event, final AggregateActionInput aggregateActionInput) { final GroupState groupState = aggregateActionInput.getGroupState(); @@ -63,6 +87,7 @@ public AggregateActionResponse handleEvent(final Event event, final AggregateAct groupState.put(startTimeKey, Instant.now()); groupState.putAll(aggregateActionInput.getIdentificationKeys()); groupState.put(countKey, 1); + exemplarList.add(createExemplar(event)); } else { Integer v = (Integer)groupState.get(countKey) + 1; groupState.put(countKey, v); @@ -98,6 +123,7 @@ public AggregateActionOutput concludeGroup(final AggregateActionInput aggregateA .withUnit(SUM_METRIC_UNIT) .withAggregationTemporality(AggregationTemporality.AGGREGATION_TEMPORALITY_DELTA.name()) .withValue((double)countValue) + .withExemplars(exemplarList) .withAttributes(attr) .build(false); event = (Event)sum; diff --git a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/HistogramAggregateAction.java b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/HistogramAggregateAction.java index f535a10e7c..6db82130ea 100644 --- a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/HistogramAggregateAction.java +++ b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/HistogramAggregateAction.java @@ -7,6 +7,9 @@ import org.opensearch.dataprepper.model.metric.JacksonHistogram; import org.opensearch.dataprepper.model.metric.Bucket; +import org.opensearch.dataprepper.model.metric.Exemplar; +import org.opensearch.dataprepper.model.trace.Span; +import org.opensearch.dataprepper.model.metric.DefaultExemplar; import org.opensearch.dataprepper.model.annotations.DataPrepperPlugin; import org.opensearch.dataprepper.model.annotations.DataPrepperPluginConstructor; import org.opensearch.dataprepper.plugins.otel.codec.OTelProtoCodec; @@ -25,6 +28,7 @@ import java.time.ZoneId; import java.time.format.DateTimeFormatter; import java.util.Map; +import java.util.Objects; import java.util.List; import java.util.HashMap; import java.util.Arrays; @@ -53,6 +57,11 @@ public class HistogramAggregateAction implements AggregateAction { private final String key; private final String units; private final boolean recordMinMax; + private List exemplarList; + private Event minEvent; + private Event maxEvent; + private double minValue; + private double maxValue; private long startTimeNanos; private double[] buckets; @@ -62,6 +71,7 @@ public HistogramAggregateAction(final HistogramAggregateActionConfig histogramAg this.key = histogramAggregateActionConfig.getKey(); List bucketList = histogramAggregateActionConfig.getBuckets(); this.buckets = new double[bucketList.size()+2]; + this.exemplarList = new ArrayList<>(); int bucketIdx = 0; this.buckets[bucketIdx++] = -Float.MAX_VALUE; for (int i = 0; i < bucketList.size(); i++) { @@ -101,6 +111,26 @@ private double convertToDouble(Number value) { return doubleValue; } + public Exemplar createExemplar(final String id, final Event event, double value) { + long curTimeNanos = getTimeNanos(Instant.now()); + Map attributes = event.toMap(); + if (Objects.nonNull(id)) { + attributes.put("exemplarId", id); + } + String spanId = null; + String traceId = null; + if (event instanceof Span) { + Span span = (Span)event; + spanId = span.getSpanId(); + traceId = span.getTraceId(); + } + return new DefaultExemplar(OTelProtoCodec.convertUnixNanosToISO8601(curTimeNanos), + value, + spanId, + traceId, + attributes); + } + @Override public AggregateActionResponse handleEvent(final Event event, final AggregateActionInput aggregateActionInput) { final GroupState groupState = aggregateActionInput.getGroupState(); @@ -126,6 +156,10 @@ public AggregateActionResponse handleEvent(final Event event, final AggregateAct if (this.recordMinMax) { groupState.put(minKey, doubleValue); groupState.put(maxKey, doubleValue); + minEvent = event; + maxEvent = event; + minValue = doubleValue; + maxValue = doubleValue; } } else { Integer v = (Integer)groupState.get(countKey) + 1; @@ -138,10 +172,14 @@ public AggregateActionResponse handleEvent(final Event event, final AggregateAct double min = (double)groupState.get(minKey); if (doubleValue < min) { groupState.put(minKey, doubleValue); + minEvent = event; + minValue = doubleValue; } double max = (double)groupState.get(maxKey); if (doubleValue > max) { groupState.put(maxKey, doubleValue); + maxEvent = event; + maxValue = doubleValue; } } } @@ -159,6 +197,8 @@ public AggregateActionOutput concludeGroup(final AggregateActionInput aggregateA long startTimeNanos = getTimeNanos(startTime); long endTimeNanos = getTimeNanos(endTime); String histogramKey = HISTOGRAM_METRIC_NAME + "_key"; + exemplarList.add(createExemplar("min", minEvent, minValue)); + exemplarList.add(createExemplar("max", maxEvent, maxValue)); if (outputFormat.equals(OutputFormat.RAW.toString())) { groupState.put(histogramKey, key); groupState.put(durationKey, endTimeNanos-startTimeNanos); @@ -203,6 +243,7 @@ public AggregateActionOutput concludeGroup(final AggregateActionInput aggregateA .withBuckets(buckets) .withBucketCountsList(bucketCounts) .withExplicitBoundsList(explicitBoundsList) + .withExemplars(exemplarList) .withAttributes(attr) .build(false); event = (Event)histogram; diff --git a/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/CountAggregateActionTest.java b/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/CountAggregateActionTest.java index 9e35507a2a..0f96a83c41 100644 --- a/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/CountAggregateActionTest.java +++ b/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/CountAggregateActionTest.java @@ -9,6 +9,7 @@ import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.model.event.JacksonEvent; import org.opensearch.dataprepper.model.metric.JacksonMetric; +import org.opensearch.dataprepper.model.metric.Exemplar; import org.junit.jupiter.api.extension.ExtendWith; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ValueSource; @@ -108,5 +109,7 @@ void testCountAggregateOTelFormat(int testCount) { assertThat(metric.toJsonString().indexOf("attributes"), not(-1)); assertThat(result.get(0).toMap(), hasKey("startTime")); assertThat(result.get(0).toMap(), hasKey("time")); + List exemplars = (List )result.get(0).toMap().get("exemplars"); + assertThat(exemplars.size(), equalTo(1)); } } diff --git a/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/HistogramAggregateActionTests.java b/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/HistogramAggregateActionTests.java index 6e9d783998..b2b498306b 100644 --- a/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/HistogramAggregateActionTests.java +++ b/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/HistogramAggregateActionTests.java @@ -9,6 +9,7 @@ import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.model.event.JacksonEvent; import org.opensearch.dataprepper.model.metric.JacksonMetric; +import org.opensearch.dataprepper.model.metric.Exemplar; import org.junit.jupiter.api.extension.ExtendWith; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ValueSource; @@ -207,6 +208,8 @@ void testHistogramAggregateOTelFormat(int testCount) throws NoSuchFieldException assertThat(expectedBucketCounts[i], equalTo(bucketCountsFromResult.get(i))); } assertThat(((Map)result.get(0).toMap().get("attributes")), hasEntry(HistogramAggregateAction.HISTOGRAM_METRIC_NAME+"_key", testKey)); + List exemplars = (List )result.get(0).toMap().get("exemplars"); + assertThat(exemplars.size(), equalTo(2)); assertThat(((Map)result.get(0).toMap().get("attributes")), hasEntry(dataKey, dataValue)); final String expectedDurationKey = histogramAggregateActionConfig.getDurationKey(); assertThat(((Map)result.get(0).toMap().get("attributes")), hasKey(expectedDurationKey)); diff --git a/data-prepper-plugins/anomaly-detector-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/anomalydetector/AnomalyDetectorProcessor.java b/data-prepper-plugins/anomaly-detector-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/anomalydetector/AnomalyDetectorProcessor.java index 0eb33f979f..28bba890b1 100644 --- a/data-prepper-plugins/anomaly-detector-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/anomalydetector/AnomalyDetectorProcessor.java +++ b/data-prepper-plugins/anomaly-detector-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/anomalydetector/AnomalyDetectorProcessor.java @@ -6,6 +6,7 @@ package org.opensearch.dataprepper.plugins.processor.anomalydetector; +import io.micrometer.core.instrument.Counter; import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.annotations.DataPrepperPlugin; import org.opensearch.dataprepper.model.annotations.DataPrepperPluginConstructor; @@ -17,7 +18,11 @@ import org.opensearch.dataprepper.model.processor.Processor; import org.opensearch.dataprepper.model.plugin.PluginFactory; import org.opensearch.dataprepper.plugins.hasher.IdentificationKeysHasher; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import java.time.Instant; +import java.time.temporal.ChronoUnit; import java.util.Collection; import java.util.HashMap; import java.util.LinkedList; @@ -30,15 +35,19 @@ public class AnomalyDetectorProcessor extends AbstractProcessor, R public static final String DEVIATION_KEY = "deviation_from_expected"; public static final String GRADE_KEY = "grade"; static final String NUMBER_RCF_INSTANCES = "RCFInstances"; + static final String CARDINALITY_OVERFLOW = "cardinalityOverflow"; private final Boolean verbose; + private final int cardinalityLimit; private final IdentificationKeysHasher identificationKeysHasher; private final List keys; private final PluginFactory pluginFactory; private final HashMap forestMap; private final AtomicInteger cardinality; private final AnomalyDetectorProcessorConfig anomalyDetectorProcessorConfig; - + private static final Logger LOG = LoggerFactory.getLogger(AnomalyDetectorProcessor.class); + private final Counter cardinalityOverflowCounter; + Instant nextWarnTime = Instant.MIN; @DataPrepperPluginConstructor public AnomalyDetectorProcessor(final AnomalyDetectorProcessorConfig anomalyDetectorProcessorConfig, final PluginMetrics pluginMetrics, final PluginFactory pluginFactory) { super(pluginMetrics); @@ -48,6 +57,8 @@ public AnomalyDetectorProcessor(final AnomalyDetectorProcessorConfig anomalyDete this.keys = anomalyDetectorProcessorConfig.getKeys(); this.verbose = anomalyDetectorProcessorConfig.getVerbose(); this.cardinality = pluginMetrics.gauge(NUMBER_RCF_INSTANCES, new AtomicInteger()); + this.cardinalityLimit = anomalyDetectorProcessorConfig.getCardinalityLimit(); + this.cardinalityOverflowCounter = pluginMetrics.counter(CARDINALITY_OVERFLOW); forestMap = new HashMap<>(); } @@ -68,12 +79,20 @@ public Collection> doExecute(Collection> records) { final IdentificationKeysHasher.IdentificationKeysMap identificationKeysMap = identificationKeysHasher.createIdentificationKeysMapFromEvent(event); AnomalyDetectorMode forest = forestMap.get(identificationKeysMap.hashCode()); - if (Objects.isNull(forest)) { + if (Objects.nonNull(forest)) { + recordsOut.addAll(forest.handleEvents(List.of(record))); + } else if (forestMap.size() < cardinalityLimit) { forest = loadAnomalyDetectorMode(pluginFactory); forest.initialize(keys, verbose); forestMap.put(identificationKeysMap.hashCode(), forest); + recordsOut.addAll(forest.handleEvents(List.of(record))); + } else { + if (Instant.now().isAfter(nextWarnTime)) { + LOG.warn("Cardinality limit reached, see cardinalityOverflow metric for count of skipped records"); + nextWarnTime = Instant.now().plus(5, ChronoUnit.MINUTES); + } + cardinalityOverflowCounter.increment(); } - recordsOut.addAll(forestMap.get(identificationKeysMap.hashCode()).handleEvents(List.of(record))); } cardinality.set(forestMap.size()); return recordsOut; diff --git a/data-prepper-plugins/anomaly-detector-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/anomalydetector/AnomalyDetectorProcessorConfig.java b/data-prepper-plugins/anomaly-detector-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/anomalydetector/AnomalyDetectorProcessorConfig.java index 7e796e660a..6331ee1f21 100644 --- a/data-prepper-plugins/anomaly-detector-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/anomalydetector/AnomalyDetectorProcessorConfig.java +++ b/data-prepper-plugins/anomaly-detector-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/anomalydetector/AnomalyDetectorProcessorConfig.java @@ -28,6 +28,9 @@ public class AnomalyDetectorProcessorConfig { @JsonProperty("verbose") private Boolean verbose = false; + @JsonProperty("cardinality_limit") + private int cardinalityLimit = 5000; + public PluginModel getDetectorMode() { return detectorMode; } @@ -47,6 +50,9 @@ public List getIdentificationKeys() { public boolean getVerbose() { return verbose; } + public int getCardinalityLimit() { + return cardinalityLimit; + } } diff --git a/data-prepper-plugins/anomaly-detector-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/anomalydetector/AnomalyDetectorProcessorTests.java b/data-prepper-plugins/anomaly-detector-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/anomalydetector/AnomalyDetectorProcessorTests.java index 302a692dd7..c7ab98a4d4 100644 --- a/data-prepper-plugins/anomaly-detector-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/anomalydetector/AnomalyDetectorProcessorTests.java +++ b/data-prepper-plugins/anomaly-detector-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/anomalydetector/AnomalyDetectorProcessorTests.java @@ -55,6 +55,8 @@ public class AnomalyDetectorProcessorTests { @Mock private Counter recordsOut; + @Mock + private Counter cardinalityOverflow; @Mock private Timer timeElapsed; @@ -77,6 +79,7 @@ void setUp() { RandomCutForestModeConfig randomCutForestModeConfig = new RandomCutForestModeConfig(); when(mockConfig.getDetectorMode()).thenReturn(modeConfiguration); + when(mockConfig.getCardinalityLimit()).thenReturn(5000); when(modeConfiguration.getPluginName()).thenReturn(UUID.randomUUID().toString()); when(modeConfiguration.getPluginSettings()).thenReturn(Collections.emptyMap()); when(pluginFactory.loadPlugin(eq(AnomalyDetectorMode.class), any(PluginSetting.class))) @@ -215,6 +218,43 @@ void testAnomalyDetectorCardinality() { } + @Test + void testAnomalyDetectorMaxCardinality() { + List identificationKeyList = new ArrayList(); + identificationKeyList.add("ip"); + when(mockConfig.getIdentificationKeys()).thenReturn(identificationKeyList); + when(mockConfig.getCardinalityLimit()).thenReturn(2); + when(pluginMetrics.counter(AnomalyDetectorProcessor.CARDINALITY_OVERFLOW)).thenReturn(cardinalityOverflow); + + anomalyDetectorProcessor = new AnomalyDetectorProcessor(mockConfig, pluginMetrics, pluginFactory); + final int numSamples = 1024; + final List> records = new ArrayList>(); + for (int i = 0; i < numSamples; i++) { + if (i % 2 == 0) { + records.add(getLatencyBytesMessageWithIp(UUID.randomUUID().toString(), ThreadLocalRandom.current().nextDouble(0.5, 0.6), ThreadLocalRandom.current().nextLong(100, 110), "1.1.1.1")); + } else { + records.add(getLatencyBytesMessageWithIp(UUID.randomUUID().toString(), ThreadLocalRandom.current().nextDouble(15.5, 15.6), ThreadLocalRandom.current().nextLong(1000, 1110), "255.255.255.255")); + } + } + // Since limit is 2, the next two IPs will not have anomaly detection + for (int i = 0; i < numSamples; i++) { + if (i % 2 == 0) { + records.add(getLatencyBytesMessageWithIp(UUID.randomUUID().toString(), ThreadLocalRandom.current().nextDouble(0.5, 0.6), ThreadLocalRandom.current().nextLong(100, 110), "2.2.2.2")); + } else { + records.add(getLatencyBytesMessageWithIp(UUID.randomUUID().toString(), ThreadLocalRandom.current().nextDouble(15.5, 15.6), ThreadLocalRandom.current().nextLong(1000, 1110), "254.254.254.254")); + } + } + + anomalyDetectorProcessor.doExecute(records); + + final List> newIpOne = (List>) anomalyDetectorProcessor.doExecute(Collections.singletonList(getLatencyBytesMessageWithIp(UUID.randomUUID().toString(), ThreadLocalRandom.current().nextDouble(1000.5, 1000.6), ThreadLocalRandom.current().nextLong(1000, 1110), "3.3.3.3"))); + assertThat(newIpOne.size(), equalTo(0)); + + final List> newIpTwo = (List>) anomalyDetectorProcessor.doExecute(Collections.singletonList(getLatencyBytesMessageWithIp(UUID.randomUUID().toString(), ThreadLocalRandom.current().nextDouble(1500.5, 1500.6), ThreadLocalRandom.current().nextLong(1000, 1110), "144.123.412.123"))); + assertThat(newIpTwo.size(), equalTo(0)); + + } + static Record buildRecordWithEvent(final Map data) { return new Record<>(JacksonEvent.builder() .withData(data) diff --git a/data-prepper-plugins/avro-codecs/src/main/java/org/opensearch/dataprepper/avro/AvroAutoSchemaGenerator.java b/data-prepper-plugins/avro-codecs/src/main/java/org/opensearch/dataprepper/avro/AvroAutoSchemaGenerator.java new file mode 100644 index 0000000000..533fd60178 --- /dev/null +++ b/data-prepper-plugins/avro-codecs/src/main/java/org/opensearch/dataprepper/avro/AvroAutoSchemaGenerator.java @@ -0,0 +1,91 @@ +package org.opensearch.dataprepper.avro; + +import org.apache.avro.Schema; +import org.apache.avro.SchemaBuilder; +import org.opensearch.dataprepper.model.sink.OutputCodecContext; + +import java.util.List; +import java.util.Map; + +/** + * Generates an Avro schema from Event data. + */ +public class AvroAutoSchemaGenerator { + public Schema autoDetermineSchema(final Map data, + final OutputCodecContext outputCodecContext) { + return autoGenerateRecordSchema(data, outputCodecContext, "Event"); + } + + private Schema autoGenerateRecordSchema(final Map eventData, OutputCodecContext codecContext, String typeName) { + SchemaBuilder.FieldAssembler fieldsAssembler = SchemaBuilder.record(typeName).fields(); + for (final String key : eventData.keySet()) { + if (codecContext != null && codecContext.getExcludeKeys().contains(key)) { + continue; + } + Schema schemaForValue = createSchemaForValue(key, eventData.get(key), codecContext); + fieldsAssembler = fieldsAssembler.name(key).type().unionOf() + .nullType() + .and() + .type(schemaForValue) + .endUnion() + .nullDefault(); + } + return fieldsAssembler.endRecord(); + } + + + private Schema createSchemaForValue(String key, final Object value, OutputCodecContext codecContext) { + if(value == null) + throw new SchemaGenerationException("Unable to auto-generate a schema because a provided value is null. key='" + key + "'."); + if (value instanceof String) { + return SchemaBuilder.builder().stringType(); + } else if (value instanceof Long) { + return SchemaBuilder.builder().longType(); + } else if (value instanceof Integer) { + return SchemaBuilder.builder().intType(); + } else if (value instanceof Float) { + return SchemaBuilder.builder().floatType(); + } else if (value instanceof Double) { + return SchemaBuilder.builder().doubleType(); + } else if (value instanceof Boolean) { + return SchemaBuilder.builder().booleanType(); + } else if (value instanceof Byte[] || value instanceof byte[]) { + return SchemaBuilder.builder().bytesType(); + } else if (value instanceof Map) { + return autoGenerateRecordSchema((Map) value, codecContext, convertFieldNameToTypeName(key)); + } else if (value instanceof List) { + List listProvided = (List) value; + if(listProvided.isEmpty()) { + throw new SchemaGenerationException("Cannot determine the element type for the Avro array because a provided list is empty and has no type information. key='" + key + "'."); + } + Object sampleElement = listProvided.get(0); + return SchemaBuilder.builder() + .array() + .items(nullableType(createSchemaForValue(null, sampleElement, codecContext))); + } + throw new SchemaGenerationException("Unable to auto-generate a schema for field '" + + key + + "' because the the type '" + value.getClass() + "' is not a recognized type for auto-generation."); + } + + private Schema nullableType(Schema schema) { + return SchemaBuilder.unionOf() + .nullType() + .and() + .type(schema) + .endUnion(); + } + + private String convertFieldNameToTypeName(String fieldName) { + char startCharacter = fieldName.charAt(0); + if(Character.isAlphabetic(startCharacter)) { + startCharacter = Character.toUpperCase(startCharacter); + } + + char[] typeChars = new char[fieldName.length()]; + typeChars[0] = startCharacter; + fieldName.getChars(1, fieldName.length(), typeChars, 1); + return new String(typeChars); + } + +} diff --git a/data-prepper-plugins/avro-codecs/src/main/java/org/opensearch/dataprepper/avro/AvroEventConverter.java b/data-prepper-plugins/avro-codecs/src/main/java/org/opensearch/dataprepper/avro/AvroEventConverter.java new file mode 100644 index 0000000000..6366bda3ef --- /dev/null +++ b/data-prepper-plugins/avro-codecs/src/main/java/org/opensearch/dataprepper/avro/AvroEventConverter.java @@ -0,0 +1,58 @@ +package org.opensearch.dataprepper.avro; + +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericRecord; +import org.opensearch.dataprepper.model.sink.OutputCodecContext; + +import java.util.List; +import java.util.Map; + +/** + * Converts an Event into an Avro record. + *

+ * It might be a good idea to consolidate similar logic for input. + */ +public class AvroEventConverter { + + private final SchemaChooser schemaChooser; + + public AvroEventConverter() { + schemaChooser = new SchemaChooser(); + } + + public GenericRecord convertEventDataToAvro(final Schema schema, + final Map eventData, + OutputCodecContext codecContext) { + final GenericRecord avroRecord = new GenericData.Record(schema); + final boolean isExcludeKeyAvailable = !codecContext.getExcludeKeys().isEmpty(); + for (final String key : eventData.keySet()) { + if (isExcludeKeyAvailable && codecContext.getExcludeKeys().contains(key)) { + continue; + } + final Schema.Field field = schema.getField(key); + if (field == null) { + throw new RuntimeException("The event has a key ('" + key + "') which is not included in the schema."); + } + final Object value = schemaMapper(field, eventData.get(key), codecContext); + avroRecord.put(key, value); + } + return avroRecord; + } + + private Object schemaMapper(final Schema.Field field, final Object rawValue, OutputCodecContext codecContext) { + Schema providedSchema = schemaChooser.chooseSchema(field.schema()); + + if (providedSchema.getType() == Schema.Type.RECORD && rawValue instanceof Map) { + return convertEventDataToAvro(providedSchema, (Map) rawValue, codecContext); + } else if (providedSchema.getType() == Schema.Type.ARRAY && rawValue instanceof List) { + GenericData.Array avroArray = + new GenericData.Array<>(((List) rawValue).size(), providedSchema); + for (String element : ((List) rawValue)) { + avroArray.add(element); + } + return avroArray; + } + return rawValue; + } +} diff --git a/data-prepper-plugins/avro-codecs/src/main/java/org/opensearch/dataprepper/avro/SchemaChooser.java b/data-prepper-plugins/avro-codecs/src/main/java/org/opensearch/dataprepper/avro/SchemaChooser.java new file mode 100644 index 0000000000..4ce0c99437 --- /dev/null +++ b/data-prepper-plugins/avro-codecs/src/main/java/org/opensearch/dataprepper/avro/SchemaChooser.java @@ -0,0 +1,21 @@ +package org.opensearch.dataprepper.avro; + +import org.apache.avro.Schema; + +class SchemaChooser { + Schema chooseSchema(Schema providedSchema) { + if(providedSchema.isNullable()) { + return getFirstNonNullable(providedSchema); + } + + return providedSchema; + } + + private Schema getFirstNonNullable(Schema providedSchema) { + return providedSchema.getTypes() + .stream() + .filter(s -> s.getType() != Schema.Type.NULL) + .findFirst() + .orElse(providedSchema); + } +} diff --git a/data-prepper-plugins/avro-codecs/src/main/java/org/opensearch/dataprepper/avro/SchemaGenerationException.java b/data-prepper-plugins/avro-codecs/src/main/java/org/opensearch/dataprepper/avro/SchemaGenerationException.java new file mode 100644 index 0000000000..1940370ab7 --- /dev/null +++ b/data-prepper-plugins/avro-codecs/src/main/java/org/opensearch/dataprepper/avro/SchemaGenerationException.java @@ -0,0 +1,7 @@ +package org.opensearch.dataprepper.avro; + +public class SchemaGenerationException extends RuntimeException { + SchemaGenerationException(String message) { + super(message); + } +} 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 fc5e298be1..f9ce073c2a 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 @@ -6,10 +6,11 @@ import org.apache.avro.Schema; import org.apache.avro.file.DataFileWriter; -import org.apache.avro.generic.GenericData; import org.apache.avro.generic.GenericDatumWriter; import org.apache.avro.generic.GenericRecord; import org.apache.avro.io.DatumWriter; +import org.opensearch.dataprepper.avro.AvroAutoSchemaGenerator; +import org.opensearch.dataprepper.avro.AvroEventConverter; import org.opensearch.dataprepper.model.annotations.DataPrepperPlugin; import org.opensearch.dataprepper.model.annotations.DataPrepperPluginConstructor; import org.opensearch.dataprepper.model.codec.OutputCodec; @@ -20,9 +21,6 @@ import java.io.IOException; import java.io.OutputStream; -import java.nio.charset.StandardCharsets; -import java.util.Arrays; -import java.util.List; import java.util.Map; import java.util.Objects; @@ -32,14 +30,12 @@ */ @DataPrepperPlugin(name = "avro", pluginType = OutputCodec.class, pluginConfigurationType = AvroOutputCodecConfig.class) public class AvroOutputCodec implements OutputCodec { - - private static final List PRIMITIVE_TYPES = Arrays.asList("int", "long", "string", "float", "double", "bytes"); private static final Logger LOG = LoggerFactory.getLogger(AvroOutputCodec.class); private static final String AVRO = "avro"; - private static final String BASE_SCHEMA_STRING = "{\"type\":\"record\",\"name\":\"AvroRecords\",\"fields\":["; - private static final String END_SCHEMA_STRING = "]}"; private final AvroOutputCodecConfig config; + private final AvroEventConverter avroEventConverter; + private final AvroAutoSchemaGenerator avroAutoSchemaGenerator; private DataFileWriter dataFileWriter; private Schema schema; @@ -51,6 +47,9 @@ public AvroOutputCodec(final AvroOutputCodecConfig config) { Objects.requireNonNull(config); this.config = config; + avroEventConverter = new AvroEventConverter(); + avroAutoSchemaGenerator = new AvroAutoSchemaGenerator(); + if (config.getSchema() != null) { schema = parseSchema(config.getSchema()); } else if (config.getFileLocation() != null) { @@ -76,64 +75,14 @@ public void start(final OutputStream outputStream, final Event event, final Outp } public Schema buildInlineSchemaFromEvent(final Event event) throws IOException { + final Map data; if (codecContext != null && codecContext.getTagsTargetKey() != null) { - return parseSchema(buildSchemaStringFromEventMap(addTagsToEvent(event, codecContext.getTagsTargetKey()).toMap(), false)); + data = addTagsToEvent(event, codecContext.getTagsTargetKey()).toMap(); } else { - return parseSchema(buildSchemaStringFromEventMap(event.toMap(), false)); + data = event.toMap(); } - } - private String buildSchemaStringFromEventMap(final Map eventData, boolean nestedRecordFlag) { - final StringBuilder builder = new StringBuilder(); - int nestedRecordIndex = 1; - if (!nestedRecordFlag) { - builder.append(BASE_SCHEMA_STRING); - } else { - builder.append("{\"type\":\"record\",\"name\":\"" + "NestedRecord" + nestedRecordIndex + "\",\"fields\":["); - nestedRecordIndex++; - } - String fields; - int index = 0; - for (final String key : eventData.keySet()) { - if (codecContext != null && codecContext.getExcludeKeys().contains(key)) { - continue; - } - if (index == 0) { - if (!(eventData.get(key) instanceof Map)) { - fields = "{\"name\":\"" + key + "\",\"type\":\"" + typeMapper(eventData.get(key)) + "\"}"; - } else { - fields = "{\"name\":\"" + key + "\",\"type\":" + typeMapper(eventData.get(key)) + "}"; - } - } else { - if (!(eventData.get(key) instanceof Map)) { - fields = "," + "{\"name\":\"" + key + "\",\"type\":\"" + typeMapper(eventData.get(key)) + "\"}"; - } else { - fields = "," + "{\"name\":\"" + key + "\",\"type\":" + typeMapper(eventData.get(key)) + "}"; - } - } - builder.append(fields); - index++; - } - builder.append(END_SCHEMA_STRING); - return builder.toString(); - } - - private String typeMapper(final Object value) { - if (value instanceof Integer || value.getClass().equals(int.class)) { - return "int"; - } else if (value instanceof Float || value.getClass().equals(float.class)) { - return "float"; - } else if (value instanceof Double || value.getClass().equals(double.class)) { - return "double"; - } else if (value instanceof Long || value.getClass().equals(long.class)) { - return "long"; - } else if (value instanceof Byte[]) { - return "bytes"; - } else if (value instanceof Map) { - return buildSchemaStringFromEventMap((Map) value, true); - } else { - return "string"; - } + return avroAutoSchemaGenerator.autoDetermineSchema(data, codecContext); } @Override @@ -145,13 +94,14 @@ public void complete(final OutputStream outputStream) throws IOException { @Override public void writeEvent(final Event event, final OutputStream outputStream) throws IOException { Objects.requireNonNull(event); + final Map data; if (codecContext.getTagsTargetKey() != null) { - final GenericRecord avroRecord = buildAvroRecord(schema, addTagsToEvent(event, codecContext.getTagsTargetKey()).toMap()); - dataFileWriter.append(avroRecord); + data = addTagsToEvent(event, codecContext.getTagsTargetKey()).toMap(); } else { - final GenericRecord avroRecord = buildAvroRecord(schema, event.toMap()); - dataFileWriter.append(avroRecord); + data = event.toMap(); } + final GenericRecord avroRecord = avroEventConverter.convertEventDataToAvro(schema, data, codecContext); + dataFileWriter.append(avroRecord); } @Override @@ -169,65 +119,6 @@ Schema parseSchema(final String schemaString) { } } - private GenericRecord buildAvroRecord(final Schema schema, final Map eventData) { - final GenericRecord avroRecord = new GenericData.Record(schema); - final boolean isExcludeKeyAvailable = !codecContext.getExcludeKeys().isEmpty(); - for (final String key : eventData.keySet()) { - if (isExcludeKeyAvailable && codecContext.getExcludeKeys().contains(key)) { - continue; - } - final Schema.Field field = schema.getField(key); - if (field == null) { - throw new RuntimeException("The event has a key ('" + key + "') which is not included in the schema."); - } - final Object value = schemaMapper(field, eventData.get(key)); - avroRecord.put(key, value); - } - return avroRecord; - } - - private Object schemaMapper(final Schema.Field field, final Object rawValue) { - Object finalValue = null; - final String fieldType = field.schema().getType().name().toLowerCase(); - if (PRIMITIVE_TYPES.contains(fieldType)) { - switch (fieldType) { - case "string": - finalValue = rawValue.toString(); - break; - case "int": - finalValue = Integer.parseInt(rawValue.toString()); - break; - case "float": - finalValue = Float.parseFloat(rawValue.toString()); - break; - case "double": - finalValue = Double.parseDouble(rawValue.toString()); - break; - case "long": - finalValue = Long.parseLong(rawValue.toString()); - break; - case "bytes": - finalValue = rawValue.toString().getBytes(StandardCharsets.UTF_8); - break; - default: - LOG.error("Unrecognised Field name : '{}' & type : '{}'", field.name(), fieldType); - break; - } - } else { - if (fieldType.equals("record") && rawValue instanceof Map) { - finalValue = buildAvroRecord(field.schema(), (Map) rawValue); - } else if (fieldType.equals("array") && rawValue instanceof List) { - GenericData.Array avroArray = - new GenericData.Array<>(((List) rawValue).size(), field.schema()); - for (String element : ((List) rawValue)) { - avroArray.add(element); - } - finalValue = avroArray; - } - } - return finalValue; - } - private boolean checkS3SchemaValidity() { return config.getBucketName() != null && config.getFileKey() != null && config.getRegion() != null; } diff --git a/data-prepper-plugins/avro-codecs/src/test/java/org/opensearch/dataprepper/avro/AvroAutoSchemaGeneratorTest.java b/data-prepper-plugins/avro-codecs/src/test/java/org/opensearch/dataprepper/avro/AvroAutoSchemaGeneratorTest.java new file mode 100644 index 0000000000..9cdb5c9d69 --- /dev/null +++ b/data-prepper-plugins/avro-codecs/src/test/java/org/opensearch/dataprepper/avro/AvroAutoSchemaGeneratorTest.java @@ -0,0 +1,201 @@ +package org.opensearch.dataprepper.avro; + +import org.apache.avro.Schema; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtensionContext; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.ArgumentsProvider; +import org.junit.jupiter.params.provider.ArgumentsSource; +import org.mockito.Mock; +import org.opensearch.dataprepper.model.sink.OutputCodecContext; + +import java.io.File; +import java.io.InputStream; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Random; +import java.util.UUID; +import java.util.stream.Stream; + +import static org.hamcrest.CoreMatchers.containsString; +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.notNullValue; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.params.provider.Arguments.arguments; +import static org.mockito.Mockito.mock; + +class AvroAutoSchemaGeneratorTest { + @Mock + private OutputCodecContext outputCodecContext; + + private AvroAutoSchemaGenerator createObjectUnderTest() { + return new AvroAutoSchemaGenerator(); + } + + @ParameterizedTest + @ArgumentsSource(PrimitiveClassesToTypesArgumentsProvider.class) + void autoDetermineSchema_with_primitive_type_returns_nullable_of_that_type(Object value, Schema.Type expectedType) { + String key = randomAvroName(); + + Schema schema = createObjectUnderTest().autoDetermineSchema(Map.of(key, value), outputCodecContext); + + assertThat(schema, notNullValue()); + assertThat(schema.getName(), equalTo("Event")); + assertThat(schema.getFields(), notNullValue()); + assertThat(schema.getFields().size(), equalTo(1)); + Schema.Field field = schema.getField(key); + assertThat(field, notNullValue()); + assertThat(field.defaultVal(), equalTo(Schema.NULL_VALUE)); + assertThat(field.schema(), notNullValue()); + assertThat(field.schema().isNullable(), equalTo(true)); + assertThat(field.schema().isUnion(), equalTo(true)); + assertThat(field.schema().getTypes(), notNullValue()); + assertThat(field.schema().getTypes().size(), equalTo(2)); + assertThat(field.schema().getTypes().get(0), notNullValue()); + assertThat(field.schema().getTypes().get(0).getType(), equalTo(Schema.Type.NULL)); + assertThat(field.schema().getTypes().get(1), notNullValue()); + assertThat(field.schema().getTypes().get(1).getType(), equalTo(expectedType)); + } + + @ParameterizedTest + @ArgumentsSource(PrimitiveClassesToTypesArgumentsProvider.class) + void autoDetermineSchema_with_map_returns_nullable_record_with_nullable_primitives(Object primitiveType, Schema.Type expectedType) { + String recordKey = randomAvroName(); + String innerKey = randomAvroName(); + + Map inputMap = Map.of(recordKey, Map.of(innerKey, primitiveType)); + Schema actualEventSchema = createObjectUnderTest().autoDetermineSchema(inputMap, outputCodecContext); + + assertThat(actualEventSchema, notNullValue()); + assertThat(actualEventSchema.getName(), equalTo("Event")); + assertThat(actualEventSchema.getFields(), notNullValue()); + assertThat(actualEventSchema.getFields().size(), equalTo(1)); + Schema.Field field = actualEventSchema.getField(recordKey); + assertThat(field, notNullValue()); + assertThat(field.defaultVal(), equalTo(Schema.NULL_VALUE)); + assertThat(field.schema(), notNullValue()); + assertThat(field.schema().isNullable(), equalTo(true)); + assertThat(field.schema().isUnion(), equalTo(true)); + assertThat(field.schema().getTypes(), notNullValue()); + assertThat(field.schema().getTypes().size(), equalTo(2)); + assertThat(field.schema().getTypes().get(0), notNullValue()); + assertThat(field.schema().getTypes().get(0).getType(), equalTo(Schema.Type.NULL)); + Schema actualRecordSchema = field.schema().getTypes().get(1); + assertThat(actualRecordSchema, notNullValue()); + assertThat(actualRecordSchema.getType(), equalTo(Schema.Type.RECORD)); + + assertThat(actualRecordSchema, notNullValue()); + assertThat(actualRecordSchema.getName(), equalTo(recordKey.replaceFirst("a", "A"))); + assertThat(actualRecordSchema.getFields(), notNullValue()); + assertThat(actualRecordSchema.getFields().size(), equalTo(1)); + Schema.Field actualInnerField = actualRecordSchema.getField(innerKey); + assertThat(actualInnerField, notNullValue()); + assertThat(actualInnerField.defaultVal(), equalTo(Schema.NULL_VALUE)); + assertThat(actualInnerField.schema(), notNullValue()); + assertThat(actualInnerField.schema().isNullable(), equalTo(true)); + assertThat(actualInnerField.schema().isUnion(), equalTo(true)); + assertThat(actualInnerField.schema().getTypes(), notNullValue()); + assertThat(actualInnerField.schema().getTypes().size(), equalTo(2)); + assertThat(actualInnerField.schema().getTypes().get(0), notNullValue()); + assertThat(actualInnerField.schema().getTypes().get(0).getType(), equalTo(Schema.Type.NULL)); + assertThat(actualInnerField.schema().getTypes().get(1), notNullValue()); + assertThat(actualInnerField.schema().getTypes().get(1).getType(), equalTo(expectedType)); + } + + @ParameterizedTest + @ArgumentsSource(PrimitiveClassesToTypesArgumentsProvider.class) + void autoDetermineSchema_with_list_returns_nullable_array_with_nullable_primitives(Object primitiveType, Schema.Type expectedType) { + String arrayKey = randomAvroName(); + + Map inputMap = Map.of(arrayKey, List.of(primitiveType)); + Schema actualEventSchema = createObjectUnderTest().autoDetermineSchema(inputMap, outputCodecContext); + + assertThat(actualEventSchema, notNullValue()); + assertThat(actualEventSchema.getName(), equalTo("Event")); + assertThat(actualEventSchema.getFields(), notNullValue()); + assertThat(actualEventSchema.getFields().size(), equalTo(1)); + Schema.Field field = actualEventSchema.getField(arrayKey); + assertThat(field, notNullValue()); + assertThat(field.defaultVal(), equalTo(Schema.NULL_VALUE)); + assertThat(field.schema(), notNullValue()); + assertThat(field.schema().isNullable(), equalTo(true)); + assertThat(field.schema().isUnion(), equalTo(true)); + assertThat(field.schema().getTypes(), notNullValue()); + assertThat(field.schema().getTypes().size(), equalTo(2)); + assertThat(field.schema().getTypes().get(0), notNullValue()); + assertThat(field.schema().getTypes().get(0).getType(), equalTo(Schema.Type.NULL)); + Schema actualArraySchema = field.schema().getTypes().get(1); + assertThat(actualArraySchema, notNullValue()); + assertThat(actualArraySchema.getType(), equalTo(Schema.Type.ARRAY)); + + assertThat(actualArraySchema, notNullValue()); + Schema actualElementType = actualArraySchema.getElementType(); + assertThat(actualElementType, notNullValue()); + assertThat(actualElementType.isNullable(), equalTo(true)); + assertThat(actualElementType.isUnion(), equalTo(true)); + assertThat(actualElementType.getTypes(), notNullValue()); + assertThat(actualElementType.getTypes().size(), equalTo(2)); + assertThat(actualElementType.getTypes().get(0), notNullValue()); + assertThat(actualElementType.getTypes().get(0).getType(), equalTo(Schema.Type.NULL)); + assertThat(actualElementType.getTypes().get(1), notNullValue()); + assertThat(actualElementType.getTypes().get(1).getType(), equalTo(expectedType)); + } + + @Test + void autoDetermineSchema_with_empty_list_throws() { + String arrayKey = randomAvroName(); + + Map inputMap = Map.of(arrayKey, List.of()); + AvroAutoSchemaGenerator objectUnderTest = createObjectUnderTest(); + SchemaGenerationException actualException = assertThrows(SchemaGenerationException.class, () -> objectUnderTest.autoDetermineSchema(inputMap, outputCodecContext)); + + assertThat(actualException.getMessage(), notNullValue()); + assertThat(actualException.getMessage(), containsString(arrayKey)); + } + + @Test + void autoDetermineSchema_with_null_value_throws() { + String fieldKey = randomAvroName(); + + Map inputMap = Collections.singletonMap(fieldKey, null); + AvroAutoSchemaGenerator objectUnderTest = createObjectUnderTest(); + SchemaGenerationException actualException = assertThrows(SchemaGenerationException.class, () -> objectUnderTest.autoDetermineSchema(inputMap, outputCodecContext)); + + assertThat(actualException.getMessage(), notNullValue()); + assertThat(actualException.getMessage(), containsString(fieldKey)); + assertThat(actualException.getMessage(), containsString("null")); + } + + @ParameterizedTest + @ArgumentsSource(SomeUnknownTypesArgumentsProvider.class) + void autoDetermineSchema_with_unknown_type_throws(Class unknownType) { + Object value = mock(unknownType); + String fieldKey = randomAvroName(); + + AvroAutoSchemaGenerator objectUnderTest = createObjectUnderTest(); + Map inputMap = Map.of(fieldKey, value); + SchemaGenerationException actualException = assertThrows(SchemaGenerationException.class, () -> objectUnderTest.autoDetermineSchema(inputMap, outputCodecContext)); + + assertThat(actualException.getMessage(), notNullValue()); + assertThat(actualException.getMessage(), containsString(fieldKey)); + assertThat(actualException.getMessage(), containsString(value.getClass().toString())); + } + + static class SomeUnknownTypesArgumentsProvider implements ArgumentsProvider { + @Override + public Stream provideArguments(ExtensionContext context) { + return Stream.of( + arguments(Random.class), + arguments(InputStream.class), + arguments(File.class) + ); + } + } + + private static String randomAvroName() { + return "a" + UUID.randomUUID().toString().replaceAll("-", ""); + } +} \ No newline at end of file diff --git a/data-prepper-plugins/avro-codecs/src/test/java/org/opensearch/dataprepper/avro/AvroEventConverterTest.java b/data-prepper-plugins/avro-codecs/src/test/java/org/opensearch/dataprepper/avro/AvroEventConverterTest.java new file mode 100644 index 0000000000..bbe5c20991 --- /dev/null +++ b/data-prepper-plugins/avro-codecs/src/test/java/org/opensearch/dataprepper/avro/AvroEventConverterTest.java @@ -0,0 +1,53 @@ +package org.opensearch.dataprepper.avro; + +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ArgumentsSource; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.dataprepper.model.sink.OutputCodecContext; + +import java.util.Collections; +import java.util.Map; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.notNullValue; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +@ExtendWith(MockitoExtension.class) +class AvroEventConverterTest { + @Mock + private Schema schema; + + @Mock + private OutputCodecContext codecContext; + + @BeforeEach + void setUp() { + when(schema.getType()).thenReturn(Schema.Type.RECORD); + } + + private AvroEventConverter createObjectUnderTest() { + return new AvroEventConverter(); + } + + @ParameterizedTest + @ArgumentsSource(PrimitiveClassesToTypesArgumentsProvider.class) + void convertEventDataToAvro_does_not_need_to_getField_on_empty_map() { + Map data = Collections.emptyMap(); + GenericRecord actualRecord = createObjectUnderTest().convertEventDataToAvro(schema, data, codecContext); + + assertThat(actualRecord, notNullValue()); + assertThat(actualRecord.getSchema(), equalTo(schema)); + + verify(schema, never()).getField(anyString()); + } + +} \ No newline at end of file diff --git a/data-prepper-plugins/avro-codecs/src/test/java/org/opensearch/dataprepper/avro/PrimitiveClassesToTypesArgumentsProvider.java b/data-prepper-plugins/avro-codecs/src/test/java/org/opensearch/dataprepper/avro/PrimitiveClassesToTypesArgumentsProvider.java new file mode 100644 index 0000000000..ad996c6a2b --- /dev/null +++ b/data-prepper-plugins/avro-codecs/src/test/java/org/opensearch/dataprepper/avro/PrimitiveClassesToTypesArgumentsProvider.java @@ -0,0 +1,31 @@ +package org.opensearch.dataprepper.avro; + +import org.apache.avro.Schema; +import org.junit.jupiter.api.extension.ExtensionContext; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.ArgumentsProvider; + +import java.util.Random; +import java.util.UUID; +import java.util.stream.Stream; + +import static org.junit.jupiter.params.provider.Arguments.arguments; + +class PrimitiveClassesToTypesArgumentsProvider implements ArgumentsProvider { + private static final Random RANDOM = new Random(); + + @Override + public Stream provideArguments(ExtensionContext context) { + byte[] bytes = new byte[10]; + RANDOM.nextBytes(bytes); + return Stream.of( + arguments(UUID.randomUUID().toString(), Schema.Type.STRING), + arguments(RANDOM.nextInt(10_000), Schema.Type.INT), + arguments(RANDOM.nextLong(), Schema.Type.LONG), + arguments(RANDOM.nextFloat(), Schema.Type.FLOAT), + arguments(RANDOM.nextDouble(), Schema.Type.DOUBLE), + arguments(RANDOM.nextBoolean(), Schema.Type.BOOLEAN), + arguments(bytes, Schema.Type.BYTES) + ); + } +} diff --git a/data-prepper-plugins/avro-codecs/src/test/java/org/opensearch/dataprepper/avro/SchemaChooserTest.java b/data-prepper-plugins/avro-codecs/src/test/java/org/opensearch/dataprepper/avro/SchemaChooserTest.java new file mode 100644 index 0000000000..bc2971e5bb --- /dev/null +++ b/data-prepper-plugins/avro-codecs/src/test/java/org/opensearch/dataprepper/avro/SchemaChooserTest.java @@ -0,0 +1,130 @@ +package org.opensearch.dataprepper.avro; + +import org.apache.avro.Schema; +import org.apache.avro.SchemaBuilder; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.extension.ExtensionContext; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.ArgumentsProvider; +import org.junit.jupiter.params.provider.ArgumentsSource; +import org.mockito.junit.jupiter.MockitoExtension; + +import java.util.UUID; +import java.util.stream.Stream; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.notNullValue; +import static org.junit.jupiter.params.provider.Arguments.arguments; + +@ExtendWith(MockitoExtension.class) +class SchemaChooserTest { + private SchemaChooser createObjectUnderTest() { + return new SchemaChooser(); + } + + @ParameterizedTest + @ArgumentsSource(PrimitiveTypesArgumentsProvider.class) + void chooseSchema_should_return_non_nullable_when_nullable_with_primitives(Schema.Type primitiveType) { + Schema innerSchema = SchemaBuilder.builder().type(primitiveType.getName()); + Schema schema = SchemaBuilder + .nullable() + .type(innerSchema); + + Schema actualSchema = createObjectUnderTest().chooseSchema(schema); + assertThat(actualSchema, notNullValue()); + assertThat(actualSchema.getType(), equalTo(primitiveType)); + } + + @ParameterizedTest + @ArgumentsSource(PrimitiveTypesArgumentsProvider.class) + void chooseSchema_should_return_non_nullable_when_nullable_with_record(Schema.Type fieldType) { + Schema innerSchema = SchemaBuilder.builder().record(randomAvroName()) + .fields() + .name(randomAvroName()).type(fieldType.getName()).noDefault() + .endRecord(); + Schema schema = SchemaBuilder + .nullable() + .type(innerSchema); + + Schema actualSchema = createObjectUnderTest().chooseSchema(schema); + assertThat(actualSchema, notNullValue()); + assertThat(actualSchema.getType(), equalTo(Schema.Type.RECORD)); + assertThat(actualSchema.getName(), equalTo(innerSchema.getName())); + } + + @ParameterizedTest + @ArgumentsSource(PrimitiveTypesArgumentsProvider.class) + void chooseSchema_should_return_non_nullable_when_nullable_with_array(Schema.Type itemType) { + Schema innerSchema = SchemaBuilder.builder() + .array() + .items(SchemaBuilder.builder().type(itemType.getName())); + + Schema schema = SchemaBuilder + .nullable() + .type(innerSchema); + + Schema actualSchema = createObjectUnderTest().chooseSchema(schema); + assertThat(actualSchema, notNullValue()); + assertThat(actualSchema.getType(), equalTo(Schema.Type.ARRAY)); + assertThat(actualSchema.getElementType(), notNullValue()); + assertThat(actualSchema.getElementType().getType(), equalTo(itemType)); + } + + @ParameterizedTest + @ArgumentsSource(PrimitiveTypesArgumentsProvider.class) + void chooseSchema_with_non_nullable_returns_input_with_non_nullable_primitive_types(Schema.Type primitiveType) { + Schema inputSchema = SchemaBuilder.builder().type(primitiveType.getName()); + + Schema actualSchema = createObjectUnderTest().chooseSchema(inputSchema); + assertThat(actualSchema, notNullValue()); + assertThat(actualSchema.getType(), equalTo(primitiveType)); + } + + @ParameterizedTest + @ArgumentsSource(PrimitiveTypesArgumentsProvider.class) + void chooseSchema_with_non_nullable_returns_input_with_non_nullable_record(Schema.Type fieldType) { + Schema schema = SchemaBuilder.builder().record(randomAvroName()) + .fields() + .name(randomAvroName()).type(fieldType.getName()).noDefault() + .endRecord(); + + Schema actualSchema = createObjectUnderTest().chooseSchema(schema); + assertThat(actualSchema, equalTo(schema)); + } + + @ParameterizedTest + @ArgumentsSource(PrimitiveTypesArgumentsProvider.class) + void chooseSchema_with_non_nullable_returns_input_with_non_nullable_array(Schema.Type itemType) { + Schema schema = SchemaBuilder.builder() + .array() + .items(SchemaBuilder.builder().type(itemType.getName())); + + + Schema actualSchema = createObjectUnderTest().chooseSchema(schema); + assertThat(actualSchema, notNullValue()); + assertThat(actualSchema.getType(), equalTo(Schema.Type.ARRAY)); + assertThat(actualSchema.getElementType(), notNullValue()); + assertThat(actualSchema.getElementType().getType(), equalTo(itemType)); + } + + private static String randomAvroName() { + return "a" + UUID.randomUUID().toString().replaceAll("-", ""); + } + + static class PrimitiveTypesArgumentsProvider implements ArgumentsProvider { + @Override + public Stream provideArguments(ExtensionContext context) { + return Stream.of( + arguments(Schema.Type.STRING), + arguments(Schema.Type.INT), + arguments(Schema.Type.LONG), + arguments(Schema.Type.FLOAT), + arguments(Schema.Type.DOUBLE), + arguments(Schema.Type.BOOLEAN), + arguments(Schema.Type.BYTES) + ); + } + } +} \ No newline at end of file diff --git a/data-prepper-plugins/avro-codecs/src/test/java/org/opensearch/dataprepper/plugins/codec/avro/AvroOutputCodecTest.java b/data-prepper-plugins/avro-codecs/src/test/java/org/opensearch/dataprepper/plugins/codec/avro/AvroOutputCodecTest.java index f632bb88dc..6e297eba6c 100644 --- a/data-prepper-plugins/avro-codecs/src/test/java/org/opensearch/dataprepper/plugins/codec/avro/AvroOutputCodecTest.java +++ b/data-prepper-plugins/avro-codecs/src/test/java/org/opensearch/dataprepper/plugins/codec/avro/AvroOutputCodecTest.java @@ -7,23 +7,21 @@ import org.apache.avro.Schema; import org.apache.avro.SchemaBuilder; import org.apache.avro.file.DataFileStream; +import org.apache.avro.generic.GenericArray; import org.apache.avro.generic.GenericDatumReader; import org.apache.avro.generic.GenericRecord; import org.apache.avro.util.Utf8; -import org.hamcrest.Matchers; 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.model.event.Event; import org.opensearch.dataprepper.model.log.JacksonLog; -import org.opensearch.dataprepper.model.record.Record; import org.opensearch.dataprepper.model.sink.OutputCodecContext; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.IOException; -import java.io.UnsupportedEncodingException; import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Collections; @@ -31,29 +29,35 @@ import java.util.List; import java.util.Map; import java.util.UUID; +import java.util.function.Function; +import java.util.stream.Collectors; +import java.util.stream.IntStream; import static org.hamcrest.CoreMatchers.containsString; +import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.notNullValue; +import static org.hamcrest.CoreMatchers.nullValue; import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; public class AvroOutputCodecTest { - private static final String EXPECTED_SCHEMA_STRING = "{\"type\":\"record\",\"name\":\"AvroRecords\",\"fields\":[{\"name\"" + - ":\"name\",\"type\":\"string\"},{\"name\":\"nestedRecord\",\"type\":{\"type\":\"record\",\"name\":" + - "\"NestedRecord1\",\"fields\":[{\"name\":\"secondFieldInNestedRecord\",\"type\":\"int\"},{\"name\":\"" + - "firstFieldInNestedRecord\",\"type\":\"string\"}]}},{\"name\":\"age\",\"type\":\"int\"}]}"; + private static final String EXPECTED_SCHEMA_STRING = "{\"type\":\"record\",\"name\":\"Event\",\"fields\":" + + "[{\"name\":\"myDouble\",\"type\":[\"null\",\"double\"],\"default\":null},{\"name\":\"myLong\",\"type\":[\"null\",\"long\"],\"default\":null}," + + "{\"name\":\"myArray\",\"type\":[\"null\",{\"type\":\"array\",\"items\":[\"null\",\"string\"]}],\"default\":null}," + + "{\"name\":\"name\",\"type\":[\"null\",\"string\"],\"default\":null}," + + "{\"name\":\"nestedRecord\",\"type\":[\"null\",{\"type\":\"record\",\"name\":\"NestedRecord\",\"fields\":[{\"name\":\"secondFieldInNestedRecord\",\"type\":[\"null\",\"int\"],\"default\":null},{\"name\":\"firstFieldInNestedRecord\",\"type\":[\"null\",\"string\"],\"default\":null}]}],\"default\":null}," + + "{\"name\":\"age\",\"type\":[\"null\",\"int\"],\"default\":null},{\"name\":\"myFloat\",\"type\":[\"null\",\"float\"],\"default\":null}]}"; + public static final int TOTAL_TOP_LEVEL_FIELDS = 7; private AvroOutputCodecConfig config; private ByteArrayOutputStream outputStream; - private static int numberOfRecords; - @BeforeEach void setUp() { config = new AvroOutputCodecConfig(); - config.setSchema(parseSchema().toString()); + config.setSchema(createStandardSchema().toString()); } private AvroOutputCodec createObjectUnderTest() { @@ -62,7 +66,7 @@ private AvroOutputCodec createObjectUnderTest() { @Test void constructor_throws_if_schema_is_invalid() { - String invalidSchema = parseSchema().toString().replaceAll(",", ";"); + String invalidSchema = createStandardSchema().toString().replaceAll(",", ";"); config.setSchema(invalidSchema); RuntimeException actualException = assertThrows(RuntimeException.class, this::createObjectUnderTest); @@ -75,39 +79,132 @@ void constructor_throws_if_schema_is_invalid() { @ParameterizedTest @ValueSource(ints = {1, 2, 10, 100}) void test_happy_case(final int numberOfRecords) throws Exception { - AvroOutputCodecTest.numberOfRecords = numberOfRecords; AvroOutputCodec avroOutputCodec = createObjectUnderTest(); outputStream = new ByteArrayOutputStream(); OutputCodecContext codecContext = new OutputCodecContext(); avroOutputCodec.start(outputStream, null, codecContext); - for (int index = 0; index < numberOfRecords; index++) { - final Event event = getRecord(index).getData(); + List> inputMaps = generateRecords(numberOfRecords); + for (Map inputMap : inputMaps) { + final Event event = createEventRecord(inputMap); avroOutputCodec.writeEvent(event, outputStream); } avroOutputCodec.complete(outputStream); - List actualRecords = createAvroRecordsList(outputStream); + + final List actualAvroRecords = createAvroRecordsList(outputStream); + assertThat(actualAvroRecords.size(), equalTo(numberOfRecords)); + int index = 0; - for (final GenericRecord actualRecord : actualRecords) { + for (final GenericRecord actualRecord : actualAvroRecords) { assertThat(actualRecord, notNullValue()); assertThat(actualRecord.getSchema(), notNullValue()); - Map expectedMap = generateRecords(numberOfRecords).get(index); - Map actualMap = new HashMap(); + final Map expectedMap = inputMaps.get(index); + final Map actualMap = new HashMap<>(); + for (Schema.Field field : actualRecord.getSchema().getFields()) { if (actualRecord.get(field.name()) instanceof GenericRecord) { GenericRecord nestedRecord = (GenericRecord) actualRecord.get(field.name()); actualMap.put(field.name(), convertRecordToMap(nestedRecord)); + } else if(actualRecord.get(field.name()) instanceof GenericArray) { + GenericArray genericArray = (GenericArray) actualRecord.get(field.name()); + actualMap.put(field.name(), genericArray.stream().map(AvroOutputCodecTest::decodeOutputIfEncoded).collect(Collectors.toList())); + } + else { + Object decodedActualOutput = decodeOutputIfEncoded(actualRecord.get(field.name())); + actualMap.put(field.name(), decodedActualOutput); + } + } + assertThat(actualMap, equalTo(expectedMap)); + index++; + } + } + + @ParameterizedTest + @ValueSource(ints = {1, 2, 10, 100}) + void test_happy_case_nullable_records(final int numberOfRecords) throws Exception { + config.setSchema(createStandardSchemaNullable().toString()); + AvroOutputCodec avroOutputCodec = createObjectUnderTest(); + outputStream = new ByteArrayOutputStream(); + OutputCodecContext codecContext = new OutputCodecContext(); + avroOutputCodec.start(outputStream, null, codecContext); + List> inputMaps = generateRecords(numberOfRecords); + for (Map inputMap : inputMaps) { + final Event event = createEventRecord(inputMap); + avroOutputCodec.writeEvent(event, outputStream); + } + avroOutputCodec.complete(outputStream); + + final List actualAvroRecords = createAvroRecordsList(outputStream); + assertThat(actualAvroRecords.size(), equalTo(numberOfRecords)); + + + int index = 0; + for (final GenericRecord actualRecord : actualAvroRecords) { + + assertThat(actualRecord, notNullValue()); + assertThat(actualRecord.getSchema(), notNullValue()); + + final Map expectedMap = inputMaps.get(index); + final Map actualMap = new HashMap<>(); + + for (Schema.Field field : actualRecord.getSchema().getFields()) { + if (field.name().equals("nestedRecord")) { + GenericRecord nestedRecord = (GenericRecord) actualRecord.get(field.name()); + actualMap.put(field.name(), convertRecordToMap(nestedRecord)); + } else if(field.name().equals("myArray")) { + GenericArray genericArray = (GenericArray) actualRecord.get(field.name()); + List items = genericArray.stream() + .map(AvroOutputCodecTest::decodeOutputIfEncoded) + .collect(Collectors.toList()); + actualMap.put(field.name(), items); } else { Object decodedActualOutput = decodeOutputIfEncoded(actualRecord.get(field.name())); actualMap.put(field.name(), decodedActualOutput); } } - assertThat(expectedMap, Matchers.equalTo(actualMap)); + assertThat(actualMap, equalTo(expectedMap)); index++; } } + @ParameterizedTest + @ValueSource(ints = {1, 2, 10, 100}) + void test_happy_case_nullable_records_with_empty_maps(final int numberOfRecords) throws Exception { + config.setSchema(createStandardSchemaNullable().toString()); + AvroOutputCodec objectUnderTest = createObjectUnderTest(); + outputStream = new ByteArrayOutputStream(); + OutputCodecContext codecContext = new OutputCodecContext(); + objectUnderTest.start(outputStream, null, codecContext); + List> inputMaps = generateEmptyRecords(numberOfRecords); + for (Map inputMap : inputMaps) { + final Event event = createEventRecord(inputMap); + objectUnderTest.writeEvent(event, outputStream); + } + objectUnderTest.complete(outputStream); + + final List actualAvroRecords = createAvroRecordsList(outputStream); + assertThat(actualAvroRecords.size(), equalTo(numberOfRecords)); + + + int count = 0; + for (final GenericRecord actualRecord : actualAvroRecords) { + + assertThat(actualRecord, notNullValue()); + assertThat(actualRecord.getSchema(), notNullValue()); + + List fields = actualRecord.getSchema().getFields(); + assertThat(fields.size(), equalTo(TOTAL_TOP_LEVEL_FIELDS)); + for (Schema.Field field : fields) { + Object actualValue = actualRecord.get(field.name()); + assertThat(actualValue, nullValue()); + } + count++; + } + + assertThat(count, equalTo(inputMaps.size())); + } + @Test void writeEvent_throws_exception_when_field_does_not_exist() throws IOException { final Event eventWithInvalidField = mock(Event.class); @@ -128,64 +225,89 @@ void writeEvent_throws_exception_when_field_does_not_exist() throws IOException public void testInlineSchemaBuilder() throws IOException { Schema expectedSchema = new Schema.Parser().parse(EXPECTED_SCHEMA_STRING); AvroOutputCodec avroOutputCodec = createObjectUnderTest(); - numberOfRecords = 1; - Event event = getRecord(0).getData(); + Event event = createEventRecord(generateRecords(1).get(0)); Schema actualSchema = avroOutputCodec.buildInlineSchemaFromEvent(event); - assertThat(actualSchema, Matchers.equalTo(expectedSchema)); + assertThat(actualSchema, equalTo(expectedSchema)); } - 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 Event createEventRecord(final Map eventData) { + return JacksonLog.builder().withData(eventData).build(); } - private static List generateRecords(int numberOfRecords) { - - List recordList = new ArrayList<>(); + private static List> generateRecords(final int numberOfRecords) { + final List> recordList = new ArrayList<>(); for (int rows = 0; rows < numberOfRecords; rows++) { - HashMap eventData = new HashMap<>(); + final Map eventData = new HashMap<>(); eventData.put("name", "Person" + rows); eventData.put("age", rows); - HashMap nestedRecord = new HashMap<>(); + eventData.put("myLong", (long) rows + (long) Integer.MAX_VALUE); + eventData.put("myFloat", rows * 1.5f); + eventData.put("myDouble", rows * 1.89d); + eventData.put("myArray", List.of(UUID.randomUUID().toString(), UUID.randomUUID().toString())); + final Map nestedRecord = new HashMap<>(); nestedRecord.put("firstFieldInNestedRecord", "testString" + rows); nestedRecord.put("secondFieldInNestedRecord", rows); eventData.put("nestedRecord", nestedRecord); - recordList.add((eventData)); - + recordList.add(eventData); } return recordList; } - private static Schema parseSchema() { - Schema innerSchema = parseInnerSchemaForNestedRecord(); - return SchemaBuilder.record("Person") - .fields() - .name("name").type().stringType().noDefault() - .name("age").type().intType().noDefault() - .name("nestedRecord").type(innerSchema).noDefault() - .endRecord(); + private static List> generateEmptyRecords(final int numberOfRecords) { + return IntStream.range(0, numberOfRecords) + .mapToObj(i -> Collections.emptyMap()) + .collect(Collectors.toList()); + } + private static Schema createStandardSchema() { + return createStandardSchema(false); } - private static Schema parseInnerSchemaForNestedRecord() { - return SchemaBuilder - .record("nestedRecord") - .fields() - .name("firstFieldInNestedRecord") - .type(Schema.create(Schema.Type.STRING)) - .noDefault() - .name("secondFieldInNestedRecord") - .type(Schema.create(Schema.Type.INT)) - .noDefault() - .endRecord(); + private static Schema createStandardSchemaNullable() { + return createStandardSchema(true); + } + + private static Schema createStandardSchema( + final boolean useNullable) { + final Function, SchemaBuilder.BaseFieldTypeBuilder> typeModifier; + if(useNullable) { + typeModifier = SchemaBuilder.FieldTypeBuilder::nullable; + } else { + typeModifier = schemaFieldTypeBuilder -> schemaFieldTypeBuilder; + } + SchemaBuilder.FieldAssembler assembler = SchemaBuilder.record("Person") + .fields(); + assembler = typeModifier.apply(assembler.name("name").type()).stringType().noDefault(); + assembler = typeModifier.apply(assembler.name("age").type()).intType().noDefault(); + assembler = typeModifier.apply(assembler.name("myLong").type()).longType().noDefault(); + assembler = typeModifier.apply(assembler.name("myFloat").type()).floatType().noDefault(); + assembler = typeModifier.apply(assembler.name("myDouble").type()).doubleType().noDefault(); + assembler = typeModifier.apply(assembler.name("myArray").type()).array().items().stringType().noDefault(); + final Schema innerSchema = createStandardInnerSchemaForNestedRecord(useNullable, typeModifier); + assembler = assembler.name("nestedRecord").type(innerSchema).noDefault(); + + return assembler.endRecord(); + } + + private static Schema createStandardInnerSchemaForNestedRecord( + boolean useNullable, final Function, SchemaBuilder.BaseFieldTypeBuilder> typeModifier) { + SchemaBuilder.RecordBuilder nestedRecord; + if(useNullable) { + nestedRecord = SchemaBuilder.nullable().record("nestedRecord"); + } else { + nestedRecord = SchemaBuilder.record("nestedRecord"); + } + SchemaBuilder.FieldAssembler assembler = nestedRecord.fields(); + assembler = typeModifier.apply(assembler.name("firstFieldInNestedRecord").type()).stringType().noDefault(); + assembler = typeModifier.apply(assembler.name("secondFieldInNestedRecord").type()).intType().noDefault(); + return assembler.endRecord(); } - private static Object decodeOutputIfEncoded(Object encodedActualOutput) throws UnsupportedEncodingException { + private static Object decodeOutputIfEncoded(Object encodedActualOutput) { if (encodedActualOutput instanceof Utf8) { byte[] utf8Bytes = encodedActualOutput.toString().getBytes(StandardCharsets.UTF_8); return new String(utf8Bytes, StandardCharsets.UTF_8); @@ -197,7 +319,7 @@ private static Object decodeOutputIfEncoded(Object encodedActualOutput) throws U private static List createAvroRecordsList(ByteArrayOutputStream outputStream) throws IOException { final byte[] avroData = outputStream.toByteArray(); ByteArrayInputStream byteArrayInputStream = new ByteArrayInputStream(avroData); - DataFileStream stream = new DataFileStream(byteArrayInputStream, new GenericDatumReader()); + DataFileStream stream = new DataFileStream(byteArrayInputStream, new GenericDatumReader<>()); List actualRecords = new ArrayList<>(); while (stream.hasNext()) { diff --git a/data-prepper-plugins/kafka-plugins/README.md b/data-prepper-plugins/kafka-plugins/README.md index 19153dd4be..31463d45ff 100644 --- a/data-prepper-plugins/kafka-plugins/README.md +++ b/data-prepper-plugins/kafka-plugins/README.md @@ -1,144 +1,11 @@ # Kafka source -This is the Data Prepper Kafka source plugin that reads records from Kafka topic. It uses the consumer API provided by Kafka to read messages from the kafka broker. +This source allows Data Prepper to use Kafka as source. This source reads records from one or more Kafka topics. It uses the consumer API provided by Kafka to read messages from the kafka broker to create DataPrepper events for further processing by the Data Prepper pipeline. +## Basic Usage and Configuration -## Usages +For usage and configuration, please refer to the documentation [here] (https://opensearch.org/docs/2.9/data-prepper/pipelines/configuration/sources/sources/kafka-source). -The Kafka source should be configured as part of Data Prepper pipeline yaml file. - -## Configuration Options - -``` -log-pipeline: - source: - kafka: - bootstrap_servers: - - 127.0.0.1:9093 - topics: - - name: my-topic-1 - workers: 10 - autocommit: false - autocommit_interval: 5s - session_timeout: 45s - max_retry_delay: 1s - auto_offset_reset: earliest - thread_waiting_time: 1s - max_record_fetch_time: 4s - heart_beat_interval: 3s - buffer_default_timeout: 5s - fetch_max_bytes: 52428800 - fetch_max_wait: 500 - fetch_min_bytes: 1 - retry_backoff: 100s - max_poll_interval: 300000s - consumer_max_poll_records: 500 - - name: my-topic-2 - workers: 10 - schema: - registry_url: http://localhost:8081/ - version: 1 - authentication: - sasl_plaintext: - username: admin - password: admin-secret - sasl_oauth: - oauth_client_id: 0oa9wc21447Pc5vsV5d8 - oauth_client_secret: aGmOfHqIEvBJGDxXAOOcatiE9PvsPgoEePx8IPPb - oauth_login_server: https://dev-1365.okta.com - oauth_login_endpoint: /oauth2/default/v1/token - oauth_login_grant_type: refresh_token - oauth_login_scope: kafka - oauth_introspect_server: https://dev-1365.okta.com - oauth_introspect_endpoint: /oauth2/default/v1/introspect - oauth_sasl_mechanism: OAUTHBEARER - oauth_security_protocol: SASL_PLAINTEXT - oauth_sasl_login_callback_handler_class: org.apache.kafka.common.security.oauthbearer.secured.OAuthBearerLoginCallbackHandler - oauth_jwks_endpoint_url: https://dev-1365.okta.com/oauth2/default/v1/keys - sink: - - stdout: - -``` - -## Configuration - -- `bootstrap_servers` (Required) : It is a host/port to use for establishing the initial connection to the Kafka cluster. Multiple brokers can be configured. - -- `topics` (Required) : The topic in which kafka source plugin associated with to read the messages.The maximum number of topics should be 10. - -- `name` (Required) : This denotes the name of the topic, and it is a mandatory one. Multiple list can be configured and the maximum number of topic should be 10. - -- `workers` (Optional) : Number of multithreaded consumers associated with each topic. Defaults to `10` and its maximum value should be 200. - -- `autocommit` (Optional) : If false, the consumer's offset will not be periodically committed in the background. Defaults to `false`. - -- `autocommit_interval` (Optional) : The frequency in seconds that the consumer offsets are auto-committed to Kafka. Defaults to `1s`. - -- `session_timeout` (Optional) : The timeout used to detect client failures when using Kafka's group management. It is used for the rebalance. - -- `max_retry_delay` (Optional) : By default the Kafka source will retry for every 1 second when there is a buffer write error. Defaults to `1s`. - -- `auto_offset_reset` (Optional) : automatically reset the offset to the earliest or latest offset. Defaults to `earliest`. - -- `thread_waiting_time` (Optional) : It is the time for thread to wait until other thread completes the task and signal it. - -- `max_record_fetch_time` (Optional) : maximum time to fetch the record from the topic. -Defaults to `4s`. - -- `heart_beat_interval` (Optional) : The expected time between heartbeats to the consumer coordinator when using Kafka's group management facilities. Defaults to `1s`. - -- `buffer_default_timeout` (Optional) : The maximum time to write data to the buffer. Defaults to `1s`. - -- `fetch_max_bytes` (Optional) : The maximum record batch size accepted by the broker. -Defaults to `52428800`. - -- `fetch_max_wait` (Optional) : The maximum amount of time the server will block before answering the fetch request if there isn't sufficient data to immediately satisfy the requirement. Defaults to `500`. - -- `fetch_min_bytes` (Optional) : The minimum amount of data the server should return for a fetch request. Defaults to `1`. - -- `retry_backoff` (Optional) : The amount of time to wait before attempting to retry a failed request to a given topic partition. Defaults to `5s`. - -- `max_poll_interval` (Optional) : The maximum delay between invocations of poll() when using consumer group management. Defaults to `1s`. - -- `consumer_max_poll_records` (Optional) : The maximum number of records returned in a single call to poll(). Defaults to `1s`. - -### Schema Configuration - -- `registry_url` (Optional) : Deserialize a record value from a bytearray into a String. Defaults to `org.apache.kafka.common.serialization.StringDeserializer`. - -- `version` (Optional) : Deserialize a record key from a bytearray into a String. Defaults to `org.apache.kafka.common.serialization.StringDeserializer`. - -### Auth Configuration for SASL PLAINTEXT - -- `username` (Optional) : The username for the Plaintext authentication. - -- `password` (Optional) : The password for the Plaintext authentication. - -### OAuth Configuration for SASLOAUTH - -- `oauth_client_id`: It is the client id is the public identifier of your authorization server. - -- `oauth_client_secret` : It is a secret known only to the application and the authorization server. - -- `oauth_login_server` : The URL of the OAuth server.(Eg: https://dev.okta.com) - -- `oauth_login_endpoint`: The End point URL of the OAuth server.(Eg: /oauth2/default/v1/token) - -- `oauth_login_grant_type` (Optional) : This grant type refers to the way an application gets an access token. - -- `oauth_login_scope` (Optional) : This scope limit an application's access to a user's account. - -- `oauth_introspect_server` (Optional) : The URL of the introspect server. Most of the cases it should be similar to the oauth_login_server URL (Eg:https://dev.okta.com) - -- `oauth_introspect_endpoint` (Optional) : The end point of the introspect server URL.(Eg: /oauth2/default/v1/introspect) - -- `oauth_sasl_mechanism` (Optional) : It describes the authentication mechanism. - -- `oauth_security_protocol` (Optional) : It is the SASL security protocol like PLAINTEXT or SSL. - -- `oauth_sasl_login_callback_handler_class` (Optional) : It is the user defined or built in Kafka class to handle login and its callbeck. - -- `oauth_jwks_endpoint_url` (Optional) : The absolute URL for the oauth token refresh. ## Integration Tests @@ -161,18 +28,33 @@ ssl.keystore.password= ``` The truststore must have "localhost" certificates in them. -Command to start kafka server +3. Create a file with name `kafka_server_jaas.conf` and with the following contents: +``` +KafkaClient { + org.apache.kafka.common.security.plain.PlainLoginModule required + username="admin" + password="admin" + user_admin="admin"; +}; +``` + +4. Export `KAFKA_OPTS` environment variable +``` +export KAFKA_OPTS=-Djava.security.auth.login.config=kafka_server_jaas.conf +``` + +5. start kafka server ``` bin/kafka-server-start.sh config/server.properties ``` -3. Command to run multi auth type integration tests +6. Command to run multi auth type integration tests ``` -./gradlew data-prepper-plugins:kafka-plugins:integrationTest -Dtests.kafka.bootstrap_servers= -Dtests.kafka.trust_store_location= -Dtests.kafka.trust_store_password= -Dtests.kafka.saslssl_bootstrap_servers= -Dtests.kafka.ssl_bootstrap_servers= -Dtests.kafka.saslplain_bootstrap_servers= -Dtests.kafka.username= -Dtests.kafka.password= --tests "*KafkaSourceMultipleAuthTypeIT*" +./gradlew data-prepper-plugins:kafka-plugins:integrationTest -Dtests.kafka.bootstrap_servers=localhost:9092 -Dtests.kafka.saslssl_bootstrap_servers=localhost:9093 -Dtests.kafka.ssl_bootstrap_servers=localhost:9094 -Dtests.kafka.saslplain_bootstrap_servers=localhost:9095 -Dtests.kafka.username=admin -Dtests.kafka.password=admin --tests "*KafkaSourceMultipleAuthTypeIT*" ``` -4. Command to run msk glue integration tests +7. Command to run msk glue integration tests ``` ./gradlew data-prepper-plugins:kafka-plugins:integrationTest -Dtests.kafka.bootstrap_servers= -Dtests.kafka.glue_registry_name= -Dtests.kafka.glue_avro_schema_name= -Dtests.kafka.glue_json_schema_name= -Dtests.msk.region= -Dtests.msk.arn= --tests "*TestAvroRecordConsumer*" diff --git a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/AuthConfig.java b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/AuthConfig.java index 4447f81a97..fc469950de 100644 --- a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/AuthConfig.java +++ b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/AuthConfig.java @@ -54,25 +54,29 @@ public boolean hasOnlyOneConfig() { } - public static class SslAuthConfig { - // TODO Add Support for SSL authentication types like - // one-way or two-way authentication + /* + * TODO + public static class SslAuthConfig { + // TODO Add Support for SSL authentication types like + // one-way or two-way authentication - public SslAuthConfig() { + public SslAuthConfig() { + } + } + + @JsonProperty("ssl") + private SslAuthConfig sslAuthConfig; + + public SslAuthConfig getSslAuthConfig() { + return sslAuthConfig; } - } - @JsonProperty("ssl") - private SslAuthConfig sslAuthConfig; + */ @Valid @JsonProperty("sasl") private SaslAuthConfig saslAuthConfig; - public SslAuthConfig getSslAuthConfig() { - return sslAuthConfig; - } - public SaslAuthConfig getSaslAuthConfig() { return saslAuthConfig; } diff --git a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/SchemaConfig.java b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/SchemaConfig.java index 049a261e95..8ad509dc56 100644 --- a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/SchemaConfig.java +++ b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/SchemaConfig.java @@ -52,23 +52,23 @@ public class SchemaConfig { @JsonProperty("s3_file_config") private S3FileConfig s3FileConfig; - @JsonProperty("is_create") + @JsonProperty("is_schema_create") @NotNull - private Boolean isCreate; + private Boolean isSchemaCreate=Boolean.FALSE; public static class S3FileConfig { @Valid - @Size(max = 0, message = "Schema from file is not supported.") + @Size(max = 0, message = "bucket is mandatory.") @JsonProperty("bucket_name") private String bucketName; @Valid - @Size(max = 0, message = "Schema from file is not supported.") + @Size(max = 0, message = "file key is mandatory.") @JsonProperty("file_key") private String fileKey; @Valid - @Size(max = 0, message = "Schema from file is not supported.") + @Size(max = 0, message = "region is mandatory") @JsonProperty("region") private String region; @@ -129,15 +129,15 @@ public S3FileConfig getS3FileConfig() { return s3FileConfig; } - @AssertTrue(message = "Only one of Inline schema or Schema file location or S3 file config config must be specified") + @AssertTrue(message = "Only one of Inline schema or Schema file location or S3 file config must be specified") public boolean hasOnlyOneConfig() { - if(isCreate) { + if(isSchemaCreate) { return Stream.of(inlineSchema, schemaFileLocation, s3FileConfig).filter(n -> n != null).count() == 1; } return true; } public Boolean isCreate() { - return isCreate; + return isSchemaCreate; } } 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 d6f2406794..9778c12933 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 @@ -9,6 +9,7 @@ import jakarta.validation.Valid; import jakarta.validation.constraints.NotNull; import jakarta.validation.constraints.Size; +import org.opensearch.dataprepper.model.types.ByteCount; import org.opensearch.dataprepper.plugins.kafka.util.MessageFormat; @@ -21,26 +22,24 @@ public class TopicConfig { static final boolean DEFAULT_AUTO_COMMIT = false; static final Duration DEFAULT_COMMIT_INTERVAL = Duration.ofSeconds(5); static final Duration DEFAULT_SESSION_TIMEOUT = Duration.ofSeconds(45); - static final int DEFAULT_MAX_RETRY_ATTEMPT = Integer.MAX_VALUE; static final String DEFAULT_AUTO_OFFSET_RESET = "earliest"; static final Duration DEFAULT_THREAD_WAITING_TIME = Duration.ofSeconds(5); static final Duration DEFAULT_MAX_RECORD_FETCH_TIME = Duration.ofSeconds(4); - static final Duration DEFAULT_BUFFER_TIMEOUT = Duration.ofSeconds(5); - static final Duration DEFAULT_MAX_RETRY_DELAY = Duration.ofSeconds(1); - static final Integer DEFAULT_FETCH_MAX_BYTES = 52428800; + static final String DEFAULT_FETCH_MAX_BYTES = "50mb"; static final Integer DEFAULT_FETCH_MAX_WAIT = 500; - static final Integer DEFAULT_FETCH_MIN_BYTES = 1; + static final String DEFAULT_FETCH_MIN_BYTES = "1b"; static final Duration DEFAULT_RETRY_BACKOFF = Duration.ofSeconds(10); static final Duration DEFAULT_RECONNECT_BACKOFF = Duration.ofSeconds(10); - static final Integer DEFAULT_MAX_PARTITION_FETCH_BYTES = 1048576; - static final Duration DEFAULT_MAX_POLL_INTERVAL = Duration.ofSeconds(300000); + static final String DEFAULT_MAX_PARTITION_FETCH_BYTES = "1mb"; + static final Duration DEFAULT_MAX_POLL_INTERVAL = Duration.ofSeconds(300); static final Integer DEFAULT_CONSUMER_MAX_POLL_RECORDS = 500; static final Integer DEFAULT_NUM_OF_WORKERS = 2; static final Duration DEFAULT_HEART_BEAT_INTERVAL_DURATION = Duration.ofSeconds(5); - private static final Integer DEFAULT_NUM_OF_PARTITIONS = 3; + private static final Integer DEFAULT_NUM_OF_PARTITIONS = 1; private static final Short DEFAULT_REPLICATION_FACTOR = 1; + private static final Long DEFAULT_RETENTION_PERIOD=604800000L; @JsonProperty("name") @@ -58,16 +57,6 @@ public class TopicConfig { @Size(min = 1, max = 200, message = "Number of worker threads should lies between 1 and 200") private Integer workers = DEFAULT_NUM_OF_WORKERS; - @JsonProperty("max_retry_attempts") - @Valid - @Size(min = 1, max = Integer.MAX_VALUE, message = " Max retry attempts should lies between 1 and Integer.MAX_VALUE") - private Integer maxRetryAttempts = DEFAULT_MAX_RETRY_ATTEMPT; - - @JsonProperty("max_retry_delay") - @Valid - @Size(min = 1) - private Duration maxRetryDelay = DEFAULT_MAX_RETRY_DELAY; - @JsonProperty("serde_format") private MessageFormat serdeFormat= MessageFormat.PLAINTEXT; @@ -87,29 +76,14 @@ public class TopicConfig { @JsonProperty("auto_offset_reset") private String autoOffsetReset = DEFAULT_AUTO_OFFSET_RESET; - @JsonProperty("group_name") - @Valid - @Size(min = 1, max = 255, message = "size of group name should be between 1 and 255") - private String groupName; - @JsonProperty("thread_waiting_time") private Duration threadWaitingTime = DEFAULT_THREAD_WAITING_TIME; - @JsonProperty("max_record_fetch_time") - private Duration maxRecordFetchTime = DEFAULT_MAX_RECORD_FETCH_TIME; - @JsonProperty("max_partition_fetch_bytes") - private Integer maxPartitionFetchBytes = DEFAULT_MAX_PARTITION_FETCH_BYTES; - - @JsonProperty("buffer_default_timeout") - @Valid - @Size(min = 1) - private Duration bufferDefaultTimeout = DEFAULT_BUFFER_TIMEOUT; + private String maxPartitionFetchBytes = DEFAULT_MAX_PARTITION_FETCH_BYTES; @JsonProperty("fetch_max_bytes") - @Valid - @Size(min = 1, max = 52428800) - private Integer fetchMaxBytes = DEFAULT_FETCH_MAX_BYTES; + private String fetchMaxBytes = DEFAULT_FETCH_MAX_BYTES; @JsonProperty("fetch_max_wait") @Valid @@ -117,9 +91,7 @@ public class TopicConfig { private Integer fetchMaxWait = DEFAULT_FETCH_MAX_WAIT; @JsonProperty("fetch_min_bytes") - @Size(min = 1) - @Valid - private Integer fetchMinBytes = DEFAULT_FETCH_MIN_BYTES; + private String fetchMinBytes = DEFAULT_FETCH_MIN_BYTES; @JsonProperty("key_mode") private KafkaKeyMode kafkaKeyMode = KafkaKeyMode.INCLUDE_AS_FIELD; @@ -141,8 +113,8 @@ public class TopicConfig { @Size(min = 1) private Duration heartBeatInterval= DEFAULT_HEART_BEAT_INTERVAL_DURATION; - @JsonProperty("is_create") - private Boolean isCreate=Boolean.FALSE; + @JsonProperty("is_topic_create") + private Boolean isTopicCreate =Boolean.FALSE; @JsonProperty("number_of_partitions") private Integer numberOfPartions = DEFAULT_NUM_OF_PARTITIONS; @@ -150,6 +122,13 @@ public class TopicConfig { @JsonProperty("replication_factor") private Short replicationFactor = DEFAULT_REPLICATION_FACTOR; + @JsonProperty("retention_period") + private Long retentionPeriod=DEFAULT_RETENTION_PERIOD; + + public Long getRetentionPeriod() { + return retentionPeriod; + } + public String getGroupId() { return groupId; } @@ -158,10 +137,6 @@ public void setGroupId(String groupId) { this.groupId = groupId; } - public void setMaxRetryAttempts(Integer maxRetryAttempts) { - this.maxRetryAttempts = maxRetryAttempts; - } - public MessageFormat getSerdeFormat() { return serdeFormat; } @@ -190,14 +165,6 @@ public void setAutoOffsetReset(String autoOffsetReset) { this.autoOffsetReset = autoOffsetReset; } - public String getGroupName() { - return groupName; - } - - public void setGroupName(String groupName) { - this.groupName = groupName; - } - public Duration getThreadWaitingTime() { return threadWaitingTime; } @@ -206,28 +173,16 @@ public void setThreadWaitingTime(Duration threadWaitingTime) { this.threadWaitingTime = threadWaitingTime; } - public Duration getMaxRecordFetchTime() { - return maxRecordFetchTime; - } - - public Integer getMaxPartitionFetchBytes() { - return maxPartitionFetchBytes; - } - - public void setMaxRecordFetchTime(Duration maxRecordFetchTime) { - this.maxRecordFetchTime = maxRecordFetchTime; + public long getMaxPartitionFetchBytes() { + return ByteCount.parse(maxPartitionFetchBytes).getBytes(); } - public Duration getBufferDefaultTimeout() { - return bufferDefaultTimeout; - } - - public void setBufferDefaultTimeout(Duration bufferDefaultTimeout) { - this.bufferDefaultTimeout = bufferDefaultTimeout; - } - - public Integer getFetchMaxBytes() { - return fetchMaxBytes; + public long getFetchMaxBytes() { + long value = ByteCount.parse(fetchMaxBytes).getBytes(); + if (value < 1 || value > 50*1024*1024) { + throw new RuntimeException("Invalid Fetch Max Bytes"); + } + return value; } public void setAutoCommit(Boolean autoCommit) { @@ -238,8 +193,12 @@ public Integer getFetchMaxWait() { return fetchMaxWait; } - public Integer getFetchMinBytes() { - return fetchMinBytes; + public long getFetchMinBytes() { + long value = ByteCount.parse(fetchMinBytes).getBytes(); + if (value < 1) { + throw new RuntimeException("Invalid Fetch Min Bytes"); + } + return value; } public Duration getRetryBackoff() { @@ -278,14 +237,6 @@ public void setWorkers(Integer workers) { this.workers = workers; } - public Duration getMaxRetryDelay() { - return maxRetryDelay; - } - - public void setMaxRetryDelay(Duration maxRetryDelay) { - this.maxRetryDelay = maxRetryDelay; - } - public Duration getHeartBeatInterval() { return heartBeatInterval; } @@ -307,7 +258,7 @@ public KafkaKeyMode getKafkaKeyMode() { } public Boolean isCreate() { - return isCreate; + return isTopicCreate; } public Integer getNumberOfPartions() { 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 805cfb6497..96767016c5 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 @@ -48,6 +48,7 @@ import org.opensearch.dataprepper.plugins.kafka.util.MessageFormat; import org.opensearch.dataprepper.plugins.kafka.util.KafkaTopicMetrics; import com.amazonaws.services.schemaregistry.serializers.json.JsonDataWithSchema; +import com.amazonaws.services.schemaregistry.exception.AWSSchemaRegistryException; import org.apache.commons.lang3.Range; /** @@ -161,10 +162,16 @@ public void consumeRecords() throws Exception { topicMetrics.getNumberOfPollAuthErrors().increment(); Thread.sleep(10000); } catch (RecordDeserializationException e) { - LOG.warn("Deserialization error - topic {} partition {} offset {}, seeking past the error record", - e.topicPartition().topic(), e.topicPartition().partition(), e.offset(), e); + LOG.warn("Deserialization error - topic {} partition {} offset {}", + e.topicPartition().topic(), e.topicPartition().partition(), e.offset()); + if (e.getCause() instanceof AWSSchemaRegistryException) { + LOG.warn("AWSSchemaRegistryException: {}. Retrying after 30 seconds", e.getMessage()); + Thread.sleep(30000); + } else { + LOG.warn("Seeking past the error record", e); + consumer.seek(e.topicPartition(), e.offset()+1); + } topicMetrics.getNumberOfDeserializationErrors().increment(); - consumer.seek(e.topicPartition(), e.offset()+1); } } @@ -173,7 +180,11 @@ private void resetOffsets() { partitionsToReset.forEach(partition -> { try { final OffsetAndMetadata offsetAndMetadata = consumer.committed(partition); - consumer.seek(partition, offsetAndMetadata); + if (Objects.isNull(offsetAndMetadata)) { + consumer.seek(partition, 0L); + } else { + consumer.seek(partition, offsetAndMetadata); + } } catch (Exception e) { LOG.error("Failed to seek to last committed offset upon negative acknowledgement {}", partition, e); } @@ -234,14 +245,20 @@ Map getOffsetsToCommit() { @Override public void run() { consumer.subscribe(Arrays.asList(topicName)); + boolean retryingAfterException = false; while (!shutdownInProgress.get()) { try { + if (retryingAfterException) { + Thread.sleep(10000); + } resetOffsets(); commitOffsets(); consumeRecords(); topicMetrics.update(consumer); + retryingAfterException = false; } catch (Exception exp) { - LOG.error("Error while reading the records from the topic {}", topicName, exp); + LOG.error("Error while reading the records from the topic {}. Retry after 10 seconds", topicName, exp); + retryingAfterException = true; } } } @@ -292,7 +309,7 @@ private Record getRecord(ConsumerRecord consumerRecord, in eventMetadata.setAttribute("kafka_key", key); } eventMetadata.setAttribute("kafka_topic", topicName); - eventMetadata.setAttribute("kafka_partition", partition); + eventMetadata.setAttribute("kafka_partition", String.valueOf(partition)); return new Record(event); } 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 a388d3ee6e..f67720f5db 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 @@ -9,7 +9,7 @@ import io.confluent.kafka.schemaregistry.client.rest.exceptions.RestClientException; import io.confluent.kafka.serializers.KafkaAvroDeserializerConfig; import io.confluent.kafka.serializers.KafkaJsonDeserializer; -import kafka.common.BrokerEndPointNotAvailableException; +//import kafka.common.BrokerEndPointNotAvailableException; import org.apache.avro.generic.GenericRecord; import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.ObjectMapper; @@ -148,12 +148,12 @@ public void start(Buffer> buffer) { executorService.submit(consumer); }); } catch (Exception e) { - if (e instanceof BrokerNotAvailableException || + /* if (e instanceof BrokerNotAvailableException || e instanceof BrokerEndPointNotAvailableException || e instanceof TimeoutException) { LOG.error("The kafka broker is not available..."); } else { LOG.error("Failed to setup the Kafka Source Plugin.", e); - } + }*/ throw new RuntimeException(e); } LOG.info("Started Kafka source for topic " + topic.getName()); @@ -358,7 +358,7 @@ private void setPropertiesForSchemaType(Properties properties, TopicConfig topic private void setConsumerTopicProperties(Properties properties, TopicConfig topicConfig) { properties.put(ConsumerConfig.GROUP_ID_CONFIG, consumerGroupID); - properties.put(ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG, topicConfig.getMaxPartitionFetchBytes()); + properties.put(ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG, (int)topicConfig.getMaxPartitionFetchBytes()); properties.put(ConsumerConfig.RETRY_BACKOFF_MS_CONFIG, ((Long)topicConfig.getRetryBackoff().toMillis()).intValue()); properties.put(ConsumerConfig.RECONNECT_BACKOFF_MS_CONFIG, ((Long)topicConfig.getReconnectBackoff().toMillis()).intValue()); properties.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, @@ -373,9 +373,9 @@ private void setConsumerTopicProperties(Properties properties, TopicConfig topic ((Long)topicConfig.getMaxPollInterval().toMillis()).intValue()); properties.put(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, ((Long)topicConfig.getSessionTimeOut().toMillis()).intValue()); properties.put(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG, ((Long)topicConfig.getHeartBeatInterval().toMillis()).intValue()); - properties.put(ConsumerConfig.FETCH_MAX_BYTES_CONFIG, topicConfig.getFetchMaxBytes().intValue()); + properties.put(ConsumerConfig.FETCH_MAX_BYTES_CONFIG, (int)topicConfig.getFetchMaxBytes()); properties.put(ConsumerConfig.FETCH_MAX_WAIT_MS_CONFIG, topicConfig.getFetchMaxWait()); - properties.put(ConsumerConfig.FETCH_MIN_BYTES_CONFIG, topicConfig.getFetchMinBytes()); + properties.put(ConsumerConfig.FETCH_MIN_BYTES_CONFIG, (int)topicConfig.getFetchMinBytes()); } private void setPropertiesForSchemaRegistryConnectivity(Properties properties) { diff --git a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/util/MessageFormat.java b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/util/MessageFormat.java index 8cf19a32bf..f2074f1ab6 100644 --- a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/util/MessageFormat.java +++ b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/util/MessageFormat.java @@ -5,9 +5,10 @@ package org.opensearch.dataprepper.plugins.kafka.util; +import com.fasterxml.jackson.annotation.JsonCreator; + import java.util.Arrays; import java.util.Map; -import java.util.function.Function; import java.util.stream.Collectors; /** @@ -19,19 +20,18 @@ public enum MessageFormat { PLAINTEXT("plaintext"), JSON("json"), AVRO("avro"); private static final Map MESSAGE_FORMAT_MAP = Arrays.stream(MessageFormat.values()) - .collect(Collectors.toMap(MessageFormat::toString, Function.identity())); - - private final String messageFormatName; + .collect(Collectors.toMap( + value -> value.type, + value -> value + )); - MessageFormat(final String name) { - this.messageFormatName = name; - } + private final String type; - @Override - public String toString() { - return this.messageFormatName; + MessageFormat(final String type) { + this.type = type; } + @JsonCreator public static MessageFormat getByMessageFormatByName(final String name) { return MESSAGE_FORMAT_MAP.get(name.toLowerCase()); } diff --git a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/util/SinkPropertyConfigurer.java b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/util/SinkPropertyConfigurer.java index 19889e91bb..2871b03459 100644 --- a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/util/SinkPropertyConfigurer.java +++ b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/util/SinkPropertyConfigurer.java @@ -9,6 +9,7 @@ import io.confluent.kafka.serializers.KafkaAvroSerializer; import org.apache.commons.lang3.ObjectUtils; import org.apache.kafka.clients.CommonClientConfigs; +import org.apache.kafka.common.config.TopicConfig; import org.apache.kafka.common.serialization.StringSerializer; import org.apache.kafka.connect.json.JsonSerializer; import org.opensearch.dataprepper.model.types.ByteCount; @@ -326,6 +327,7 @@ public static Properties getPropertiesForAdmintClient(final KafkaSinkConfig kafk Properties properties = new Properties(); setCommonServerProperties(properties, kafkaSinkConfig); setAuthProperties(kafkaSinkConfig, properties); + properties.put(TopicConfig.RETENTION_MS_CONFIG,kafkaSinkConfig.getTopic().getRetentionPeriod()); return properties; } 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 0c4db63f8e..a85ed92727 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 @@ -5,13 +5,13 @@ package org.opensearch.dataprepper.plugins.kafka.configuration; import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.datatype.jsr310.JavaTimeModule; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Tag; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.TestInfo; import org.mockito.Mock; import org.yaml.snakeyaml.Yaml; +import com.fasterxml.jackson.datatype.jsr310.JavaTimeModule; import java.io.FileReader; import java.io.IOException; @@ -25,11 +25,15 @@ import static org.hamcrest.Matchers.notNullValue; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertThrows; + +import org.opensearch.dataprepper.model.types.ByteCount; +import static org.opensearch.dataprepper.test.helper.ReflectivelySetField.setField; class TopicConfigTest { @Mock - TopicConfig topicsConfig; + TopicConfig topicConfig; private static final String YAML_FILE_WITH_CONSUMER_CONFIG = "sample-pipelines.yaml"; @@ -38,7 +42,7 @@ class TopicConfigTest { @BeforeEach void setUp(TestInfo testInfo) throws IOException { String fileName = testInfo.getTags().stream().findFirst().orElse(""); - topicsConfig = new TopicConfig(); + topicConfig = new TopicConfig(); Yaml yaml = new Yaml(); FileReader fileReader = new FileReader(getClass().getClassLoader().getResource(fileName).getFile()); Object data = yaml.load(fileReader); @@ -52,85 +56,87 @@ void setUp(TestInfo testInfo) throws IOException { String json = mapper.writeValueAsString(kafkaConfigMap); Reader reader = new StringReader(json); KafkaSourceConfig kafkaSourceConfig = mapper.readValue(reader, KafkaSourceConfig.class); - List topicsConfigList = kafkaSourceConfig.getTopics(); - topicsConfig = topicsConfigList.get(0); + List topicConfigList = kafkaSourceConfig.getTopics(); + topicConfig = topicConfigList.get(0); } } @Test @Tag(YAML_FILE_WITH_CONSUMER_CONFIG) void test_topicsConfig_not_null() { - assertThat(topicsConfig, notNullValue()); + assertThat(topicConfig, notNullValue()); } @Test @Tag(YAML_FILE_WITH_MISSING_CONSUMER_CONFIG) void testConfigValues_default() { - assertEquals("my-topic-2", topicsConfig.getName()); - assertEquals("my-test-group", topicsConfig.getGroupId()); - assertEquals("kafka-consumer-group-2", topicsConfig.getGroupName()); - assertEquals(false, topicsConfig.getAutoCommit()); - assertEquals(Duration.ofSeconds(5), topicsConfig.getCommitInterval()); - assertEquals(Duration.ofSeconds(45), topicsConfig.getSessionTimeOut()); - assertEquals("latest", topicsConfig.getAutoOffsetReset()); - assertEquals(Duration.ofSeconds(5), topicsConfig.getThreadWaitingTime()); - assertEquals(Duration.ofSeconds(4), topicsConfig.getMaxRecordFetchTime()); - assertEquals(Duration.ofSeconds(5), topicsConfig.getBufferDefaultTimeout()); - assertEquals(52428800L, topicsConfig.getFetchMaxBytes().longValue()); - assertEquals(500L, topicsConfig.getFetchMaxWait().longValue()); - assertEquals(1L, topicsConfig.getFetchMinBytes().longValue()); - assertEquals(Duration.ofSeconds(10), topicsConfig.getRetryBackoff()); - assertEquals(Duration.ofSeconds(300000), topicsConfig.getMaxPollInterval()); - assertEquals(500L, topicsConfig.getConsumerMaxPollRecords().longValue()); - assertEquals(2, topicsConfig.getWorkers().intValue()); - assertEquals(Duration.ofSeconds(5), topicsConfig.getHeartBeatInterval()); + assertEquals("my-topic-2", topicConfig.getName()); + assertEquals("my-test-group", topicConfig.getGroupId()); + assertEquals(TopicConfig.DEFAULT_AUTO_COMMIT, topicConfig.getAutoCommit()); + assertEquals(TopicConfig.DEFAULT_COMMIT_INTERVAL, topicConfig.getCommitInterval()); + assertEquals(TopicConfig.DEFAULT_SESSION_TIMEOUT, topicConfig.getSessionTimeOut()); + assertEquals(TopicConfig.DEFAULT_AUTO_OFFSET_RESET, topicConfig.getAutoOffsetReset()); + assertEquals(TopicConfig.DEFAULT_THREAD_WAITING_TIME, topicConfig.getThreadWaitingTime()); + assertEquals(ByteCount.parse(TopicConfig.DEFAULT_FETCH_MAX_BYTES).getBytes(), topicConfig.getFetchMaxBytes()); + assertEquals(TopicConfig.DEFAULT_FETCH_MAX_WAIT, topicConfig.getFetchMaxWait()); + assertEquals(ByteCount.parse(TopicConfig.DEFAULT_FETCH_MIN_BYTES).getBytes(), topicConfig.getFetchMinBytes()); + assertEquals(TopicConfig.DEFAULT_RETRY_BACKOFF, topicConfig.getRetryBackoff()); + assertEquals(TopicConfig.DEFAULT_RECONNECT_BACKOFF, topicConfig.getReconnectBackoff()); + assertEquals(TopicConfig.DEFAULT_MAX_POLL_INTERVAL, topicConfig.getMaxPollInterval()); + assertEquals(TopicConfig.DEFAULT_CONSUMER_MAX_POLL_RECORDS, topicConfig.getConsumerMaxPollRecords()); + assertEquals(TopicConfig.DEFAULT_NUM_OF_WORKERS, topicConfig.getWorkers()); + assertEquals(TopicConfig.DEFAULT_HEART_BEAT_INTERVAL_DURATION, topicConfig.getHeartBeatInterval()); + assertEquals(ByteCount.parse(TopicConfig.DEFAULT_MAX_PARTITION_FETCH_BYTES).getBytes(), topicConfig.getMaxPartitionFetchBytes()); } @Test @Tag(YAML_FILE_WITH_CONSUMER_CONFIG) void testConfigValues_from_yaml() { - - assertEquals("my-topic-1", topicsConfig.getName()); - assertEquals("my-test-group", topicsConfig.getGroupId()); - assertEquals(null, topicsConfig.getGroupName()); - assertEquals(false, topicsConfig.getAutoCommit()); - assertEquals(Duration.ofSeconds(5), topicsConfig.getCommitInterval()); - assertEquals(Duration.ofSeconds(45), topicsConfig.getSessionTimeOut()); - assertEquals("earliest", topicsConfig.getAutoOffsetReset()); - assertEquals(Duration.ofSeconds(1), topicsConfig.getThreadWaitingTime()); - assertEquals(Duration.ofSeconds(4), topicsConfig.getMaxRecordFetchTime()); - assertEquals(Duration.ofSeconds(5), topicsConfig.getBufferDefaultTimeout()); - assertEquals(52428800L, topicsConfig.getFetchMaxBytes().longValue()); - assertEquals(500L, topicsConfig.getFetchMaxWait().longValue()); - assertEquals(1L, topicsConfig.getFetchMinBytes().longValue()); - assertEquals(Duration.ofSeconds(100), topicsConfig.getRetryBackoff()); - assertEquals(Duration.ofSeconds(300000), topicsConfig.getMaxPollInterval()); - assertEquals(500L, topicsConfig.getConsumerMaxPollRecords().longValue()); - assertEquals(5, topicsConfig.getWorkers().intValue()); - assertEquals(Duration.ofSeconds(3), topicsConfig.getHeartBeatInterval()); + assertEquals("my-topic-1", topicConfig.getName()); + assertEquals(false, topicConfig.getAutoCommit()); + assertEquals(Duration.ofSeconds(5), topicConfig.getCommitInterval()); + assertEquals(45000, topicConfig.getSessionTimeOut().toMillis()); + assertEquals("earliest", topicConfig.getAutoOffsetReset()); + assertEquals(Duration.ofSeconds(1), topicConfig.getThreadWaitingTime()); + assertEquals(52428800L, topicConfig.getFetchMaxBytes()); + assertEquals(500L, topicConfig.getFetchMaxWait().longValue()); + assertEquals(1L, topicConfig.getFetchMinBytes()); + assertEquals(Duration.ofSeconds(100), topicConfig.getRetryBackoff()); + assertEquals(Duration.ofSeconds(300), topicConfig.getMaxPollInterval()); + assertEquals(500L, topicConfig.getConsumerMaxPollRecords().longValue()); + assertEquals(5, topicConfig.getWorkers().intValue()); + assertEquals(Duration.ofSeconds(3), topicConfig.getHeartBeatInterval()); + assertEquals(10*ByteCount.parse(TopicConfig.DEFAULT_MAX_PARTITION_FETCH_BYTES).getBytes(), topicConfig.getMaxPartitionFetchBytes()); } @Test @Tag(YAML_FILE_WITH_CONSUMER_CONFIG) void testConfigValues_from_yaml_not_null() { + assertNotNull(topicConfig.getName()); + assertNotNull(topicConfig.getAutoCommit()); + assertNotNull(topicConfig.getCommitInterval()); + assertNotNull(topicConfig.getSessionTimeOut()); + assertNotNull(topicConfig.getAutoOffsetReset()); + assertNotNull(topicConfig.getThreadWaitingTime()); + assertNotNull(topicConfig.getFetchMaxBytes()); + assertNotNull(topicConfig.getFetchMaxWait()); + assertNotNull(topicConfig.getFetchMinBytes()); + assertNotNull(topicConfig.getRetryBackoff()); + assertNotNull(topicConfig.getMaxPollInterval()); + assertNotNull(topicConfig.getConsumerMaxPollRecords()); + assertNotNull(topicConfig.getWorkers()); + assertNotNull(topicConfig.getHeartBeatInterval()); + } - assertNotNull(topicsConfig.getName()); - assertNotNull(topicsConfig.getGroupId()); - assertNotNull(topicsConfig.getAutoCommit()); - assertNotNull(topicsConfig.getCommitInterval()); - assertNotNull(topicsConfig.getSessionTimeOut()); - assertNotNull(topicsConfig.getAutoOffsetReset()); - assertNotNull(topicsConfig.getThreadWaitingTime()); - assertNotNull(topicsConfig.getMaxRecordFetchTime()); - assertNotNull(topicsConfig.getBufferDefaultTimeout()); - assertNotNull(topicsConfig.getFetchMaxBytes()); - assertNotNull(topicsConfig.getFetchMaxWait()); - assertNotNull(topicsConfig.getFetchMinBytes()); - assertNotNull(topicsConfig.getRetryBackoff()); - assertNotNull(topicsConfig.getMaxPollInterval()); - assertNotNull(topicsConfig.getConsumerMaxPollRecords()); - assertNotNull(topicsConfig.getWorkers()); - assertNotNull(topicsConfig.getHeartBeatInterval()); + @Test + @Tag(YAML_FILE_WITH_CONSUMER_CONFIG) + void TestInvalidConfigValues() throws NoSuchFieldException, IllegalAccessException { + setField(TopicConfig.class, topicConfig, "fetchMaxBytes", "60mb"); + assertThrows(RuntimeException.class, () -> topicConfig.getFetchMaxBytes()); + setField(TopicConfig.class, topicConfig, "fetchMaxBytes", "0b"); + assertThrows(RuntimeException.class, () -> topicConfig.getFetchMaxBytes()); + setField(TopicConfig.class, topicConfig, "fetchMinBytes", "0b"); + assertThrows(RuntimeException.class, () -> topicConfig.getFetchMinBytes()); } } diff --git a/data-prepper-plugins/kafka-plugins/src/test/resources/sample-pipelines-1.yaml b/data-prepper-plugins/kafka-plugins/src/test/resources/sample-pipelines-1.yaml index 67f655e167..6f3502a175 100644 --- a/data-prepper-plugins/kafka-plugins/src/test/resources/sample-pipelines-1.yaml +++ b/data-prepper-plugins/kafka-plugins/src/test/resources/sample-pipelines-1.yaml @@ -5,7 +5,6 @@ log-pipeline: - 127.0.0.1:9093 topics: - name: my-topic-2 - group_name: kafka-consumer-group-2 group_id: my-test-group - name: my-topic-1 group_id: my-test-group 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 e6ecc1a243..18eb541d87 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 @@ -12,18 +12,15 @@ log-pipeline: auto_commit: false commit_interval: PT5S session_timeout: PT45S - max_retry_attempts: 1000 auto_offset_reset: earliest thread_waiting_time: PT1S - max_record_fetch_time: PT4S heart_beat_interval: PT3S - buffer_default_timeout: PT5S - fetch_max_bytes: 52428800 + fetch_max_bytes: "50mb" fetch_max_wait: 500 - fetch_min_bytes: 1 + fetch_min_bytes: "1b" retry_backoff: PT100S consumer_max_poll_records: 500 - max_partition_fetch_bytes: 10485760 + max_partition_fetch_bytes: "10mb" schema: registry_url: http://localhost:8081/ version: 1 diff --git a/data-prepper-plugins/opensearch/README.md b/data-prepper-plugins/opensearch/README.md index 2f08f8d3c5..2bede56c52 100644 --- a/data-prepper-plugins/opensearch/README.md +++ b/data-prepper-plugins/opensearch/README.md @@ -95,7 +95,7 @@ Default is null. - `index_type` (optional): a String from the list [`custom`, `trace-analytics-raw`, `trace-analytics-service-map`, `management_disabled`], which represents an index type. Defaults to `custom` if `serverless` is `false` in [AWS Configuration](#aws_configuration), otherwise defaults to `management_disabled`. This index_type instructs Sink plugin what type of data it is handling. -- `enable_request_compression` (optional): A boolean that enables or disables request compression when sending requests to OpenSearch. Default is true. +- `enable_request_compression` (optional): A boolean that enables or disables request compression when sending requests to OpenSearch. For `distribution_version` set to `es6`, default value is `false`, otherwise default value is `true`. ``` APM trace analytics raw span data type example: @@ -209,7 +209,7 @@ With the `document_root_key` set to `status`. The document structure would be `{ duration: "15 ms" } ``` -- `include_keys`: A list of keys to be included (retained). The key in the list can be a valid JSON path, such as 'request/status'. This option can work together with `document_root_key`. +- `include_keys`: A list of keys to be included (retained). The key in the list cannot contain '/'. This option can work together with `document_root_key`. For example, If we have the following sample event: ``` @@ -224,7 +224,7 @@ For example, If we have the following sample event: } } ``` -if `include_keys` is set to ["status", "metadata/sourceIp"], the document written to OpenSearch would be: +if `include_keys` is set to ["status", "metadata"], the document written to OpenSearch would be: ``` { status: 200, @@ -256,11 +256,11 @@ For example, If we have the following sample event: } } ``` -if `exclude_keys` is set to ["status", "metadata/sourceIp"], the document written to OpenSearch would be: +if `exclude_keys` is set to ["message", "status"], the document written to OpenSearch would be: ``` { - message: null, metadata: { + sourceIp: "123.212.49.58", destinationIp: "79.54.67.231", bytes: 3545, duration: "15 ms" diff --git a/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/ConnectionConfiguration.java b/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/ConnectionConfiguration.java index a517f34ded..0eb173b7e5 100644 --- a/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/ConnectionConfiguration.java +++ b/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/ConnectionConfiguration.java @@ -58,6 +58,7 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; +import static org.opensearch.dataprepper.plugins.sink.opensearch.index.IndexConfiguration.DISTRIBUTION_VERSION; public class ConnectionConfiguration { private static final Logger LOG = LoggerFactory.getLogger(OpenSearchSink.class); @@ -234,7 +235,11 @@ public static ConnectionConfiguration readConnectionConfiguration(final PluginSe final String proxy = pluginSetting.getStringOrDefault(PROXY, null); builder = builder.withProxy(proxy); - final boolean requestCompressionEnabled = pluginSetting.getBooleanOrDefault(REQUEST_COMPRESSION_ENABLED, true); + final String distributionVersionName = pluginSetting.getStringOrDefault(DISTRIBUTION_VERSION, + DistributionVersion.DEFAULT.getVersion()); + final DistributionVersion distributionVersion = DistributionVersion.fromTypeName(distributionVersionName); + final boolean requestCompressionEnabled = pluginSetting.getBooleanOrDefault( + REQUEST_COMPRESSION_ENABLED, !DistributionVersion.ES6.equals(distributionVersion)); builder = builder.withRequestCompressionEnabled(requestCompressionEnabled); return builder.build(); diff --git a/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/OpenSearchSink.java b/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/OpenSearchSink.java index ccd430c982..04147619b7 100644 --- a/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/OpenSearchSink.java +++ b/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/OpenSearchSink.java @@ -101,6 +101,7 @@ public class OpenSearchSink extends AbstractSink> { private final long flushTimeout; private final IndexType indexType; private final String documentIdField; + private final String documentId; private final String routingField; private final String action; private final String documentRootKey; @@ -117,7 +118,6 @@ public class OpenSearchSink extends AbstractSink> { private PluginSetting pluginSetting; private final SinkContext sinkContext; private final ExpressionEvaluator expressionEvaluator; - private final boolean isDocumentIdAnExpression; private FailedBulkOperationConverter failedBulkOperationConverter; @@ -145,7 +145,7 @@ public OpenSearchSink(final PluginSetting pluginSetting, this.flushTimeout = openSearchSinkConfig.getIndexConfiguration().getFlushTimeout(); this.indexType = openSearchSinkConfig.getIndexConfiguration().getIndexType(); this.documentIdField = openSearchSinkConfig.getIndexConfiguration().getDocumentIdField(); - this.isDocumentIdAnExpression = expressionEvaluator.isValidExpressionStatement(documentIdField); + this.documentId = openSearchSinkConfig.getIndexConfiguration().getDocumentId(); this.routingField = openSearchSinkConfig.getIndexConfiguration().getRoutingField(); this.action = openSearchSinkConfig.getIndexConfiguration().getAction(); this.documentRootKey = openSearchSinkConfig.getIndexConfiguration().getDocumentRootKey(); @@ -337,14 +337,14 @@ private SerializedJson getDocument(final Event event) { String docId = null; - if (isDocumentIdAnExpression) { + if (Objects.nonNull(documentIdField)) { + docId = event.get(documentIdField, String.class); + } else if (Objects.nonNull(documentId)) { try { - docId = (String) expressionEvaluator.evaluate(documentIdField, event); - } catch (final ExpressionEvaluationException e) { - LOG.error("Unable to construct document_id_field from expression {}, the document_id will be generated by OpenSearch", documentIdField); + docId = event.formatString(documentId, expressionEvaluator); + } catch (final ExpressionEvaluationException | EventKeyNotFoundException e) { + LOG.error("Unable to construct document_id with format {}, the document_id will be generated by OpenSearch", documentId, e); } - } else if (Objects.nonNull(documentIdField)) { - docId = event.get(documentIdField, String.class); } String routing = (routingField != null) ? event.get(routingField, String.class) : null; diff --git a/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/index/IndexConfiguration.java b/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/index/IndexConfiguration.java index ccc2e1c951..e23fdd4e26 100644 --- a/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/index/IndexConfiguration.java +++ b/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/index/IndexConfiguration.java @@ -9,11 +9,14 @@ import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.commons.lang3.EnumUtils; +import org.opensearch.dataprepper.model.plugin.InvalidPluginConfigurationException; import org.opensearch.dataprepper.plugins.sink.opensearch.DistributionVersion; import org.opensearch.dataprepper.plugins.sink.opensearch.bulk.BulkAction; import org.opensearch.dataprepper.plugins.sink.opensearch.s3.FileReader; import org.opensearch.dataprepper.plugins.sink.opensearch.s3.S3ClientProvider; import org.opensearch.dataprepper.plugins.sink.opensearch.s3.S3FileReader; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import software.amazon.awssdk.arns.Arn; import software.amazon.awssdk.services.s3.S3Client; @@ -23,12 +26,15 @@ import java.net.URL; import java.util.HashMap; import java.util.Map; +import java.util.Objects; import java.util.Optional; import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; public class IndexConfiguration { + private static final Logger LOG = LoggerFactory.getLogger(IndexConfiguration.class); + public static final String SETTINGS = "settings"; public static final String INDEX_ALIAS = "index"; public static final String INDEX_TYPE = "index_type"; @@ -41,6 +47,7 @@ public class IndexConfiguration { public static final String MAX_LOCAL_COMPRESSIONS_FOR_ESTIMATION = "max_local_compressions_for_estimation"; public static final String FLUSH_TIMEOUT = "flush_timeout"; public static final String DOCUMENT_ID_FIELD = "document_id_field"; + public static final String DOCUMENT_ID = "document_id"; public static final String ROUTING_FIELD = "routing_field"; public static final String ISM_POLICY_FILE = "ism_policy_file"; public static final long DEFAULT_BULK_SIZE = 5L; @@ -61,6 +68,7 @@ public class IndexConfiguration { private final String indexAlias; private final Map indexTemplate; private final String documentIdField; + private final String documentId; private final String routingField; private final long bulkSize; private final boolean estimateBulkSizeUsingCompression; @@ -119,12 +127,14 @@ private IndexConfiguration(final Builder builder) { this.routingField = builder.routingField; String documentIdField = builder.documentIdField; + String documentId = builder.documentId; if (indexType.equals(IndexType.TRACE_ANALYTICS_RAW)) { - documentIdField = "spanId"; + documentId = "${spanId}"; } else if (indexType.equals(IndexType.TRACE_ANALYTICS_SERVICE_MAP)) { - documentIdField = "hashId"; + documentId = "${hashId}"; } this.documentIdField = documentIdField; + this.documentId = documentId; this.ismPolicyFile = builder.ismPolicyFile; this.action = builder.action; this.documentRootKey = builder.documentRootKey; @@ -180,10 +190,21 @@ public static IndexConfiguration readIndexConfig(final PluginSetting pluginSetti final long flushTimeout = pluginSetting.getLongOrDefault(FLUSH_TIMEOUT, DEFAULT_FLUSH_TIMEOUT); builder = builder.withFlushTimeout(flushTimeout); - final String documentId = pluginSetting.getStringOrDefault(DOCUMENT_ID_FIELD, null); - if (documentId != null) { - builder = builder.withDocumentIdField(documentId); + final String documentIdField = pluginSetting.getStringOrDefault(DOCUMENT_ID_FIELD, null); + final String documentId = pluginSetting.getStringOrDefault(DOCUMENT_ID, null); + + + if (Objects.nonNull(documentIdField) && Objects.nonNull(documentId)) { + throw new InvalidPluginConfigurationException("Both document_id_field and document_id cannot be used at the same time. It is preferred to only use document_id as document_id_field is deprecated."); + } + + if (documentIdField != null) { + LOG.warn("document_id_field is deprecated in favor of document_id, and support for document_id_field will be removed in a future major version release."); + builder = builder.withDocumentIdField(documentIdField); + } else if (documentId != null) { + builder = builder.withDocumentId(documentId); } + final String routingField = pluginSetting.getStringOrDefault(ROUTING_FIELD, null); if (routingField != null) { builder = builder.withRoutingField(routingField); @@ -242,6 +263,8 @@ public String getDocumentIdField() { return documentIdField; } + public String getDocumentId() { return documentId; } + public String getRoutingField() { return routingField; } @@ -349,6 +372,7 @@ public static class Builder { private int numReplicas; private String routingField; private String documentIdField; + private String documentId; private long bulkSize = DEFAULT_BULK_SIZE; private boolean estimateBulkSizeUsingCompression = DEFAULT_ESTIMATE_BULK_SIZE_USING_COMPRESSION; private int maxLocalCompressionsForEstimation = DEFAULT_MAX_LOCAL_COMPRESSIONS_FOR_ESTIMATION; @@ -391,11 +415,17 @@ public Builder withTemplateFile(final String templateFile) { } public Builder withDocumentIdField(final String documentIdField) { - checkNotNull(documentIdField, "documentId field cannot be null"); + checkNotNull(documentIdField, "document_id_field cannot be null"); this.documentIdField = documentIdField; return this; } + public Builder withDocumentId(final String documentId) { + checkNotNull(documentId, "document_id cannot be null"); + this.documentId = documentId; + return this; + } + public Builder withRoutingField(final String routingField) { this.routingField = routingField; return this; diff --git a/data-prepper-plugins/opensearch/src/test/java/org/opensearch/dataprepper/plugins/sink/opensearch/ConnectionConfigurationTests.java b/data-prepper-plugins/opensearch/src/test/java/org/opensearch/dataprepper/plugins/sink/opensearch/ConnectionConfigurationTests.java index 70dc05e04c..d5da3e6d3d 100644 --- a/data-prepper-plugins/opensearch/src/test/java/org/opensearch/dataprepper/plugins/sink/opensearch/ConnectionConfigurationTests.java +++ b/data-prepper-plugins/opensearch/src/test/java/org/opensearch/dataprepper/plugins/sink/opensearch/ConnectionConfigurationTests.java @@ -50,6 +50,7 @@ import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; import static org.opensearch.dataprepper.plugins.sink.opensearch.ConnectionConfiguration.SERVERLESS; +import static org.opensearch.dataprepper.plugins.sink.opensearch.index.IndexConfiguration.DISTRIBUTION_VERSION; @ExtendWith(MockitoExtension.class) class ConnectionConfigurationTests { @@ -84,6 +85,18 @@ void testReadConnectionConfigurationDefault() { assertNull(connectionConfiguration.getConnectTimeout()); assertNull(connectionConfiguration.getSocketTimeout()); assertEquals(TEST_PIPELINE_NAME, connectionConfiguration.getPipelineName()); + assertTrue(connectionConfiguration.isRequestCompressionEnabled()); + } + + @Test + void testReadConnectionConfigurationES6Default() { + final Map configMetadata = generateConfigurationMetadata( + TEST_HOSTS, null, null, null, null, true, null, null, null, false); + configMetadata.put(DISTRIBUTION_VERSION, "es6"); + final PluginSetting pluginSetting = getPluginSettingByConfigurationMetadata(configMetadata); + final ConnectionConfiguration connectionConfiguration = + ConnectionConfiguration.readConnectionConfiguration(pluginSetting); + assertFalse(connectionConfiguration.isRequestCompressionEnabled()); } @Test diff --git a/data-prepper-plugins/opensearch/src/test/java/org/opensearch/dataprepper/plugins/sink/opensearch/index/IndexConfigurationTests.java b/data-prepper-plugins/opensearch/src/test/java/org/opensearch/dataprepper/plugins/sink/opensearch/index/IndexConfigurationTests.java index 64810bf1f2..cda9476743 100644 --- a/data-prepper-plugins/opensearch/src/test/java/org/opensearch/dataprepper/plugins/sink/opensearch/index/IndexConfigurationTests.java +++ b/data-prepper-plugins/opensearch/src/test/java/org/opensearch/dataprepper/plugins/sink/opensearch/index/IndexConfigurationTests.java @@ -286,7 +286,7 @@ public void testReadIndexConfig_RawIndexType() { assertEquals(60_000L, indexConfiguration.getFlushTimeout()); assertEquals(false, indexConfiguration.isEstimateBulkSizeUsingCompression()); assertEquals(2, indexConfiguration.getMaxLocalCompressionsForEstimation()); - assertEquals("spanId", indexConfiguration.getDocumentIdField()); + assertEquals("${spanId}", indexConfiguration.getDocumentId()); } @Test @@ -312,7 +312,7 @@ public void testReadIndexConfig_ServiceMapIndexType() { assertEquals(60_000L, indexConfiguration.getFlushTimeout()); assertEquals(false, indexConfiguration.isEstimateBulkSizeUsingCompression()); assertEquals(2, indexConfiguration.getMaxLocalCompressionsForEstimation()); - assertEquals("hashId", indexConfiguration.getDocumentIdField()); + assertEquals("${hashId}", indexConfiguration.getDocumentId()); } @Test @@ -335,7 +335,7 @@ public void testReadIndexConfigCustom() { assertEquals(testFlushTimeout, indexConfiguration.getFlushTimeout()); assertEquals(true, indexConfiguration.isEstimateBulkSizeUsingCompression()); assertEquals(5, indexConfiguration.getMaxLocalCompressionsForEstimation()); - assertEquals(testIdField, indexConfiguration.getDocumentIdField()); + assertEquals(testIdField, indexConfiguration.getDocumentId()); } @Test @@ -356,7 +356,7 @@ public void testReadIndexConfig_ExplicitCustomIndexType() { assertFalse(indexConfiguration.getIndexTemplate().isEmpty()); assertEquals(testBulkSize, indexConfiguration.getBulkSize()); assertEquals(testFlushTimeout, indexConfiguration.getFlushTimeout()); - assertEquals(testIdField, indexConfiguration.getDocumentIdField()); + assertEquals(testIdField, indexConfiguration.getDocumentId()); } @Test @@ -458,7 +458,7 @@ private PluginSetting getPluginSetting(Map metadata) { } private Map initializeConfigMetaData( - String indexType, String indexAlias, String templateFilePath, Long bulkSize, Long flushTimeout, String documentIdField) { + String indexType, String indexAlias, String templateFilePath, Long bulkSize, Long flushTimeout, String documentId) { final Map metadata = new HashMap<>(); if (indexType != null) { metadata.put(IndexConfiguration.INDEX_TYPE, indexType); @@ -475,8 +475,8 @@ private Map initializeConfigMetaData( if (flushTimeout != null) { metadata.put(IndexConfiguration.FLUSH_TIMEOUT, flushTimeout); } - if (documentIdField != null) { - metadata.put(IndexConfiguration.DOCUMENT_ID_FIELD, documentIdField); + if (documentId != null) { + metadata.put(IndexConfiguration.DOCUMENT_ID, documentId); } return metadata; } 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 deleted file mode 100644 index 3b61c19220..0000000000 --- a/data-prepper-plugins/parquet-codecs/src/main/java/org/opensearch/dataprepper/plugins/codec/parquet/ParquetOutputCodec.java +++ /dev/null @@ -1,323 +0,0 @@ -/* - * Copyright OpenSearch Contributors - * SPDX-License-Identifier: Apache-2.0 - */ -package org.opensearch.dataprepper.plugins.codec.parquet; - -import org.apache.avro.Schema; -import org.apache.avro.generic.GenericData; -import org.apache.avro.generic.GenericRecord; -import org.apache.parquet.avro.AvroParquetWriter; -import org.apache.parquet.hadoop.ParquetWriter; -import org.apache.parquet.io.OutputFile; -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 org.opensearch.dataprepper.model.sink.OutputCodecContext; -import org.opensearch.dataprepper.plugins.fs.LocalInputFile; -import org.opensearch.dataprepper.plugins.fs.LocalOutputFile; -import org.opensearch.dataprepper.plugins.s3keyindex.S3ObjectIndexUtility; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; -import software.amazon.awssdk.auth.credentials.AwsCredentialsProviderChain; -import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider; -import software.amazon.awssdk.http.apache.ApacheHttpClient; -import software.amazon.awssdk.regions.Region; -import software.amazon.awssdk.services.s3.S3Client; -import software.amazon.awssdk.services.s3.model.DeleteObjectRequest; - -import java.io.File; -import java.io.IOException; -import java.io.OutputStream; -import java.nio.charset.StandardCharsets; -import java.util.Arrays; -import java.util.List; -import java.util.Map; -import java.util.Objects; -import java.util.regex.Pattern; - -@DataPrepperPlugin(name = "parquet", pluginType = OutputCodec.class, pluginConfigurationType = ParquetOutputCodecConfig.class) -public class ParquetOutputCodec implements OutputCodec { - private static final Logger LOG = LoggerFactory.getLogger(ParquetOutputCodec.class); - - private final ParquetOutputCodecConfig config; - private static final String BASE_SCHEMA_STRING = "{\"type\":\"record\",\"name\":\"ParquetRecords\",\"fields\":["; - private static final String END_SCHEMA_STRING = "]}"; - private static Schema schema; - private ParquetWriter writer; - private final ApacheHttpClient.Builder apacheHttpClientBuilder = ApacheHttpClient.builder(); - private S3Client s3Client; - private OutputCodecContext codecContext; - private static final String PARQUET = "parquet"; - - private static final String TIME_PATTERN_REGULAR_EXPRESSION = "\\%\\{.*?\\}"; - private static final Pattern SIMPLE_DURATION_PATTERN = Pattern.compile(TIME_PATTERN_REGULAR_EXPRESSION); - private String key; - private final String bucket; - private final String region; - private static final List primitiveTypes = Arrays.asList("int", "long", "string", "float", "double", "bytes"); - - - @DataPrepperPluginConstructor - public ParquetOutputCodec(final ParquetOutputCodecConfig config) { - Objects.requireNonNull(config); - this.config = config; - this.region = config.getRegion(); - this.bucket = config.getBucket(); - } - - @Override - public synchronized void start(final OutputStream outputStream, final Event event, final OutputCodecContext codecContext) throws IOException { - Objects.requireNonNull(outputStream); - Objects.requireNonNull(codecContext); - this.codecContext = codecContext; - this.s3Client = buildS3Client(); - buildSchemaAndKey(event, codecContext.getTagsTargetKey()); - final S3OutputFile s3OutputFile = new S3OutputFile(s3Client, bucket, key); - buildWriter(s3OutputFile); - } - - public synchronized void start(File file, final OutputCodecContext codecContext) throws IOException { - Objects.requireNonNull(codecContext); - this.codecContext = codecContext; - LocalOutputFile localOutputFile = new LocalOutputFile(file); - buildSchemaAndKey(null, null); - buildWriter(localOutputFile); - } - - void buildSchemaAndKey(final Event event, final String tagsTargetKey) throws IOException { - if (config.getSchema() != null) { - schema = parseSchema(config.getSchema()); - } else if (config.getFileLocation() != null) { - schema = ParquetSchemaParser.parseSchemaFromJsonFile(config.getFileLocation()); - } else if (config.getSchemaRegistryUrl() != null) { - schema = parseSchema(ParquetSchemaParserFromSchemaRegistry.getSchemaType(config.getSchemaRegistryUrl())); - } else if (checkS3SchemaValidity()) { - schema = ParquetSchemaParserFromS3.parseSchema(config); - } else { - schema = buildInlineSchemaFromEvent(event, tagsTargetKey); - } - key = generateKey(); - } - - public Schema buildInlineSchemaFromEvent(final Event event, final String tagsTargetKey) throws IOException { - if (tagsTargetKey != null) { - return parseSchema(buildSchemaStringFromEventMap(addTagsToEvent(event, tagsTargetKey).toMap(), false)); - } else { - return parseSchema(buildSchemaStringFromEventMap(event.toMap(), false)); - } - } - - private String buildSchemaStringFromEventMap(final Map eventData, boolean nestedRecordFlag) { - final StringBuilder builder = new StringBuilder(); - int nestedRecordIndex = 1; - if (!nestedRecordFlag) { - builder.append(BASE_SCHEMA_STRING); - } else { - builder.append("{\"type\":\"record\",\"name\":\"" + "NestedRecord" + nestedRecordIndex + "\",\"fields\":["); - nestedRecordIndex++; - } - String fields; - int index = 0; - for (final String key : eventData.keySet()) { - if (codecContext.getExcludeKeys().contains(key)) { - continue; - } - if (index == 0) { - if (!(eventData.get(key) instanceof Map)) { - fields = "{\"name\":\"" + key + "\",\"type\":\"" + typeMapper(eventData.get(key)) + "\"}"; - } else { - fields = "{\"name\":\"" + key + "\",\"type\":" + typeMapper(eventData.get(key)) + "}"; - } - } else { - if (!(eventData.get(key) instanceof Map)) { - fields = "," + "{\"name\":\"" + key + "\",\"type\":\"" + typeMapper(eventData.get(key)) + "\"}"; - } else { - fields = "," + "{\"name\":\"" + key + "\",\"type\":" + typeMapper(eventData.get(key)) + "}"; - } - } - builder.append(fields); - index++; - } - builder.append(END_SCHEMA_STRING); - return builder.toString(); - } - - private String typeMapper(final Object value) { - if (value instanceof Integer || value.getClass().equals(int.class)) { - return "int"; - } else if (value instanceof Float || value.getClass().equals(float.class)) { - return "float"; - } else if (value instanceof Double || value.getClass().equals(double.class)) { - return "double"; - } else if (value instanceof Long || value.getClass().equals(long.class)) { - return "long"; - } else if (value instanceof Byte[]) { - return "bytes"; - } else if (value instanceof Map) { - return buildSchemaStringFromEventMap((Map) value, true); - } else { - return "string"; - } - } - - private void buildWriter(OutputFile outputFile) throws IOException { - writer = AvroParquetWriter.builder(outputFile) - .withSchema(schema) - .build(); - } - - @Override - public void writeEvent(final Event event, final OutputStream outputStream) throws IOException { - final GenericData.Record parquetRecord = new GenericData.Record(schema); - final Event modifiedEvent; - if (codecContext.getTagsTargetKey() != null) { - modifiedEvent = addTagsToEvent(event, codecContext.getTagsTargetKey()); - } else { - modifiedEvent = event; - } - for (final String key : modifiedEvent.toMap().keySet()) { - if (codecContext.getExcludeKeys().contains(key)) { - continue; - } - final Schema.Field field = schema.getField(key); - final Object value = schemaMapper(field, modifiedEvent.toMap().get(key)); - parquetRecord.put(key, value); - } - writer.write(parquetRecord); - } - - @Override - public synchronized void complete(final OutputStream outputStream) throws IOException { - writer.close(); - final S3ObjectReference s3ObjectReference = S3ObjectReference.bucketAndKey(bucket, key).build(); - final S3InputFile inputFile = new S3InputFile(s3Client, s3ObjectReference); - byte[] byteBuffer = inputFile.newStream().readAllBytes(); - outputStream.write(byteBuffer); - final DeleteObjectRequest deleteRequest = DeleteObjectRequest.builder() - .bucket(bucket) - .key(key) - .build(); - s3Client.deleteObject(deleteRequest); - } - - public void closeWriter(final OutputStream outputStream, File file) throws IOException { - final LocalInputFile inputFile = new LocalInputFile(file); - byte[] byteBuffer = inputFile.newStream().readAllBytes(); - outputStream.write(byteBuffer); - writer.close(); - } - - @Override - public String getExtension() { - return PARQUET; - } - - static Schema parseSchema(final String schemaString) { - return new Schema.Parser().parse(schemaString); - } - - private S3Client buildS3Client() { - final AwsCredentialsProvider credentialsProvider = AwsCredentialsProviderChain.builder() - .addCredentialsProvider(DefaultCredentialsProvider.create()).build(); - return S3Client.builder() - .region(Region.of(region)) - .credentialsProvider(credentialsProvider) - .httpClientBuilder(apacheHttpClientBuilder) - .build(); - } - - /** - * Generate the s3 object path prefix and object file name. - * - * @return object key path. - */ - protected String generateKey() { - final String pathPrefix = buildObjectPath(config.getPathPrefix()); - final String namePattern = buildObjectFileName(config.getNamePattern()); - return (!pathPrefix.isEmpty()) ? pathPrefix + namePattern : namePattern; - } - - private static String buildObjectPath(final String pathPrefix) { - final StringBuilder s3ObjectPath = new StringBuilder(); - if (pathPrefix != null && !pathPrefix.isEmpty()) { - String[] pathPrefixList = pathPrefix.split("\\/"); - for (final String prefixPath : pathPrefixList) { - if (SIMPLE_DURATION_PATTERN.matcher(prefixPath).find()) { - s3ObjectPath.append(S3ObjectIndexUtility.getObjectPathPrefix(prefixPath)).append("/"); - } else { - s3ObjectPath.append(prefixPath).append("/"); - } - } - } - return s3ObjectPath.toString(); - } - - private String buildObjectFileName(final String configNamePattern) { - return S3ObjectIndexUtility.getObjectNameWithDateTimeId(configNamePattern) + "." + getExtension(); - } - - private static Object schemaMapper(final Schema.Field field, final Object rawValue) { - Object finalValue = null; - final String fieldType = field.schema().getType().name().toLowerCase(); - if (field.schema().getLogicalType() == null && primitiveTypes.contains(fieldType)) { - switch (fieldType) { - case "string": - finalValue = rawValue.toString(); - break; - case "int": - finalValue = Integer.parseInt(rawValue.toString()); - break; - case "float": - finalValue = Float.parseFloat(rawValue.toString()); - break; - case "double": - finalValue = Double.parseDouble(rawValue.toString()); - break; - case "long": - finalValue = Long.parseLong(rawValue.toString()); - break; - case "bytes": - finalValue = rawValue.toString().getBytes(StandardCharsets.UTF_8); - break; - default: - LOG.error("Unrecognised Field name : '{}' & type : '{}'", field.name(), fieldType); - break; - } - } else { - final String logicalTypeName = field.schema().getLogicalType().getName(); - switch (logicalTypeName) { - case "date": - finalValue = Integer.parseInt(rawValue.toString()); - break; - case "time-millis": - case "timestamp-millis": - case "time-micros": - case "timestamp-micros": - finalValue = Long.parseLong(rawValue.toString()); - break; - case "decimal": - Double.parseDouble(rawValue.toString()); - break; - case "uuid": - finalValue = rawValue.toString().getBytes(StandardCharsets.UTF_8); - break; - default: - LOG.error("Unrecognised Logical Datatype for field : '{}' & type : '{}'", field.name(), logicalTypeName); - break; - } - } - return finalValue; - } - - boolean checkS3SchemaValidity() throws IOException { - if (config.getSchemaBucket() != null && config.getFileKey() != null && config.getSchemaRegion() != null) { - return true; - } else { - LOG.error("Invalid S3 credentials, can't reach the schema file."); - throw new IOException("Can't proceed without schema."); - } - } -} diff --git a/data-prepper-plugins/parquet-codecs/src/main/java/org/opensearch/dataprepper/plugins/codec/parquet/S3InputFile.java b/data-prepper-plugins/parquet-codecs/src/main/java/org/opensearch/dataprepper/plugins/codec/parquet/S3InputFile.java deleted file mode 100644 index 6b5630c342..0000000000 --- a/data-prepper-plugins/parquet-codecs/src/main/java/org/opensearch/dataprepper/plugins/codec/parquet/S3InputFile.java +++ /dev/null @@ -1,78 +0,0 @@ -/* - * Copyright OpenSearch Contributors - * SPDX-License-Identifier: Apache-2.0 - */ -package org.opensearch.dataprepper.plugins.codec.parquet; - -import org.apache.parquet.io.SeekableInputStream; -import org.opensearch.dataprepper.model.io.InputFile; -import software.amazon.awssdk.services.s3.S3Client; -import software.amazon.awssdk.services.s3.model.HeadObjectRequest; -import software.amazon.awssdk.services.s3.model.HeadObjectResponse; - -import java.util.concurrent.atomic.LongAdder; - -public class S3InputFile implements InputFile { - - private S3Client s3Client; - - private S3ObjectReference s3ObjectReference; - - private LongAdder bytesCounter; - - private HeadObjectResponse metadata; - - public S3InputFile(final S3Client s3Client, final S3ObjectReference s3ObjectReference) { - this.s3Client = s3Client; - this.s3ObjectReference = s3ObjectReference; - } - - /** - * Note: this may be stale if file was deleted since metadata is cached for size/existence checks. - * - * @return content length - */ - @Override - public long getLength() { - return getMetadata().contentLength(); - } - - /** - * Create an input stream from the input file - * @return an implementation of a SeekableInputStream into the S3 object. - */ - @Override - public SeekableInputStream newStream() { - bytesCounter = new LongAdder(); - - return new S3InputStream(s3Client, s3ObjectReference, getMetadata(), bytesCounter); - } - - /** - * Get the count of bytes read from the S3 object - * @return - */ - public long getBytesCount() { - if (bytesCounter == null) { - return 0; - } - - return bytesCounter.longValue(); - } - - /** - * Get the metadata of the S3 object. Cache the metadata to avoid subsequent headObject calls to S3 - * @return the metadata of the S3 object - */ - private synchronized HeadObjectResponse getMetadata() { - if (metadata == null) { - final HeadObjectRequest request = HeadObjectRequest.builder() - .bucket(s3ObjectReference.getBucketName()) - .key(s3ObjectReference.getKey()) - .build(); - metadata = s3Client.headObject(request); - } - - return metadata; - } -} diff --git a/data-prepper-plugins/parquet-codecs/src/main/java/org/opensearch/dataprepper/plugins/codec/parquet/S3InputStream.java b/data-prepper-plugins/parquet-codecs/src/main/java/org/opensearch/dataprepper/plugins/codec/parquet/S3InputStream.java deleted file mode 100644 index fceebe1eac..0000000000 --- a/data-prepper-plugins/parquet-codecs/src/main/java/org/opensearch/dataprepper/plugins/codec/parquet/S3InputStream.java +++ /dev/null @@ -1,605 +0,0 @@ -/* - * Copyright OpenSearch Contributors - * SPDX-License-Identifier: Apache-2.0 - */ -package org.opensearch.dataprepper.plugins.codec.parquet; - -import com.google.common.base.Preconditions; -import com.google.common.io.ByteStreams; -import org.apache.parquet.io.SeekableInputStream; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import software.amazon.awssdk.core.sync.ResponseTransformer; -import software.amazon.awssdk.http.Abortable; -import software.amazon.awssdk.services.s3.S3Client; -import software.amazon.awssdk.services.s3.model.GetObjectRequest; -import software.amazon.awssdk.services.s3.model.HeadObjectResponse; -import software.amazon.awssdk.services.s3.model.NoSuchKeyException; - -import java.io.EOFException; -import java.io.IOException; -import java.io.InputStream; -import java.nio.ByteBuffer; -import java.util.concurrent.atomic.LongAdder; - - -class S3InputStream extends SeekableInputStream { - - private static final int COPY_BUFFER_SIZE = 8192; - - private static final Logger LOG = LoggerFactory.getLogger(S3InputStream.class); - - private static final int SKIP_SIZE = 1024 * 1024; - - private final S3Client s3Client; - - private final S3ObjectReference s3ObjectReference; - - private final HeadObjectResponse metadata; - - private final LongAdder bytesCounter; - - private final GetObjectRequest.Builder getObjectRequestBuilder; - - private InputStream stream; - - private final byte[] temp = new byte[COPY_BUFFER_SIZE]; - - private long pos = 0; - private long next = 0; - - private long mark = 0; - - private long markLimit = 0; - - private boolean closed = false; - - public S3InputStream( - final S3Client s3Client, - final S3ObjectReference s3ObjectReference, - final HeadObjectResponse metadata, - final LongAdder bytesCounter) { - this.s3Client = s3Client; - this.s3ObjectReference = s3ObjectReference; - this.metadata = metadata; - this.bytesCounter = bytesCounter; - - this.getObjectRequestBuilder = GetObjectRequest.builder() - .bucket(this.s3ObjectReference.getBucketName()) - .key(this.s3ObjectReference.getKey()); - } - - // Implement all InputStream methods first: - - /** - * Returns bytes available to read. - * - * @throws IOException If the underlying stream throws IOException - */ - @Override - public int available() throws IOException { - Preconditions.checkState(!closed, "Cannot read: already closed"); - positionStream(); - - return stream.available(); - } - - /** - * Close a stream. - * - * @throws IOException If the underlying stream throws IOException - */ - @Override - public void close() throws IOException { - super.close(); - closed = true; - closeStream(); - } - - /** - * Mark the current position of the input stream - * - * @param readlimit the maximum limit of bytes that can be read before - * the mark position becomes invalid. - */ - @Override - public synchronized void mark(int readlimit) { - mark = next; - markLimit = mark + readlimit; - } - - /** - * Whether this stream supports mark or not. - * @return Whether mark is supported or not. - */ - @Override - public synchronized boolean markSupported() { - return true; - } - - - /** - * Read a single byte from the stream - * @return the number of bytes read - * @throws IOException if data cannoy be read. - */ - @Override - public int read() throws IOException { - Preconditions.checkState(!closed, "Cannot read: already closed"); - positionStream(); - - pos += 1; - next += 1; - bytesCounter.increment(); - - return stream.read(); - } - - /** - * Read data into the provided byte array - * @param b the buffer into which the data is read. - * @return number of bytes read - * @throws IOException if data cannot be read - */ - @Override - public int read(byte[] b) throws IOException { - return read(b, 0, b.length); - } - - /** - * Read len bytes into the provided byte array starting at off - * @param b the buffer into which the data is read. - * @param off the start offset in array b - * at which the data is written. - * @param len the maximum number of bytes to read. - * @return number of bytes read - * @throws IOException if data cannot be read - */ - @Override - 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); - - return bytesRead; - } - - /** - * Read all bytes from this input stream. - * @return Array of bytes read - * @throws IOException - */ - @Override - public byte[] readAllBytes() throws IOException { - Preconditions.checkState(!closed, "Cannot read: already closed"); - positionStream(); - - final byte[] bytesRead = stream.readAllBytes(); - - pos += bytesRead.length; - next += bytesRead.length; - bytesCounter.add(bytesRead.length); - - return bytesRead; - } - - /** - * - * @param b the byte array into which the data is read - * @param off the start offset in {@code b} at which the data is written - * @param len the maximum number of bytes to read - * @return number of bytes read - * @throws IOException if underlying stream cannot be read from - */ - @Override - public int readNBytes(byte[] b, int off, int len) throws IOException { - Preconditions.checkState(!closed, "Cannot read: already closed"); - positionStream(); - - final int bytesRead = stream.readNBytes(b, off, len); - - pos += bytesRead; - next += bytesRead; - bytesCounter.add(bytesRead); - - return bytesRead; - } - - /** - * @param len the number of bytes to read - * @return array of bytes read - * @throws IOException if stream cannot be read from - */ - @Override - public byte[] readNBytes(int len) throws IOException { - Preconditions.checkState(!closed, "Cannot read: already closed"); - positionStream(); - - final byte[] bytesRead = stream.readNBytes(len); - - pos += bytesRead.length; - next += bytesRead.length; - bytesCounter.add(bytesRead.length); - - return bytesRead; - } - - /** - * Reset the stream to the marked position - * @throws IOException if the stream that was marked is no longer valid - */ - @Override - public synchronized void reset() throws IOException { - if (next > markLimit) { - throw new IOException("Cannot reset stream because mark limit exceeded"); - } - - next = mark; - } - - /** - * Skip n number of bytes in the stream. - * @param n the number of bytes to be skipped. - * @return the number of bytes skipped. - */ - @Override - public long skip(long n) { - if (next >= metadata.contentLength()) { - return 0; - } - - long toSkip = Math.min(n, metadata.contentLength() - next); - - next += toSkip; - - return toSkip; - } - - // Override all SeekableInputStream methods - - /** - * Get the offset into the stream - * @return the offset into the stream - */ - @Override - public long getPos() { - return next; - } - - /** - * Seek the specified offset into the input stream. - * @param newPos the new position to seek to - */ - @Override - public void seek(long newPos) { - Preconditions.checkState(!closed, "Cannot read: already closed"); - Preconditions.checkArgument(newPos >= 0, "position is negative: %s", newPos); - - // this allows a seek beyond the end of the stream but the next read will fail - next = newPos; - } - - // Implement all SeekableInputStream methods - - /** - * Read a byte array of data, from position 0 to the end of the array. - *

- * This method is equivalent to {@code read(bytes, 0, bytes.length)}. - *

- * This method will block until len bytes are available to copy into the - * array, or will throw {@link EOFException} if the stream ends before the - * array is full. - * - * @param bytes a byte array to fill with data from the stream - * @throws IOException If the underlying stream throws IOException - * @throws EOFException If the stream has fewer bytes left than are needed to - * fill the array, {@code bytes.length} - */ - @Override - public void readFully(byte[] bytes) throws IOException { - readFully(bytes, 0, bytes.length); - } - - /** - * Read {@code len} bytes of data into an array, at position {@code start}. - *

- * This method will block until len bytes are available to copy into the - * array, or will throw {@link EOFException} if the stream ends before the - * array is full. - * - * @param bytes a byte array to fill with data from the stream - * @param start the starting position in the byte array for data - * @param len the length of bytes to read into the byte array - * @throws IOException If the underlying stream throws IOException - * @throws EOFException If the stream has fewer than {@code len} bytes left - */ - @Override - public void readFully(byte[] bytes, int start, int len) throws IOException { - Preconditions.checkState(!closed, "Cannot read: already closed"); - positionStream(); - - int bytesRead = readFully(stream, bytes, start, len); - - this.pos += bytesRead; - this.next += bytesRead; - this.bytesCounter.add(bytesRead); - } - - /** - * Read {@code buf.remaining()} bytes of data into a {@link ByteBuffer}. - *

- * This method will copy available bytes into the buffer, reading at most - * {@code buf.remaining()} bytes. The number of bytes actually copied is - * returned by the method, or -1 is returned to signal that the end of the - * underlying stream has been reached. - * - * @param buf a byte buffer to fill with data from the stream - * @return the number of bytes read or -1 if the stream ended - * @throws IOException If the underlying stream throws IOException - */ - @Override - public int read(ByteBuffer buf) throws IOException { - Preconditions.checkState(!closed, "Cannot read: already closed"); - positionStream(); - - int bytesRead = 0; - if (buf.hasArray()) { - bytesRead = readHeapBuffer(stream, buf); - } else { - bytesRead = readDirectBuffer(stream, buf, temp); - } - - this.pos += bytesRead; - this.next += bytesRead; - this.bytesCounter.add(bytesRead); - - return bytesRead; - } - - /** - * Read {@code buf.remaining()} bytes of data into a {@link ByteBuffer}. - *

- * This method will block until {@code buf.remaining()} bytes are available - * to copy into the buffer, or will throw {@link EOFException} if the stream - * ends before the buffer is full. - * - * @param buf a byte buffer to fill with data from the stream - * @throws IOException If the underlying stream throws IOException - * @throws EOFException If the stream has fewer bytes left than are needed to - * fill the buffer, {@code buf.remaining()} - */ - @Override - public void readFully(ByteBuffer buf) throws IOException { - Preconditions.checkState(!closed, "Cannot read: already closed"); - positionStream(); - - int bytesRead = 0; - if (buf.hasArray()) { - bytesRead = readFullyHeapBuffer(stream, buf); - } else { - bytesRead = readFullyDirectBuffer(stream, buf, temp); - } - - this.pos += bytesRead; - this.next += bytesRead; - this.bytesCounter.add(bytesRead); - } - - /** - * Position the stream for reading bytes starting at next offset - * @throws IOException if stream cannot be set correctly - */ - private void positionStream() throws IOException { - - if ((stream != null) && (next == pos)) { - // already at specified position - return; - } - - if ((stream != null) && (next > pos)) { - // seeking forwards - long skip = next - pos; - if (skip <= Math.max(stream.available(), SKIP_SIZE)) { - // already buffered or seek is small enough - LOG.debug("Read-through seek for {} to offset {}", s3ObjectReference, next); - try { - ByteStreams.skipFully(stream, skip); - pos = next; - return; - } catch (IOException ignored) { - // will retry by re-opening the stream - } - } - } - - // close the stream and open at desired position - LOG.debug("Seek with new stream for {} to offset {}", s3ObjectReference, next); - pos = next; - openStream(); - } - - /** - * Open the stream to the S3 object - * @throws IOException if the stream cannot be opened. - */ - private void openStream() throws IOException { - closeStream(); - - if (pos >= metadata.contentLength()) { - stream = InputStream.nullInputStream(); - return; - } - - final GetObjectRequest request = this.getObjectRequestBuilder - .range(String.format("bytes=%s-", pos)) - .build(); - - try { - stream = s3Client.getObject(request, ResponseTransformer.toInputStream()); - } catch (NoSuchKeyException e) { - throw new IOException("Location does not exist: " + s3ObjectReference.toString(), e); - } - } - - /** - * Close the input stream from the S3 object - * @throws IOException if the stream cannot be closed. - */ - private void closeStream() throws IOException { - if (stream != null) { - // if we aren't at the end of the stream, and the stream is abortable, then - // call abort() so we don't read the remaining data with the Apache HTTP client - abortStream(); - try { - stream.close(); - } catch (IOException e) { - // the Apache HTTP client will throw a ConnectionClosedException - // when closing an aborted stream, which is expected - if (!e.getClass().getSimpleName().equals("ConnectionClosedException")) { - throw e; - } - } - stream = null; - } - } - - /** - * Abort the stream to the S3 object. - */ - private void abortStream() { - try { - if (stream instanceof Abortable && stream.read() != -1) { - ((Abortable) stream).abort(); - } - } catch (Exception e) { - LOG.warn("An error occurred while aborting the stream", e); - } - } - - /** - * Read the input stream into the byte buffer with the assumption that the byte buffer is backed by some bytes. - * @param f input stream - * @param buf byte buffer wrapper - * @return bytes read into the buffer - * @throws IOException if bytes cannot be read from input stream into the byte buffer - */ - // Visible for testing - static int readHeapBuffer(InputStream f, ByteBuffer buf) throws IOException { - int bytesRead = f.read(buf.array(), buf.arrayOffset() + buf.position(), buf.remaining()); - if (bytesRead < 0) { - // if this resulted in EOF, don't update position - return bytesRead; - } else { - buf.position(buf.position() + bytesRead); - return bytesRead; - } - } - - /** - * Helper method to read bytes from an input stream into a byte array - * @param f input stream - * @param bytes byte array - * @param start offset into byte array to start reading to - * @param len number of bytes to read into the byte array - * @return number of bytes read into buffer - * @throws IOException if input stream cannot be read - */ - static int readFully(InputStream f, byte[] bytes, int start, int len) throws IOException { - int totalBytesRead = 0; - int offset = start; - int remaining = len; - while (remaining > 0) { - int bytesRead = f.read(bytes, offset, remaining); - if (bytesRead < 0) { - throw new EOFException( - "Reached the end of stream with " + remaining + " bytes left to read"); - } - - remaining -= bytesRead; - offset += bytesRead; - totalBytesRead += bytesRead; - } - return totalBytesRead; - } - - /** - * Read fully into the bytes buffer assuming that the byte buffer is backed by a byte array - * @param f input stream - * @param buf byte buffer - * @return number of bytes read into buffer - * @throws IOException if bytes cannot be read into the byte buffer - */ - // Visible for testing - static int readFullyHeapBuffer(InputStream f, ByteBuffer buf) throws IOException { - int bytesRead = readFully(f, buf.array(), buf.arrayOffset() + buf.position(), buf.remaining()); - buf.position(buf.limit()); - return bytesRead; - } - - /** - * Read into a direct buffer with the assumption that the byte buffer has no backing byte array - * @param f input stream - * @param buf byte buffer - * @param temp byte array to use as a buffer - * @return the number of bytes read - * @throws IOException if the bytes cannot be read from the input stream - */ - // Visible for testing - static int readDirectBuffer(InputStream f, ByteBuffer buf, byte[] temp) throws IOException { - // copy all the bytes that return immediately, stopping at the first - // read that doesn't return a full buffer. - int nextReadLength = Math.min(buf.remaining(), temp.length); - int totalBytesRead = 0; - int bytesRead; - - while ((bytesRead = f.read(temp, 0, nextReadLength)) == temp.length) { - buf.put(temp); - totalBytesRead += bytesRead; - nextReadLength = Math.min(buf.remaining(), temp.length); - } - - if (bytesRead < 0) { - // return -1 if nothing was read - return totalBytesRead == 0 ? -1 : totalBytesRead; - } else { - // copy the last partial buffer - buf.put(temp, 0, bytesRead); - totalBytesRead += bytesRead; - return totalBytesRead; - } - } - - /** - * Read into from the input stream into the byte buffer using the provided byte array as a buffer - * @param f input sream to read from - * @param buf byte buffer to read data into - * @param temp The byte array to use as a buffer for reading. - * @return number of bytes read into buffer - * @throws IOException if the bytes cannot be read - */ - // Visible for testing - static int readFullyDirectBuffer(InputStream f, ByteBuffer buf, byte[] temp) throws IOException { - int totalBytesRead = 0; - int nextReadLength = Math.min(buf.remaining(), temp.length); - int bytesRead = 0; - - while (nextReadLength > 0 && (bytesRead = f.read(temp, 0, nextReadLength)) >= 0) { - buf.put(temp, 0, bytesRead); - nextReadLength = Math.min(buf.remaining(), temp.length); - if (bytesRead >= 0) { - totalBytesRead += bytesRead; - } - } - - if (bytesRead < 0 && buf.remaining() > 0) { - throw new EOFException( - "Reached the end of stream with " + buf.remaining() + " bytes left to read"); - } - - return totalBytesRead; - } -} diff --git a/data-prepper-plugins/parquet-codecs/src/main/java/org/opensearch/dataprepper/plugins/codec/parquet/S3ObjectReference.java b/data-prepper-plugins/parquet-codecs/src/main/java/org/opensearch/dataprepper/plugins/codec/parquet/S3ObjectReference.java deleted file mode 100644 index 09cb196626..0000000000 --- a/data-prepper-plugins/parquet-codecs/src/main/java/org/opensearch/dataprepper/plugins/codec/parquet/S3ObjectReference.java +++ /dev/null @@ -1,72 +0,0 @@ -/* - * Copyright OpenSearch Contributors - * SPDX-License-Identifier: Apache-2.0 - */ - -package org.opensearch.dataprepper.plugins.codec.parquet; - -import java.util.Objects; -import java.util.Optional; - -/** - * Reference to an S3 object. - */ -class S3ObjectReference { - private final String bucketName; - private final String key; - private final String owner; - - private S3ObjectReference(final String bucketName, final String key, final String owner) { - this.bucketName = bucketName; - this.key = key; - this.owner = owner; - } - - static Builder bucketAndKey(final String bucketName, final String key) { - Objects.requireNonNull(bucketName, "bucketName must be non null"); - Objects.requireNonNull(key, "key must be non null"); - return new Builder(bucketName, key); - } - - String getBucketName() { - return bucketName; - } - - String getKey() { - return key; - } - - Optional getBucketOwner() { - return Optional.ofNullable(owner); - } - - @Override - public String toString() { - return "[bucketName=" + bucketName + ", key=" + key + "]"; - } - - public String uri() { - return String.format("s3://%s/%s", bucketName, key); - } - - public static final class Builder { - - private final String bucketName; - private final String key; - private String owner; - - private Builder(final String bucketName, final String key) { - this.bucketName = bucketName; - this.key = key; - } - - public Builder owner(final String owner) { - this.owner = owner; - return this; - } - - public S3ObjectReference build() { - return new S3ObjectReference(bucketName, key, owner); - } - } -} diff --git a/data-prepper-plugins/parquet-codecs/src/test/java/org/opensearch/dataprepper/plugins/codec/parquet/ParquetOutputCodecTest.java b/data-prepper-plugins/parquet-codecs/src/test/java/org/opensearch/dataprepper/plugins/codec/parquet/ParquetOutputCodecTest.java deleted file mode 100644 index 6017f09049..0000000000 --- a/data-prepper-plugins/parquet-codecs/src/test/java/org/opensearch/dataprepper/plugins/codec/parquet/ParquetOutputCodecTest.java +++ /dev/null @@ -1,202 +0,0 @@ -/* - * Copyright OpenSearch Contributors - * SPDX-License-Identifier: Apache-2.0 - */ -package org.opensearch.dataprepper.plugins.codec.parquet; - -import org.apache.avro.Schema; -import org.apache.avro.SchemaBuilder; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.Path; -import org.apache.parquet.ParquetReadOptions; -import org.apache.parquet.column.page.PageReadStore; -import org.apache.parquet.example.data.Group; -import org.apache.parquet.example.data.simple.SimpleGroup; -import org.apache.parquet.example.data.simple.convert.GroupRecordConverter; -import org.apache.parquet.hadoop.ParquetFileReader; -import org.apache.parquet.hadoop.metadata.ParquetMetadata; -import org.apache.parquet.hadoop.util.HadoopInputFile; -import org.apache.parquet.io.ColumnIOFactory; -import org.apache.parquet.io.MessageColumnIO; -import org.apache.parquet.io.RecordReader; -import org.apache.parquet.schema.MessageType; -import org.apache.parquet.schema.Type; -import org.hamcrest.Matchers; -import org.junit.jupiter.api.Test; -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 org.opensearch.dataprepper.model.sink.OutputCodecContext; - -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.io.File; -import java.io.IOException; -import java.io.InputStream; -import java.nio.file.Files; -import java.nio.file.StandardCopyOption; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -import static org.hamcrest.CoreMatchers.equalTo; -import static org.hamcrest.CoreMatchers.notNullValue; -import static org.hamcrest.MatcherAssert.assertThat; -import static org.junit.jupiter.api.Assertions.assertThrows; - -public class ParquetOutputCodecTest { - private static final String FILE_NAME = "parquet-data"; - private static final String FILE_SUFFIX = ".parquet"; - private static int numberOfRecords; - private ParquetOutputCodecConfig config; - - 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); - eventData.put("doubleType", Double.valueOf(rows)); - eventData.put("floatType", Float.valueOf(rows)); - eventData.put("longType", Long.valueOf(rows)); - eventData.put("bytesType", ("Person" + rows).getBytes()); - recordList.add((eventData)); - - } - return recordList; - } - - private static Schema parseSchema() { - return SchemaBuilder.record("Person") - .fields() - .name("name").type().stringType().noDefault() - .name("age").type().intType().noDefault() - .name("doubleType").type().doubleType().noDefault() - .name("floatType").type().floatType().noDefault() - .name("longType").type().longType().noDefault() - .name("bytesType").type().bytesType().noDefault() - .endRecord(); - - } - - private ParquetOutputCodec createObjectUnderTest() { - config = new ParquetOutputCodecConfig(); - config.setSchema(parseSchema().toString()); - config.setBucket("test"); - config.setRegion("test"); - return new ParquetOutputCodec(config); - - } - - @ParameterizedTest - @ValueSource(ints = {1, 2, 10, 100}) - void test_happy_case(final int numberOfRecords) throws Exception { - ParquetOutputCodecTest.numberOfRecords = numberOfRecords; - ParquetOutputCodec parquetOutputCodec = createObjectUnderTest(); - ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); - final File tempFile = File.createTempFile(FILE_NAME, FILE_SUFFIX); - OutputCodecContext codecContext = new OutputCodecContext(); - parquetOutputCodec.start(tempFile, codecContext); - for (int index = 0; index < numberOfRecords; index++) { - final Event event = (Event) getRecord(index).getData(); - parquetOutputCodec.writeEvent(event, outputStream); - } - parquetOutputCodec.closeWriter(outputStream, tempFile); - List> actualRecords = createParquetRecordsList(new ByteArrayInputStream(tempFile.toString().getBytes())); - int index = 0; - for (final HashMap actualMap : actualRecords) { - assertThat(actualMap, notNullValue()); - Map expectedMap = generateRecords(numberOfRecords).get(index); - assertThat(expectedMap, Matchers.equalTo(actualMap)); - index++; - } - tempFile.delete(); - } - - @Test - public void test_getExtension() { - ParquetOutputCodec parquetOutputCodec = createObjectUnderTest(); - String extension = parquetOutputCodec.getExtension(); - - assertThat(extension, equalTo("parquet")); - } - - @Test - public void whenNoSchemaProvided_thenThrowsException() { - config = new ParquetOutputCodecConfig(); - config.setSchema(null); - config.setFileLocation(null); - config.setSchemaRegistryUrl(null); - ParquetOutputCodec parquetOutputCodec = new ParquetOutputCodec(config); - assertThrows(IOException.class, () -> - parquetOutputCodec.buildSchemaAndKey(null, null)); - } - - @Test - public void test_s3SchemaValidity() throws IOException { - config = new ParquetOutputCodecConfig(); - config.setSchema(parseSchema().toString()); - config.setSchemaBucket("test"); - config.setSchemaRegion("test"); - config.setFileKey("test"); - ParquetOutputCodec parquetOutputCodec = new ParquetOutputCodec(config); - assertThat(parquetOutputCodec.checkS3SchemaValidity(), equalTo(Boolean.TRUE)); - ParquetOutputCodec parquetOutputCodecFalse = createObjectUnderTest(); - assertThrows(IOException.class, () -> - parquetOutputCodecFalse.checkS3SchemaValidity()); - } - - private List> createParquetRecordsList(final InputStream inputStream) throws IOException { - - final File tempFile = File.createTempFile(FILE_NAME, FILE_SUFFIX); - Files.copy(inputStream, tempFile.toPath(), StandardCopyOption.REPLACE_EXISTING); - List> actualRecordList = new ArrayList<>(); - try (ParquetFileReader parquetFileReader = new ParquetFileReader(HadoopInputFile.fromPath(new Path(tempFile.toURI()), new Configuration()), ParquetReadOptions.builder().build())) { - final ParquetMetadata footer = parquetFileReader.getFooter(); - final MessageType schema = createdParquetSchema(footer); - PageReadStore pages; - - while ((pages = parquetFileReader.readNextRowGroup()) != null) { - final long rows = pages.getRowCount(); - final MessageColumnIO columnIO = new ColumnIOFactory().getColumnIO(schema); - final RecordReader recordReader = columnIO.getRecordReader(pages, new GroupRecordConverter(schema)); - for (int row = 0; row < rows; row++) { - final Map eventData = new HashMap<>(); - int fieldIndex = 0; - final SimpleGroup simpleGroup = (SimpleGroup) recordReader.read(); - for (Type field : schema.getFields()) { - try { - eventData.put(field.getName(), simpleGroup.getValueToString(fieldIndex, 0)); - } catch (Exception parquetException) { - parquetException.printStackTrace(); - } - fieldIndex++; - } - actualRecordList.add((HashMap) eventData); - } - } - } catch (Exception parquetException) { - parquetException.printStackTrace(); - } finally { - Files.delete(tempFile.toPath()); - } - return actualRecordList; - } - - private MessageType createdParquetSchema(ParquetMetadata parquetMetadata) { - return parquetMetadata.getFileMetaData().getSchema(); - } -} \ No newline at end of file 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 30cc3a053b..9e31a368d9 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 @@ -7,6 +7,8 @@ import com.fasterxml.jackson.core.JsonEncoding; import com.fasterxml.jackson.core.JsonFactory; import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.core.JsonProcessingException; +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; @@ -15,7 +17,10 @@ import java.io.IOException; import java.io.OutputStream; +import java.util.Map; import java.util.Objects; +import java.util.function.Function; +import java.util.stream.Collectors; /** * An implementation of {@link OutputCodec} which deserializes Data-Prepper events @@ -23,10 +28,10 @@ */ @DataPrepperPlugin(name = "json", pluginType = OutputCodec.class, pluginConfigurationType = JsonOutputCodecConfig.class) public class JsonOutputCodec implements OutputCodec { - + private final ObjectMapper objectMapper = new ObjectMapper(); private static final String JSON = "json"; private static final JsonFactory factory = new JsonFactory(); - JsonOutputCodecConfig config; + private final JsonOutputCodecConfig config; private JsonGenerator generator; private OutputCodecContext codecContext; @@ -36,6 +41,11 @@ public JsonOutputCodec(final JsonOutputCodecConfig config) { this.config = config; } + @Override + public String getExtension() { + return JSON; + } + @Override public void start(final OutputStream outputStream, Event event, final OutputCodecContext codecContext) throws IOException { Objects.requireNonNull(outputStream); @@ -59,27 +69,30 @@ public void complete(final OutputStream outputStream) throws IOException { @Override public synchronized void writeEvent(final Event event, final OutputStream outputStream) throws IOException { Objects.requireNonNull(event); + Map dataMap = getDataMapToSerialize(event); + objectMapper.writeValue(generator, dataMap); + generator.flush(); + } + + private Map getDataMapToSerialize(Event event) throws JsonProcessingException { final Event modifiedEvent; if (codecContext.getTagsTargetKey() != null) { modifiedEvent = addTagsToEvent(event, codecContext.getTagsTargetKey()); } else { modifiedEvent = event; } - generator.writeStartObject(); - final boolean isExcludeKeyAvailable = !codecContext.getExcludeKeys().isEmpty(); - for (final String key : modifiedEvent.toMap().keySet()) { - if (isExcludeKeyAvailable && codecContext.getExcludeKeys().contains(key)) { - continue; - } - generator.writeStringField(key, modifiedEvent.toMap().get(key).toString()); - } - generator.writeEndObject(); - generator.flush(); - } + Map dataMap = modifiedEvent.toMap(); - @Override - public String getExtension() { - return JSON; + if ((codecContext.getIncludeKeys() != null && !codecContext.getIncludeKeys().isEmpty()) || + (codecContext.getExcludeKeys() != null && !codecContext.getExcludeKeys().isEmpty())) { + + Map finalDataMap = dataMap; + dataMap = dataMap.keySet() + .stream() + .filter(codecContext::shouldIncludeKey) + .collect(Collectors.toMap(Function.identity(), finalDataMap::get)); + } + return dataMap; } } diff --git a/data-prepper-plugins/parse-json-processor/src/test/java/org/opensearch/dataprepper/plugins/codec/json/JsonOutputCodecTest.java b/data-prepper-plugins/parse-json-processor/src/test/java/org/opensearch/dataprepper/plugins/codec/json/JsonOutputCodecTest.java index 073a768cfc..7376300440 100644 --- a/data-prepper-plugins/parse-json-processor/src/test/java/org/opensearch/dataprepper/plugins/codec/json/JsonOutputCodecTest.java +++ b/data-prepper-plugins/parse-json-processor/src/test/java/org/opensearch/dataprepper/plugins/codec/json/JsonOutputCodecTest.java @@ -8,13 +8,11 @@ import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.node.JsonNodeType; -import org.hamcrest.Matchers; import org.junit.jupiter.api.Test; 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 org.opensearch.dataprepper.model.sink.OutputCodecContext; import java.io.ByteArrayOutputStream; @@ -30,50 +28,103 @@ import static org.hamcrest.MatcherAssert.assertThat; class JsonOutputCodecTest { - - private static int numberOfRecords; private ByteArrayOutputStream outputStream; - 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 JsonOutputCodec createObjectUnderTest() { + return new JsonOutputCodec(new JsonOutputCodecConfig()); } - private static List generateRecords(int numberOfRecords) { + @ParameterizedTest + @ValueSource(ints = {1, 2, 10, 100}) + void test_happy_case(final int numberOfRecords) throws IOException { + JsonOutputCodec jsonOutputCodec = createObjectUnderTest(); + outputStream = new ByteArrayOutputStream(); + OutputCodecContext codecContext = new OutputCodecContext(); + jsonOutputCodec.start(outputStream, null, codecContext); - List recordList = new ArrayList<>(); + final List> expectedData = generateRecords(numberOfRecords); + for (int index = 0; index < numberOfRecords; index++) { + final Event event = convertToEvent(expectedData.get(index)); + jsonOutputCodec.writeEvent(event, outputStream); + } + jsonOutputCodec.complete(outputStream); - for (int rows = 0; rows < numberOfRecords; rows++) { + int index = 0; + ObjectMapper mapper = new ObjectMapper(); + JsonNode jsonNode = mapper.readTree(outputStream.toByteArray()); + assertThat(jsonNode.getNodeType(), equalTo(JsonNodeType.OBJECT)); + Map.Entry nextField = jsonNode.fields().next(); + assertThat(nextField, notNullValue()); + assertThat(nextField.getKey(), equalTo(JsonOutputCodecConfig.DEFAULT_KEY_NAME)); + jsonNode = nextField.getValue(); + assertThat(jsonNode, notNullValue()); + assertThat(jsonNode.getNodeType(), equalTo(JsonNodeType.ARRAY)); + for (JsonNode actualElement : jsonNode) { + Map expectedMap = expectedData.get(index); + Set keys = expectedMap.keySet(); + Map actualMap = new HashMap<>(); + for (String key : keys) { + actualMap.put(key, getValue(actualElement.get(key))); + } + assertThat(actualMap, equalTo(expectedMap)); + index++; + } - HashMap eventData = new HashMap<>(); + assertThat(index, equalTo(numberOfRecords)); + } - eventData.put("name", "Person" + rows); - eventData.put("age", Integer.toString(rows)); - recordList.add((eventData)); + @ParameterizedTest + @ValueSource(ints = {1, 2, 10, 100}) + void writeEvent_with_include_keys(final int numberOfRecords) throws IOException { + JsonOutputCodec jsonOutputCodec = createObjectUnderTest(); + outputStream = new ByteArrayOutputStream(); + OutputCodecContext codecContext = new OutputCodecContext(null, List.of("name"), null); + jsonOutputCodec.start(outputStream, null, codecContext); + final List> expectedData = generateRecords(numberOfRecords); + for (int index = 0; index < numberOfRecords; index++) { + final Event event = convertToEvent(expectedData.get(index)); + jsonOutputCodec.writeEvent(event, outputStream); } - return recordList; - } + jsonOutputCodec.complete(outputStream); - private JsonOutputCodec createObjectUnderTest() { - return new JsonOutputCodec(new JsonOutputCodecConfig()); + int index = 0; + ObjectMapper mapper = new ObjectMapper(); + JsonNode jsonNode = mapper.readTree(outputStream.toByteArray()); + assertThat(jsonNode.getNodeType(), equalTo(JsonNodeType.OBJECT)); + Map.Entry nextField = jsonNode.fields().next(); + assertThat(nextField, notNullValue()); + assertThat(nextField.getKey(), equalTo(JsonOutputCodecConfig.DEFAULT_KEY_NAME)); + jsonNode = nextField.getValue(); + assertThat(jsonNode, notNullValue()); + assertThat(jsonNode.getNodeType(), equalTo(JsonNodeType.ARRAY)); + for (JsonNode actualElement : jsonNode) { + Map expectedMap = expectedData.get(index); + assertThat(actualElement.has("age"), equalTo(false)); + assertThat(actualElement.has("name"), equalTo(true)); + assertThat(actualElement.get("name").getNodeType(), equalTo(JsonNodeType.STRING)); + assertThat(actualElement.get("name").asText(), equalTo(expectedMap.get("name"))); + index++; + } + + assertThat(index, equalTo(numberOfRecords)); } @ParameterizedTest @ValueSource(ints = {1, 2, 10, 100}) - void test_happy_case(final int numberOfRecords) throws IOException { - JsonOutputCodecTest.numberOfRecords = numberOfRecords; + void writeEvent_with_exclude_keys(final int numberOfRecords) throws IOException { JsonOutputCodec jsonOutputCodec = createObjectUnderTest(); outputStream = new ByteArrayOutputStream(); - OutputCodecContext codecContext = new OutputCodecContext(); + OutputCodecContext codecContext = new OutputCodecContext(null, null, List.of("age")); jsonOutputCodec.start(outputStream, null, codecContext); + + final List> expectedData = generateRecords(numberOfRecords); for (int index = 0; index < numberOfRecords; index++) { - final Event event = (Event) getRecord(index).getData(); + final Event event = convertToEvent(expectedData.get(index)); jsonOutputCodec.writeEvent(event, outputStream); } jsonOutputCodec.complete(outputStream); - List expectedRecords = generateRecords(numberOfRecords); + int index = 0; ObjectMapper mapper = new ObjectMapper(); JsonNode jsonNode = mapper.readTree(outputStream.toByteArray()); @@ -84,16 +135,16 @@ void test_happy_case(final int numberOfRecords) throws IOException { jsonNode = nextField.getValue(); assertThat(jsonNode, notNullValue()); assertThat(jsonNode.getNodeType(), equalTo(JsonNodeType.ARRAY)); - for (JsonNode element : jsonNode) { - Set keys = expectedRecords.get(index).keySet(); - Map actualMap = new HashMap<>(); - for (String key : keys) { - actualMap.put(key, element.get(key).asText()); - } - assertThat(expectedRecords.get(index), Matchers.equalTo(actualMap)); + for (JsonNode actualElement : jsonNode) { + Map expectedMap = expectedData.get(index); + assertThat(actualElement.has("age"), equalTo(false)); + assertThat(actualElement.has("name"), equalTo(true)); + assertThat(actualElement.get("name").getNodeType(), equalTo(JsonNodeType.STRING)); + assertThat(actualElement.get("name").asText(), equalTo(expectedMap.get("name"))); index++; - } + + assertThat(index, equalTo(numberOfRecords)); } @Test @@ -102,4 +153,36 @@ void testGetExtension() { assertThat("json", equalTo(jsonOutputCodec.getExtension())); } + + private static Event convertToEvent(Map data) { + return JacksonLog.builder().withData(data).build(); + } + + private static List> generateRecords(int numberOfRecords) { + + List> recordList = new ArrayList<>(); + + for (int rows = 0; rows < numberOfRecords; rows++) { + + Map eventData = new HashMap<>(); + + eventData.put("name", "Person" + rows); + eventData.put("age", rows); + recordList.add(eventData); + + } + + return recordList; + } + + + private Object getValue(JsonNode jsonNode) { + if(jsonNode.isTextual()) + return jsonNode.asText(); + + if(jsonNode.isInt()) + return jsonNode.asInt(); + + throw new RuntimeException("Test not setup correctly."); + } } \ No newline at end of file diff --git a/data-prepper-plugins/s3-sink/build.gradle b/data-prepper-plugins/s3-sink/build.gradle index 5d74fd169d..c45d640cc9 100644 --- a/data-prepper-plugins/s3-sink/build.gradle +++ b/data-prepper-plugins/s3-sink/build.gradle @@ -17,16 +17,20 @@ dependencies { implementation 'software.amazon.awssdk:s3' implementation 'software.amazon.awssdk:sts' implementation 'org.jetbrains.kotlin:kotlin-stdlib:1.8.21' + implementation project(':data-prepper-plugins:avro-codecs') implementation 'org.apache.avro:avro:1.11.1' implementation 'org.apache.hadoop:hadoop-common:3.3.6' implementation 'org.apache.parquet:parquet-avro:1.13.1' + implementation 'software.amazon.awssdk:apache-client' implementation 'org.jetbrains.kotlin:kotlin-stdlib-common:1.8.21' implementation libs.commons.lang3 testImplementation project(':data-prepper-test-common') testImplementation project(':data-prepper-plugins:parquet-codecs') testImplementation project(':data-prepper-plugins:parse-json-processor') testImplementation project(':data-prepper-plugins:csv-processor') - testImplementation project(':data-prepper-plugins:avro-codecs') + testImplementation testLibs.slf4j.simple + testImplementation 'software.amazon.awssdk:s3-transfer-manager' + testImplementation 'software.amazon.awssdk.crt:aws-crt:0.25.0' } test { @@ -56,10 +60,12 @@ task integrationTest(type: Test) { useJUnitPlatform() classpath = sourceSets.integrationTest.runtimeClasspath + + systemProperty 'log4j.configurationFile', 'src/test/resources/log4j2.properties' systemProperty 'tests.s3sink.bucket', System.getProperty('tests.s3sink.bucket') systemProperty 'tests.s3sink.region', System.getProperty('tests.s3sink.region') filter { includeTestsMatching '*IT' } -} \ No newline at end of file +} diff --git a/data-prepper-plugins/s3-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/s3/BufferScenario.java b/data-prepper-plugins/s3-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/s3/BufferScenario.java new file mode 100644 index 0000000000..3c67de1093 --- /dev/null +++ b/data-prepper-plugins/s3-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/s3/BufferScenario.java @@ -0,0 +1,13 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.sink.s3; + +import org.opensearch.dataprepper.plugins.sink.s3.accumulator.BufferTypeOptions; + +public interface BufferScenario { + BufferTypeOptions getBufferType(); + int getMaximumNumberOfEvents(); +} diff --git a/data-prepper-plugins/s3-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/s3/CompressionScenario.java b/data-prepper-plugins/s3-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/s3/CompressionScenario.java new file mode 100644 index 0000000000..953bbe468a --- /dev/null +++ b/data-prepper-plugins/s3-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/s3/CompressionScenario.java @@ -0,0 +1,22 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.sink.s3; + +import org.opensearch.dataprepper.plugins.sink.s3.compression.CompressionOption; + +import java.io.IOException; +import java.io.InputStream; + +/** + * A scenario for whole-file compression. + */ +public interface CompressionScenario { + CompressionOption getCompressionOption(); + default boolean requiresDecompression() { + return true; + } + InputStream decompressingInputStream(final InputStream inputStream) throws IOException; +} diff --git a/data-prepper-plugins/s3-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/s3/GZipCompressionScenario.java b/data-prepper-plugins/s3-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/s3/GZipCompressionScenario.java new file mode 100644 index 0000000000..ec80e6bfe8 --- /dev/null +++ b/data-prepper-plugins/s3-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/s3/GZipCompressionScenario.java @@ -0,0 +1,29 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.sink.s3; + +import org.apache.commons.compress.compressors.gzip.GzipCompressorInputStream; +import org.opensearch.dataprepper.plugins.sink.s3.compression.CompressionOption; + +import java.io.IOException; +import java.io.InputStream; + +public class GZipCompressionScenario implements CompressionScenario { + @Override + public CompressionOption getCompressionOption() { + return CompressionOption.GZIP; + } + + @Override + public InputStream decompressingInputStream(final InputStream inputStream) throws IOException { + return new GzipCompressorInputStream(inputStream); + } + + @Override + public String toString() { + return "GZip"; + } +} diff --git a/data-prepper-plugins/s3-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/s3/InMemoryBufferScenario.java b/data-prepper-plugins/s3-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/s3/InMemoryBufferScenario.java new file mode 100644 index 0000000000..db58354dbe --- /dev/null +++ b/data-prepper-plugins/s3-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/s3/InMemoryBufferScenario.java @@ -0,0 +1,22 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.sink.s3; + +import org.opensearch.dataprepper.plugins.sink.s3.accumulator.BufferTypeOptions; + +import static org.opensearch.dataprepper.plugins.sink.s3.S3SinkIT.MEDIUM_OBJECT_SIZE; + +public class InMemoryBufferScenario implements BufferScenario { + @Override + public BufferTypeOptions getBufferType() { + return BufferTypeOptions.INMEMORY; + } + + @Override + public int getMaximumNumberOfEvents() { + return MEDIUM_OBJECT_SIZE; + } +} diff --git a/data-prepper-plugins/s3-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/s3/LocalFileBufferScenario.java b/data-prepper-plugins/s3-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/s3/LocalFileBufferScenario.java new file mode 100644 index 0000000000..f6c45d656f --- /dev/null +++ b/data-prepper-plugins/s3-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/s3/LocalFileBufferScenario.java @@ -0,0 +1,22 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.sink.s3; + +import org.opensearch.dataprepper.plugins.sink.s3.accumulator.BufferTypeOptions; + +import static org.opensearch.dataprepper.plugins.sink.s3.S3SinkIT.LARGE_OBJECT_SIZE; + +public class LocalFileBufferScenario implements BufferScenario { + @Override + public BufferTypeOptions getBufferType() { + return BufferTypeOptions.LOCALFILE; + } + + @Override + public int getMaximumNumberOfEvents() { + return LARGE_OBJECT_SIZE; + } +} diff --git a/data-prepper-plugins/s3-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/s3/MultiPartBufferScenario.java b/data-prepper-plugins/s3-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/s3/MultiPartBufferScenario.java new file mode 100644 index 0000000000..3d26b3dc82 --- /dev/null +++ b/data-prepper-plugins/s3-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/s3/MultiPartBufferScenario.java @@ -0,0 +1,22 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.sink.s3; + +import org.opensearch.dataprepper.plugins.sink.s3.accumulator.BufferTypeOptions; + +import static org.opensearch.dataprepper.plugins.sink.s3.S3SinkIT.LARGE_OBJECT_SIZE; + +public class MultiPartBufferScenario implements BufferScenario { + @Override + public BufferTypeOptions getBufferType() { + return BufferTypeOptions.MULTI_PART; + } + + @Override + public int getMaximumNumberOfEvents() { + return LARGE_OBJECT_SIZE; + } +} diff --git a/data-prepper-plugins/s3-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/s3/NdjsonOutputScenario.java b/data-prepper-plugins/s3-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/s3/NdjsonOutputScenario.java new file mode 100644 index 0000000000..85ae980540 --- /dev/null +++ b/data-prepper-plugins/s3-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/s3/NdjsonOutputScenario.java @@ -0,0 +1,63 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.sink.s3; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.opensearch.dataprepper.model.codec.OutputCodec; +import org.opensearch.dataprepper.plugins.codec.json.NdjsonOutputCodec; +import org.opensearch.dataprepper.plugins.codec.json.NdjsonOutputConfig; + +import java.io.BufferedInputStream; +import java.io.File; +import java.io.FileInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.util.List; +import java.util.Map; +import java.util.Scanner; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.MatcherAssert.assertThat; + +public class NdjsonOutputScenario implements OutputScenario { + public static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + + @Override + public OutputCodec getCodec() { + return new NdjsonOutputCodec(new NdjsonOutputConfig()); + } + + @Override + public void validate(int expectedRecords, final List> sampleEventData, final File actualContentFile, CompressionScenario compressionScenario) throws IOException { + final InputStream inputStream = new BufferedInputStream(new FileInputStream(actualContentFile), 64 * 1024); + + final Scanner scanner = new Scanner(inputStream); + + int count = 0; + int sampledData = 0; + while (scanner.hasNext()) { + + final String actualJsonString = scanner.next(); + + final Map actualData = OBJECT_MAPPER.readValue(actualJsonString, Map.class); + + if(count < sampleEventData.size()) { + final Map expectedData = sampleEventData.get(count); + assertThat(actualData, equalTo(expectedData)); + sampledData++; + } + count++; + } + + assertThat(count, equalTo(expectedRecords)); + assertThat(sampledData, equalTo(sampleEventData.size())); + } + + @Override + public String toString() { + return "NDJSON"; + } +} diff --git a/data-prepper-plugins/s3-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/s3/NoneCompressionScenario.java b/data-prepper-plugins/s3-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/s3/NoneCompressionScenario.java new file mode 100644 index 0000000000..df85cb27c7 --- /dev/null +++ b/data-prepper-plugins/s3-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/s3/NoneCompressionScenario.java @@ -0,0 +1,33 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.sink.s3; + +import org.opensearch.dataprepper.plugins.sink.s3.compression.CompressionOption; + +import java.io.IOException; +import java.io.InputStream; + +public class NoneCompressionScenario implements CompressionScenario { + @Override + public CompressionOption getCompressionOption() { + return CompressionOption.NONE; + } + + @Override + public InputStream decompressingInputStream(final InputStream inputStream) throws IOException { + return inputStream; + } + + @Override + public boolean requiresDecompression() { + return false; + } + + @Override + public String toString() { + return "Uncompressed"; + } +} diff --git a/data-prepper-plugins/s3-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/s3/OutputScenario.java b/data-prepper-plugins/s3-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/s3/OutputScenario.java new file mode 100644 index 0000000000..e60fce299d --- /dev/null +++ b/data-prepper-plugins/s3-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/s3/OutputScenario.java @@ -0,0 +1,53 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.sink.s3; + +import org.opensearch.dataprepper.model.codec.OutputCodec; +import org.opensearch.dataprepper.plugins.sink.s3.accumulator.BufferTypeOptions; + +import java.io.File; +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Set; + +/** + * Represents a scenario for the output format. + */ +public interface OutputScenario { + /** + * Gets the codec this scenario uses. + * + * @return The {@link OutputCodec} + */ + OutputCodec getCodec(); + + /** + * Returns true if the approach to this output is to compress internally to the data. + * For example, Parquet should not have whole-file compression. The compression only + * should happen inside individual row groups. + * @return True if the compression is internal to the format; false if whole-file compression is ok. + */ + default boolean isCompressionInternal() { + return false; + } + + default Set getIncompatibleBufferTypes() { + return Collections.emptySet(); + } + + /** + * Validates the data against the sample events provided. + * + * @param expectedRecords The total expected records + * @param sampleEventData The collection of all the sample event data as maps. This is the first N values. + * @param actualContentFile The actual file which has been downloaded and decompressed as part of the test + * @param compressionScenario The compression scenario. This should only be needed when {@link #isCompressionInternal()} is true. + * @throws IOException Some IOException + */ + void validate(int expectedRecords, List> sampleEventData, File actualContentFile, CompressionScenario compressionScenario) throws IOException; +} diff --git a/data-prepper-plugins/s3-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/s3/ParquetOutputScenario.java b/data-prepper-plugins/s3-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/s3/ParquetOutputScenario.java new file mode 100644 index 0000000000..7f67048ac1 --- /dev/null +++ b/data-prepper-plugins/s3-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/s3/ParquetOutputScenario.java @@ -0,0 +1,147 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.sink.s3; + +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.util.Utf8; +import org.apache.parquet.ParquetReadOptions; +import org.apache.parquet.avro.AvroParquetReader; +import org.apache.parquet.hadoop.ParquetFileReader; +import org.apache.parquet.hadoop.ParquetReader; +import org.apache.parquet.hadoop.metadata.BlockMetaData; +import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData; +import org.apache.parquet.hadoop.metadata.CompressionCodecName; +import org.apache.parquet.hadoop.metadata.ParquetMetadata; +import org.apache.parquet.io.InputFile; +import org.opensearch.dataprepper.model.codec.OutputCodec; +import org.opensearch.dataprepper.plugins.codec.parquet.ParquetOutputCodec; +import org.opensearch.dataprepper.plugins.codec.parquet.ParquetOutputCodecConfig; +import org.opensearch.dataprepper.plugins.fs.LocalInputFile; +import org.opensearch.dataprepper.plugins.sink.s3.accumulator.BufferTypeOptions; +import org.opensearch.dataprepper.plugins.sink.s3.compression.CompressionOption; + +import java.io.File; +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.instanceOf; +import static org.hamcrest.CoreMatchers.notNullValue; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.greaterThanOrEqualTo; + +public class ParquetOutputScenario implements OutputScenario { + @Override + public OutputCodec getCodec() { + return new ParquetOutputCodec(new ParquetOutputCodecConfig()); + } + + @Override + public boolean isCompressionInternal() { + return true; + } + + @Override + public Set getIncompatibleBufferTypes() { + return Set.of(BufferTypeOptions.LOCALFILE, BufferTypeOptions.INMEMORY); + } + + @Override + public void validate(int expectedRecords, final List> sampleEventData, final File actualContentFile, CompressionScenario compressionScenario) throws IOException { + final InputFile inputFile = new LocalInputFile(actualContentFile); + + CompressionCodecName expectedCompressionCodec = determineCompressionCodec(compressionScenario.getCompressionOption()); + validateParquetStructure(expectedRecords, sampleEventData, inputFile, expectedCompressionCodec); + + final Map> eventsById = sampleEventData.stream() + .collect(Collectors.toMap(data -> (String) data.get("id"), data -> data)); + + int validatedRecords = 0; + + int count = 0; + try (final ParquetReader reader = AvroParquetReader.builder(inputFile) + .build()) { + GenericRecord record; + + while ((record = reader.read()) != null) { + assertThat(record.getSchema(), notNullValue()); + + assertThat(record.hasField("id"), equalTo(true)); + assertThat(record.get("id"), instanceOf(Utf8.class)); + final String id = record.get("id").toString(); + + final Map expectedData = eventsById.get(id); + if(expectedData != null) { + for (String key : expectedData.keySet()) { + assertThat(record.hasField(key), notNullValue()); + assertThat(record.get(key), notNullValue()); + + final Object expectedValue = expectedData.get(key); + if (expectedValue instanceof String) { + assertThat(record.get(key), instanceOf(Utf8.class)); + assertThat(record.get(key).toString(), equalTo(expectedValue)); + } + } + validatedRecords++; + } + + count++; + } + } + + assertThat(count, equalTo(expectedRecords)); + + assertThat("Not all the sample data was validated.", validatedRecords, equalTo(sampleEventData.size())); + } + + private static void validateParquetStructure(int expectedRecords, final List> allEventData, final InputFile inputFile, CompressionCodecName expectedCompressionCodec) throws IOException { + // This test assumes that the data all has the same keys. + final Map sampleEvent = allEventData.iterator().next(); + + try (final ParquetFileReader parquetFileReader = new ParquetFileReader(inputFile, ParquetReadOptions.builder().build())) { + final ParquetMetadata footer = parquetFileReader.getFooter(); + + assertThat(footer, notNullValue()); + assertThat(footer.getFileMetaData(), notNullValue()); + + assertThat(parquetFileReader.getRecordCount(), equalTo((long) expectedRecords)); + assertThat(parquetFileReader.getFileMetaData().getSchema(), notNullValue()); + + assertThat(parquetFileReader.getRowGroups(), notNullValue()); + assertThat(parquetFileReader.getRowGroups().size(), greaterThanOrEqualTo(1)); + + for (BlockMetaData rowGroup : parquetFileReader.getRowGroups()) { + assertThat(rowGroup.getColumns(), notNullValue()); + assertThat(rowGroup.getColumns().size(), equalTo(sampleEvent.keySet().size())); + for (ColumnChunkMetaData column : rowGroup.getColumns()) { + assertThat(column.getCodec(), equalTo(expectedCompressionCodec)); + } + + } + } + } + + @Override + public String toString() { + return "Parquet"; + } + + private static CompressionCodecName determineCompressionCodec(CompressionOption compressionOption) { + switch (compressionOption) { + case NONE: + return CompressionCodecName.UNCOMPRESSED; + case GZIP: + return CompressionCodecName.GZIP; + case SNAPPY: + return CompressionCodecName.SNAPPY; + } + + throw new RuntimeException("The provided compression option is not supported by Parquet or is not configured for testing: " + compressionOption.getOption()); + } +} diff --git a/data-prepper-plugins/s3-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/s3/S3SinkIT.java b/data-prepper-plugins/s3-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/s3/S3SinkIT.java new file mode 100644 index 0000000000..dfb9b18a07 --- /dev/null +++ b/data-prepper-plugins/s3-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/s3/S3SinkIT.java @@ -0,0 +1,331 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.sink.s3; + +import org.apache.commons.io.IOUtils; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.extension.ExtensionContext; +import org.junit.jupiter.api.io.TempDir; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.ArgumentsProvider; +import org.junit.jupiter.params.provider.ArgumentsSource; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +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.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.plugin.PluginFactory; +import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.model.sink.SinkContext; +import org.opensearch.dataprepper.model.types.ByteCount; +import org.opensearch.dataprepper.plugins.sink.s3.accumulator.BufferTypeOptions; +import org.opensearch.dataprepper.plugins.sink.s3.configuration.AwsAuthenticationOptions; +import org.opensearch.dataprepper.plugins.sink.s3.configuration.ObjectKeyOptions; +import org.opensearch.dataprepper.plugins.sink.s3.configuration.ThresholdOptions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; +import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider; +import software.amazon.awssdk.regions.Region; +import software.amazon.awssdk.services.s3.S3AsyncClient; +import software.amazon.awssdk.services.s3.S3Client; +import software.amazon.awssdk.services.s3.model.GetObjectRequest; +import software.amazon.awssdk.services.s3.model.ListObjectsV2Request; +import software.amazon.awssdk.services.s3.model.ListObjectsV2Response; +import software.amazon.awssdk.services.s3.model.S3Object; +import software.amazon.awssdk.transfer.s3.S3TransferManager; +import software.amazon.awssdk.transfer.s3.model.DownloadFileRequest; +import software.amazon.awssdk.transfer.s3.model.FileDownload; + +import java.io.File; +import java.io.FileInputStream; +import java.io.FileOutputStream; +import java.io.IOException; +import java.time.Duration; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.time.format.DateTimeFormatter; +import java.util.ArrayList; +import java.util.Collections; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Random; +import java.util.UUID; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import java.util.stream.Stream; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.notNullValue; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.jupiter.params.provider.Arguments.arguments; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +@ExtendWith(MockitoExtension.class) +public class S3SinkIT { + private static final Logger LOG = LoggerFactory.getLogger(S3SinkIT.class); + private static final Random RANDOM = new Random(); + + static final int MEDIUM_OBJECT_SIZE = 50 * 500; + static final int LARGE_OBJECT_SIZE = 500 * 2_000; + + private static List reusableRandomStrings; + + @Mock + private PluginSetting pluginSetting; + @Mock + private S3SinkConfig s3SinkConfig; + @Mock + private PluginFactory pluginFactory; + @Mock + private SinkContext sinkContext; + @Mock + private AwsCredentialsSupplier awsCredentialsSupplier; + + @Mock + private ThresholdOptions thresholdOptions; + @Mock + private ObjectKeyOptions objectKeyOptions; + private String s3region; + private String bucketName; + private S3Client s3Client; + + @TempDir + private File s3FileLocation; + private S3TransferManager transferManager; + private static String pathPrefixForTestSuite; + + @BeforeAll + static void setUpAll() { + LocalDateTime now = LocalDateTime.now(); + String datePart = LocalDate.from(now).format(DateTimeFormatter.ofPattern("yyyy/MM/dd")); + String time = LocalTime.from(now).toString(); + pathPrefixForTestSuite = datePart + "/" + time + "-" + UUID.randomUUID() + "/"; + + int totalRandomStrings = 1_000; + reusableRandomStrings = new ArrayList<>(totalRandomStrings); + for (int i = 0; i < totalRandomStrings; i++) { + reusableRandomStrings.add(UUID.randomUUID().toString()); + } + } + + @BeforeEach + void setUp() { + when(pluginSetting.getPipelineName()).thenReturn(UUID.randomUUID().toString()); + when(pluginSetting.getName()).thenReturn("s3"); + + s3region = System.getProperty("tests.s3sink.region"); + + final AwsCredentialsProvider awsCredentialsProvider = DefaultCredentialsProvider.create(); + + final Region region = Region.of(s3region); + s3Client = S3Client.builder().region(region).build(); + bucketName = System.getProperty("tests.s3sink.bucket"); + + when(s3SinkConfig.getBucketName()).thenReturn(bucketName); + when(objectKeyOptions.getNamePattern()).thenReturn("events-%{yyyy-MM-dd'T'hh-mm-ss}"); + when(s3SinkConfig.getObjectKeyOptions()).thenReturn(objectKeyOptions); + + when(s3SinkConfig.getThresholdOptions()).thenReturn(thresholdOptions); + when(thresholdOptions.getEventCollectTimeOut()).thenReturn(Duration.ofDays(1)); + when(thresholdOptions.getMaximumSize()).thenReturn(ByteCount.parse("1gb")); + + final PluginModel pluginModel = mock(PluginModel.class); + when(s3SinkConfig.getCodec()).thenReturn(pluginModel); + when(pluginModel.getPluginName()).thenReturn(UUID.randomUUID().toString()); + when(pluginModel.getPluginSettings()).thenReturn(Collections.emptyMap()); + + final AwsAuthenticationOptions awsAuthenticationOptions = mock(AwsAuthenticationOptions.class); + when(s3SinkConfig.getAwsAuthenticationOptions()).thenReturn(awsAuthenticationOptions); + when(awsAuthenticationOptions.getAwsRegion()).thenReturn(region); + + when(awsCredentialsSupplier.getProvider(any())).thenReturn(awsCredentialsProvider); + + final S3AsyncClient s3AsyncClient = S3AsyncClient + .crtBuilder() + .credentialsProvider(awsCredentialsProvider) + .region(region) + .build(); + + transferManager = S3TransferManager.builder() + .s3Client(s3AsyncClient) + .build(); + } + + private S3Sink createObjectUnderTest() { + return new S3Sink(pluginSetting, s3SinkConfig, pluginFactory, sinkContext, awsCredentialsSupplier); + } + + @ParameterizedTest + @ArgumentsSource(IntegrationTestArguments.class) + void test(final OutputScenario outputScenario, final BufferTypeOptions bufferTypeOptions, final CompressionScenario compressionScenario, final int batchSize, final int numberOfBatches) throws IOException { + + String testRun = outputScenario + "-" + bufferTypeOptions + "-" + compressionScenario + "-" + batchSize + "-" + numberOfBatches; + final String pathPrefix = pathPrefixForTestSuite + testRun; + when(objectKeyOptions.getPathPrefix()).thenReturn(pathPrefix + "/"); + + when(pluginFactory.loadPlugin(eq(OutputCodec.class), any())).thenReturn(outputScenario.getCodec()); + when(s3SinkConfig.getBufferType()).thenReturn(bufferTypeOptions); + when(s3SinkConfig.getCompression()).thenReturn(compressionScenario.getCompressionOption()); + int expectedTotalSize = batchSize * numberOfBatches; + when(thresholdOptions.getEventCount()).thenReturn(expectedTotalSize); + + final S3Sink objectUnderTest = createObjectUnderTest(); + + final int maxEventDataToSample = 5000; + final List> sampleEventData = new ArrayList<>(maxEventDataToSample); + for (int batchNumber = 0; batchNumber < numberOfBatches; batchNumber++) { + final int currentBatchNumber = batchNumber; + final List> events = IntStream.range(0, batchSize) + .mapToObj(sequence -> generateEventData((currentBatchNumber+1) * (sequence+1))) + .peek(data -> { + if (sampleEventData.size() < maxEventDataToSample) + sampleEventData.add(data); + }) + .map(this::generateTestEvent) + .map(Record::new) + .collect(Collectors.toList()); + + LOG.debug("Writing batch {} with size {}.", currentBatchNumber, events.size()); + objectUnderTest.doOutput(events); + } + + LOG.info("Listing S3 path prefix: {}", pathPrefix); + + final ListObjectsV2Response listObjectsResponse = s3Client.listObjectsV2(ListObjectsV2Request.builder() + .bucket(bucketName) + .prefix(pathPrefix) + .build()); + + assertThat(listObjectsResponse.contents(), notNullValue()); + assertThat(listObjectsResponse.contents().size(), equalTo(1)); + + final S3Object s3Object = listObjectsResponse.contents().get(0); + + final File target = new File(s3FileLocation, testRun + ".original"); + + LOG.info("Downloading S3 object to local file {}.", target); + + final FileDownload fileDownload = transferManager.downloadFile(DownloadFileRequest.builder() + .destination(target) + .getObjectRequest(GetObjectRequest.builder() + .bucket(bucketName) + .key(s3Object.key()) + .build()) + .build()); + + fileDownload.completionFuture().join(); + + File actualContentFile = decompressFileIfNecessary(outputScenario, compressionScenario, testRun, target); + + LOG.info("Validating output. totalSize={}; sampleDataSize={}", expectedTotalSize, sampleEventData.size()); + outputScenario.validate(expectedTotalSize, sampleEventData, actualContentFile, compressionScenario); + } + + private File decompressFileIfNecessary(OutputScenario outputScenario, CompressionScenario compressionScenario, String pathPrefix, File target) throws IOException { + + if (outputScenario.isCompressionInternal() || !compressionScenario.requiresDecompression()) + return target; + + File actualContentFile = new File(s3FileLocation, pathPrefix + ".content"); + IOUtils.copy( + compressionScenario.decompressingInputStream(new FileInputStream(target)), + new FileOutputStream(actualContentFile)); + + return actualContentFile; + } + + private Event generateTestEvent(final Map eventData) { + final EventMetadata defaultEventMetadata = DefaultEventMetadata.builder() + .withEventType(EventType.LOG.toString()) + .build(); + final JacksonEvent event = JacksonLog.builder().withData(eventData).withEventMetadata(defaultEventMetadata).build(); + event.setEventHandle(mock(EventHandle.class)); + return JacksonEvent.builder() + .withData(eventData) + .withEventMetadata(defaultEventMetadata) + .build(); + } + + private Map generateEventData(final int sequence) { + final Map eventDataMap = new LinkedHashMap<>(); + eventDataMap.put("sequence", sequence); + eventDataMap.put("id", UUID.randomUUID().toString()); + for (int i = 0; i < 2; i++) { + eventDataMap.put("field" + i, reusableRandomString()); + eventDataMap.put("float" + i, (float) i * 1.5 * sequence); + } + for (int i = 0; i < 2; i++) { + eventDataMap.put("list" + i, + List.of(reusableRandomString(), reusableRandomString(), reusableRandomString())); + } + return eventDataMap; + } + + static class IntegrationTestArguments implements ArgumentsProvider { + @Override + public Stream provideArguments(final ExtensionContext context) { + final List bufferScenarios = List.of( + new InMemoryBufferScenario(), + new LocalFileBufferScenario(), + new MultiPartBufferScenario() + ); + final List outputScenarios = List.of( + new NdjsonOutputScenario(), + new ParquetOutputScenario() + ); + final List compressionScenarios = List.of( + new NoneCompressionScenario(), + new GZipCompressionScenario(), + new SnappyCompressionScenario() + ); + final List numberOfRecordsPerBatchList = List.of( + 1, + 500 + ); + final List numberOfBatchesList = List.of( + 1, + 50, + 1_000 + ); + + return outputScenarios + .stream() + .flatMap(outputScenario -> bufferScenarios + .stream() + .filter(bufferScenario -> !outputScenario.getIncompatibleBufferTypes().contains(bufferScenario.getBufferType())) + .flatMap(bufferScenario -> compressionScenarios + .stream() + .flatMap(compressionScenario -> numberOfRecordsPerBatchList + .stream() + .flatMap(batchRecordCount -> numberOfBatchesList + .stream() + .filter(batchCount -> batchCount * batchRecordCount <= bufferScenario.getMaximumNumberOfEvents()) + .map(batchCount -> arguments(outputScenario, bufferScenario.getBufferType(), compressionScenario, batchRecordCount, batchCount)) + )))); + } + } + + private static String reusableRandomString() { + return reusableRandomStrings.get(RANDOM.nextInt(reusableRandomStrings.size())); + } +} diff --git a/data-prepper-plugins/s3-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/s3/S3SinkServiceIT.java b/data-prepper-plugins/s3-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/s3/S3SinkServiceIT.java index d679663f11..ef01125aff 100644 --- a/data-prepper-plugins/s3-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/s3/S3SinkServiceIT.java +++ b/data-prepper-plugins/s3-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/s3/S3SinkServiceIT.java @@ -129,6 +129,7 @@ class S3SinkServiceIT { private DistributionSummary s3ObjectSizeSummary; private OutputCodec codec; + private KeyGenerator keyGenerator; @Mock NdjsonOutputConfig ndjsonOutputConfig; @@ -172,6 +173,7 @@ void verify_flushed_object_count_into_s3_bucket() { void configureNewLineCodec() { codec = new NdjsonOutputCodec(ndjsonOutputConfig); + keyGenerator = new KeyGenerator(s3SinkConfig, codec); } @Test @@ -206,7 +208,7 @@ void verify_flushed_records_into_s3_bucketNewLine() throws JsonProcessingExcepti @Test void verify_flushed_records_into_s3_bucketNewLine_with_compression() throws IOException { configureNewLineCodec(); - bufferFactory = new CompressionBufferFactory(bufferFactory, CompressionOption.GZIP.getCompressionEngine()); + bufferFactory = new CompressionBufferFactory(bufferFactory, CompressionOption.GZIP.getCompressionEngine(), codec); S3SinkService s3SinkService = createObjectUnderTest(); Collection> recordsData = setEventQueue(); @@ -240,7 +242,7 @@ void verify_flushed_records_into_s3_bucketNewLine_with_compression() throws IOEx private S3SinkService createObjectUnderTest() { OutputCodecContext codecContext = new OutputCodecContext("Tag", Collections.emptyList(), Collections.emptyList()); - return new S3SinkService(s3SinkConfig, bufferFactory, codec, codecContext, s3Client, pluginMetrics); + return new S3SinkService(s3SinkConfig, bufferFactory, codec, codecContext, s3Client, keyGenerator, pluginMetrics); } private int gets3ObjectCount() { @@ -352,11 +354,9 @@ void verify_flushed_records_into_s3_bucket_Parquet() throws IOException { private void configureParquetCodec() { parquetOutputCodecConfig = new ParquetOutputCodecConfig(); parquetOutputCodecConfig.setSchema(parseSchema().toString()); - parquetOutputCodecConfig.setBucket(bucketName); - parquetOutputCodecConfig.setRegion(s3region); parquetOutputCodecConfig.setPathPrefix(PATH_PREFIX); codec = new ParquetOutputCodec(parquetOutputCodecConfig); - + keyGenerator = new KeyGenerator(s3SinkConfig, codec); } private Collection> getRecordList() { diff --git a/data-prepper-plugins/s3-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/s3/SnappyCompressionScenario.java b/data-prepper-plugins/s3-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/s3/SnappyCompressionScenario.java new file mode 100644 index 0000000000..2990b40758 --- /dev/null +++ b/data-prepper-plugins/s3-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/s3/SnappyCompressionScenario.java @@ -0,0 +1,29 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.sink.s3; + +import org.opensearch.dataprepper.plugins.sink.s3.compression.CompressionOption; +import org.xerial.snappy.SnappyInputStream; + +import java.io.IOException; +import java.io.InputStream; + +public class SnappyCompressionScenario implements CompressionScenario { + @Override + public CompressionOption getCompressionOption() { + return CompressionOption.SNAPPY; + } + + @Override + public InputStream decompressingInputStream(final InputStream inputStream) throws IOException { + return new SnappyInputStream(inputStream); + } + + @Override + public String toString() { + return "Snappy"; + } +} diff --git a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/codec/parquet/CompressionConverter.java b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/codec/parquet/CompressionConverter.java new file mode 100644 index 0000000000..af454585a3 --- /dev/null +++ b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/codec/parquet/CompressionConverter.java @@ -0,0 +1,30 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.codec.parquet; + +import org.apache.parquet.hadoop.metadata.CompressionCodecName; +import org.opensearch.dataprepper.model.plugin.InvalidPluginDefinitionException; +import org.opensearch.dataprepper.plugins.sink.s3.compression.CompressionOption; + +class CompressionConverter { + static CompressionCodecName convertCodec(final CompressionOption compressionOption) { + switch (compressionOption) { + case NONE: + return CompressionCodecName.UNCOMPRESSED; + case GZIP: + return CompressionCodecName.GZIP; + case SNAPPY: + return CompressionCodecName.SNAPPY; + } + + throw new InvalidPluginDefinitionException( + String.format("The Parquet codec supports the following compression options: %s, %s, %s", + CompressionOption.NONE.getOption(), + CompressionOption.GZIP.getOption(), + CompressionOption.SNAPPY.getOption() + )); + } +} diff --git a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/codec/parquet/ParquetOutputCodec.java b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/codec/parquet/ParquetOutputCodec.java new file mode 100644 index 0000000000..4eb9724f27 --- /dev/null +++ b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/codec/parquet/ParquetOutputCodec.java @@ -0,0 +1,182 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.codec.parquet; + +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.parquet.avro.AvroParquetWriter; +import org.apache.parquet.hadoop.ParquetWriter; +import org.apache.parquet.hadoop.metadata.CompressionCodecName; +import org.apache.parquet.io.OutputFile; +import org.apache.parquet.io.PositionOutputStream; +import org.opensearch.dataprepper.avro.AvroAutoSchemaGenerator; +import org.opensearch.dataprepper.avro.AvroEventConverter; +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 org.opensearch.dataprepper.model.sink.OutputCodecContext; +import org.opensearch.dataprepper.plugins.fs.LocalInputFile; +import org.opensearch.dataprepper.plugins.s3keyindex.S3ObjectIndexUtility; +import org.opensearch.dataprepper.plugins.sink.s3.S3OutputCodecContext; + +import java.io.File; +import java.io.IOException; +import java.io.OutputStream; +import java.util.Map; +import java.util.Objects; +import java.util.regex.Pattern; + +@DataPrepperPlugin(name = "parquet", pluginType = OutputCodec.class, pluginConfigurationType = ParquetOutputCodecConfig.class) +public class ParquetOutputCodec implements OutputCodec { + private final ParquetOutputCodecConfig config; + private static Schema schema; + private final AvroEventConverter avroEventConverter; + private final AvroAutoSchemaGenerator avroAutoSchemaGenerator; + private ParquetWriter writer; + private OutputCodecContext codecContext; + private static final String PARQUET = "parquet"; + + private static final String TIME_PATTERN_REGULAR_EXPRESSION = "\\%\\{.*?\\}"; + private static final Pattern SIMPLE_DURATION_PATTERN = Pattern.compile(TIME_PATTERN_REGULAR_EXPRESSION); + private String key; + + + @DataPrepperPluginConstructor + public ParquetOutputCodec(final ParquetOutputCodecConfig config) { + Objects.requireNonNull(config); + this.config = config; + + avroEventConverter = new AvroEventConverter(); + avroAutoSchemaGenerator = new AvroAutoSchemaGenerator(); + } + + @Override + public synchronized void start(final OutputStream outputStream, final Event event, final OutputCodecContext codecContext) throws IOException { + Objects.requireNonNull(outputStream); + Objects.requireNonNull(codecContext); + if(!(outputStream instanceof PositionOutputStream)) { + throw new RuntimeException("The Parquet output codec only works with the S3OutputStream and thus only with multi-part uploads."); + } + if(!(codecContext instanceof S3OutputCodecContext)) { + throw new RuntimeException("The Parquet output codec only works with S3 presently"); + } + PositionOutputStream s3OutputStream = (PositionOutputStream) outputStream; + CompressionCodecName compressionCodecName = CompressionConverter.convertCodec(((S3OutputCodecContext) codecContext).getCompressionOption()); + this.codecContext = codecContext; + buildSchemaAndKey(event); + final S3OutputFile s3OutputFile = new S3OutputFile(s3OutputStream); + buildWriter(s3OutputFile, compressionCodecName); + } + + @Override + public boolean isCompressionInternal() { + return true; + } + + void buildSchemaAndKey(final Event event) throws IOException { + if (config.getSchema() != null) { + schema = parseSchema(config.getSchema()); + } else if (config.getFileLocation() != null) { + schema = ParquetSchemaParser.parseSchemaFromJsonFile(config.getFileLocation()); + } else if (config.getSchemaRegistryUrl() != null) { + schema = parseSchema(ParquetSchemaParserFromSchemaRegistry.getSchemaType(config.getSchemaRegistryUrl())); + } else if (checkS3SchemaValidity()) { + schema = ParquetSchemaParserFromS3.parseSchema(config); + } else { + schema = buildInlineSchemaFromEvent(event); + } + key = generateKey(); + } + + public Schema buildInlineSchemaFromEvent(final Event event) throws IOException { + final Map data; + if (codecContext != null && codecContext.getTagsTargetKey() != null) { + data = addTagsToEvent(event, codecContext.getTagsTargetKey()).toMap(); + } else { + data = event.toMap(); + } + + return avroAutoSchemaGenerator.autoDetermineSchema(data, codecContext); + } + + private void buildWriter(OutputFile outputFile, CompressionCodecName compressionCodecName) throws IOException { + writer = AvroParquetWriter.builder(outputFile) + .withSchema(schema) + .withCompressionCodec(compressionCodecName) + .build(); + } + + @Override + public void writeEvent(final Event event, final OutputStream outputStream) throws IOException { + final Event modifiedEvent; + if (codecContext.getTagsTargetKey() != null) { + modifiedEvent = addTagsToEvent(event, codecContext.getTagsTargetKey()); + } else { + modifiedEvent = event; + } + GenericRecord parquetRecord = avroEventConverter.convertEventDataToAvro(schema, modifiedEvent.toMap(), codecContext); + writer.write(parquetRecord); + } + + @Override + public synchronized void complete(final OutputStream outputStream) throws IOException { + writer.close(); + } + + public void closeWriter(final OutputStream outputStream, File file) throws IOException { + final LocalInputFile inputFile = new LocalInputFile(file); + byte[] byteBuffer = inputFile.newStream().readAllBytes(); + outputStream.write(byteBuffer); + writer.close(); + } + + @Override + public String getExtension() { + return PARQUET; + } + + static Schema parseSchema(final String schemaString) { + return new Schema.Parser().parse(schemaString); + } + + /** + * Generate the s3 object path prefix and object file name. + * + * @return object key path. + */ + protected String generateKey() { + final String pathPrefix = buildObjectPath(config.getPathPrefix()); + final String namePattern = buildObjectFileName(config.getNamePattern()); + return (!pathPrefix.isEmpty()) ? pathPrefix + namePattern : namePattern; + } + + private static String buildObjectPath(final String pathPrefix) { + final StringBuilder s3ObjectPath = new StringBuilder(); + if (pathPrefix != null && !pathPrefix.isEmpty()) { + String[] pathPrefixList = pathPrefix.split("\\/"); + for (final String prefixPath : pathPrefixList) { + if (SIMPLE_DURATION_PATTERN.matcher(prefixPath).find()) { + s3ObjectPath.append(S3ObjectIndexUtility.getObjectPathPrefix(prefixPath)).append("/"); + } else { + s3ObjectPath.append(prefixPath).append("/"); + } + } + } + return s3ObjectPath.toString(); + } + + private String buildObjectFileName(final String configNamePattern) { + return S3ObjectIndexUtility.getObjectNameWithDateTimeId(configNamePattern) + "." + getExtension(); + } + + boolean checkS3SchemaValidity() { + if (config.getSchemaBucket() != null && config.getFileKey() != null && config.getSchemaRegion() != null) { + return true; + } else { + return false; + } + } +} diff --git a/data-prepper-plugins/parquet-codecs/src/main/java/org/opensearch/dataprepper/plugins/codec/parquet/ParquetOutputCodecConfig.java b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/codec/parquet/ParquetOutputCodecConfig.java similarity index 86% rename from data-prepper-plugins/parquet-codecs/src/main/java/org/opensearch/dataprepper/plugins/codec/parquet/ParquetOutputCodecConfig.java rename to data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/codec/parquet/ParquetOutputCodecConfig.java index 4ade76f536..c31b6ab988 100644 --- a/data-prepper-plugins/parquet-codecs/src/main/java/org/opensearch/dataprepper/plugins/codec/parquet/ParquetOutputCodecConfig.java +++ b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/codec/parquet/ParquetOutputCodecConfig.java @@ -36,16 +36,6 @@ public class ParquetOutputCodecConfig { @JsonProperty("schema_region") private String schemaRegion; - @JsonProperty("region") - @NotNull - @Valid - private String region; - - @JsonProperty("bucket") - @NotNull - @Valid - private String bucket; - @JsonProperty("path_prefix") @NotNull @Valid @@ -72,14 +62,6 @@ public String getSchemaRegistryUrl() { return schemaRegistryUrl; } - public String getRegion() { - return region; - } - - public String getBucket() { - return bucket; - } - public String getPathPrefix() { return pathPrefix; } @@ -93,14 +75,6 @@ public String getNamePattern() { return DEFAULT_OBJECT_NAME_PATTERN; } - public void setRegion(String region) { - this.region = region; - } - - public void setBucket(String bucket) { - this.bucket = bucket; - } - public void setPathPrefix(String pathPrefix) { this.pathPrefix = pathPrefix; } diff --git a/data-prepper-plugins/parquet-codecs/src/main/java/org/opensearch/dataprepper/plugins/codec/parquet/ParquetSchemaParser.java b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/codec/parquet/ParquetSchemaParser.java similarity index 100% rename from data-prepper-plugins/parquet-codecs/src/main/java/org/opensearch/dataprepper/plugins/codec/parquet/ParquetSchemaParser.java rename to data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/codec/parquet/ParquetSchemaParser.java diff --git a/data-prepper-plugins/parquet-codecs/src/main/java/org/opensearch/dataprepper/plugins/codec/parquet/ParquetSchemaParserFromS3.java b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/codec/parquet/ParquetSchemaParserFromS3.java similarity index 100% rename from data-prepper-plugins/parquet-codecs/src/main/java/org/opensearch/dataprepper/plugins/codec/parquet/ParquetSchemaParserFromS3.java rename to data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/codec/parquet/ParquetSchemaParserFromS3.java diff --git a/data-prepper-plugins/parquet-codecs/src/main/java/org/opensearch/dataprepper/plugins/codec/parquet/ParquetSchemaParserFromSchemaRegistry.java b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/codec/parquet/ParquetSchemaParserFromSchemaRegistry.java similarity index 100% rename from data-prepper-plugins/parquet-codecs/src/main/java/org/opensearch/dataprepper/plugins/codec/parquet/ParquetSchemaParserFromSchemaRegistry.java rename to data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/codec/parquet/ParquetSchemaParserFromSchemaRegistry.java diff --git a/data-prepper-plugins/parquet-codecs/src/main/java/org/opensearch/dataprepper/plugins/codec/parquet/S3OutputFile.java b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/codec/parquet/S3OutputFile.java similarity index 58% rename from data-prepper-plugins/parquet-codecs/src/main/java/org/opensearch/dataprepper/plugins/codec/parquet/S3OutputFile.java rename to data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/codec/parquet/S3OutputFile.java index cc5020a3f7..9f6af3f67b 100644 --- a/data-prepper-plugins/parquet-codecs/src/main/java/org/opensearch/dataprepper/plugins/codec/parquet/S3OutputFile.java +++ b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/codec/parquet/S3OutputFile.java @@ -6,30 +6,20 @@ import org.apache.parquet.io.OutputFile; import org.apache.parquet.io.PositionOutputStream; -import software.amazon.awssdk.services.s3.S3Client; import java.io.IOException; -public class S3OutputFile implements OutputFile { +class S3OutputFile implements OutputFile { + private final PositionOutputStream outputStream; - private S3Client s3Client; - - private String bucketName; - - private String key; - - - public S3OutputFile(final S3Client s3Client, final String bucketName, - final String key) { - this.s3Client = s3Client; - this.bucketName = bucketName; - this.key = key; + public S3OutputFile(PositionOutputStream outputStream) { + this.outputStream = outputStream; } @Override public PositionOutputStream create(long blockSizeHint) throws IOException { - return new S3OutputStream(s3Client, bucketName, key); + return outputStream; } @Override diff --git a/data-prepper-plugins/parquet-codecs/src/main/java/org/opensearch/dataprepper/plugins/codec/parquet/S3OutputStream.java b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/codec/parquet/S3OutputStream.java similarity index 66% rename from data-prepper-plugins/parquet-codecs/src/main/java/org/opensearch/dataprepper/plugins/codec/parquet/S3OutputStream.java rename to data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/codec/parquet/S3OutputStream.java index 388a1e31cc..f1d8e037f6 100644 --- a/data-prepper-plugins/parquet-codecs/src/main/java/org/opensearch/dataprepper/plugins/codec/parquet/S3OutputStream.java +++ b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/codec/parquet/S3OutputStream.java @@ -13,7 +13,6 @@ import software.amazon.awssdk.services.s3.model.CompletedPart; import software.amazon.awssdk.services.s3.model.CreateMultipartUploadRequest; import software.amazon.awssdk.services.s3.model.CreateMultipartUploadResponse; -import software.amazon.awssdk.services.s3.model.PutObjectRequest; import software.amazon.awssdk.services.s3.model.UploadPartRequest; import software.amazon.awssdk.services.s3.model.UploadPartResponse; @@ -21,13 +20,14 @@ import java.io.IOException; import java.util.ArrayList; import java.util.List; +import java.util.function.Supplier; public class S3OutputStream extends PositionOutputStream { /** * Default chunk size is 10MB */ - protected static final int BUFFER_SIZE = 10000000; + protected static final int BUFFER_SIZE = 10 * 1024 * 1024; /** * The bucket-name on Amazon S3 @@ -35,9 +35,9 @@ public class S3OutputStream extends PositionOutputStream { private final String bucket; /** - * The path (key) name within the bucket + * The key (path) name within the bucket */ - private final String path; + private final String key; /** * The temporary buffer used for storing the chunks @@ -66,13 +66,13 @@ public class S3OutputStream extends PositionOutputStream { * Creates a new S3 OutputStream * * @param s3Client the AmazonS3 client - * @param bucket name of the bucket - * @param path path within the bucket + * @param bucketSupplier name of the bucket + * @param keySupplier path within the bucket */ - public S3OutputStream(S3Client s3Client, String bucket, String path) { + public S3OutputStream(final S3Client s3Client, Supplier bucketSupplier, Supplier keySupplier) { this.s3Client = s3Client; - this.bucket = bucket; - this.path = path; + this.bucket = bucketSupplier.get(); + this.key = keySupplier.get(); buf = new byte[BUFFER_SIZE]; position = 0; etags = new ArrayList<>(); @@ -125,77 +125,73 @@ public void write(byte[] byteArray, int o, int l) { } /** - * Flushes the buffer by uploading a part to S3. + * Flushing is not available because the parts must be of the same size. */ @Override - public synchronized void flush() { - assertOpen(); + public void flush() { } @Override public void close() { if (open) { open = false; - if (uploadId != null) { - if (position > 0) { - uploadPart(); - } - - CompletedPart[] completedParts = new CompletedPart[etags.size()]; - for (int i = 0; i < etags.size(); i++) { - completedParts[i] = CompletedPart.builder() - .eTag(etags.get(i)) - .partNumber(i + 1) - .build(); - } - - CompletedMultipartUpload completedMultipartUpload = CompletedMultipartUpload.builder() - .parts(completedParts) - .build(); - CompleteMultipartUploadRequest completeMultipartUploadRequest = CompleteMultipartUploadRequest.builder() - .bucket(bucket) - .key(path) - .uploadId(uploadId) - .multipartUpload(completedMultipartUpload) - .build(); - s3Client.completeMultipartUpload(completeMultipartUploadRequest); - } else { - PutObjectRequest putRequest = PutObjectRequest.builder() - .bucket(bucket) - .key(path) - .contentLength((long) position) - .build(); + possiblyStartMultipartUpload(); + if (position > 0) { + uploadPart(); + } - RequestBody requestBody = RequestBody.fromInputStream(new ByteArrayInputStream(buf, 0, position), - position); - s3Client.putObject(putRequest, requestBody); + CompletedPart[] completedParts = new CompletedPart[etags.size()]; + for (int i = 0; i < etags.size(); i++) { + completedParts[i] = CompletedPart.builder() + .eTag(etags.get(i)) + .partNumber(i + 1) + .build(); } + + CompletedMultipartUpload completedMultipartUpload = CompletedMultipartUpload.builder() + .parts(completedParts) + .build(); + CompleteMultipartUploadRequest completeMultipartUploadRequest = CompleteMultipartUploadRequest.builder() + .bucket(bucket) + .key(key) + .uploadId(uploadId) + .multipartUpload(completedMultipartUpload) + .build(); + s3Client.completeMultipartUpload(completeMultipartUploadRequest); } } + public String getKey() { + return key; + } + private void assertOpen() { if (!open) { throw new IllegalStateException("Closed"); } } - protected void flushBufferAndRewind() { + private void flushBufferAndRewind() { + possiblyStartMultipartUpload(); + uploadPart(); + position = 0; + } + + private void possiblyStartMultipartUpload() { if (uploadId == null) { CreateMultipartUploadRequest uploadRequest = CreateMultipartUploadRequest.builder() .bucket(bucket) - .key(path) + .key(key) .build(); CreateMultipartUploadResponse multipartUpload = s3Client.createMultipartUpload(uploadRequest); uploadId = multipartUpload.uploadId(); } - uploadPart(); - position = 0; } - protected void uploadPart() { + private void uploadPart() { UploadPartRequest uploadRequest = UploadPartRequest.builder() .bucket(bucket) - .key(path) + .key(key) .uploadId(uploadId) .partNumber(etags.size() + 1) .contentLength((long) position) @@ -208,7 +204,7 @@ protected void uploadPart() { @Override public long getPos() throws IOException { - return position; + return position + (long) etags.size() * (long) BUFFER_SIZE; } } diff --git a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/KeyGenerator.java b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/KeyGenerator.java new file mode 100644 index 0000000000..2087554c7c --- /dev/null +++ b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/KeyGenerator.java @@ -0,0 +1,30 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.sink.s3; + +import org.opensearch.dataprepper.model.codec.OutputCodec; +import org.opensearch.dataprepper.plugins.sink.s3.accumulator.ObjectKey; + +public class KeyGenerator { + private final S3SinkConfig s3SinkConfig; + private final OutputCodec outputCodec; + + public KeyGenerator(S3SinkConfig s3SinkConfig, OutputCodec outputCodec) { + this.s3SinkConfig = s3SinkConfig; + this.outputCodec = outputCodec; + } + + /** + * Generate the s3 object path prefix and object file name. + * + * @return object key path. + */ + String generateKey() { + final String pathPrefix = ObjectKey.buildingPathPrefix(s3SinkConfig); + final String namePattern = ObjectKey.objectFileName(s3SinkConfig, outputCodec.getExtension()); + return (!pathPrefix.isEmpty()) ? pathPrefix + namePattern : namePattern; + } +} diff --git a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/S3OutputCodecContext.java b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/S3OutputCodecContext.java new file mode 100644 index 0000000000..85b3e7a9bb --- /dev/null +++ b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/S3OutputCodecContext.java @@ -0,0 +1,22 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.sink.s3; + +import org.opensearch.dataprepper.model.sink.OutputCodecContext; +import org.opensearch.dataprepper.plugins.sink.s3.compression.CompressionOption; + +public class S3OutputCodecContext extends OutputCodecContext { + private final CompressionOption compressionOption; + + public S3OutputCodecContext(OutputCodecContext outputCodecContext, CompressionOption compressionOption) { + super(outputCodecContext.getTagsTargetKey(), outputCodecContext.getIncludeKeys(), outputCodecContext.getExcludeKeys()); + this.compressionOption = compressionOption; + } + + public CompressionOption getCompressionOption() { + return compressionOption; + } +} diff --git a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/S3Sink.java b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/S3Sink.java index c880a72464..351ebcf0e1 100644 --- a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/S3Sink.java +++ b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/S3Sink.java @@ -20,10 +20,7 @@ import org.opensearch.dataprepper.model.sink.Sink; import org.opensearch.dataprepper.model.sink.SinkContext; import org.opensearch.dataprepper.plugins.sink.s3.accumulator.BufferFactory; -import org.opensearch.dataprepper.plugins.sink.s3.accumulator.BufferTypeOptions; import org.opensearch.dataprepper.plugins.sink.s3.accumulator.CompressionBufferFactory; -import org.opensearch.dataprepper.plugins.sink.s3.accumulator.InMemoryBufferFactory; -import org.opensearch.dataprepper.plugins.sink.s3.accumulator.LocalFileBufferFactory; import org.opensearch.dataprepper.plugins.sink.s3.compression.CompressionEngine; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -66,16 +63,15 @@ public S3Sink(final PluginSetting pluginSetting, codec = pluginFactory.loadPlugin(OutputCodec.class, codecPluginSettings); sinkInitialized = Boolean.FALSE; - final BufferFactory innerBufferFactory; - if (s3SinkConfig.getBufferType().equals(BufferTypeOptions.LOCALFILE)) { - innerBufferFactory = new LocalFileBufferFactory(); - } else { - innerBufferFactory = new InMemoryBufferFactory(); - } - final CompressionEngine compressionEngine = s3SinkConfig.getCompression().getCompressionEngine(); - bufferFactory = new CompressionBufferFactory(innerBufferFactory, compressionEngine); final S3Client s3Client = ClientFactory.createS3Client(s3SinkConfig, awsCredentialsSupplier); - s3SinkService = new S3SinkService(s3SinkConfig, bufferFactory, codec, OutputCodecContext.fromSinkContext(sinkContext), s3Client, pluginMetrics); + KeyGenerator keyGenerator = new KeyGenerator(s3SinkConfig, codec); + final BufferFactory innerBufferFactory = s3SinkConfig.getBufferType().getBufferFactory(); + final CompressionEngine compressionEngine = s3SinkConfig.getCompression().getCompressionEngine(); + bufferFactory = new CompressionBufferFactory(innerBufferFactory, compressionEngine, codec); + + S3OutputCodecContext s3OutputCodecContext = new S3OutputCodecContext(OutputCodecContext.fromSinkContext(sinkContext), s3SinkConfig.getCompression()); + + s3SinkService = new S3SinkService(s3SinkConfig, bufferFactory, codec, s3OutputCodecContext, s3Client, keyGenerator, pluginMetrics); } @Override @@ -110,9 +106,6 @@ private void doInitializeInternal() { */ @Override public void doOutput(final Collection> records) { - if (records.isEmpty()) { - return; - } s3SinkService.output(records); } } diff --git a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/S3SinkService.java b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/S3SinkService.java index 0b56890f8e..7007259ebf 100644 --- a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/S3SinkService.java +++ b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/S3SinkService.java @@ -16,7 +16,6 @@ import org.opensearch.dataprepper.model.types.ByteCount; import org.opensearch.dataprepper.plugins.sink.s3.accumulator.Buffer; import org.opensearch.dataprepper.plugins.sink.s3.accumulator.BufferFactory; -import org.opensearch.dataprepper.plugins.sink.s3.accumulator.ObjectKey; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import software.amazon.awssdk.awscore.exception.AwsServiceException; @@ -24,7 +23,6 @@ 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; @@ -60,6 +58,7 @@ public class S3SinkService { private final Counter numberOfRecordsFailedCounter; private final DistributionSummary s3ObjectSizeSummary; private final OutputCodecContext codecContext; + private final KeyGenerator keyGenerator; /** * @param s3SinkConfig s3 sink related configuration. @@ -69,12 +68,13 @@ public class S3SinkService { * @param pluginMetrics metrics. */ public S3SinkService(final S3SinkConfig s3SinkConfig, final BufferFactory bufferFactory, - final OutputCodec codec, final OutputCodecContext codecContext, final S3Client s3Client, final PluginMetrics pluginMetrics) { + final OutputCodec codec, final OutputCodecContext codecContext, final S3Client s3Client, final KeyGenerator keyGenerator, final PluginMetrics pluginMetrics) { this.s3SinkConfig = s3SinkConfig; this.bufferFactory = bufferFactory; this.codec = codec; this.s3Client = s3Client; this.codecContext = codecContext; + this.keyGenerator = keyGenerator; reentrantLock = new ReentrantLock(); bufferedEventHandles = new LinkedList<>(); @@ -91,58 +91,44 @@ public S3SinkService(final S3SinkConfig s3SinkConfig, final BufferFactory buffer numberOfRecordsSuccessCounter = pluginMetrics.counter(NUMBER_OF_RECORDS_FLUSHED_TO_S3_SUCCESS); numberOfRecordsFailedCounter = pluginMetrics.counter(NUMBER_OF_RECORDS_FLUSHED_TO_S3_FAILED); s3ObjectSizeSummary = pluginMetrics.summary(S3_OBJECTS_SIZE); + + currentBuffer = bufferFactory.getBuffer(s3Client, () -> bucket, keyGenerator::generateKey); } /** * @param records received records and add into buffer. */ void output(Collection> records) { - reentrantLock.lock(); - if (currentBuffer == null) { - currentBuffer = bufferFactory.getBuffer(); + // Don't acquire the lock if there's no work to be done + if (records.isEmpty() && currentBuffer.getEventCount() == 0) { + return; } - try { - OutputStream outputStream = currentBuffer.getOutputStream(); + reentrantLock.lock(); + try { for (Record record : records) { if (currentBuffer.getEventCount() == 0) { final Event eventForSchemaAutoGenerate = record.getData(); - codec.start(outputStream, eventForSchemaAutoGenerate, codecContext); + codec.start(currentBuffer.getOutputStream(), eventForSchemaAutoGenerate, codecContext); } final Event event = record.getData(); - codec.writeEvent(event, outputStream); + codec.writeEvent(event, currentBuffer.getOutputStream()); int count = currentBuffer.getEventCount() + 1; currentBuffer.setEventCount(count); if (event.getEventHandle() != null) { bufferedEventHandles.add(event.getEventHandle()); } - if (ThresholdCheck.checkThresholdExceed(currentBuffer, maxEvents, maxBytes, maxCollectionDuration)) { - 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); - if (isFlushToS3) { - LOG.info("Successfully saved {} to S3.", s3Key); - numberOfRecordsSuccessCounter.increment(currentBuffer.getEventCount()); - objectsSucceededCounter.increment(); - s3ObjectSizeSummary.record(currentBuffer.getSize()); - releaseEventHandles(true); - } else { - LOG.error("Failed to save {} to S3.", s3Key); - numberOfRecordsFailedCounter.increment(currentBuffer.getEventCount()); - objectsFailedCounter.increment(); - releaseEventHandles(false); - } - currentBuffer = bufferFactory.getBuffer(); - } + flushToS3IfNeeded(); } - } catch (IOException | InterruptedException e) { + } catch (IOException e) { LOG.error("Exception while write event into buffer :", e); } + + flushToS3IfNeeded(); + reentrantLock.unlock(); } @@ -154,43 +140,63 @@ private void releaseEventHandles(final boolean result) { bufferedEventHandles.clear(); } + private void flushToS3IfNeeded() { + if (ThresholdCheck.checkThresholdExceed(currentBuffer, maxEvents, maxBytes, maxCollectionDuration)) { + try { + codec.complete(currentBuffer.getOutputStream()); + String s3Key = currentBuffer.getKey(); + LOG.info("Writing {} to S3 with {} events and size of {} bytes.", + s3Key, currentBuffer.getEventCount(), currentBuffer.getSize()); + final boolean isFlushToS3 = retryFlushToS3(currentBuffer, s3Key); + if (isFlushToS3) { + LOG.info("Successfully saved {} to S3.", s3Key); + numberOfRecordsSuccessCounter.increment(currentBuffer.getEventCount()); + objectsSucceededCounter.increment(); + s3ObjectSizeSummary.record(currentBuffer.getSize()); + releaseEventHandles(true); + } else { + LOG.error("Failed to save {} to S3.", s3Key); + numberOfRecordsFailedCounter.increment(currentBuffer.getEventCount()); + objectsFailedCounter.increment(); + releaseEventHandles(false); + } + currentBuffer = bufferFactory.getBuffer(s3Client, () -> bucket, keyGenerator::generateKey); + } catch (final IOException e) { + LOG.error("Exception while completing codec", e); + } + } + } + /** * perform retry in-case any issue occurred, based on max_upload_retries configuration. * * @param currentBuffer current buffer. * @param s3Key * @return boolean based on object upload status. - * @throws InterruptedException interruption during sleep. */ - protected boolean retryFlushToS3(final Buffer currentBuffer, final String s3Key) throws InterruptedException { + protected boolean retryFlushToS3(final Buffer currentBuffer, final String s3Key) { boolean isUploadedToS3 = Boolean.FALSE; int retryCount = maxRetries; do { try { - currentBuffer.flushToS3(s3Client, bucket, s3Key); + currentBuffer.flushToS3(); isUploadedToS3 = Boolean.TRUE; } catch (AwsServiceException | SdkClientException e) { LOG.error("Exception occurred while uploading records to s3 bucket. Retry countdown : {} | exception:", retryCount, e); - LOG.info("Error Massage {}", e.getMessage()); + LOG.info("Error Message {}", e.getMessage()); --retryCount; if (retryCount == 0) { return isUploadedToS3; } - Thread.sleep(5000); + + try { + Thread.sleep(5000); + } catch (final InterruptedException ex) { + LOG.warn("Interrupted while backing off before retrying S3 upload", ex); + } } } while (!isUploadedToS3); return isUploadedToS3; } - - /** - * Generate the s3 object path prefix and object file name. - * - * @return object key path. - */ - protected String generateKey(OutputCodec codec) { - final String pathPrefix = ObjectKey.buildingPathPrefix(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/s3/accumulator/Buffer.java b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/Buffer.java index afd695db2b..a56e124632 100644 --- a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/Buffer.java +++ b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/Buffer.java @@ -5,14 +5,12 @@ package org.opensearch.dataprepper.plugins.sink.s3.accumulator; -import software.amazon.awssdk.services.s3.S3Client; import java.io.OutputStream; /** * A buffer can hold data before flushing it to S3. */ public interface Buffer { - /** * Gets the current size of the buffer. This should be the number of bytes. * @return buffer size. @@ -22,9 +20,11 @@ public interface Buffer { long getDuration(); - void flushToS3(S3Client s3Client, String bucket, String key) ; + void flushToS3(); OutputStream getOutputStream(); void setEventCount(int eventCount); + + String getKey(); } diff --git a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/BufferFactory.java b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/BufferFactory.java index 0235642bfd..17d8275f3d 100644 --- a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/BufferFactory.java +++ b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/BufferFactory.java @@ -5,6 +5,10 @@ package org.opensearch.dataprepper.plugins.sink.s3.accumulator; +import software.amazon.awssdk.services.s3.S3Client; + +import java.util.function.Supplier; + public interface BufferFactory { - Buffer getBuffer(); + Buffer getBuffer(S3Client s3Client, Supplier bucketSupplier, Supplier keySupplier); } diff --git a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/BufferTypeOptions.java b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/BufferTypeOptions.java index 494ce36a14..868d6a8a31 100644 --- a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/BufferTypeOptions.java +++ b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/BufferTypeOptions.java @@ -16,7 +16,8 @@ public enum BufferTypeOptions { INMEMORY("in_memory", new InMemoryBufferFactory()), - LOCALFILE("local_file", new LocalFileBufferFactory()); + LOCALFILE("local_file", new LocalFileBufferFactory()), + MULTI_PART("multipart", new MultipartBufferFactory()); private final String option; private final BufferFactory bufferType; @@ -28,7 +29,7 @@ public enum BufferTypeOptions { this.bufferType = bufferType; } - public BufferFactory getBufferType() { + public BufferFactory getBufferFactory() { return bufferType; } diff --git a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/CompressionBuffer.java b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/CompressionBuffer.java index 440c030ac0..47203d69c2 100644 --- a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/CompressionBuffer.java +++ b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/CompressionBuffer.java @@ -6,7 +6,6 @@ package org.opensearch.dataprepper.plugins.sink.s3.accumulator; import org.opensearch.dataprepper.plugins.sink.s3.compression.CompressionEngine; -import software.amazon.awssdk.services.s3.S3Client; import java.io.IOException; import java.io.OutputStream; @@ -38,8 +37,8 @@ public long getDuration() { } @Override - public void flushToS3(final S3Client s3Client, final String bucket, final String key) { - innerBuffer.flushToS3(s3Client, bucket, key); + public void flushToS3() { + innerBuffer.flushToS3(); } @Override @@ -63,4 +62,9 @@ public OutputStream getOutputStream() { public void setEventCount(final int eventCount) { innerBuffer.setEventCount(eventCount); } + + @Override + public String getKey() { + return innerBuffer.getKey(); + } } diff --git a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/CompressionBufferFactory.java b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/CompressionBufferFactory.java index 5dcb652f0f..81bef2c100 100644 --- a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/CompressionBufferFactory.java +++ b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/CompressionBufferFactory.java @@ -5,21 +5,30 @@ package org.opensearch.dataprepper.plugins.sink.s3.accumulator; +import org.opensearch.dataprepper.model.codec.OutputCodec; import org.opensearch.dataprepper.plugins.sink.s3.compression.CompressionEngine; +import software.amazon.awssdk.services.s3.S3Client; import java.util.Objects; +import java.util.function.Supplier; public class CompressionBufferFactory implements BufferFactory { private final BufferFactory innerBufferFactory; private final CompressionEngine compressionEngine; + private final boolean compressionInternal; - public CompressionBufferFactory(final BufferFactory innerBufferFactory, final CompressionEngine compressionEngine) { + public CompressionBufferFactory(final BufferFactory innerBufferFactory, final CompressionEngine compressionEngine, final OutputCodec codec) { this.innerBufferFactory = Objects.requireNonNull(innerBufferFactory); this.compressionEngine = Objects.requireNonNull(compressionEngine); + compressionInternal = Objects.requireNonNull(codec).isCompressionInternal(); } @Override - public Buffer getBuffer() { - return new CompressionBuffer(innerBufferFactory.getBuffer(), compressionEngine); + public Buffer getBuffer(S3Client s3Client, Supplier bucketSupplier, Supplier keySupplier) { + final Buffer internalBuffer = innerBufferFactory.getBuffer(s3Client, bucketSupplier, keySupplier); + if(compressionInternal) + return internalBuffer; + + return new CompressionBuffer(internalBuffer, compressionEngine); } } diff --git a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/InMemoryBuffer.java b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/InMemoryBuffer.java index 58121912d7..bba1c9be8a 100644 --- a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/InMemoryBuffer.java +++ b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/InMemoryBuffer.java @@ -12,6 +12,7 @@ import java.io.ByteArrayOutputStream; import java.io.OutputStream; import java.util.concurrent.TimeUnit; +import java.util.function.Supplier; /** * A buffer can hold in memory data and flushing it to S3. @@ -19,11 +20,19 @@ public class InMemoryBuffer implements Buffer { private static final ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream(); + private final S3Client s3Client; + private final Supplier bucketSupplier; + private final Supplier keySupplier; private int eventCount; private final StopWatch watch; private boolean isCodecStarted; + private String bucket; + private String key; - InMemoryBuffer() { + InMemoryBuffer(S3Client s3Client, Supplier bucketSupplier, Supplier keySupplier) { + this.s3Client = s3Client; + this.bucketSupplier = bucketSupplier; + this.keySupplier = keySupplier; byteArrayOutputStream.reset(); eventCount = 0; watch = new StopWatch(); @@ -47,23 +56,33 @@ public long getDuration() { /** * Upload accumulated data to s3 bucket. - * - * @param s3Client s3 client object. - * @param bucket bucket name. - * @param key s3 object key path. */ @Override - public void flushToS3(S3Client s3Client, String bucket, String key) { + public void flushToS3() { final byte[] byteArray = byteArrayOutputStream.toByteArray(); s3Client.putObject( - PutObjectRequest.builder().bucket(bucket).key(key).build(), + PutObjectRequest.builder().bucket(getBucket()).key(getKey()).build(), RequestBody.fromBytes(byteArray)); } + private String getBucket() { + if(bucket == null) + bucket = bucketSupplier.get(); + return bucket; + } + @Override public void setEventCount(int eventCount) { this.eventCount = eventCount; } + + @Override + public String getKey() { + if(key == null) + key = keySupplier.get(); + return key; + } + @Override public OutputStream getOutputStream() { return byteArrayOutputStream; diff --git a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/InMemoryBufferFactory.java b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/InMemoryBufferFactory.java index d26c2a317d..a631972b93 100644 --- a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/InMemoryBufferFactory.java +++ b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/InMemoryBufferFactory.java @@ -5,9 +5,13 @@ package org.opensearch.dataprepper.plugins.sink.s3.accumulator; +import software.amazon.awssdk.services.s3.S3Client; + +import java.util.function.Supplier; + public class InMemoryBufferFactory implements BufferFactory { @Override - public Buffer getBuffer() { - return new InMemoryBuffer(); + public Buffer getBuffer(S3Client s3Client, Supplier bucketSupplier, Supplier keySupplier) { + return new InMemoryBuffer(s3Client, bucketSupplier, keySupplier); } } diff --git a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/LocalFileBuffer.java b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/LocalFileBuffer.java index 52b6229d92..6fb0d6c043 100644 --- a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/LocalFileBuffer.java +++ b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/LocalFileBuffer.java @@ -20,6 +20,7 @@ import java.nio.file.Files; import java.nio.file.Paths; import java.util.concurrent.TimeUnit; +import java.util.function.Supplier; /** * A buffer can hold local file data and flushing it to S3. @@ -28,14 +29,23 @@ public class LocalFileBuffer implements Buffer { private static final Logger LOG = LoggerFactory.getLogger(LocalFileBuffer.class); private final OutputStream outputStream; + private final S3Client s3Client; + private final Supplier bucketSupplier; + private final Supplier keySupplier; private int eventCount; private final StopWatch watch; private final File localFile; private boolean isCodecStarted; + private String bucket; + private String key; - LocalFileBuffer(File tempFile) throws FileNotFoundException { + + LocalFileBuffer(File tempFile, S3Client s3Client, Supplier bucketSupplier, Supplier keySupplier) throws FileNotFoundException { localFile = tempFile; - outputStream = new BufferedOutputStream(new FileOutputStream(tempFile)); + outputStream = new BufferedOutputStream(new FileOutputStream(tempFile), 32 * 1024); + this.s3Client = s3Client; + this.bucketSupplier = bucketSupplier; + this.keySupplier = keySupplier; eventCount = 0; watch = new StopWatch(); watch.start(); @@ -64,15 +74,12 @@ public long getDuration(){ /** * Upload accumulated data to amazon s3. - * @param s3Client s3 client object. - * @param bucket bucket name. - * @param key s3 object key path. */ @Override - public void flushToS3(S3Client s3Client, String bucket, String key) { + public void flushToS3() { flushAndCloseStream(); s3Client.putObject( - PutObjectRequest.builder().bucket(bucket).key(key).build(), + PutObjectRequest.builder().bucket(getBucket()).key(getKey()).build(), RequestBody.fromFile(localFile)); removeTemporaryFile(); } @@ -112,4 +119,17 @@ public OutputStream getOutputStream() { return outputStream; } + + private String getBucket() { + if(bucket == null) + bucket = bucketSupplier.get(); + return bucket; + } + + @Override + public String getKey() { + if(key == null) + key = keySupplier.get(); + return key; + } } \ No newline at end of file diff --git a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/LocalFileBufferFactory.java b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/LocalFileBufferFactory.java index e50c5e6dd6..d3c068923f 100644 --- a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/LocalFileBufferFactory.java +++ b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/LocalFileBufferFactory.java @@ -7,21 +7,25 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import software.amazon.awssdk.services.s3.S3Client; + import java.io.File; import java.io.IOException; +import java.util.function.Supplier; 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() { + public Buffer getBuffer(S3Client s3Client, Supplier bucketSupplier, Supplier keySupplier) { File tempFile = null; Buffer localfileBuffer = null; try { tempFile = File.createTempFile(PREFIX, SUFFIX); - localfileBuffer = new LocalFileBuffer(tempFile); + localfileBuffer = new LocalFileBuffer(tempFile, s3Client, bucketSupplier, keySupplier); } catch (IOException e) { LOG.error("Unable to create temp file ", e); } diff --git a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/MultipartBuffer.java b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/MultipartBuffer.java new file mode 100644 index 0000000000..e0fadd87f3 --- /dev/null +++ b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/MultipartBuffer.java @@ -0,0 +1,69 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.sink.s3.accumulator; + +import org.apache.commons.lang3.time.StopWatch; +import org.opensearch.dataprepper.plugins.codec.parquet.S3OutputStream; + +import java.io.IOException; +import java.util.concurrent.TimeUnit; + +public class MultipartBuffer implements Buffer { + + private int eventCount; + private final StopWatch watch; + private boolean isCodecStarted; + private S3OutputStream s3OutputStream; + + MultipartBuffer(S3OutputStream s3OutputStream) { + this.s3OutputStream = s3OutputStream; + eventCount = 0; + watch = new StopWatch(); + watch.start(); + isCodecStarted = false; + } + + @Override + public long getSize() { + try { + return s3OutputStream.getPos(); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + @Override + public int getEventCount() { + return eventCount; + } + + public long getDuration() { + return watch.getTime(TimeUnit.SECONDS); + } + + /** + * Upload accumulated data to s3 bucket. + */ + @Override + public void flushToS3() { + s3OutputStream.close(); + } + + @Override + public void setEventCount(int eventCount) { + this.eventCount = eventCount; + } + + @Override + public String getKey() { + return s3OutputStream.getKey(); + } + + @Override + public S3OutputStream getOutputStream() { + return s3OutputStream; + } +} diff --git a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/MultipartBufferFactory.java b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/MultipartBufferFactory.java new file mode 100644 index 0000000000..6bff4331bb --- /dev/null +++ b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/MultipartBufferFactory.java @@ -0,0 +1,18 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.sink.s3.accumulator; + +import org.opensearch.dataprepper.plugins.codec.parquet.S3OutputStream; +import software.amazon.awssdk.services.s3.S3Client; + +import java.util.function.Supplier; + +public class MultipartBufferFactory implements BufferFactory { + @Override + public Buffer getBuffer(S3Client s3Client, Supplier bucketSupplier, Supplier keySupplier) { + return new MultipartBuffer(new S3OutputStream(s3Client, bucketSupplier, keySupplier)); + } +} diff --git a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/codec/Codec.java b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/codec/Codec.java deleted file mode 100644 index 6cedf86779..0000000000 --- a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/codec/Codec.java +++ /dev/null @@ -1,23 +0,0 @@ -/* - * Copyright OpenSearch Contributors - * SPDX-License-Identifier: Apache-2.0 - */ - -package org.opensearch.dataprepper.plugins.sink.s3.codec; - -import java.io.IOException; -import org.opensearch.dataprepper.model.event.Event; - -/** - * Each implementation of this class should support parsing a specific type or format of data. See - * sub-classes for examples. - */ -public interface Codec { - /** - * @param event input data. - * @param tagsTargetKey key name for including tags if not null - * @return parse string. - * @throws IOException exception. - */ - String parse(final Event event, final String tagsTargetKey) throws IOException; -} diff --git a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/codec/JsonCodec.java b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/codec/JsonCodec.java deleted file mode 100644 index 15ac0ba7ec..0000000000 --- a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/codec/JsonCodec.java +++ /dev/null @@ -1,26 +0,0 @@ -/* - * Copyright OpenSearch Contributors - * SPDX-License-Identifier: Apache-2.0 - */ - -package org.opensearch.dataprepper.plugins.sink.s3.codec; - -import java.io.IOException; -import java.util.Objects; -import org.opensearch.dataprepper.model.annotations.DataPrepperPlugin; -import org.opensearch.dataprepper.model.event.Event; - -/** - * An implementation of {@link Codec} which serializes to JSON. - */ -@DataPrepperPlugin(name = "ndjson", pluginType = Codec.class) -public class JsonCodec implements Codec { - /** - * Generates a serialized json string of the Event - */ - @Override - public String parse(final Event event, final String tagsTargetKey) throws IOException { - Objects.requireNonNull(event); - return event.jsonBuilder().includeTags(tagsTargetKey).toJsonString(); - } -} diff --git a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/compression/CompressionOption.java b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/compression/CompressionOption.java index 7e759909d5..86715f806d 100644 --- a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/compression/CompressionOption.java +++ b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/compression/CompressionOption.java @@ -14,7 +14,8 @@ public enum CompressionOption { NONE("none", NoneCompressionEngine::new), - GZIP("gzip", GZipCompressionEngine::new); + GZIP("gzip", GZipCompressionEngine::new), + SNAPPY("snappy", SnappyCompressionEngine::new); private static final Map OPTIONS_MAP = Arrays.stream(CompressionOption.values()) .collect(Collectors.toMap( @@ -23,8 +24,8 @@ public enum CompressionOption { )); private final String option; - private final Supplier compressionEngineSupplier; + private final Supplier compressionEngineSupplier; CompressionOption(final String option, final Supplier compressionEngineSupplier) { this.option = option.toLowerCase(); this.compressionEngineSupplier = compressionEngineSupplier; @@ -34,8 +35,12 @@ public CompressionEngine getCompressionEngine() { return compressionEngineSupplier.get(); } + public String getOption() { + return option; + } + @JsonCreator public static CompressionOption fromOptionValue(final String option) { - return OPTIONS_MAP.get(option.toLowerCase()); + return OPTIONS_MAP.get(option); } } diff --git a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/compression/GZipCompressionEngine.java b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/compression/GZipCompressionEngine.java index f59956a8ed..3fd045b714 100644 --- a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/compression/GZipCompressionEngine.java +++ b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/compression/GZipCompressionEngine.java @@ -10,7 +10,7 @@ import java.io.IOException; import java.io.OutputStream; -public class GZipCompressionEngine implements CompressionEngine { +class GZipCompressionEngine implements CompressionEngine { @Override public OutputStream createOutputStream(final OutputStream outputStream) throws IOException { return new GzipCompressorOutputStream(outputStream); diff --git a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/compression/NoneCompressionEngine.java b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/compression/NoneCompressionEngine.java index 9c852b4f85..e7eed68da2 100644 --- a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/compression/NoneCompressionEngine.java +++ b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/compression/NoneCompressionEngine.java @@ -7,7 +7,7 @@ import java.io.OutputStream; -public class NoneCompressionEngine implements CompressionEngine { +class NoneCompressionEngine implements CompressionEngine { @Override public OutputStream createOutputStream(final OutputStream outputStream) { return outputStream; diff --git a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/compression/SnappyCompressionEngine.java b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/compression/SnappyCompressionEngine.java new file mode 100644 index 0000000000..03bf0eec1b --- /dev/null +++ b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/compression/SnappyCompressionEngine.java @@ -0,0 +1,18 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.sink.s3.compression; + +import org.xerial.snappy.SnappyOutputStream; + +import java.io.IOException; +import java.io.OutputStream; + +class SnappyCompressionEngine implements CompressionEngine { + @Override + public OutputStream createOutputStream(final OutputStream outputStream) throws IOException { + return new SnappyOutputStream(outputStream); + } +} diff --git a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/configuration/ObjectKeyOptions.java b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/configuration/ObjectKeyOptions.java index bd0b07cf1a..bc2c93eb3f 100644 --- a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/configuration/ObjectKeyOptions.java +++ b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/configuration/ObjectKeyOptions.java @@ -11,7 +11,7 @@ * An implementation class of path prefix and file pattern configuration Options */ public class ObjectKeyOptions { - private static final String DEFAULT_OBJECT_NAME_PATTERN = "events-%{yyyy-MM-dd'T'hh-mm-ss}"; + private static final String DEFAULT_OBJECT_NAME_PATTERN = "events-%{yyyy-MM-dd'T'HH-mm-ss'Z'}"; @JsonProperty("path_prefix") private String pathPrefix; diff --git a/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/codec/parquet/CompressionConverterTest.java b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/codec/parquet/CompressionConverterTest.java new file mode 100644 index 0000000000..0f966b9199 --- /dev/null +++ b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/codec/parquet/CompressionConverterTest.java @@ -0,0 +1,39 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.codec.parquet; + +import org.apache.parquet.hadoop.metadata.CompressionCodecName; +import org.junit.jupiter.api.extension.ExtensionContext; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.ArgumentsProvider; +import org.junit.jupiter.params.provider.ArgumentsSource; +import org.opensearch.dataprepper.plugins.sink.s3.compression.CompressionOption; + +import java.util.stream.Stream; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.jupiter.params.provider.Arguments.arguments; + +class CompressionConverterTest { + @ParameterizedTest + @ArgumentsSource(OptionToParquetCodec.class) + void convertCodec_with_known_codecs(final CompressionOption compressionOption, final CompressionCodecName expectedParquetCodec) { + assertThat(CompressionConverter.convertCodec(compressionOption), equalTo(expectedParquetCodec)); + } + + static class OptionToParquetCodec implements ArgumentsProvider { + @Override + public Stream provideArguments(final ExtensionContext context) { + return Stream.of( + arguments(CompressionOption.NONE, CompressionCodecName.UNCOMPRESSED), + arguments(CompressionOption.GZIP, CompressionCodecName.GZIP), + arguments(CompressionOption.SNAPPY, CompressionCodecName.SNAPPY) + ); + } + } +} \ No newline at end of file diff --git a/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/codec/parquet/ParquetOutputCodecTest.java b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/codec/parquet/ParquetOutputCodecTest.java new file mode 100644 index 0000000000..94bed11435 --- /dev/null +++ b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/codec/parquet/ParquetOutputCodecTest.java @@ -0,0 +1,319 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.codec.parquet; + +import org.apache.avro.Schema; +import org.apache.avro.SchemaBuilder; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.parquet.ParquetReadOptions; +import org.apache.parquet.column.page.PageReadStore; +import org.apache.parquet.example.data.Group; +import org.apache.parquet.example.data.simple.SimpleGroup; +import org.apache.parquet.example.data.simple.convert.GroupRecordConverter; +import org.apache.parquet.hadoop.ParquetFileReader; +import org.apache.parquet.hadoop.metadata.ParquetMetadata; +import org.apache.parquet.hadoop.util.HadoopInputFile; +import org.apache.parquet.io.ColumnIOFactory; +import org.apache.parquet.io.MessageColumnIO; +import org.apache.parquet.io.RecordReader; +import org.apache.parquet.schema.MessageType; +import org.apache.parquet.schema.Type; +import org.hamcrest.Matchers; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.io.TempDir; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.log.JacksonLog; +import org.opensearch.dataprepper.plugins.fs.LocalFilePositionOutputStream; +import org.opensearch.dataprepper.plugins.sink.s3.S3OutputCodecContext; +import org.opensearch.dataprepper.plugins.sink.s3.compression.CompressionOption; + +import java.io.ByteArrayInputStream; +import java.io.File; +import java.io.IOException; +import java.io.InputStream; +import java.nio.file.Files; +import java.nio.file.StandardCopyOption; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.function.Function; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +import static org.hamcrest.CoreMatchers.containsString; +import static org.hamcrest.CoreMatchers.equalTo; +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.mock; +import static org.mockito.Mockito.when; + +@ExtendWith(MockitoExtension.class) +public class ParquetOutputCodecTest { + private static final String FILE_NAME = "parquet-data.parquet"; + private ParquetOutputCodecConfig config; + @Mock + private S3OutputCodecContext codecContext; + + @TempDir + private File tempDirectory; + + @BeforeEach + void setUp() { + config = new ParquetOutputCodecConfig(); + } + + private ParquetOutputCodec createObjectUnderTest() { + return new ParquetOutputCodec(config); + } + + @ParameterizedTest + @ValueSource(ints = {1, 2, 10, 100}) + void test_happy_case(final int numberOfRecords) throws Exception { + config.setSchema(createStandardSchema().toString()); + when(codecContext.getCompressionOption()).thenReturn(CompressionOption.NONE); + ParquetOutputCodec parquetOutputCodec = createObjectUnderTest(); + final File tempFile = new File(tempDirectory, FILE_NAME); + LocalFilePositionOutputStream outputStream = LocalFilePositionOutputStream.create(tempFile); + parquetOutputCodec.start(outputStream, null, codecContext); + List> inputMaps = generateRecords(numberOfRecords); + for (Map inputMap : inputMaps) { + final Event event = createEventRecord(inputMap); + parquetOutputCodec.writeEvent(event, outputStream); + } + parquetOutputCodec.closeWriter(outputStream, tempFile); + List> actualRecords = createParquetRecordsList(new ByteArrayInputStream(tempFile.toString().getBytes())); + int index = 0; + for (final Map actualMap : actualRecords) { + assertThat(actualMap, notNullValue()); + Map expectedMap = generateRecords(numberOfRecords).get(index); + assertThat(expectedMap, Matchers.equalTo(actualMap)); + index++; + } + tempFile.delete(); + } + + @ParameterizedTest + @ValueSource(ints = {1, 2, 10, 100}) + void test_happy_case_nullable_records(final int numberOfRecords) throws Exception { + config.setSchema(createStandardSchemaNullable().toString()); + when(codecContext.getCompressionOption()).thenReturn(CompressionOption.NONE); + ParquetOutputCodec parquetOutputCodec = createObjectUnderTest(); + final File tempFile = new File(tempDirectory, FILE_NAME); + LocalFilePositionOutputStream outputStream = LocalFilePositionOutputStream.create(tempFile); + parquetOutputCodec.start(outputStream, null, codecContext); + List> inputMaps = generateRecords(numberOfRecords); + for (Map inputMap : inputMaps) { + final Event event = createEventRecord(inputMap); + parquetOutputCodec.writeEvent(event, outputStream); + } + parquetOutputCodec.closeWriter(outputStream, tempFile); + List> actualRecords = createParquetRecordsList(new ByteArrayInputStream(tempFile.toString().getBytes())); + int index = 0; + for (final Map actualMap : actualRecords) { + assertThat(actualMap, notNullValue()); + Map expectedMap = generateRecords(numberOfRecords).get(index); + assertThat(expectedMap, Matchers.equalTo(actualMap)); + index++; + } + tempFile.delete(); + } + + @ParameterizedTest + @ValueSource(ints = {1, 2, 10, 100}) + void test_happy_case_nullable_records_with_empty_maps(final int numberOfRecords) throws Exception { + config.setSchema(createStandardSchemaNullable().toString()); + when(codecContext.getCompressionOption()).thenReturn(CompressionOption.NONE); + ParquetOutputCodec parquetOutputCodec = createObjectUnderTest(); + final File tempFile = new File(tempDirectory, FILE_NAME); + LocalFilePositionOutputStream outputStream = LocalFilePositionOutputStream.create(tempFile); + parquetOutputCodec.start(outputStream, null, codecContext); + List> inputMaps = generateEmptyRecords(numberOfRecords); + for (Map inputMap : inputMaps) { + final Event event = createEventRecord(inputMap); + parquetOutputCodec.writeEvent(event, outputStream); + } + parquetOutputCodec.closeWriter(outputStream, tempFile); + List> actualRecords = createParquetRecordsList(new ByteArrayInputStream(tempFile.toString().getBytes())); + int index = 0; + for (final Map actualMap : actualRecords) { + assertThat(actualMap, notNullValue()); + Map expectedMap = generateRecords(numberOfRecords).get(index); + assertThat(expectedMap, Matchers.equalTo(actualMap)); + index++; + } + tempFile.delete(); + } + + @Test + public void test_getExtension() { + ParquetOutputCodec parquetOutputCodec = createObjectUnderTest(); + String extension = parquetOutputCodec.getExtension(); + + assertThat(extension, equalTo("parquet")); + } + + @Test + void writeEvent_throws_exception_when_field_does_not_exist() throws IOException { + config.setSchema(createStandardSchema().toString()); + when(codecContext.getCompressionOption()).thenReturn(CompressionOption.NONE); + final Event eventWithInvalidField = mock(Event.class); + final String invalidFieldName = UUID.randomUUID().toString(); + when(eventWithInvalidField.toMap()).thenReturn(Collections.singletonMap(invalidFieldName, UUID.randomUUID().toString())); + final ParquetOutputCodec objectUnderTest = createObjectUnderTest(); + + final File tempFile = new File(tempDirectory, FILE_NAME); + LocalFilePositionOutputStream outputStream = LocalFilePositionOutputStream.create(tempFile); + objectUnderTest.start(outputStream, null, codecContext); + + final RuntimeException actualException = assertThrows(RuntimeException.class, () -> objectUnderTest.writeEvent(eventWithInvalidField, outputStream)); + + assertThat(actualException.getMessage(), notNullValue()); + assertThat(actualException.getMessage(), containsString(invalidFieldName)); + } + + @Test + @Disabled("This feature is not present anyway. But, this test case may be quite correct because it does not account for auto-schema generation.") + public void test_s3SchemaValidity() throws IOException { + config = new ParquetOutputCodecConfig(); + config.setSchema(createStandardSchema().toString()); + config.setSchemaBucket("test"); + config.setSchemaRegion("test"); + config.setFileKey("test"); + ParquetOutputCodec parquetOutputCodec = new ParquetOutputCodec(config); + assertThat(parquetOutputCodec.checkS3SchemaValidity(), equalTo(Boolean.TRUE)); + ParquetOutputCodec parquetOutputCodecFalse = createObjectUnderTest(); + assertThrows(IOException.class, () -> + parquetOutputCodecFalse.checkS3SchemaValidity()); + } + + private static Event createEventRecord(final Map eventData) { + return JacksonLog.builder().withData(eventData).build(); + } + + private static List> generateRecords(final int numberOfRecords) { + final List> recordList = new ArrayList<>(); + + for (int rows = 0; rows < numberOfRecords; rows++) { + + final Map eventData = new HashMap<>(); + + eventData.put("name", "Person" + rows); + eventData.put("age", rows); + eventData.put("myLong", (long) rows + (long) Integer.MAX_VALUE); + eventData.put("myFloat", rows * 1.5f); + eventData.put("myDouble", rows * 1.89d); + eventData.put("myArray", List.of(UUID.randomUUID().toString(), UUID.randomUUID().toString())); + final Map nestedRecord = new HashMap<>(); + nestedRecord.put("firstFieldInNestedRecord", "testString" + rows); + nestedRecord.put("secondFieldInNestedRecord", rows); + eventData.put("nestedRecord", nestedRecord); + recordList.add(eventData); + } + return recordList; + } + + private static List> generateEmptyRecords(final int numberOfRecords) { + return IntStream.range(0, numberOfRecords) + .mapToObj(i -> Collections.emptyMap()) + .collect(Collectors.toList()); + } + + private static Schema createStandardSchema() { + return createStandardSchema(false); + } + + private static Schema createStandardSchemaNullable() { + return createStandardSchema(true); + } + + private static Schema createStandardSchema( + final boolean useNullable) { + final Function, SchemaBuilder.BaseFieldTypeBuilder> typeModifier; + if(useNullable) { + typeModifier = SchemaBuilder.FieldTypeBuilder::nullable; + } else { + typeModifier = schemaFieldTypeBuilder -> schemaFieldTypeBuilder; + } + SchemaBuilder.FieldAssembler assembler = SchemaBuilder.record("Person") + .fields(); + assembler = typeModifier.apply(assembler.name("name").type()).stringType().noDefault(); + assembler = typeModifier.apply(assembler.name("age").type()).intType().noDefault(); + assembler = typeModifier.apply(assembler.name("myLong").type()).longType().noDefault(); + assembler = typeModifier.apply(assembler.name("myFloat").type()).floatType().noDefault(); + assembler = typeModifier.apply(assembler.name("myDouble").type()).doubleType().noDefault(); + assembler = typeModifier.apply(assembler.name("myArray").type()).array().items().stringType().noDefault(); + final Schema innerSchema = createStandardInnerSchemaForNestedRecord(useNullable, typeModifier); + assembler = assembler.name("nestedRecord").type(innerSchema).noDefault(); + + return assembler.endRecord(); + } + + private static Schema createStandardInnerSchemaForNestedRecord( + boolean useNullable, final Function, SchemaBuilder.BaseFieldTypeBuilder> typeModifier) { + SchemaBuilder.RecordBuilder nestedRecord; + if(useNullable) { + nestedRecord = SchemaBuilder.nullable().record("nestedRecord"); + } else { + nestedRecord = SchemaBuilder.record("nestedRecord"); + } + SchemaBuilder.FieldAssembler assembler = nestedRecord.fields(); + assembler = typeModifier.apply(assembler.name("firstFieldInNestedRecord").type()).stringType().noDefault(); + assembler = typeModifier.apply(assembler.name("secondFieldInNestedRecord").type()).intType().noDefault(); + return assembler.endRecord(); + } + + private List> createParquetRecordsList(final InputStream inputStream) throws IOException { + + final File tempFile = new File(tempDirectory, FILE_NAME); + Files.copy(inputStream, tempFile.toPath(), StandardCopyOption.REPLACE_EXISTING); + List> actualRecordList = new ArrayList<>(); + try (ParquetFileReader parquetFileReader = new ParquetFileReader(HadoopInputFile.fromPath(new Path(tempFile.toURI()), new Configuration()), ParquetReadOptions.builder().build())) { + final ParquetMetadata footer = parquetFileReader.getFooter(); + final MessageType schema = createdParquetSchema(footer); + PageReadStore pages; + + while ((pages = parquetFileReader.readNextRowGroup()) != null) { + final long rows = pages.getRowCount(); + final MessageColumnIO columnIO = new ColumnIOFactory().getColumnIO(schema); + final RecordReader recordReader = columnIO.getRecordReader(pages, new GroupRecordConverter(schema)); + for (int row = 0; row < rows; row++) { + final Map eventData = new HashMap<>(); + int fieldIndex = 0; + final SimpleGroup simpleGroup = (SimpleGroup) recordReader.read(); + for (Type field : schema.getFields()) { + try { + eventData.put(field.getName(), simpleGroup.getValueToString(fieldIndex, 0)); + } catch (Exception parquetException) { + parquetException.printStackTrace(); + } + fieldIndex++; + } + actualRecordList.add((HashMap) eventData); + } + } + } catch (Exception parquetException) { + parquetException.printStackTrace(); + } finally { + Files.delete(tempFile.toPath()); + } + return actualRecordList; + } + + private MessageType createdParquetSchema(ParquetMetadata parquetMetadata) { + return parquetMetadata.getFileMetaData().getSchema(); + } +} \ No newline at end of file diff --git a/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/KeyGeneratorTest.java b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/KeyGeneratorTest.java new file mode 100644 index 0000000000..f080163914 --- /dev/null +++ b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/KeyGeneratorTest.java @@ -0,0 +1,77 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.sink.s3; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.dataprepper.model.codec.OutputCodec; +import org.opensearch.dataprepper.plugins.sink.s3.configuration.ObjectKeyOptions; + +import java.time.LocalDateTime; +import java.time.ZoneId; +import java.time.ZonedDateTime; +import java.time.format.DateTimeFormatter; +import java.util.TimeZone; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.mockito.Mockito.when; + +@ExtendWith(MockitoExtension.class) +class KeyGeneratorTest { + public static final String OBJECT_KEY_NAME_PATTERN_START = "events-"; + public static final String OBJECT_KEY_NAME_PATTERN = OBJECT_KEY_NAME_PATTERN_START + "%{yyyy-MM-dd'T'hh-mm-ss}"; + + @Mock + private S3SinkConfig s3SinkConfig; + + @Mock + private OutputCodec outputCodec; + + @Mock + private ObjectKeyOptions objectKeyOptions; + + @BeforeEach + void setUp() { + when(s3SinkConfig.getObjectKeyOptions()).thenReturn(objectKeyOptions); + when(objectKeyOptions.getNamePattern()).thenReturn(OBJECT_KEY_NAME_PATTERN); + } + + private KeyGenerator createObjectUnderTest() { + return new KeyGenerator(s3SinkConfig, outputCodec); + } + + @Test + void test_generateKey_with_general_prefix() { + String pathPrefix = "events/"; + when(s3SinkConfig.getObjectKeyOptions().getPathPrefix()).thenReturn(pathPrefix); + String key = createObjectUnderTest().generateKey(); + assertNotNull(key); + assertThat(key, true); + assertThat(key, key.contains(pathPrefix)); + } + + @Test + void test_generateKey_with_date_prefix() { + String pathPrefix = "logdata/"; + String datePattern = "%{yyyy}/%{MM}/%{dd}/"; + + DateTimeFormatter fomatter = DateTimeFormatter.ofPattern("yyyy/MM/dd"); + ZonedDateTime zdt = LocalDateTime.now().atZone(ZoneId.systemDefault()) + .withZoneSameInstant(ZoneId.of(TimeZone.getTimeZone("UTC").getID())); + String dateString = fomatter.format(zdt); + + when(s3SinkConfig.getObjectKeyOptions() + .getPathPrefix()).thenReturn(pathPrefix + datePattern); + String key = createObjectUnderTest().generateKey(); + assertNotNull(key); + assertThat(key, true); + assertThat(key, key.contains(pathPrefix + dateString)); + } +} \ No newline at end of file diff --git a/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/S3SinkServiceTest.java b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/S3SinkServiceTest.java index 3334a8a6df..cb311d178a 100644 --- a/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/S3SinkServiceTest.java +++ b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/S3SinkServiceTest.java @@ -36,16 +36,11 @@ import java.io.IOException; import java.io.OutputStream; import java.time.Duration; -import java.time.LocalDateTime; -import java.time.ZoneId; -import java.time.ZonedDateTime; -import java.time.format.DateTimeFormatter; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; import java.util.List; import java.util.Random; -import java.util.TimeZone; import java.util.UUID; import java.util.stream.Collectors; @@ -56,7 +51,6 @@ import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyLong; -import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.doNothing; import static org.mockito.Mockito.doThrow; @@ -81,6 +75,7 @@ class S3SinkServiceTest { private S3Client s3Client; private OutputCodec codec; private OutputCodecContext codecContext; + private KeyGenerator keyGenerator = mock(KeyGenerator.class); private PluginMetrics pluginMetrics; private BufferFactory bufferFactory; private Counter snapshotSuccessCounter; @@ -138,7 +133,7 @@ void setUp() { } private S3SinkService createObjectUnderTest() { - return new S3SinkService(s3SinkConfig, bufferFactory, codec, codecContext, s3Client, pluginMetrics); + return new S3SinkService(s3SinkConfig, bufferFactory, codec, codecContext, s3Client, keyGenerator, pluginMetrics); } @Test @@ -148,43 +143,13 @@ void test_s3SinkService_notNull() { assertThat(s3SinkService, instanceOf(S3SinkService.class)); } - @Test - void test_generateKey_with_general_prefix() { - String pathPrefix = "events/"; - when(s3SinkConfig.getObjectKeyOptions().getPathPrefix()).thenReturn(pathPrefix); - S3SinkService s3SinkService = createObjectUnderTest(); - String key = s3SinkService.generateKey(codec); - assertNotNull(key); - assertThat(key, true); - assertThat(key, key.contains(pathPrefix)); - } - - @Test - void test_generateKey_with_date_prefix() { - String pathPrefix = "logdata/"; - String datePattern = "%{yyyy}/%{MM}/%{dd}/"; - - DateTimeFormatter fomatter = DateTimeFormatter.ofPattern("yyyy/MM/dd"); - ZonedDateTime zdt = LocalDateTime.now().atZone(ZoneId.systemDefault()) - .withZoneSameInstant(ZoneId.of(TimeZone.getTimeZone("UTC").getID())); - String dateString = fomatter.format(zdt); - - when(s3SinkConfig.getObjectKeyOptions() - .getPathPrefix()).thenReturn(pathPrefix + datePattern); - S3SinkService s3SinkService = createObjectUnderTest(); - String key = s3SinkService.generateKey(codec); - assertNotNull(key); - assertThat(key, true); - assertThat(key, key.contains(pathPrefix + dateString)); - } - @Test void test_output_with_threshold_set_as_more_then_zero_event_count() throws IOException { bufferFactory = mock(BufferFactory.class); InMemoryBuffer buffer = mock(InMemoryBuffer.class); when(buffer.getEventCount()).thenReturn(10); - doNothing().when(buffer).flushToS3(any(S3Client.class), anyString(), any(String.class)); - when(bufferFactory.getBuffer()).thenReturn(buffer); + doNothing().when(buffer).flushToS3(); + when(bufferFactory.getBuffer(any(S3Client.class), any(), any())).thenReturn(buffer); when(s3SinkConfig.getThresholdOptions().getEventCount()).thenReturn(5); final OutputStream outputStream = mock(OutputStream.class); @@ -193,7 +158,7 @@ void test_output_with_threshold_set_as_more_then_zero_event_count() throws IOExc S3SinkService s3SinkService = createObjectUnderTest(); assertNotNull(s3SinkService); s3SinkService.output(generateRandomStringEventRecord()); - verify(snapshotSuccessCounter, times(50)).increment(); + verify(snapshotSuccessCounter, times(51)).increment(); } @@ -205,8 +170,8 @@ void test_output_with_threshold_set_as_zero_event_count() throws IOException { bufferFactory = mock(BufferFactory.class); InMemoryBuffer buffer = mock(InMemoryBuffer.class); when(buffer.getSize()).thenReturn(25500L); - doNothing().when(buffer).flushToS3(any(S3Client.class), anyString(), any(String.class)); - when(bufferFactory.getBuffer()).thenReturn(buffer); + doNothing().when(buffer).flushToS3(); + when(bufferFactory.getBuffer(any(S3Client.class), any(), any())).thenReturn(buffer); when(s3SinkConfig.getThresholdOptions().getEventCount()).thenReturn(0); when(s3SinkConfig.getThresholdOptions().getMaximumSize()).thenReturn(ByteCount.parse("2kb")); @@ -216,7 +181,7 @@ void test_output_with_threshold_set_as_zero_event_count() throws IOException { S3SinkService s3SinkService = createObjectUnderTest(); assertNotNull(s3SinkService); s3SinkService.output(generateRandomStringEventRecord()); - verify(snapshotSuccessCounter, times(50)).increment(); + verify(snapshotSuccessCounter, times(51)).increment(); } @Test @@ -225,8 +190,8 @@ void test_output_with_uploadedToS3_success() throws IOException { bufferFactory = mock(BufferFactory.class); InMemoryBuffer buffer = mock(InMemoryBuffer.class); when(buffer.getEventCount()).thenReturn(10); - doNothing().when(buffer).flushToS3(any(S3Client.class), anyString(), any(String.class)); - when(bufferFactory.getBuffer()).thenReturn(buffer); + doNothing().when(buffer).flushToS3(); + when(bufferFactory.getBuffer(any(S3Client.class), any(), any())).thenReturn(buffer); final OutputStream outputStream = mock(OutputStream.class); final Event event = JacksonEvent.fromMessage(UUID.randomUUID().toString()); @@ -235,7 +200,7 @@ void test_output_with_uploadedToS3_success() throws IOException { assertNotNull(s3SinkService); assertThat(s3SinkService, instanceOf(S3SinkService.class)); s3SinkService.output(generateRandomStringEventRecord()); - verify(snapshotSuccessCounter, times(50)).increment(); + verify(snapshotSuccessCounter, times(51)).increment(); } @Test @@ -243,7 +208,7 @@ void test_output_with_uploadedToS3_success_records_byte_count() throws IOExcepti bufferFactory = mock(BufferFactory.class); Buffer buffer = mock(Buffer.class); - when(bufferFactory.getBuffer()).thenReturn(buffer); + when(bufferFactory.getBuffer(any(S3Client.class), any(), any())).thenReturn(buffer); final long objectSize = random.nextInt(1_000_000) + 10_000; when(buffer.getSize()).thenReturn(objectSize); @@ -254,7 +219,35 @@ void test_output_with_uploadedToS3_success_records_byte_count() throws IOExcepti final S3SinkService s3SinkService = createObjectUnderTest(); s3SinkService.output(generateRandomStringEventRecord()); - verify(s3ObjectSizeSummary, times(50)).record(objectSize); + verify(s3ObjectSizeSummary, times(51)).record(objectSize); + } + + @Test + void test_output_with_uploadedToS3_midBatch_generatesNewOutputStream() throws IOException { + + bufferFactory = mock(BufferFactory.class); + InMemoryBuffer buffer = mock(InMemoryBuffer.class); + when(buffer.getEventCount()).thenReturn(10); + doNothing().when(buffer).flushToS3(); + when(bufferFactory.getBuffer(any(S3Client.class), any(), any())).thenReturn(buffer); + final OutputStream outputStream1 = mock(OutputStream.class); + final OutputStream outputStream2 = mock(OutputStream.class); + when(buffer.getOutputStream()) + .thenReturn(outputStream1) + .thenReturn(outputStream2); + + doNothing().when(codec).writeEvent(any(), eq(outputStream1)); + doNothing().when(codec).writeEvent(any(), eq(outputStream2)); + + S3SinkService s3SinkService = createObjectUnderTest(); + assertNotNull(s3SinkService); + assertThat(s3SinkService, instanceOf(S3SinkService.class)); + + s3SinkService.output(generateEventRecords(2)); + + verify(snapshotSuccessCounter, times(3)).increment(); + verify(codec).writeEvent(any(), eq(outputStream1)); + verify(codec).writeEvent(any(), eq(outputStream2)); } @Test @@ -276,9 +269,9 @@ void test_output_with_uploadedToS3_failure_does_not_record_byte_count() throws I bufferFactory = mock(BufferFactory.class); Buffer buffer = mock(Buffer.class); - when(bufferFactory.getBuffer()).thenReturn(buffer); + when(bufferFactory.getBuffer(any(S3Client.class), any(), any())).thenReturn(buffer); - doThrow(AwsServiceException.class).when(buffer).flushToS3(any(), anyString(), anyString()); + doThrow(AwsServiceException.class).when(buffer).flushToS3(); final long objectSize = random.nextInt(1_000_000) + 10_000; when(buffer.getSize()).thenReturn(objectSize); @@ -290,7 +283,45 @@ void test_output_with_uploadedToS3_failure_does_not_record_byte_count() throws I s3SinkService.output(Collections.singletonList(new Record<>(event))); verify(s3ObjectSizeSummary, never()).record(anyLong()); - verify(buffer, times(3)).flushToS3(any(), anyString(), anyString()); + verify(buffer, times(6)).flushToS3(); + } + + @Test + void test_output_with_no_incoming_records_flushes_batch() throws IOException { + + bufferFactory = mock(BufferFactory.class); + Buffer buffer = mock(Buffer.class); + when(bufferFactory.getBuffer(any(S3Client.class), any(), any())).thenReturn(buffer); + when(buffer.getEventCount()).thenReturn(10); + + final OutputStream outputStream = mock(OutputStream.class); + final Event event = JacksonEvent.fromMessage(UUID.randomUUID().toString()); + doNothing().when(codec).writeEvent(event, outputStream); + final S3SinkService s3SinkService = createObjectUnderTest(); + s3SinkService.output(Collections.emptyList()); + + verify(snapshotSuccessCounter, times(1)).increment(); + verify(buffer, times(1)).flushToS3(); + } + + @Test + void test_output_with_no_incoming_records_or_buffered_records_short_circuits() throws IOException { + + bufferFactory = mock(BufferFactory.class); + Buffer buffer = mock(Buffer.class); + when(bufferFactory.getBuffer(any(S3Client.class), any(), any())).thenReturn(buffer); + when(buffer.getEventCount()).thenReturn(0); + final long objectSize = random.nextInt(1_000_000) + 10_000; + when(buffer.getSize()).thenReturn(objectSize); + + final OutputStream outputStream = mock(OutputStream.class); + final Event event = JacksonEvent.fromMessage(UUID.randomUUID().toString()); + doNothing().when(codec).writeEvent(event, outputStream); + final S3SinkService s3SinkService = createObjectUnderTest(); + s3SinkService.output(Collections.emptyList()); + + verify(snapshotSuccessCounter, times(0)).increment(); + verify(buffer, times(0)).flushToS3(); } @Test @@ -298,8 +329,8 @@ void test_retryFlushToS3_positive() throws InterruptedException, IOException { bufferFactory = mock(BufferFactory.class); InMemoryBuffer buffer = mock(InMemoryBuffer.class); - doNothing().when(buffer).flushToS3(any(S3Client.class), anyString(), any(String.class)); - when(bufferFactory.getBuffer()).thenReturn(buffer); + doNothing().when(buffer).flushToS3(); + when(bufferFactory.getBuffer(any(S3Client.class), any(), any())).thenReturn(buffer); S3SinkService s3SinkService = createObjectUnderTest(); assertNotNull(s3SinkService); @@ -316,7 +347,7 @@ void test_retryFlushToS3_positive() throws InterruptedException, IOException { void test_retryFlushToS3_negative() throws InterruptedException, IOException { bufferFactory = mock(BufferFactory.class); InMemoryBuffer buffer = mock(InMemoryBuffer.class); - when(bufferFactory.getBuffer()).thenReturn(buffer); + when(bufferFactory.getBuffer(any(S3Client.class), any(), any())).thenReturn(buffer); when(s3SinkConfig.getBucketName()).thenReturn(""); S3SinkService s3SinkService = createObjectUnderTest(); assertNotNull(s3SinkService); @@ -324,7 +355,7 @@ void test_retryFlushToS3_negative() throws InterruptedException, IOException { final Event event = JacksonEvent.fromMessage(UUID.randomUUID().toString()); codec.writeEvent(event, outputStream); final String s3Key = UUID.randomUUID().toString(); - doThrow(AwsServiceException.class).when(buffer).flushToS3(eq(s3Client), anyString(), anyString()); + doThrow(AwsServiceException.class).when(buffer).flushToS3(); boolean isUploadedToS3 = s3SinkService.retryFlushToS3(buffer, s3Key); assertFalse(isUploadedToS3); } @@ -334,7 +365,7 @@ void test_retryFlushToS3_negative() throws InterruptedException, IOException { void output_will_release_all_handles_since_a_flush() throws IOException { bufferFactory = mock(BufferFactory.class); final Buffer buffer = mock(Buffer.class); - when(bufferFactory.getBuffer()).thenReturn(buffer); + when(bufferFactory.getBuffer(any(S3Client.class), any(), any())).thenReturn(buffer); final long objectSize = random.nextInt(1_000_000) + 10_000; when(buffer.getSize()).thenReturn(objectSize); @@ -357,7 +388,7 @@ void output_will_release_all_handles_since_a_flush() throws IOException { void output_will_skip_releasing_events_without_EventHandle_objects() throws IOException { bufferFactory = mock(BufferFactory.class); final Buffer buffer = mock(Buffer.class); - when(bufferFactory.getBuffer()).thenReturn(buffer); + when(bufferFactory.getBuffer(any(S3Client.class), any(), any())).thenReturn(buffer); final long objectSize = random.nextInt(1_000_000) + 10_000; when(buffer.getSize()).thenReturn(objectSize); @@ -388,9 +419,9 @@ void output_will_skip_releasing_events_without_EventHandle_objects() throws IOEx void output_will_release_all_handles_since_a_flush_when_S3_fails() throws IOException { bufferFactory = mock(BufferFactory.class); final Buffer buffer = mock(Buffer.class); - when(bufferFactory.getBuffer()).thenReturn(buffer); + when(bufferFactory.getBuffer(any(S3Client.class), any(), any())).thenReturn(buffer); - doThrow(AwsServiceException.class).when(buffer).flushToS3(any(), anyString(), anyString()); + doThrow(AwsServiceException.class).when(buffer).flushToS3(); final long objectSize = random.nextInt(1_000_000) + 10_000; when(buffer.getSize()).thenReturn(objectSize); @@ -413,7 +444,7 @@ void output_will_release_all_handles_since_a_flush_when_S3_fails() throws IOExce void output_will_release_only_new_handles_since_a_flush() throws IOException { bufferFactory = mock(BufferFactory.class); final Buffer buffer = mock(Buffer.class); - when(bufferFactory.getBuffer()).thenReturn(buffer); + when(bufferFactory.getBuffer(any(S3Client.class), any(), any())).thenReturn(buffer); final long objectSize = random.nextInt(1_000_000) + 10_000; when(buffer.getSize()).thenReturn(objectSize); @@ -444,9 +475,9 @@ void output_will_release_only_new_handles_since_a_flush() throws IOException { void output_will_release_only_new_handles_since_a_flush_when_S3_fails() throws IOException { bufferFactory = mock(BufferFactory.class); final Buffer buffer = mock(Buffer.class); - when(bufferFactory.getBuffer()).thenReturn(buffer); + when(bufferFactory.getBuffer(any(S3Client.class), any(), any())).thenReturn(buffer); - doThrow(AwsServiceException.class).when(buffer).flushToS3(any(), anyString(), anyString()); + doThrow(AwsServiceException.class).when(buffer).flushToS3(); final long objectSize = random.nextInt(1_000_000) + 10_000; when(buffer.getSize()).thenReturn(objectSize); diff --git a/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/ThresholdCheckTest.java b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/ThresholdCheckTest.java index 864421accb..2c454fe568 100644 --- a/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/ThresholdCheckTest.java +++ b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/ThresholdCheckTest.java @@ -23,7 +23,7 @@ class ThresholdCheckTest { @BeforeEach void setUp() throws IOException { - inMemoryBuffer = new InMemoryBufferFactory().getBuffer(); + inMemoryBuffer = new InMemoryBufferFactory().getBuffer(null, null, null); while (inMemoryBuffer.getEventCount() < 100) { OutputStream outputStream = inMemoryBuffer.getOutputStream(); @@ -80,7 +80,7 @@ void test_exceedThreshold_ture_dueTo_maxCollectionDuration_is_less_than_buffered final ByteCount maxBytes = ByteCount.parse("500mb"); final long maxCollectionDuration = 10; - inMemoryBuffer = new InMemoryBufferFactory().getBuffer(); + inMemoryBuffer = new InMemoryBufferFactory().getBuffer(null, null, null); boolean isThresholdExceed = Boolean.FALSE; synchronized (this) { while (inMemoryBuffer.getEventCount() < 100) { @@ -106,7 +106,7 @@ void test_exceedThreshold_ture_dueTo_maxCollectionDuration_is_greater_than_buffe final ByteCount maxBytes = ByteCount.parse("500mb"); final long maxCollectionDuration = 240; - inMemoryBuffer = new InMemoryBufferFactory().getBuffer(); + inMemoryBuffer = new InMemoryBufferFactory().getBuffer(null,null, null); boolean isThresholdExceed = Boolean.FALSE; synchronized (this) { diff --git a/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/BufferTypeOptionsTest.java b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/BufferTypeOptionsTest.java index 65531acc32..ebc2cd9351 100644 --- a/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/BufferTypeOptionsTest.java +++ b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/BufferTypeOptionsTest.java @@ -23,7 +23,7 @@ void notNull_test() { @Test void get_buffer_type_test() { - assertNotNull(BufferTypeOptions.INMEMORY.getBufferType()); + assertNotNull(BufferTypeOptions.INMEMORY.getBufferFactory()); } @Test diff --git a/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/CompressionBufferFactoryTest.java b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/CompressionBufferFactoryTest.java index a27798f3df..00bd76e239 100644 --- a/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/CompressionBufferFactoryTest.java +++ b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/CompressionBufferFactoryTest.java @@ -11,13 +11,20 @@ import org.junit.jupiter.api.extension.ExtendWith; import org.mockito.Mock; import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.dataprepper.model.codec.OutputCodec; import org.opensearch.dataprepper.plugins.sink.s3.compression.CompressionEngine; +import software.amazon.awssdk.services.s3.S3Client; + +import java.util.function.Supplier; import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.instanceOf; import static org.hamcrest.CoreMatchers.not; +import static org.hamcrest.CoreMatchers.sameInstance; import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; @ExtendWith(MockitoExtension.class) @@ -28,8 +35,20 @@ class CompressionBufferFactoryTest { @Mock private CompressionEngine compressionEngine; + @Mock + private S3Client s3Client; + + @Mock + private Supplier bucketSupplier; + + @Mock + private Supplier keySupplier; + + @Mock + private OutputCodec codec; + private CompressionBufferFactory createObjectUnderTest() { - return new CompressionBufferFactory(innerBufferFactory, compressionEngine); + return new CompressionBufferFactory(innerBufferFactory, compressionEngine, codec); } @Test @@ -46,6 +65,13 @@ void constructor_throws_if_CompressionEngine_is_null() { assertThrows(NullPointerException.class, this::createObjectUnderTest); } + @Test + void constructor_throws_if_Codec_is_null() { + codec = null; + + assertThrows(NullPointerException.class, this::createObjectUnderTest); + } + @Nested class WithBuffer { @Mock @@ -53,21 +79,44 @@ class WithBuffer { @BeforeEach void setUp() { - when(innerBufferFactory.getBuffer()).thenReturn(innerBuffer); + when(innerBufferFactory.getBuffer(s3Client, bucketSupplier, keySupplier)).thenReturn(innerBuffer); } @Test void getBuffer_returns_CompressionBuffer() { - final Buffer buffer = createObjectUnderTest().getBuffer(); + final Buffer buffer = createObjectUnderTest().getBuffer(s3Client, bucketSupplier, keySupplier); assertThat(buffer, instanceOf(CompressionBuffer.class)); } @Test void getBuffer_returns_new_on_each_call() { final CompressionBufferFactory objectUnderTest = createObjectUnderTest(); - final Buffer firstBuffer = objectUnderTest.getBuffer(); + final Buffer firstBuffer = objectUnderTest.getBuffer(s3Client, bucketSupplier, keySupplier); + + assertThat(objectUnderTest.getBuffer(s3Client, bucketSupplier, keySupplier), not(equalTo(firstBuffer))); + } + + @Nested + class WithInternalCompression { + @BeforeEach + void setUp() { + when(codec.isCompressionInternal()).thenReturn(true); + } + + @Test + void getBuffer_returns_innerBuffer_directly() { + final Buffer buffer = createObjectUnderTest().getBuffer(s3Client, bucketSupplier, keySupplier); + assertThat(buffer, sameInstance(innerBuffer)); + } + + @Test + void getBuffer_calls_on_each_call() { + final CompressionBufferFactory objectUnderTest = createObjectUnderTest(); + objectUnderTest.getBuffer(s3Client, bucketSupplier, keySupplier); + objectUnderTest.getBuffer(s3Client, bucketSupplier, keySupplier); - assertThat(objectUnderTest.getBuffer(), not(equalTo(firstBuffer))); + verify(innerBufferFactory, times(2)).getBuffer(s3Client, bucketSupplier, keySupplier); + } } } } \ No newline at end of file diff --git a/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/CompressionBufferTest.java b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/CompressionBufferTest.java index 3a7055414b..0195d1ae41 100644 --- a/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/CompressionBufferTest.java +++ b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/CompressionBufferTest.java @@ -95,9 +95,9 @@ void flushToS3_calls_inner_flushToS3() { final String bucket = UUID.randomUUID().toString(); final String key = UUID.randomUUID().toString(); - createObjectUnderTest().flushToS3(s3Client, bucket, key); + createObjectUnderTest().flushToS3(); - verify(innerBuffer).flushToS3(s3Client, bucket, key); + verify(innerBuffer).flushToS3(); } @Test diff --git a/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/InMemoryBufferFactoryTest.java b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/InMemoryBufferFactoryTest.java index 0964cbece4..0196657f3b 100644 --- a/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/InMemoryBufferFactoryTest.java +++ b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/InMemoryBufferFactoryTest.java @@ -22,7 +22,7 @@ void test_inMemoryBufferFactory_notNull(){ void test_buffer_notNull(){ InMemoryBufferFactory inMemoryBufferFactory = new InMemoryBufferFactory(); Assertions.assertNotNull(inMemoryBufferFactory); - Buffer buffer = inMemoryBufferFactory.getBuffer(); + Buffer buffer = inMemoryBufferFactory.getBuffer(null, null, null); Assertions.assertNotNull(buffer); assertThat(buffer, instanceOf(Buffer.class)); } diff --git a/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/InMemoryBufferTest.java b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/InMemoryBufferTest.java index e43f801eae..9a681b7354 100644 --- a/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/InMemoryBufferTest.java +++ b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/InMemoryBufferTest.java @@ -7,18 +7,28 @@ import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.MatcherAssert.assertThat; + +import org.hamcrest.Matchers; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; import org.mockito.Mock; import org.mockito.junit.jupiter.MockitoExtension; +import software.amazon.awssdk.core.exception.SdkClientException; +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.IOException; import java.io.OutputStream; +import java.util.function.Supplier; import static org.hamcrest.Matchers.greaterThanOrEqualTo; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; @ExtendWith(MockitoExtension.class) class InMemoryBufferTest { @@ -26,11 +36,15 @@ class InMemoryBufferTest { public static final int MAX_EVENTS = 55; @Mock private S3Client s3Client; + @Mock + private Supplier bucketSupplier; + @Mock + private Supplier keySupplier; private InMemoryBuffer inMemoryBuffer; @Test void test_with_write_event_into_buffer() throws IOException { - inMemoryBuffer = new InMemoryBuffer(); + inMemoryBuffer = new InMemoryBuffer(s3Client, bucketSupplier, keySupplier); while (inMemoryBuffer.getEventCount() < MAX_EVENTS) { OutputStream outputStream = inMemoryBuffer.getOutputStream(); @@ -46,7 +60,7 @@ void test_with_write_event_into_buffer() throws IOException { @Test void test_with_write_event_into_buffer_and_flush_toS3() throws IOException { - inMemoryBuffer = new InMemoryBuffer(); + inMemoryBuffer = new InMemoryBuffer(s3Client, bucketSupplier, keySupplier); while (inMemoryBuffer.getEventCount() < MAX_EVENTS) { OutputStream outputStream = inMemoryBuffer.getOutputStream(); @@ -55,24 +69,29 @@ void test_with_write_event_into_buffer_and_flush_toS3() throws IOException { inMemoryBuffer.setEventCount(eventCount); } assertDoesNotThrow(() -> { - inMemoryBuffer.flushToS3(s3Client, "data-prepper", "log.txt"); + inMemoryBuffer.flushToS3(); }); } @Test void test_uploadedToS3_success() { - inMemoryBuffer = new InMemoryBuffer(); + inMemoryBuffer = new InMemoryBuffer(s3Client, bucketSupplier, keySupplier); Assertions.assertNotNull(inMemoryBuffer); assertDoesNotThrow(() -> { - inMemoryBuffer.flushToS3(s3Client, "data-prepper", "log.txt"); + inMemoryBuffer.flushToS3(); }); } @Test void test_uploadedToS3_fails() { - inMemoryBuffer = new InMemoryBuffer(); + inMemoryBuffer = new InMemoryBuffer(s3Client, bucketSupplier, keySupplier); Assertions.assertNotNull(inMemoryBuffer); - assertThrows(Exception.class, () -> inMemoryBuffer.flushToS3(null, null, null)); + SdkClientException sdkClientException = mock(SdkClientException.class); + when(s3Client.putObject(any(PutObjectRequest.class), any(RequestBody.class))) + .thenThrow(sdkClientException); + SdkClientException actualException = assertThrows(SdkClientException.class, () -> inMemoryBuffer.flushToS3()); + + assertThat(actualException, Matchers.equalTo(sdkClientException)); } private byte[] generateByteArray() { diff --git a/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/LocalFileBufferFactoryTest.java b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/LocalFileBufferFactoryTest.java index 4eb509dff5..834a1af710 100644 --- a/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/LocalFileBufferFactoryTest.java +++ b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/LocalFileBufferFactoryTest.java @@ -21,7 +21,7 @@ void test_localFileBufferFactory_notNull() { void test_buffer_notNull() { LocalFileBufferFactory localFileBufferFactory = new LocalFileBufferFactory(); Assertions.assertNotNull(localFileBufferFactory); - Buffer buffer = localFileBufferFactory.getBuffer(); + Buffer buffer = localFileBufferFactory.getBuffer(null, null, null); Assertions.assertNotNull(buffer); assertThat(buffer, instanceOf(LocalFileBuffer.class)); } diff --git a/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/LocalFileBufferTest.java b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/LocalFileBufferTest.java index f05a688936..5dd1044d5c 100644 --- a/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/LocalFileBufferTest.java +++ b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/LocalFileBufferTest.java @@ -15,6 +15,8 @@ import java.io.IOException; import java.io.OutputStream; import java.util.UUID; +import java.util.function.Supplier; + import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.notNullValue; import static org.hamcrest.CoreMatchers.nullValue; @@ -25,6 +27,7 @@ import static org.junit.jupiter.api.Assertions.assertFalse; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; @ExtendWith(MockitoExtension.class) class LocalFileBufferTest { @@ -35,13 +38,17 @@ class LocalFileBufferTest { public static final String SUFFIX = ".log"; @Mock private S3Client s3Client; + @Mock + private Supplier bucketSupplier; + @Mock + private Supplier keySupplier; private LocalFileBuffer localFileBuffer; private File tempFile; @BeforeEach void setUp() throws IOException { tempFile = File.createTempFile(PREFIX, SUFFIX); - localFileBuffer = new LocalFileBuffer(tempFile); + localFileBuffer = new LocalFileBuffer(tempFile, s3Client, bucketSupplier, keySupplier); } @Test @@ -82,8 +89,11 @@ void test_with_write_events_into_buffer_and_flush_toS3() throws IOException { assertThat(localFileBuffer.getEventCount(), equalTo(55)); assertThat(localFileBuffer.getDuration(), greaterThanOrEqualTo(0L)); + when(keySupplier.get()).thenReturn(KEY); + when(bucketSupplier.get()).thenReturn(BUCKET_NAME); + assertDoesNotThrow(() -> { - localFileBuffer.flushToS3(s3Client, BUCKET_NAME, KEY); + localFileBuffer.flushToS3(); }); ArgumentCaptor putObjectRequestArgumentCaptor = ArgumentCaptor.forClass(PutObjectRequest.class); @@ -100,9 +110,12 @@ void test_with_write_events_into_buffer_and_flush_toS3() throws IOException { @Test void test_uploadedToS3_success() { + when(keySupplier.get()).thenReturn(KEY); + when(bucketSupplier.get()).thenReturn(BUCKET_NAME); + Assertions.assertNotNull(localFileBuffer); assertDoesNotThrow(() -> { - localFileBuffer.flushToS3(s3Client, BUCKET_NAME, KEY); + localFileBuffer.flushToS3(); }); ArgumentCaptor putObjectRequestArgumentCaptor = ArgumentCaptor.forClass(PutObjectRequest.class); diff --git a/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/codec/JsonCodecTest.java b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/codec/JsonCodecTest.java deleted file mode 100644 index 35fd55c9f1..0000000000 --- a/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/codec/JsonCodecTest.java +++ /dev/null @@ -1,73 +0,0 @@ -/* - * Copyright OpenSearch Contributors - * SPDX-License-Identifier: Apache-2.0 - */ - -package org.opensearch.dataprepper.plugins.sink.s3.codec; - -import static org.hamcrest.CoreMatchers.equalTo; -import static org.hamcrest.CoreMatchers.notNullValue; -import static org.hamcrest.MatcherAssert.assertThat; -import static org.junit.jupiter.api.Assertions.assertNotNull; -import java.io.IOException; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.UUID; -import org.junit.jupiter.api.Test; -import org.opensearch.dataprepper.model.event.JacksonEvent; -import org.opensearch.dataprepper.model.log.JacksonLog; -import com.fasterxml.jackson.databind.ObjectMapper; - -class JsonCodecTest { - - @Test - void parse_with_events_output_stream_json_codec() throws IOException { - - final Map eventData = new HashMap<>(); - String value1 = UUID.randomUUID().toString(); - eventData.put("key1", value1); - String value2 = UUID.randomUUID().toString(); - eventData.put("key2", value2); - final JacksonEvent event = JacksonLog.builder().withData(eventData).withEventType("LOG").build(); - String output = createObjectUnderTest().parse(event, null); - assertNotNull(output); - - ObjectMapper objectMapper = new ObjectMapper(); - Map deserializedData = objectMapper.readValue(output, Map.class); - assertThat(deserializedData, notNullValue()); - assertThat(deserializedData.get("key1"), notNullValue()); - assertThat(deserializedData.get("key1"), equalTo(value1)); - assertThat(deserializedData.get("key2"), notNullValue()); - assertThat(deserializedData.get("key2"), equalTo(value2)); - } - - @Test - void parse_with_events_output_stream_json_codec_with_tags() throws IOException { - - final Map eventData = new HashMap<>(); - String value1 = UUID.randomUUID().toString(); - eventData.put("key1", value1); - String value2 = UUID.randomUUID().toString(); - eventData.put("key2", value2); - final JacksonEvent event = JacksonLog.builder().withData(eventData).withEventType("LOG").build(); - List tagsList = List.of("tag1"); - event.getMetadata().addTags(tagsList); - String output = createObjectUnderTest().parse(event, "tags"); - assertNotNull(output); - - ObjectMapper objectMapper = new ObjectMapper(); - Map deserializedData = objectMapper.readValue(output, Map.class); - assertThat(deserializedData, notNullValue()); - assertThat(deserializedData.get("key1"), notNullValue()); - assertThat(deserializedData.get("key1"), equalTo(value1)); - assertThat(deserializedData.get("key2"), notNullValue()); - assertThat(deserializedData.get("key2"), equalTo(value2)); - assertThat(deserializedData.get("tags"), notNullValue()); - assertThat(deserializedData.get("tags"), equalTo(tagsList)); - } - - private JsonCodec createObjectUnderTest() { - return new JsonCodec(); - } -} diff --git a/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/compression/CompressionOptionTest.java b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/compression/CompressionOptionTest.java new file mode 100644 index 0000000000..15a13b31db --- /dev/null +++ b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/compression/CompressionOptionTest.java @@ -0,0 +1,51 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.sink.s3.compression; + +import org.junit.jupiter.api.extension.ExtensionContext; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.ArgumentsProvider; +import org.junit.jupiter.params.provider.ArgumentsSource; +import org.junit.jupiter.params.provider.EnumSource; + +import java.util.stream.Stream; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.instanceOf; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.jupiter.params.provider.Arguments.arguments; + +class CompressionOptionTest { + @ParameterizedTest + @EnumSource(CompressionOption.class) + void fromOptionValue_returns_expected_value(final CompressionOption option) { + assertThat(CompressionOption.fromOptionValue(option.getOption()), equalTo(option)); + } + + @ParameterizedTest + @EnumSource(CompressionOption.class) + void getCompressionEngine_returns_a_CompressionEngine(final CompressionOption option) { + assertThat(option.getCompressionEngine(), instanceOf(CompressionEngine.class)); + } + + @ParameterizedTest + @ArgumentsSource(OptionToExpectedEngine.class) + void getCompressionEngine_returns_expected_engine_type(final CompressionOption option, final Class expectedEngineType) { + assertThat(option.getCompressionEngine(), instanceOf(expectedEngineType)); + } + + static class OptionToExpectedEngine implements ArgumentsProvider { + @Override + public Stream provideArguments(final ExtensionContext context) { + return Stream.of( + arguments(CompressionOption.NONE, NoneCompressionEngine.class), + arguments(CompressionOption.GZIP, GZipCompressionEngine.class), + arguments(CompressionOption.SNAPPY, SnappyCompressionEngine.class) + ); + } + } +} \ No newline at end of file diff --git a/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/compression/SnappyCompressionEngineTest.java b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/compression/SnappyCompressionEngineTest.java new file mode 100644 index 0000000000..709d445d05 --- /dev/null +++ b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/compression/SnappyCompressionEngineTest.java @@ -0,0 +1,61 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.sink.s3.compression; + +import org.junit.jupiter.api.Test; +import org.xerial.snappy.SnappyCodec; +import org.xerial.snappy.SnappyInputStream; +import org.xerial.snappy.SnappyOutputStream; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.OutputStream; +import java.nio.charset.StandardCharsets; +import java.util.UUID; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.instanceOf; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.Mockito.mock; + +class SnappyCompressionEngineTest { + + private SnappyCompressionEngine createObjectUnderTest() { + return new SnappyCompressionEngine(); + } + + @Test + void createOutputStream_should_return_SnappyOutputStream() throws IOException { + final OutputStream innerOutputStream = mock(OutputStream.class); + final OutputStream outputStream = createObjectUnderTest().createOutputStream(innerOutputStream); + + assertThat(outputStream, instanceOf(SnappyOutputStream.class)); + } + + @Test + void createOutputStream_should_write_compressed_data() throws IOException { + final ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream(); + + final OutputStream outputStream = createObjectUnderTest().createOutputStream(byteArrayOutputStream); + + final byte[] inputBytes = UUID.randomUUID().toString().getBytes(StandardCharsets.UTF_8); + + outputStream.write(inputBytes); + outputStream.close(); + + final byte[] writtenBytes = byteArrayOutputStream.toByteArray(); + + assertTrue(SnappyCodec.hasMagicHeaderPrefix(writtenBytes)); + + final ByteArrayInputStream verificationInputStream = new ByteArrayInputStream(writtenBytes); + + final SnappyInputStream uncompressingInputStream = new SnappyInputStream(verificationInputStream); + final byte[] uncompressedBytes = uncompressingInputStream.readAllBytes(); + assertThat(uncompressedBytes, equalTo(inputBytes)); + } +} \ No newline at end of file diff --git a/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/configuration/ObjectKeyOptionsTest.java b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/configuration/ObjectKeyOptionsTest.java index 5f52015eaf..6dd3f9c8bf 100644 --- a/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/configuration/ObjectKeyOptionsTest.java +++ b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/configuration/ObjectKeyOptionsTest.java @@ -12,7 +12,7 @@ class ObjectKeyOptionsTest { - private static final String DEFAULT_FILE_PATTERN = "events-%{yyyy-MM-dd'T'hh-mm-ss}"; + private static final String DEFAULT_FILE_PATTERN = "events-%{yyyy-MM-dd'T'HH-mm-ss'Z'}"; @Test void default_file_pattern_test() { diff --git a/data-prepper-plugins/s3-sink/src/test/resources/simplelogger.properties b/data-prepper-plugins/s3-sink/src/test/resources/simplelogger.properties new file mode 100644 index 0000000000..a1f54ead37 --- /dev/null +++ b/data-prepper-plugins/s3-sink/src/test/resources/simplelogger.properties @@ -0,0 +1,7 @@ +# +# Copyright OpenSearch Contributors +# SPDX-License-Identifier: Apache-2.0 +# + +org.slf4j.simpleLogger.showDateTime=true +org.slf4j.simpleLogger.dateTimeFormat=yyyy-MM-dd' 'HH:mm:ss.SSS diff --git a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/CustomLocalDateTimeDeserializer.java b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/CustomLocalDateTimeDeserializer.java deleted file mode 100644 index c118e95cbc..0000000000 --- a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/CustomLocalDateTimeDeserializer.java +++ /dev/null @@ -1,45 +0,0 @@ -/* - * Copyright OpenSearch Contributors - * * SPDX-License-Identifier: Apache-2.0 - * - */ -package org.opensearch.dataprepper.plugins.source; - -import com.fasterxml.jackson.core.JsonParser; -import com.fasterxml.jackson.databind.DeserializationContext; -import com.fasterxml.jackson.databind.deser.std.StdDeserializer; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.IOException; -import java.time.LocalDateTime; -import java.time.format.DateTimeParseException; - -public class CustomLocalDateTimeDeserializer extends StdDeserializer { - private static final Logger LOG = LoggerFactory.getLogger(CustomLocalDateTimeDeserializer.class); - static final String CURRENT_LOCAL_DATE_TIME_STRING = "now"; - - public CustomLocalDateTimeDeserializer() { - this(null); - } - - public CustomLocalDateTimeDeserializer(Class vc) { - super(vc); - } - - @Override - public LocalDateTime deserialize(JsonParser parser, DeserializationContext context) throws IOException { - final String valueAsString = parser.getValueAsString(); - - if (valueAsString.equals(CURRENT_LOCAL_DATE_TIME_STRING)) { - return LocalDateTime.now(); - } else { - try { - return LocalDateTime.parse(valueAsString); - } catch (final DateTimeParseException e) { - LOG.error("Unable to parse {} to LocalDateTime.", valueAsString, e); - throw new IllegalArgumentException("Unable to obtain instance of LocalDateTime from " + valueAsString, e); - } - } - } -} diff --git a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/S3ScanPartitionCreationSupplier.java b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/S3ScanPartitionCreationSupplier.java index e1ebea9fa0..5ec4848354 100644 --- a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/S3ScanPartitionCreationSupplier.java +++ b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/S3ScanPartitionCreationSupplier.java @@ -36,6 +36,7 @@ public class S3ScanPartitionCreationSupplier implements Function globalStateMap) { globalStateMap.put(SCAN_COUNT, 0); + globalStateMap.put(SINGLE_SCAN_COMPLETE, false); } private boolean isLastModifiedTimeAfterMostRecentScanForBucket(final String bucketName, @@ -178,14 +186,22 @@ private Instant getMostRecentLastModifiedTimestamp(final ListObjectsV2Response l } private boolean shouldScanBeSkipped(final Map globalStateMap) { + if (Objects.isNull(schedulingOptions) && hasAlreadyBeenScanned(globalStateMap)) { - LOG.info("Skipping scan because the buckets have already been scanned once"); + + if (!(Boolean) globalStateMap.get(SINGLE_SCAN_COMPLETE)) { + LOG.info("Single S3 scan has already been completed"); + globalStateMap.put(SINGLE_SCAN_COMPLETE, true); + } + return true; } if (Objects.nonNull(schedulingOptions) && (hasReachedMaxScanCount(globalStateMap) || !hasReachedScheduledScanTime(globalStateMap))) { + + if (hasReachedMaxScanCount(globalStateMap)) { LOG.info("Skipping scan as the max scan count {} has been reached", schedulingOptions.getCount()); } else { diff --git a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/S3ScanService.java b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/S3ScanService.java index 19d8a9d679..9b2ba8999b 100644 --- a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/S3ScanService.java +++ b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/S3ScanService.java @@ -65,6 +65,10 @@ public void start() { scanObjectWorkerThread.start(); } + public void stop() { + scanObjectWorkerThread.interrupt(); + } + /** * This Method Used to fetch the scan options details from {@link S3SourceConfig} amd build the * all the s3 scan buckets information in list. diff --git a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/S3Source.java b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/S3Source.java index 7a0717d707..2e1b861032 100644 --- a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/S3Source.java +++ b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/S3Source.java @@ -131,8 +131,13 @@ public void start(Buffer> buffer) { @Override public void stop() { - sqsService.stop(); - if (Objects.nonNull(sourceCoordinator)) { + + if (Objects.nonNull(sqsService)) { + sqsService.stop(); + } + + if (Objects.nonNull(s3ScanService) && Objects.nonNull(sourceCoordinator)) { + s3ScanService.stop(); sourceCoordinator.giveUpPartitions(); } } diff --git a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/ScanObjectWorker.java b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/ScanObjectWorker.java index 39a251cddc..8e962a16d0 100644 --- a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/ScanObjectWorker.java +++ b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/ScanObjectWorker.java @@ -65,7 +65,7 @@ public class ScanObjectWorker implements Runnable{ private final AcknowledgementSetManager acknowledgementSetManager; // Should there be a duration or time that is configured in the source to stop processing? Otherwise will only stop when data prepper is stopped - private final boolean shouldStopProcessing = false; + private boolean shouldStopProcessing = false; private final boolean deleteS3ObjectsOnRead; private final S3ObjectDeleteWorker s3ObjectDeleteWorker; private final PluginMetrics pluginMetrics; @@ -109,6 +109,7 @@ public void run() { Thread.sleep(RETRY_BACKOFF_ON_EXCEPTION_MILLIS); } catch (InterruptedException ex) { LOG.error("S3 Scan worker thread interrupted while backing off.", ex); + return; } } @@ -129,7 +130,7 @@ private void startProcessingObject(final int waitTimeMillis) { try { Thread.sleep(waitTimeMillis); } catch (InterruptedException e) { - e.printStackTrace(); + shouldStopProcessing = true; } return; } diff --git a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/ScanOptions.java b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/ScanOptions.java index b1a33db719..eeba77d066 100644 --- a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/ScanOptions.java +++ b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/ScanOptions.java @@ -5,6 +5,8 @@ package org.opensearch.dataprepper.plugins.source; import org.opensearch.dataprepper.plugins.source.configuration.S3ScanBucketOption; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.time.Duration; import java.time.LocalDateTime; @@ -15,18 +17,18 @@ * Class consists the scan related properties. */ public class ScanOptions { + private static final Logger LOG = LoggerFactory.getLogger(ScanOptions.class); + private final LocalDateTime startDateTime; - private LocalDateTime startDateTime; + private final Duration range; - private Duration range; + private final S3ScanBucketOption bucketOption; - private S3ScanBucketOption bucketOption; + private final LocalDateTime endDateTime; - private LocalDateTime endDateTime; + private final LocalDateTime useStartDateTime; - private LocalDateTime useStartDateTime; - - private LocalDateTime useEndDateTime; + private final LocalDateTime useEndDateTime; private ScanOptions(Builder builder){ this.startDateTime = builder.startDateTime; @@ -94,53 +96,44 @@ public Builder setBucketOption(S3ScanBucketOption bucketOption) { } public ScanOptions build() { - LocalDateTime bucketStartDateTime = Objects.isNull(bucketOption.getStartTime()) ? startDateTime : bucketOption.getStartTime(); - LocalDateTime bucketEndDateTime = Objects.isNull(bucketOption.getEndTime()) ? endDateTime : bucketOption.getEndTime(); - Duration bucketRange = Objects.isNull(bucketOption.getRange()) ? range : bucketOption.getRange(); + long globalLevelNonNullCount = Stream.of(startDateTime, endDateTime, range) + .filter(Objects::nonNull) + .count(); - long nonNullCount = Stream.of(bucketStartDateTime, bucketEndDateTime, bucketRange) + long originalBucketLevelNonNullCount = Stream.of( + bucketOption.getStartTime(), bucketOption.getEndTime(), bucketOption.getRange()) .filter(Objects::nonNull) .count(); - if (nonNullCount == 0 || nonNullCount == 2) { - setDateTimeToUse(bucketStartDateTime, bucketEndDateTime, bucketRange); - } else if (nonNullCount == 3) { - long originalBucketLevelNonNullCount = Stream.of( - bucketOption.getStartTime(), bucketOption.getEndTime(), bucketOption.getRange()) - .filter(Objects::nonNull) - .count(); - - if (originalBucketLevelNonNullCount == 2) { - setDateTimeToUse(bucketOption.getStartTime(), bucketOption.getEndTime(), bucketOption.getRange()); - } else { - scanRangeDateValidationError(); - } - } else { - scanRangeDateValidationError(); + if (originalBucketLevelNonNullCount != 0) { + setDateTimeToUse(bucketOption.getStartTime(), bucketOption.getEndTime(), bucketOption.getRange()); + } else if (globalLevelNonNullCount != 0) { + setDateTimeToUse(startDateTime, endDateTime, range); } + return new ScanOptions(this); } private void setDateTimeToUse(LocalDateTime bucketStartDateTime, LocalDateTime bucketEndDateTime, Duration bucketRange) { - if (Objects.nonNull(bucketStartDateTime) && Objects.nonNull(bucketEndDateTime)) { this.useStartDateTime = bucketStartDateTime; this.useEndDateTime = bucketEndDateTime; - } else if (Objects.nonNull(bucketStartDateTime) && Objects.nonNull(bucketRange)) { + LOG.info("Scanning objects modified from {} to {} from bucket: {}", useStartDateTime, useEndDateTime, bucketOption.getName()); + } else if (Objects.nonNull(bucketStartDateTime)) { this.useStartDateTime = bucketStartDateTime; - this.useEndDateTime = bucketStartDateTime.plus(bucketRange); - } else if (Objects.nonNull(bucketEndDateTime) && Objects.nonNull(bucketRange)) { - this.useStartDateTime = bucketEndDateTime.minus(bucketRange); + LOG.info("Scanning objects modified after {} from bucket: {}", useStartDateTime, bucketOption.getName()); + } else if (Objects.nonNull(bucketEndDateTime)) { this.useEndDateTime = bucketEndDateTime; + LOG.info("Scanning objects modified before {} from bucket: {}", useEndDateTime, bucketOption.getName()); + } else if (Objects.nonNull(bucketRange)) { + this.useEndDateTime = LocalDateTime.now(); + this.useStartDateTime = this.useEndDateTime.minus(bucketRange); + LOG.info("Scanning objects modified from {} to {} from bucket: {}", useStartDateTime, useEndDateTime, bucketOption.getName()); + } else { + LOG.info("Scanning all objects from bucket: {}", bucketOption.getName()); } } - private void scanRangeDateValidationError(){ - String message = "To set a time range for the bucket with name " + bucketOption.getName() + - ", specify any two configurations from start_time, end_time and range"; - throw new IllegalArgumentException(message); - } - @Override public String toString() { return "startDateTime=" + startDateTime + diff --git a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/configuration/S3ScanBucketOption.java b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/configuration/S3ScanBucketOption.java index b54dab4075..58492d0cce 100644 --- a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/configuration/S3ScanBucketOption.java +++ b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/configuration/S3ScanBucketOption.java @@ -6,10 +6,11 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.databind.annotation.JsonDeserialize; +import com.fasterxml.jackson.datatype.jsr310.deser.LocalDateTimeDeserializer; +import jakarta.validation.constraints.AssertFalse; import jakarta.validation.constraints.AssertTrue; import jakarta.validation.constraints.NotEmpty; import jakarta.validation.constraints.Size; -import org.opensearch.dataprepper.plugins.source.CustomLocalDateTimeDeserializer; import java.time.Duration; import java.time.LocalDateTime; @@ -27,11 +28,11 @@ public class S3ScanBucketOption { @Size(min = 3, max = 500, message = "bucket length should be at least 3 characters") private String name; - @JsonDeserialize(using = CustomLocalDateTimeDeserializer.class) + @JsonDeserialize(using = LocalDateTimeDeserializer.class) @JsonProperty("start_time") private LocalDateTime startTime; - @JsonDeserialize(using = CustomLocalDateTimeDeserializer.class) + @JsonDeserialize(using = LocalDateTimeDeserializer.class) @JsonProperty("end_time") private LocalDateTime endTime; @@ -46,6 +47,11 @@ public boolean hasValidTimeOptions() { return Stream.of(startTime, endTime, range).filter(Objects::nonNull).count() < 3; } + @AssertFalse(message = "bucket start_time or end_time cannot be used along with range") + public boolean hasValidTimeAndRangeOptions() { + return (startTime != null || endTime != null) && range != null; + } + public String getName() { if (name.startsWith(S3_PREFIX)) { return name.substring(S3_PREFIX.length()); diff --git a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/configuration/S3ScanBucketOptions.java b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/configuration/S3ScanBucketOptions.java index 86292feffe..1bf077813d 100644 --- a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/configuration/S3ScanBucketOptions.java +++ b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/configuration/S3ScanBucketOptions.java @@ -5,11 +5,14 @@ package org.opensearch.dataprepper.plugins.source.configuration; import com.fasterxml.jackson.annotation.JsonProperty; +import jakarta.validation.Valid; + /** * Class consists the bucket properties. */ public class S3ScanBucketOptions { @JsonProperty("bucket") + @Valid private S3ScanBucketOption scanBucketOption; public S3ScanBucketOption getS3ScanBucketOption() { diff --git a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/configuration/S3ScanScanOptions.java b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/configuration/S3ScanScanOptions.java index 3db6abb179..1c0df901d0 100644 --- a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/configuration/S3ScanScanOptions.java +++ b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/configuration/S3ScanScanOptions.java @@ -6,9 +6,10 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.databind.annotation.JsonDeserialize; +import com.fasterxml.jackson.datatype.jsr310.deser.LocalDateTimeDeserializer; import jakarta.validation.Valid; +import jakarta.validation.constraints.AssertFalse; import jakarta.validation.constraints.AssertTrue; -import org.opensearch.dataprepper.plugins.source.CustomLocalDateTimeDeserializer; import java.time.Duration; import java.time.LocalDateTime; @@ -24,11 +25,11 @@ public class S3ScanScanOptions { @JsonProperty("range") private Duration range; - @JsonDeserialize(using = CustomLocalDateTimeDeserializer.class) + @JsonDeserialize(using = LocalDateTimeDeserializer.class) @JsonProperty("start_time") private LocalDateTime startTime; - @JsonDeserialize(using = CustomLocalDateTimeDeserializer.class) + @JsonDeserialize(using = LocalDateTimeDeserializer.class) @JsonProperty("end_time") private LocalDateTime endTime; @@ -45,6 +46,11 @@ public boolean hasValidTimeOptions() { return Stream.of(startTime, endTime, range).filter(Objects::nonNull).count() < 3; } + @AssertFalse(message = "start_time or end_time cannot be used along with range") + public boolean hasValidTimeAndRangeOptions() { + return (startTime != null || endTime != null) && range != null; + } + @AssertTrue(message = "start_time, end_time, and range are not valid options when using scheduling with s3 scan") public boolean hasValidTimeOptionsWithScheduling() { return !Objects.nonNull(schedulingOptions) || Stream.of(startTime, endTime, range).noneMatch(Objects::nonNull); diff --git a/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/CustomLocalDateTimeDeserializerTest.java b/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/CustomLocalDateTimeDeserializerTest.java deleted file mode 100644 index e814da31f1..0000000000 --- a/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/CustomLocalDateTimeDeserializerTest.java +++ /dev/null @@ -1,64 +0,0 @@ -/* - * Copyright OpenSearch Contributors - * * SPDX-License-Identifier: Apache-2.0 - * - */ - -package org.opensearch.dataprepper.plugins.source; - -import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.databind.module.SimpleModule; -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 java.time.Duration; -import java.time.LocalDateTime; -import java.time.temporal.ChronoUnit; - -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.greaterThan; -import static org.hamcrest.Matchers.lessThan; -import static org.junit.Assert.assertThrows; -import static org.opensearch.dataprepper.plugins.source.CustomLocalDateTimeDeserializer.CURRENT_LOCAL_DATE_TIME_STRING; - -class CustomLocalDateTimeDeserializerTest { - private ObjectMapper objectMapper; - - @BeforeEach - void setUp() { - objectMapper = new ObjectMapper(); - - final SimpleModule simpleModule = new SimpleModule(); - simpleModule.addDeserializer(LocalDateTime.class, new CustomLocalDateTimeDeserializer()); - objectMapper.registerModule(simpleModule); - } - - @ParameterizedTest - @ValueSource(strings = {"2023-01-2118:00:00", "2023-01-21T8:00:00"}) - void deserialize_with_invalid_values_throws(final String invalidDateTimeString) { - assertThrows(IllegalArgumentException.class, () -> objectMapper.convertValue(invalidDateTimeString, LocalDateTime.class)); - } - - @Test - void deserialize_with_predefined_custom_value_returns_current_local_datetime() { - final LocalDateTime expectedDateTime = objectMapper.convertValue(CURRENT_LOCAL_DATE_TIME_STRING, LocalDateTime.class); - assertThat(expectedDateTime, lessThan(LocalDateTime.now())); - assertThat(expectedDateTime, greaterThan(LocalDateTime.now().minus(Duration.of(5, ChronoUnit.SECONDS)))); - } - - @Test - void deserialize_with_iso_local_date_time_string_returns_correct_local_datetime() { - final String testLocalDateTimeString = "2023-01-21T18:30:45"; - final LocalDateTime expectedDateTime = objectMapper.convertValue(testLocalDateTimeString, LocalDateTime.class); - assertThat(expectedDateTime, equalTo(LocalDateTime.of(2023, 1, 21, 18, 30, 45))); - assertThat(expectedDateTime.getYear(), equalTo(2023)); - assertThat(expectedDateTime.getMonthValue(), equalTo(1)); - assertThat(expectedDateTime.getDayOfMonth(), equalTo(21)); - assertThat(expectedDateTime.getHour(), equalTo(18)); - assertThat(expectedDateTime.getMinute(), equalTo(30)); - assertThat(expectedDateTime.getSecond(), equalTo(45)); - } -} \ No newline at end of file diff --git a/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/S3ScanServiceTest.java b/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/S3ScanServiceTest.java index 09807e24b8..de40f6d79d 100644 --- a/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/S3ScanServiceTest.java +++ b/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/S3ScanServiceTest.java @@ -24,6 +24,8 @@ import static org.hamcrest.CoreMatchers.sameInstance; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.greaterThanOrEqualTo; +import static org.hamcrest.Matchers.lessThanOrEqualTo; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; @@ -57,12 +59,10 @@ void scan_service_test_and_verify_thread_invoking() { void scan_service_with_valid_s3_scan_configuration_test_and_verify() { final String bucketName="my-bucket-5"; final LocalDateTime startDateTime = LocalDateTime.parse("2023-03-07T10:00:00"); - final Duration range = Duration.parse("P2DT1H"); final List includeKeyPathList = List.of("file1.csv","file2.csv"); final S3SourceConfig s3SourceConfig = mock(S3SourceConfig.class); final S3ScanScanOptions s3ScanScanOptions = mock(S3ScanScanOptions.class); when(s3ScanScanOptions.getStartTime()).thenReturn(startDateTime); - when(s3ScanScanOptions.getRange()).thenReturn(range); S3ScanBucketOptions bucket = mock(S3ScanBucketOptions.class); final S3ScanBucketOption s3ScanBucketOption = mock(S3ScanBucketOption.class); when(s3ScanBucketOption.getName()).thenReturn(bucketName); @@ -78,13 +78,12 @@ void scan_service_with_valid_s3_scan_configuration_test_and_verify() { assertThat(scanOptionsBuilder.get(0).getBucketOption().getS3ScanFilter().getS3scanIncludePrefixOptions(),sameInstance(includeKeyPathList)); assertThat(scanOptionsBuilder.get(0).getBucketOption().getName(),sameInstance(bucketName)); assertThat(scanOptionsBuilder.get(0).getUseStartDateTime(),equalTo(startDateTime)); - assertThat(scanOptionsBuilder.get(0).getUseEndDateTime(),equalTo(startDateTime.plus(range))); + assertThat(scanOptionsBuilder.get(0).getUseEndDateTime(),equalTo(null)); } @Test void scan_service_with_valid_bucket_time_range_configuration_test_and_verify() { final String bucketName="my-bucket-5"; - final LocalDateTime startDateTime = LocalDateTime.parse("2023-03-07T10:00:00"); final Duration range = Duration.parse("P2DT1H"); final List includeKeyPathList = List.of("file1.csv","file2.csv"); final S3SourceConfig s3SourceConfig = mock(S3SourceConfig.class); @@ -94,7 +93,6 @@ void scan_service_with_valid_bucket_time_range_configuration_test_and_verify() { when(s3ScanBucketOption.getName()).thenReturn(bucketName); S3ScanKeyPathOption s3ScanKeyPathOption = mock(S3ScanKeyPathOption.class); when(s3ScanKeyPathOption.getS3scanIncludePrefixOptions()).thenReturn(includeKeyPathList); - when(s3ScanBucketOption.getStartTime()).thenReturn(startDateTime); when(s3ScanBucketOption.getRange()).thenReturn(range); when(s3ScanBucketOption.getS3ScanFilter()).thenReturn(s3ScanKeyPathOption); when(bucket.getS3ScanBucketOption()).thenReturn(s3ScanBucketOption); @@ -104,8 +102,10 @@ void scan_service_with_valid_bucket_time_range_configuration_test_and_verify() { final List scanOptionsBuilder = service.getScanOptions(); assertThat(scanOptionsBuilder.get(0).getBucketOption().getS3ScanFilter().getS3scanIncludePrefixOptions(),sameInstance(includeKeyPathList)); assertThat(scanOptionsBuilder.get(0).getBucketOption().getName(),sameInstance(bucketName)); - assertThat(scanOptionsBuilder.get(0).getUseStartDateTime(),equalTo(startDateTime)); - assertThat(scanOptionsBuilder.get(0).getUseEndDateTime(),equalTo(startDateTime.plus(range))); + assertThat(scanOptionsBuilder.get(0).getUseStartDateTime(), lessThanOrEqualTo(LocalDateTime.now().minus(range))); + assertThat(scanOptionsBuilder.get(0).getUseStartDateTime(), greaterThanOrEqualTo(LocalDateTime.now().minus(range).minus(Duration.parse("PT5S")))); + assertThat(scanOptionsBuilder.get(0).getUseEndDateTime(), lessThanOrEqualTo(LocalDateTime.now())); + assertThat(scanOptionsBuilder.get(0).getUseEndDateTime(), greaterThanOrEqualTo(LocalDateTime.now().minus(Duration.parse("PT5S")))); } @Test diff --git a/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/ScanOptionsTest.java b/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/ScanOptionsTest.java index 804d6e1e52..6ac0b2b3cf 100644 --- a/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/ScanOptionsTest.java +++ b/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/ScanOptionsTest.java @@ -17,46 +17,34 @@ import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.instanceOf; import static org.hamcrest.MatcherAssert.assertThat; -import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.hamcrest.Matchers.greaterThanOrEqualTo; +import static org.hamcrest.Matchers.lessThanOrEqualTo; import static org.opensearch.dataprepper.test.helper.ReflectivelySetField.setField; class ScanOptionsTest { @ParameterizedTest @MethodSource("validGlobalTimeRangeOptions") - public void s3scan_options_with_valid_global_time_range_build_success( + void s3scan_options_with_valid_global_time_range_build_success( LocalDateTime startDateTime, LocalDateTime endDateTime, Duration range, - LocalDateTime useStartDateTime, LocalDateTime useEndDateTime) { + LocalDateTime useStartDateTime, LocalDateTime useEndDateTime) throws NoSuchFieldException, IllegalAccessException { + S3ScanBucketOption bucketOption = new S3ScanBucketOption(); + setField(S3ScanBucketOption.class, bucketOption, "name", "bucket_name"); final ScanOptions scanOptions = ScanOptions.builder() .setStartDateTime(startDateTime) .setEndDateTime(endDateTime) .setRange(range) - .setBucketOption(new S3ScanBucketOption()) + .setBucketOption(bucketOption) .build(); - assertThat(scanOptions.getUseStartDateTime(), equalTo(useStartDateTime)); - assertThat(scanOptions.getUseEndDateTime(), equalTo(useEndDateTime)); assertThat(scanOptions.getBucketOption(), instanceOf(S3ScanBucketOption.class)); - } - - @ParameterizedTest - @MethodSource("invalidTimeRangeOptions") - public void s3scan_options_with_invalid_global_time_range_throws_exception_when_build( - LocalDateTime startDateTime, LocalDateTime endDateTime, Duration range) throws NoSuchFieldException, IllegalAccessException { - S3ScanBucketOption bucketOption = new S3ScanBucketOption(); - setField(S3ScanBucketOption.class, bucketOption, "name", "bucket_name"); - - assertThrows(IllegalArgumentException.class, () -> ScanOptions.builder() - .setStartDateTime(startDateTime) - .setEndDateTime(endDateTime) - .setRange(range) - .setBucketOption(bucketOption) - .build()); + assertThat(scanOptions.getBucketOption().getName(), equalTo("bucket_name")); + validateStartAndEndTime(useStartDateTime, useEndDateTime, scanOptions); } @ParameterizedTest @MethodSource("validBucketTimeRangeOptions") - public void s3scan_options_with_valid_bucket_time_range_build_success( + void s3scan_options_with_valid_bucket_time_range_build_success( LocalDateTime bucketStartDateTime, LocalDateTime bucketEndDateTime, Duration bucketRange, LocalDateTime useStartDateTime, LocalDateTime useEndDateTime) throws NoSuchFieldException, IllegalAccessException { S3ScanBucketOption bucketOption = new S3ScanBucketOption(); @@ -68,30 +56,14 @@ public void s3scan_options_with_valid_bucket_time_range_build_success( .setBucketOption(bucketOption) .build(); - assertThat(scanOptions.getUseStartDateTime(), equalTo(useStartDateTime)); - assertThat(scanOptions.getUseEndDateTime(), equalTo(useEndDateTime)); assertThat(scanOptions.getBucketOption(), instanceOf(S3ScanBucketOption.class)); assertThat(scanOptions.getBucketOption().getName(), equalTo("bucket_name")); - } - - @ParameterizedTest - @MethodSource("invalidTimeRangeOptions") - public void s3scan_options_with_invalid_bucket_time_range_throws_exception_when_build( - LocalDateTime bucketStartDateTime, LocalDateTime bucketEndDateTime, Duration bucketRange - ) throws NoSuchFieldException, IllegalAccessException { - S3ScanBucketOption bucketOption = new S3ScanBucketOption(); - setField(S3ScanBucketOption.class, bucketOption, "name", "bucket_name"); - setField(S3ScanBucketOption.class, bucketOption, "startTime", bucketStartDateTime); - setField(S3ScanBucketOption.class, bucketOption, "endTime", bucketEndDateTime); - setField(S3ScanBucketOption.class, bucketOption, "range", bucketRange); - assertThrows(IllegalArgumentException.class, () -> ScanOptions.builder() - .setBucketOption(bucketOption) - .build()); + validateStartAndEndTime(useStartDateTime, useEndDateTime, scanOptions); } @ParameterizedTest @MethodSource("validCombinedTimeRangeOptions") - public void s3scan_options_with_valid_combined_time_range_build_success( + void s3scan_options_with_valid_combined_time_range_build_success( LocalDateTime globalStartDateTime, LocalDateTime globeEndDateTime, Duration globalRange, LocalDateTime bucketStartDateTime, LocalDateTime bucketEndDateTime, Duration bucketRange, LocalDateTime useStartDateTime, LocalDateTime useEndDateTime) throws NoSuchFieldException, IllegalAccessException { @@ -107,63 +79,42 @@ public void s3scan_options_with_valid_combined_time_range_build_success( .setBucketOption(bucketOption) .build(); - assertThat(scanOptions.getUseStartDateTime(), equalTo(useStartDateTime)); - assertThat(scanOptions.getUseEndDateTime(), equalTo(useEndDateTime)); assertThat(scanOptions.getBucketOption(), instanceOf(S3ScanBucketOption.class)); assertThat(scanOptions.getBucketOption().getName(), equalTo("bucket_name")); + validateStartAndEndTime(useStartDateTime, useEndDateTime, scanOptions); } - @ParameterizedTest - @MethodSource("invalidCombinedTimeRangeOptions") - public void s3scan_options_with_invalid_combined_time_range_throws_exception_when_build( - LocalDateTime globalStartDateTime, LocalDateTime globeEndDateTime, Duration globalRange, - LocalDateTime bucketStartDateTime, LocalDateTime bucketEndDateTime, Duration bucketRange - ) throws NoSuchFieldException, IllegalAccessException { - S3ScanBucketOption bucketOption = new S3ScanBucketOption(); - setField(S3ScanBucketOption.class, bucketOption, "name", "bucket_name"); - setField(S3ScanBucketOption.class, bucketOption, "startTime", bucketStartDateTime); - setField(S3ScanBucketOption.class, bucketOption, "endTime", bucketEndDateTime); - setField(S3ScanBucketOption.class, bucketOption, "range", bucketRange); - - assertThrows(IllegalArgumentException.class, () -> ScanOptions.builder() - .setStartDateTime(globalStartDateTime) - .setEndDateTime(globeEndDateTime) - .setRange(globalRange) - .setBucketOption(bucketOption) - .build()); + private static void validateStartAndEndTime(final LocalDateTime useStartDateTime, + final LocalDateTime useEndDateTime, + final ScanOptions scanOptions) { + if (useStartDateTime != null) { + assertThat(scanOptions.getUseStartDateTime(), lessThanOrEqualTo(useStartDateTime.plus(Duration.parse("PT5S")))); + assertThat(scanOptions.getUseStartDateTime(), greaterThanOrEqualTo(useStartDateTime)); + } + if (useEndDateTime != null) { + assertThat(scanOptions.getUseEndDateTime(), lessThanOrEqualTo(useEndDateTime.plus(Duration.parse("PT5S")))); + assertThat(scanOptions.getUseEndDateTime(), greaterThanOrEqualTo(useEndDateTime)); + } } - + private static Stream validGlobalTimeRangeOptions() { return Stream.of( - Arguments.of( - LocalDateTime.parse("2023-01-21T18:00:00"), LocalDateTime.parse("2023-01-24T18:00:00"), null, - LocalDateTime.parse("2023-01-21T18:00:00"), LocalDateTime.parse("2023-01-24T18:00:00")), - Arguments.of(LocalDateTime.parse("2023-01-21T18:00:00"), null, Duration.parse("P3D"), - LocalDateTime.parse("2023-01-21T18:00:00"), LocalDateTime.parse("2023-01-24T18:00:00")), - Arguments.of(null, LocalDateTime.parse("2023-01-24T18:00:00"), Duration.parse("P3D"), + Arguments.of(LocalDateTime.parse("2023-01-21T18:00:00"), LocalDateTime.parse("2023-01-24T18:00:00"), null, LocalDateTime.parse("2023-01-21T18:00:00"), LocalDateTime.parse("2023-01-24T18:00:00")), + Arguments.of(null, null, Duration.parse("P90D"), LocalDateTime.now().minus(Duration.parse("P90D")), LocalDateTime.now()), + Arguments.of(LocalDateTime.parse("2023-01-21T18:00:00"), null, null, LocalDateTime.parse("2023-01-21T18:00:00"), null), + Arguments.of(null, LocalDateTime.parse("2023-01-21T18:00:00"), null, null, LocalDateTime.parse("2023-01-21T18:00:00")), Arguments.of(null, null, null, null, null) ); } - private static Stream invalidTimeRangeOptions() { - return Stream.of( - Arguments.of(LocalDateTime.parse("2023-01-21T18:00:00"), LocalDateTime.parse("2023-04-21T18:00:00"), - Duration.parse("P90DT3H4M")), - Arguments.of(LocalDateTime.parse("2023-01-21T18:00:00"), null, null), - Arguments.of(null, LocalDateTime.parse("2023-04-21T18:00:00"), null), - Arguments.of(null, null, Duration.parse("P90DT3H4M")) - ); - } - private static Stream validBucketTimeRangeOptions() { return Stream.of( Arguments.of(LocalDateTime.parse("2023-01-21T18:00:00"), LocalDateTime.parse("2023-01-24T18:00:00"), null, LocalDateTime.parse("2023-01-21T18:00:00"), LocalDateTime.parse("2023-01-24T18:00:00")), - Arguments.of(LocalDateTime.parse("2023-01-21T18:00:00"), null, Duration.ofDays(3L), - LocalDateTime.parse("2023-01-21T18:00:00"), LocalDateTime.parse("2023-01-24T18:00:00")), - Arguments.of(null, LocalDateTime.parse("2023-01-24T18:00:00"), Duration.ofDays(3L), - LocalDateTime.parse("2023-01-21T18:00:00"), LocalDateTime.parse("2023-01-24T18:00:00")), + Arguments.of(null, null, Duration.parse("P90D"), LocalDateTime.now().minus(Duration.parse("P90D")), LocalDateTime.now()), + Arguments.of(LocalDateTime.parse("2023-01-21T18:00:00"), null, null, LocalDateTime.parse("2023-01-21T18:00:00"), null), + Arguments.of(null, LocalDateTime.parse("2023-01-21T18:00:00"), null, null, LocalDateTime.parse("2023-01-21T18:00:00")), Arguments.of(null, null, null, null, null) ); } @@ -171,51 +122,33 @@ private static Stream validBucketTimeRangeOptions() { private static Stream validCombinedTimeRangeOptions() { return Stream.of( Arguments.of( - LocalDateTime.parse("2023-05-21T18:00:00"), null, null, - null, LocalDateTime.parse("2023-05-24T18:00:00"), null, - LocalDateTime.parse("2023-05-21T18:00:00"), LocalDateTime.parse("2023-05-24T18:00:00")), + LocalDateTime.parse("2023-05-21T18:00:00"), LocalDateTime.parse("2023-05-24T18:00:00"), null, + LocalDateTime.parse("2023-08-21T18:00:00"), LocalDateTime.parse("2023-08-24T18:00:00"), null, + LocalDateTime.parse("2023-08-21T18:00:00"), LocalDateTime.parse("2023-08-24T18:00:00")), Arguments.of( - null, LocalDateTime.parse("2023-05-24T18:00:00"), null, - LocalDateTime.parse("2023-05-21T18:00:00"), null, null, - LocalDateTime.parse("2023-05-21T18:00:00"), LocalDateTime.parse("2023-05-24T18:00:00")), + LocalDateTime.parse("2023-05-21T18:00:00"), LocalDateTime.parse("2023-05-24T18:00:00"), null, + LocalDateTime.parse("2023-08-21T18:00:00"), null, null, + LocalDateTime.parse("2023-08-21T18:00:00"), null), Arguments.of( - LocalDateTime.parse("2023-05-21T18:00:00"), null, null, - null, null, Duration.ofDays(3L), - LocalDateTime.parse("2023-05-21T18:00:00"), LocalDateTime.parse("2023-05-24T18:00:00")), + LocalDateTime.parse("2023-05-21T18:00:00"), LocalDateTime.parse("2023-05-24T18:00:00"), null, + null, LocalDateTime.parse("2023-08-24T18:00:00"), null, + null, LocalDateTime.parse("2023-08-24T18:00:00")), Arguments.of( - LocalDateTime.parse("2023-01-21T18:00:00"), LocalDateTime.parse("2023-01-24T18:00:00"), null, LocalDateTime.parse("2023-05-21T18:00:00"), LocalDateTime.parse("2023-05-24T18:00:00"), null, + null, null, null, LocalDateTime.parse("2023-05-21T18:00:00"), LocalDateTime.parse("2023-05-24T18:00:00")), Arguments.of( - LocalDateTime.parse("2023-01-21T18:00:00"), LocalDateTime.parse("2023-01-24T18:00:00"), null, - LocalDateTime.parse("2023-05-21T18:00:00"), null, Duration.ofDays(3L), - LocalDateTime.parse("2023-05-21T18:00:00"), LocalDateTime.parse("2023-05-24T18:00:00")) - ); - } - - private static Stream invalidCombinedTimeRangeOptions() { - return Stream.of( - Arguments.of( - LocalDateTime.parse("2023-05-21T18:00:00"), null, null, - LocalDateTime.parse("2023-05-24T18:00:00"), null, null), - Arguments.of( - null, LocalDateTime.parse("2023-05-24T18:00:00"), null, - null, LocalDateTime.parse("2023-05-21T18:00:00"), null), - Arguments.of( - null, null, Duration.ofDays(3L), - null, null, Duration.ofDays(3L)), - Arguments.of( - LocalDateTime.parse("2023-01-21T18:00:00"), LocalDateTime.parse("2023-01-24T18:00:00"), null, - null, null, Duration.ofDays(3L)), - Arguments.of( - LocalDateTime.parse("2023-01-21T18:00:00"), null, Duration.ofDays(3L), - null, LocalDateTime.parse("2023-05-24T18:00:00"), null), + LocalDateTime.parse("2023-05-21T18:00:00"), LocalDateTime.parse("2023-05-24T18:00:00"), null, + null, null, Duration.parse("P90D"), + LocalDateTime.now().minus(Duration.parse("P90D")), LocalDateTime.now()), Arguments.of( - null, LocalDateTime.parse("2023-01-24T18:00:00"), Duration.ofDays(3L), - LocalDateTime.parse("2023-05-21T18:00:00"), null, null), + null, null, Duration.parse("P30D"), + null, null, Duration.parse("P90D"), + LocalDateTime.now().minus(Duration.parse("P90D")), LocalDateTime.now()), Arguments.of( - LocalDateTime.parse("2023-01-21T18:00:00"), LocalDateTime.parse("2023-01-24T18:00:00"), Duration.ofDays(3L), - LocalDateTime.parse("2023-05-21T18:00:00"), LocalDateTime.parse("2023-05-24T18:00:00"), Duration.ofDays(3L)) + null, null, Duration.parse("P30D"), + null, null, null, + LocalDateTime.now().minus(Duration.parse("P30D")), LocalDateTime.now()) ); } -} \ No newline at end of file +} diff --git a/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/configuration/S3ScanBucketOptionTest.java b/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/configuration/S3ScanBucketOptionTest.java index 629c6726e7..6e4a6df6fc 100644 --- a/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/configuration/S3ScanBucketOptionTest.java +++ b/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/configuration/S3ScanBucketOptionTest.java @@ -18,7 +18,7 @@ public class S3ScanBucketOptionTest { - private ObjectMapper objectMapper = new ObjectMapper(new YAMLFactory().enable(YAMLGenerator.Feature.USE_PLATFORM_LINE_BREAKS)); + private final ObjectMapper objectMapper = new ObjectMapper(new YAMLFactory().enable(YAMLGenerator.Feature.USE_PLATFORM_LINE_BREAKS)); @Test public void s3scan_bucket_options_with_scan_buckets_yaml_configuration_test() throws JsonProcessingException { final String bucketOptionsYaml = " name: test-s3-source-test-output\n" + diff --git a/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/configuration/S3ScanScanOptionsTest.java b/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/configuration/S3ScanScanOptionsTest.java index f0e31de164..013d82aaeb 100644 --- a/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/configuration/S3ScanScanOptionsTest.java +++ b/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/configuration/S3ScanScanOptionsTest.java @@ -19,7 +19,7 @@ public class S3ScanScanOptionsTest { - private ObjectMapper objectMapper = new ObjectMapper(new YAMLFactory().enable(YAMLGenerator.Feature.USE_PLATFORM_LINE_BREAKS)); + private final ObjectMapper objectMapper = new ObjectMapper(new YAMLFactory().enable(YAMLGenerator.Feature.USE_PLATFORM_LINE_BREAKS)); @Test public void s3scan_options_test_with_scan_yaml_configuration_test() throws JsonProcessingException { diff --git a/release/build-resources.gradle b/release/build-resources.gradle index 8070ad86be..67e2ce8c61 100644 --- a/release/build-resources.gradle +++ b/release/build-resources.gradle @@ -9,7 +9,7 @@ ext { linux: ['x64'] ] jdkSources = [ - linux_x64: 'https://github.com/adoptium/temurin17-binaries/releases/download/jdk-17.0.4.1%2B1/OpenJDK17U-jdk_x64_linux_hotspot_17.0.4.1_1.tar.gz', + linux_x64: 'https://github.com/adoptium/temurin17-binaries/releases/download/jdk-17.0.8%2B7/OpenJDK17U-jdk_x64_linux_hotspot_17.0.8_7.tar.gz', linux_arm64: 'https://hg.openjdk.java.net/aarch64-port/jdk8/archive/tip.tar.gz' ] awsResources = [