diff --git a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/JacksonEvent.java b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/JacksonEvent.java index 47de571672..09a0705e0e 100644 --- a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/JacksonEvent.java +++ b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/JacksonEvent.java @@ -587,7 +587,7 @@ public Builder withTimeReceived(final Instant timeReceived) { * @return returns the builder * @since 2.10 */ - protected Builder withEventHandle(final EventHandle eventHandle) { + public Builder withEventHandle(final EventHandle eventHandle) { this.eventHandle = eventHandle; return this; } diff --git a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/processor/Processor.java b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/processor/Processor.java index 551aed3d01..784758fa95 100644 --- a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/processor/Processor.java +++ b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/processor/Processor.java @@ -33,6 +33,15 @@ public interface Processor, OutputRecord extends R */ void prepareForShutdown(); + /** + * @since 2.11 + * Indicates if the processor holds the events or not + * Holding events indicates that the events are not ready to be released. + */ + default boolean holdsEvents() { + return false; + } + /** * @since 1.2 * Returns true if the Processor's internal state is safe to be shutdown. diff --git a/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/processor/ProcessorTest.java b/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/processor/ProcessorTest.java new file mode 100644 index 0000000000..2fec941c4f --- /dev/null +++ b/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/processor/ProcessorTest.java @@ -0,0 +1,23 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.model.processor; + +import org.junit.jupiter.api.Test; +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class ProcessorTest { + + @Test + public void testDefault() { + Processor processor = mock(Processor.class); + when(processor.holdsEvents()).thenCallRealMethod(); + assertThat(processor.holdsEvents(), equalTo(false)); + } +} + diff --git a/data-prepper-core/build.gradle b/data-prepper-core/build.gradle index c939129a1c..6ad15ff472 100644 --- a/data-prepper-core/build.gradle +++ b/data-prepper-core/build.gradle @@ -36,7 +36,7 @@ dependencies { implementation 'io.micrometer:micrometer-registry-cloudwatch2' implementation 'javax.ws.rs:javax.ws.rs-api:2.1.1' implementation 'software.amazon.awssdk:cloudwatch' - implementation platform('org.apache.logging.log4j:log4j-bom:2.23.1') + implementation platform('org.apache.logging.log4j:log4j-bom:2.24.3') implementation 'org.apache.logging.log4j:log4j-core' implementation 'org.apache.logging.log4j:log4j-slf4j2-impl' implementation 'javax.inject:javax.inject:1' diff --git a/data-prepper-core/src/main/java/org/opensearch/dataprepper/core/pipeline/Pipeline.java b/data-prepper-core/src/main/java/org/opensearch/dataprepper/core/pipeline/Pipeline.java index b5d3f812cf..f1a247b717 100644 --- a/data-prepper-core/src/main/java/org/opensearch/dataprepper/core/pipeline/Pipeline.java +++ b/data-prepper-core/src/main/java/org/opensearch/dataprepper/core/pipeline/Pipeline.java @@ -347,7 +347,7 @@ private void shutdownExecutorService(final ExecutorService executorService, fina * @param records records that needs to published to each sink * @return List of Future, each future for each sink */ - List> publishToSinks(final Collection records) { + public List> publishToSinks(final Collection records) { final int sinksSize = sinks.size(); final List> sinkFutures = new ArrayList<>(sinksSize); diff --git a/data-prepper-core/src/main/java/org/opensearch/dataprepper/core/pipeline/ProcessWorker.java b/data-prepper-core/src/main/java/org/opensearch/dataprepper/core/pipeline/ProcessWorker.java index e313430b49..8fb314fd83 100644 --- a/data-prepper-core/src/main/java/org/opensearch/dataprepper/core/pipeline/ProcessWorker.java +++ b/data-prepper-core/src/main/java/org/opensearch/dataprepper/core/pipeline/ProcessWorker.java @@ -137,7 +137,8 @@ private void doRun() { try { records = processor.execute(records); - if (inputEvents != null) { + // acknowledge missing events only if the processor is not holding events + if (!processor.holdsEvents() && inputEvents != null) { processAcknowledgements(inputEvents, records); } } catch (final Exception e) { diff --git a/data-prepper-core/src/test/java/org/opensearch/dataprepper/core/pipeline/ProcessWorkerTest.java b/data-prepper-core/src/test/java/org/opensearch/dataprepper/core/pipeline/ProcessWorkerTest.java index 230bb32780..1eb28f991b 100644 --- a/data-prepper-core/src/test/java/org/opensearch/dataprepper/core/pipeline/ProcessWorkerTest.java +++ b/data-prepper-core/src/test/java/org/opensearch/dataprepper/core/pipeline/ProcessWorkerTest.java @@ -6,6 +6,8 @@ import org.mockito.Mock; import org.mockito.MockedStatic; import org.mockito.junit.jupiter.MockitoExtension; +import static org.mockito.Mockito.lenient; +import static org.mockito.Mockito.atLeast; import org.opensearch.dataprepper.core.pipeline.common.FutureHelper; import org.opensearch.dataprepper.core.pipeline.common.FutureHelperResult; import org.opensearch.dataprepper.model.CheckpointState; @@ -132,6 +134,61 @@ void testProcessWorkerHappyPathWithAcknowledgments() { } } + @Test + void testProcessWorkerWithProcessorsNotHoldingEvents() { + DefaultEventHandle eventHandle = mock(DefaultEventHandle.class); + Event event = mock(Event.class); + Record record = mock(Record.class); + when(eventHandle.release(true)).thenReturn(true); + lenient().when(event.getEventHandle()).thenReturn(eventHandle); + when(record.getData()).thenReturn(event); + final List records = List.of(record); + final CheckpointState checkpointState = mock(CheckpointState.class); + final Map.Entry readResult = Map.entry(records, checkpointState); + when(buffer.read(pipeline.getReadBatchTimeoutInMillis())).thenReturn(readResult); + + final Processor processor1 = mock(Processor.class); + when(processor1.holdsEvents()).thenReturn(false); + when(processor1.execute(records)).thenReturn(List.of()); + when(processor1.isReadyForShutdown()).thenReturn(true); + processors = List.of(processor1); + when(source.areAcknowledgementsEnabled()).thenReturn(true); + + final ProcessWorker processWorker = createObjectUnderTest(); + + processWorker.run(); + + verify(eventHandle, atLeast(1)).release(true); + } + + + @Test + void testProcessWorkerWithProcessorsHoldingEvents() { + EventHandle eventHandle = mock(EventHandle.class); + Event event = mock(Event.class); + Record record = mock(Record.class); + lenient().when(event.getEventHandle()).thenReturn(eventHandle); + when(record.getData()).thenReturn(event); + final List records = List.of(record); + final CheckpointState checkpointState = mock(CheckpointState.class); + final Map.Entry readResult = Map.entry(records, checkpointState); + when(buffer.read(pipeline.getReadBatchTimeoutInMillis())).thenReturn(readResult); + + final Processor processor1 = mock(Processor.class); + when(processor1.holdsEvents()).thenReturn(true); + when(processor1.execute(records)).thenReturn(List.of()); + when(processor1.isReadyForShutdown()).thenReturn(true); + + processors = List.of(processor1); + when(source.areAcknowledgementsEnabled()).thenReturn(true); + + final ProcessWorker processWorker = createObjectUnderTest(); + + processWorker.run(); + + verify(eventHandle, never()).release(true); + } + @Test void testProcessWorkerWithProcessorThrowingExceptionIsCaughtProperly() { diff --git a/data-prepper-plugins/aggregate-processor/build.gradle b/data-prepper-plugins/aggregate-processor/build.gradle index 9a3eb4551a..bc2f398b4b 100644 --- a/data-prepper-plugins/aggregate-processor/build.gradle +++ b/data-prepper-plugins/aggregate-processor/build.gradle @@ -14,6 +14,7 @@ dependencies { implementation project(':data-prepper-expression') implementation project(':data-prepper-plugins:otel-proto-common') implementation project(':data-prepper-plugins:otel-metrics-raw-processor') + testImplementation project(':data-prepper-core') implementation libs.guava.core implementation libs.commons.lang3 implementation libs.opentelemetry.proto diff --git a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateAction.java b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateAction.java index ae798af032..541cd15d3d 100644 --- a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateAction.java +++ b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateAction.java @@ -6,6 +6,7 @@ package org.opensearch.dataprepper.plugins.processor.aggregate; import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.event.EventHandle; import java.util.Collections; @@ -29,6 +30,14 @@ default AggregateActionResponse handleEvent(final Event event, final AggregateAc return AggregateActionResponse.fromEvent(event); } + /** + * indicates if the action holds the events or not + * + */ + default boolean holdsEvents() { + return false; + } + /** * Concludes a group of Events * @@ -38,6 +47,12 @@ default AggregateActionResponse handleEvent(final Event event, final AggregateAc * @since 1.3 */ default AggregateActionOutput concludeGroup(final AggregateActionInput aggregateActionInput) { + if (aggregateActionInput != null) { + EventHandle eventHandle = aggregateActionInput.getEventHandle(); + if (eventHandle != null) { + eventHandle.release(true); + } + } return new AggregateActionOutput(Collections.emptyList()); } diff --git a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateActionInput.java b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateActionInput.java index 0bec0b2350..cd7b47d66e 100644 --- a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateActionInput.java +++ b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateActionInput.java @@ -5,6 +5,8 @@ package org.opensearch.dataprepper.plugins.processor.aggregate; +import org.opensearch.dataprepper.model.event.EventHandle; + import java.util.Map; import java.util.function.Function; import java.time.Duration; @@ -28,6 +30,12 @@ public interface AggregateActionInput { */ Map getIdentificationKeys(); + /** + * @return returns eventHandle held by the instance + * @since 2.11 + */ + EventHandle getEventHandle(); + /** * Sets custom shouldConclude function * diff --git a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateGroup.java b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateGroup.java index 09e0e97223..14adde221f 100644 --- a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateGroup.java +++ b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateGroup.java @@ -5,6 +5,11 @@ package org.opensearch.dataprepper.plugins.processor.aggregate; +import org.opensearch.dataprepper.model.event.AggregateEventHandle; +import org.opensearch.dataprepper.model.event.InternalEventHandle; +import org.opensearch.dataprepper.model.event.EventHandle; +import org.opensearch.dataprepper.model.event.Event; + import java.time.Duration; import java.time.Instant; import java.util.function.Function; @@ -19,6 +24,7 @@ class AggregateGroup implements AggregateActionInput { private final Lock handleEventForGroupLock; private final Map identificationKeys; private Function customShouldConclude; + private EventHandle eventHandle; AggregateGroup(final Map identificationKeys) { this.groupState = new DefaultGroupState(); @@ -26,6 +32,19 @@ class AggregateGroup implements AggregateActionInput { this.groupStart = Instant.now(); this.concludeGroupLock = new ReentrantLock(); this.handleEventForGroupLock = new ReentrantLock(); + this.eventHandle = new AggregateEventHandle(Instant.now()); + } + + @Override + public EventHandle getEventHandle() { + return eventHandle; + } + + public void attachToEventAcknowledgementSet(Event event) { + InternalEventHandle internalEventHandle; + EventHandle handle = event.getEventHandle(); + internalEventHandle = (InternalEventHandle)(handle); + internalEventHandle.addEventHandle(eventHandle); } public GroupState getGroupState() { @@ -63,5 +82,6 @@ boolean shouldConcludeGroup(final Duration groupDuration) { void resetGroup() { groupStart = Instant.now(); groupState.clear(); + this.eventHandle = new AggregateEventHandle(groupStart); } } diff --git a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateGroupManager.java b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateGroupManager.java index 9d271aa40b..dedf1edde0 100644 --- a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateGroupManager.java +++ b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateGroupManager.java @@ -26,6 +26,7 @@ AggregateGroup getAggregateGroup(final IdentificationKeysHasher.IdentificationKe return allGroups.computeIfAbsent(identificationKeysMap, (hash) -> new AggregateGroup(identificationKeysMap.getKeyMap())); } + List> getGroupsToConclude(final boolean forceConclude) { final List> groupsToConclude = new ArrayList<>(); for (final Map.Entry groupEntry : allGroups.entrySet()) { diff --git a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateProcessor.java b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateProcessor.java index 2b19e98516..616c3c5ea8 100644 --- a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateProcessor.java +++ b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateProcessor.java @@ -93,6 +93,12 @@ private AggregateAction loadAggregateAction(final PluginFactory pluginFactory) { return pluginFactory.loadPlugin(AggregateAction.class, actionPluginSetting); } + AggregateGroup getAggregateGroupForEvent(final IdentificationKeysHasher.IdentificationKeysMap identificationKeysMap, final Event event) { + AggregateGroup aggregateGroup = aggregateGroupManager.getAggregateGroup(identificationKeysMap); + aggregateGroup.attachToEventAcknowledgementSet(event); + return aggregateGroup; + } + @Override public Collection> doExecute(Collection> records) { final List> recordsOut = new LinkedList<>(); @@ -124,7 +130,7 @@ public Collection> doExecute(Collection> records) { continue; } final IdentificationKeysHasher.IdentificationKeysMap identificationKeysMap = identificationKeysHasher.createIdentificationKeysMapFromEvent(event); - final AggregateGroup aggregateGroupForEvent = aggregateGroupManager.getAggregateGroup(identificationKeysMap); + final AggregateGroup aggregateGroupForEvent = getAggregateGroupForEvent(identificationKeysMap, event); final AggregateActionResponse handleEventResponse = aggregateActionSynchronizer.handleEventForGroup(event, identificationKeysMap, aggregateGroupForEvent); @@ -149,6 +155,11 @@ public Collection> doExecute(Collection> records) { return recordsOut; } + @Override + public boolean holdsEvents() { + return aggregateAction.holdsEvents(); + } + public static long getTimeNanos(final Instant time) { final long NANO_MULTIPLIER = 1_000 * 1_000 * 1_000; long currentTimeNanos = time.getEpochSecond() * NANO_MULTIPLIER + time.getNano(); diff --git a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/AppendAggregateAction.java b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/AppendAggregateAction.java index ece5212ac4..0d930b94ba 100644 --- a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/AppendAggregateAction.java +++ b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/AppendAggregateAction.java @@ -84,6 +84,7 @@ public AggregateActionOutput concludeGroup(final AggregateActionInput aggregateA final Event event = JacksonEvent.builder() .withEventType(EVENT_TYPE) .withData(aggregateActionInput.getGroupState()) + .withEventHandle(aggregateActionInput.getEventHandle()) .build(); return new AggregateActionOutput(List.of(event)); } 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 8b67ca64cd..16bbf39c31 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 @@ -146,6 +146,7 @@ public AggregateActionOutput concludeGroup(final AggregateActionInput aggregateA event = JacksonEvent.builder() .withEventType(EVENT_TYPE) .withData(groupState) + .withEventHandle(aggregateActionInput.getEventHandle()) .build(); } else { Integer countValue = (Integer)groupState.get(countKey); @@ -168,6 +169,7 @@ public AggregateActionOutput concludeGroup(final AggregateActionInput aggregateA .withValue((double)countValue) .withExemplars(List.of(exemplar)) .withAttributes(attr) + .withEventHandle(aggregateActionInput.getEventHandle()) .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 22cfa7efb7..ac1a59a712 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 @@ -225,6 +225,7 @@ public AggregateActionOutput concludeGroup(final AggregateActionInput aggregateA event = JacksonEvent.builder() .withEventType(EVENT_TYPE) .withData(groupState) + .withEventHandle(aggregateActionInput.getEventHandle()) .build(); } else { List explicitBoundsList = new ArrayList(); @@ -262,6 +263,7 @@ public AggregateActionOutput concludeGroup(final AggregateActionInput aggregateA .withExplicitBoundsList(explicitBoundsList) .withExemplars(exemplarList) .withAttributes(attr) + .withEventHandle(aggregateActionInput.getEventHandle()) .build(false); event = (Event)histogram; } diff --git a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/PercentSamplerAggregateAction.java b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/PercentSamplerAggregateAction.java index b243dd5ef0..9b27a49dee 100644 --- a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/PercentSamplerAggregateAction.java +++ b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/PercentSamplerAggregateAction.java @@ -47,4 +47,5 @@ public AggregateActionResponse handleEvent(final Event event, final AggregateAct } return AggregateActionResponse.nullEventResponse(); } + } diff --git a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/PutAllAggregateAction.java b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/PutAllAggregateAction.java index 78debabb35..54e0e2c72c 100644 --- a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/PutAllAggregateAction.java +++ b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/PutAllAggregateAction.java @@ -41,6 +41,7 @@ public AggregateActionOutput concludeGroup(final AggregateActionInput aggregateA final Event event = JacksonEvent.builder() .withEventType(EVENT_TYPE) .withData(aggregateActionInput.getGroupState()) + .withEventHandle(aggregateActionInput.getEventHandle()) .build(); return new AggregateActionOutput(List.of(event)); diff --git a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/RateLimiterAggregateAction.java b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/RateLimiterAggregateAction.java index 3ea0d0b8af..ce8131b95c 100644 --- a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/RateLimiterAggregateAction.java +++ b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/RateLimiterAggregateAction.java @@ -42,4 +42,5 @@ public AggregateActionResponse handleEvent(final Event event, final AggregateAct } return new AggregateActionResponse(event); } + } diff --git a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/TailSamplerAggregateAction.java b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/TailSamplerAggregateAction.java index 26b245da73..fc347e0105 100644 --- a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/TailSamplerAggregateAction.java +++ b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/TailSamplerAggregateAction.java @@ -8,6 +8,7 @@ import org.opensearch.dataprepper.model.annotations.DataPrepperPlugin; import org.opensearch.dataprepper.model.annotations.DataPrepperPluginConstructor; import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.event.EventHandle; import org.opensearch.dataprepper.plugins.processor.aggregate.AggregateAction; import org.opensearch.dataprepper.plugins.processor.aggregate.AggregateActionInput; import org.opensearch.dataprepper.plugins.processor.aggregate.AggregateActionOutput; @@ -15,8 +16,9 @@ import org.opensearch.dataprepper.plugins.processor.aggregate.GroupState; import org.opensearch.dataprepper.expression.ExpressionEvaluator; -import java.util.List; import java.util.ArrayList; +import java.util.Collections; +import java.util.List; import java.util.Random; import java.time.Duration; import java.time.Instant; @@ -70,14 +72,27 @@ public AggregateActionResponse handleEvent(final Event event, final AggregateAct return AggregateActionResponse.nullEventResponse(); } + @Override + public boolean holdsEvents() { + return true; + } + @Override public AggregateActionOutput concludeGroup(final AggregateActionInput aggregateActionInput) { GroupState groupState = aggregateActionInput.getGroupState(); int randomInt = random.nextInt(100); + aggregateActionInput.getEventHandle().release(true); if (((groupState.containsKey(ERROR_STATUS_KEY) && (Boolean)groupState.get(ERROR_STATUS_KEY) == true)) || (randomInt < percent)) { return new AggregateActionOutput((List)groupState.getOrDefault(EVENTS_KEY, List.of())); } - return new AggregateActionOutput(List.of()); + List events = (List)groupState.getOrDefault(EVENTS_KEY, List.of()); + for (final Event event : events) { + EventHandle eventHandle = event.getEventHandle(); + if (eventHandle != null) { + eventHandle.release(true); + } + } + return new AggregateActionOutput(Collections.emptyList()); } } diff --git a/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateActionTestUtils.java b/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateActionTestUtils.java index 21e49e05be..b46d2bdaab 100644 --- a/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateActionTestUtils.java +++ b/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateActionTestUtils.java @@ -5,6 +5,11 @@ package org.opensearch.dataprepper.plugins.processor.aggregate; +import org.opensearch.dataprepper.model.event.AggregateEventHandle; +import org.opensearch.dataprepper.model.event.EventHandle; + +import java.time.Duration; +import java.time.Instant; import java.util.Map; import java.util.HashMap; import java.time.Duration; @@ -15,10 +20,12 @@ public static class TestAggregateActionInput implements AggregateActionInput { private final GroupState groupState; private final Map identificationKeys; private Function customShouldConclude; + private EventHandle eventHandle; public TestAggregateActionInput(Map identificationKeys) { this.groupState = new AggregateActionTestUtils.TestGroupState(); this.identificationKeys = identificationKeys; + this.eventHandle = new AggregateEventHandle(Instant.now()); } @Override @@ -31,6 +38,11 @@ public GroupState getGroupState() { return groupState; } + @Override + public EventHandle getEventHandle() { + return eventHandle; + } + @Override public Map getIdentificationKeys() { return identificationKeys; diff --git a/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateProcessorIT.java b/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateProcessorIT.java index a7608decec..0a5dbd6117 100644 --- a/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateProcessorIT.java +++ b/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateProcessorIT.java @@ -11,9 +11,14 @@ import org.opensearch.dataprepper.model.configuration.PluginModel; import org.opensearch.dataprepper.model.configuration.PluginSetting; import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.event.EventHandle; +import org.opensearch.dataprepper.model.event.AggregateEventHandle; +import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSet; import org.opensearch.dataprepper.model.event.JacksonEvent; import org.opensearch.dataprepper.model.plugin.PluginFactory; import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.core.acknowledgements.DefaultAcknowledgementSet; +import org.opensearch.dataprepper.core.acknowledgements.DefaultAcknowledgementSetMetrics; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.RepeatedTest; import org.junit.jupiter.api.extension.ExtendWith; @@ -52,6 +57,7 @@ import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutorService; +import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; import java.util.concurrent.ThreadLocalRandom; @@ -105,6 +111,7 @@ public class AggregateProcessorIT { private Collection> eventBatch; private ConcurrentLinkedQueue> aggregatedResult; private Set> uniqueEventMaps; + private Set eventHandles; @Mock private PluginFactory pluginFactory; @@ -114,6 +121,7 @@ public class AggregateProcessorIT { @BeforeEach void setup() { + eventHandles = new HashSet<>(); aggregatedResult = new ConcurrentLinkedQueue<>(); uniqueEventMaps = new HashSet<>(); @@ -186,7 +194,7 @@ void aggregateWithConcludingGroupsOnceReturnsExpectedResult() throws Interrupted final CountDownLatch countDownLatch = new CountDownLatch(NUM_THREADS); objectUnderTest.doExecute(eventBatch); - Thread.sleep(GROUP_DURATION_FOR_ONLY_SINGLE_CONCLUDE * 1000); + Thread.sleep(GROUP_DURATION_FOR_ONLY_SINGLE_CONCLUDE * 2000); for (int i = 0; i < NUM_THREADS; i++) { executorService.execute(() -> { @@ -213,7 +221,7 @@ void aggregateWithConcludingGroupsOnceReturnsExpectedResult() throws Interrupted } } - @RepeatedTest(value = 2) + @RepeatedTest(value = 1) void aggregateWithPutAllActionAndCondition() throws InterruptedException { aggregateAction = new PutAllAggregateAction(); when(pluginFactory.loadPlugin(eq(AggregateAction.class), any(PluginSetting.class))) @@ -222,9 +230,12 @@ void aggregateWithPutAllActionAndCondition() throws InterruptedException { when(aggregateProcessorConfig.getGroupDuration()).thenReturn(Duration.ofSeconds(GROUP_DURATION_FOR_ONLY_SINGLE_CONCLUDE)); when(aggregateProcessorConfig.getWhenCondition()).thenReturn(condition); when(expressionEvaluator.isValidExpressionStatement(condition)).thenReturn(true); + final ScheduledExecutorService scheduledExecutorService = Executors.newScheduledThreadPool(1); + AcknowledgementSet acknowledgementSet = new DefaultAcknowledgementSet(scheduledExecutorService, (result) -> {}, Duration.ofSeconds(10), new DefaultAcknowledgementSetMetrics(pluginMetrics)); int count = 0; for (Record record: eventBatch) { Event event = record.getData(); + acknowledgementSet.add(event.getEventHandle()); boolean value = (count % 2 == 0) ? true : false; when(expressionEvaluator.evaluateConditional(condition, event)).thenReturn(value); if (!value) { @@ -238,8 +249,9 @@ void aggregateWithPutAllActionAndCondition() throws InterruptedException { final CountDownLatch countDownLatch = new CountDownLatch(NUM_THREADS); objectUnderTest.doExecute(eventBatch); - Thread.sleep(GROUP_DURATION_FOR_ONLY_SINGLE_CONCLUDE * 1000); + Thread.sleep(GROUP_DURATION_FOR_ONLY_SINGLE_CONCLUDE * 2000); + final List> allRecordsOut = new ArrayList<>(); for (int i = 0; i < NUM_THREADS; i++) { executorService.execute(() -> { final List> recordsOut = (List>) objectUnderTest.doExecute(eventBatch); @@ -247,6 +259,7 @@ void aggregateWithPutAllActionAndCondition() throws InterruptedException { final Map map = record.getData().toMap(); aggregatedResult.add(map); } + allRecordsOut.addAll(recordsOut); countDownLatch.countDown(); }); } @@ -259,6 +272,11 @@ void aggregateWithPutAllActionAndCondition() throws InterruptedException { for (final Map uniqueEventMap : uniqueEventMaps) { assertThat(aggregatedResult, hasItem(uniqueEventMap)); } + for (Record record: allRecordsOut) { + EventHandle eventHandle = record.getData().getEventHandle(); + assertTrue(eventHandle instanceof AggregateEventHandle); + assertTrue(((AggregateEventHandle)eventHandle).hasAcknowledgementSet()); + } } @ParameterizedTest @@ -276,7 +294,7 @@ void aggregateWithPercentSamplerAction(double testPercent) throws InterruptedExc final CountDownLatch countDownLatch = new CountDownLatch(NUM_THREADS); objectUnderTest.doExecute(eventBatch); - Thread.sleep(GROUP_DURATION_FOR_ONLY_SINGLE_CONCLUDE * 1000); + Thread.sleep(GROUP_DURATION_FOR_ONLY_SINGLE_CONCLUDE * 2000); AtomicInteger allowedEventsCount = new AtomicInteger(0); for (int i = 0; i < NUM_THREADS; i++) { @@ -309,7 +327,7 @@ void aggregateWithRateLimiterAction() throws InterruptedException { final CountDownLatch countDownLatch = new CountDownLatch(NUM_THREADS); objectUnderTest.doExecute(eventBatch); - Thread.sleep(GROUP_DURATION_FOR_ONLY_SINGLE_CONCLUDE * 1000); + Thread.sleep(GROUP_DURATION_FOR_ONLY_SINGLE_CONCLUDE * 2000); for (int i = 0; i < NUM_THREADS; i++) { executorService.execute(() -> { @@ -344,7 +362,7 @@ void aggregateWithRateLimiterActionNoDrops() throws InterruptedException { final CountDownLatch countDownLatch = new CountDownLatch(NUM_THREADS); objectUnderTest.doExecute(eventBatch); - Thread.sleep(GROUP_DURATION_FOR_ONLY_SINGLE_CONCLUDE * 1000); + Thread.sleep(GROUP_DURATION_FOR_ONLY_SINGLE_CONCLUDE * 2000); for (int i = 0; i < NUM_THREADS; i++) { executorService.execute(() -> { @@ -364,8 +382,8 @@ void aggregateWithRateLimiterActionNoDrops() throws InterruptedException { assertThat(aggregatedResult.size(), equalTo(NUM_THREADS * NUM_EVENTS_PER_BATCH)); } - @RepeatedTest(value = 2) - void aggregateWithCountAggregateAction() throws InterruptedException, NoSuchFieldException, IllegalAccessException { + @RepeatedTest(value = 1) + void aggregateWithCountAggregateActionKK() throws InterruptedException, NoSuchFieldException, IllegalAccessException { CountAggregateActionConfig countAggregateActionConfig = new CountAggregateActionConfig(); setField(CountAggregateActionConfig.class, countAggregateActionConfig, "outputFormat", OutputFormat.RAW); aggregateAction = new CountAggregateAction(countAggregateActionConfig); @@ -373,6 +391,12 @@ void aggregateWithCountAggregateAction() throws InterruptedException, NoSuchFiel .thenReturn(aggregateAction); when(aggregateProcessorConfig.getGroupDuration()).thenReturn(Duration.ofSeconds(GROUP_DURATION_FOR_ONLY_SINGLE_CONCLUDE)); eventBatch = getBatchOfEvents(true); + final ScheduledExecutorService scheduledExecutorService = Executors.newScheduledThreadPool(1); + AcknowledgementSet acknowledgementSet = new DefaultAcknowledgementSet(scheduledExecutorService, (result) -> {}, Duration.ofSeconds(10), new DefaultAcknowledgementSetMetrics(pluginMetrics)); + for (Record record: eventBatch) { + Event event = record.getData(); + acknowledgementSet.add(event.getEventHandle()); + } final AggregateProcessor objectUnderTest = createObjectUnderTest(); @@ -385,7 +409,7 @@ void aggregateWithCountAggregateAction() throws InterruptedException, NoSuchFiel countDownLatch.countDown(); }); } - Thread.sleep(GROUP_DURATION_FOR_ONLY_SINGLE_CONCLUDE * 1000); + Thread.sleep(GROUP_DURATION_FOR_ONLY_SINGLE_CONCLUDE * 2000); boolean allThreadsFinished = countDownLatch.await(5L, TimeUnit.SECONDS); assertThat(allThreadsFinished, equalTo(true)); @@ -399,10 +423,13 @@ void aggregateWithCountAggregateAction() throws InterruptedException, NoSuchFiel final Record record = (Record)results.toArray()[0]; expectedEventMap.forEach((k, v) -> assertThat(record.getData().toMap(), hasEntry(k,v))); assertThat(record.getData().toMap(), hasKey(DEFAULT_START_TIME_KEY)); + EventHandle eventHandle = record.getData().getEventHandle(); + assertTrue(eventHandle instanceof AggregateEventHandle); + assertTrue(((AggregateEventHandle)eventHandle).hasAcknowledgementSet()); } @RepeatedTest(value = 2) - void aggregateWithCountAggregateActionWithCondition() throws InterruptedException, NoSuchFieldException, IllegalAccessException { + void aggregateWithCountAggregateActionWithConditionPP() throws InterruptedException, NoSuchFieldException, IllegalAccessException { CountAggregateActionConfig countAggregateActionConfig = new CountAggregateActionConfig(); setField(CountAggregateActionConfig.class, countAggregateActionConfig, "outputFormat", OutputFormat.RAW); aggregateAction = new CountAggregateAction(countAggregateActionConfig); @@ -414,16 +441,19 @@ void aggregateWithCountAggregateActionWithCondition() throws InterruptedExceptio when(expressionEvaluator.isValidExpressionStatement(condition)).thenReturn(true); int count = 0; eventBatch = getBatchOfEvents(true); + + final AggregateProcessor objectUnderTest = createObjectUnderTest(); + + final ExecutorService executorService = Executors.newFixedThreadPool(NUM_THREADS); + final ScheduledExecutorService scheduledExecutorService = Executors.newScheduledThreadPool(1); + AcknowledgementSet acknowledgementSet = new DefaultAcknowledgementSet(scheduledExecutorService, (result) -> {}, Duration.ofSeconds(10), new DefaultAcknowledgementSetMetrics(pluginMetrics)); for (Record record: eventBatch) { Event event = record.getData(); + acknowledgementSet.add(event.getEventHandle()); boolean value = (count % 2 == 0) ? true : false; when(expressionEvaluator.evaluateConditional(condition, event)).thenReturn(value); count++; } - - final AggregateProcessor objectUnderTest = createObjectUnderTest(); - - final ExecutorService executorService = Executors.newFixedThreadPool(NUM_THREADS); final CountDownLatch countDownLatch = new CountDownLatch(NUM_THREADS); for (int i = 0; i < NUM_THREADS; i++) { @@ -432,7 +462,7 @@ void aggregateWithCountAggregateActionWithCondition() throws InterruptedExceptio countDownLatch.countDown(); }); } - Thread.sleep(GROUP_DURATION_FOR_ONLY_SINGLE_CONCLUDE * 1000); + Thread.sleep(GROUP_DURATION_FOR_ONLY_SINGLE_CONCLUDE * 2000); boolean allThreadsFinished = countDownLatch.await(5L, TimeUnit.SECONDS); assertThat(allThreadsFinished, equalTo(true)); @@ -446,6 +476,9 @@ void aggregateWithCountAggregateActionWithCondition() throws InterruptedExceptio final Record record = (Record)results.toArray()[0]; expectedEventMap.forEach((k, v) -> assertThat(record.getData().toMap(), hasEntry(k,v))); assertThat(record.getData().toMap(), hasKey(DEFAULT_START_TIME_KEY)); + EventHandle eventHandle = record.getData().getEventHandle(); + assertTrue(eventHandle instanceof AggregateEventHandle); + assertTrue(((AggregateEventHandle)eventHandle).hasAcknowledgementSet()); } @RepeatedTest(value = 2) @@ -460,6 +493,12 @@ void aggregateWithCountAggregateActionWithUnaggregatedEvents() throws Interrupte .thenReturn(aggregateAction); when(aggregateProcessorConfig.getGroupDuration()).thenReturn(Duration.ofSeconds(GROUP_DURATION_FOR_ONLY_SINGLE_CONCLUDE)); eventBatch = getBatchOfEvents(true); + final ScheduledExecutorService scheduledExecutorService = Executors.newScheduledThreadPool(1); + AcknowledgementSet acknowledgementSet = new DefaultAcknowledgementSet(scheduledExecutorService, (result) -> {}, Duration.ofSeconds(10), new DefaultAcknowledgementSetMetrics(pluginMetrics)); + for (Record record: eventBatch) { + Event event = record.getData(); + acknowledgementSet.add(event.getEventHandle()); + } final AggregateProcessor objectUnderTest = createObjectUnderTest(); @@ -474,7 +513,7 @@ void aggregateWithCountAggregateActionWithUnaggregatedEvents() throws Interrupte }); } // wait longer so that the raw events are processed. - Thread.sleep(2*GROUP_DURATION_FOR_ONLY_SINGLE_CONCLUDE * 1000); + Thread.sleep(2*GROUP_DURATION_FOR_ONLY_SINGLE_CONCLUDE * 2000); boolean allThreadsFinished = countDownLatch.await(5L, TimeUnit.SECONDS); assertThat(allThreadsFinished, equalTo(true)); @@ -489,6 +528,9 @@ void aggregateWithCountAggregateActionWithUnaggregatedEvents() throws Interrupte assertTrue(record.getData().getMetadata().hasTags(List.of(tag))); expectedEventMap.forEach((k, v) -> assertThat(record.getData().toMap(), hasEntry(k,v))); assertThat(record.getData().toMap(), hasKey(DEFAULT_START_TIME_KEY)); + EventHandle eventHandle = record.getData().getEventHandle(); + assertTrue(eventHandle instanceof AggregateEventHandle); + assertTrue(((AggregateEventHandle)eventHandle).hasAcknowledgementSet()); } @@ -518,10 +560,13 @@ void aggregateWithHistogramAggregateAction() throws InterruptedException, NoSuch .thenReturn(aggregateAction); when(aggregateProcessorConfig.getGroupDuration()).thenReturn(Duration.ofSeconds(GROUP_DURATION_FOR_ONLY_SINGLE_CONCLUDE)); eventBatch = getBatchOfEvents(true); + final ScheduledExecutorService scheduledExecutorService = Executors.newScheduledThreadPool(1); + AcknowledgementSet acknowledgementSet = new DefaultAcknowledgementSet(scheduledExecutorService, (result) -> {}, Duration.ofSeconds(10), new DefaultAcknowledgementSetMetrics(pluginMetrics)); for (final Record record : eventBatch) { final double value = ThreadLocalRandom.current().nextDouble(TEST_VALUE_RANGE_MIN-TEST_VALUE_RANGE_STEP, TEST_VALUE_RANGE_MAX+TEST_VALUE_RANGE_STEP); Event event = record.getData(); event.put(testKey, value); + acknowledgementSet.add(event.getEventHandle()); } final AggregateProcessor objectUnderTest = createObjectUnderTest(); @@ -556,6 +601,9 @@ void aggregateWithHistogramAggregateAction() throws InterruptedException, NoSuch for (int i = 0; i < testBuckets.size(); i++) { assertThat(testBuckets.get(i).doubleValue(), equalTo(bucketsInResult.get(i))); } + EventHandle eventHandle = record.getData().getEventHandle(); + assertTrue(eventHandle instanceof AggregateEventHandle); + assertTrue(((AggregateEventHandle)eventHandle).hasAcknowledgementSet()); } @ParameterizedTest @@ -581,7 +629,7 @@ void aggregateWithTailSamplerAction(final int testPercent) throws InterruptedExc final int numberOfSpans = 5; eventBatch = getBatchOfEventsForTailSampling(numberOfErrorTraces, numberOfSpans); objectUnderTest.doExecute(eventBatch); - Thread.sleep(GROUP_DURATION_FOR_ONLY_SINGLE_CONCLUDE * 1000); + Thread.sleep(GROUP_DURATION_FOR_ONLY_SINGLE_CONCLUDE * 2000); final CountDownLatch countDownLatch = new CountDownLatch(NUM_THREADS); for (int i = 0; i < NUM_THREADS; i++) { @@ -590,7 +638,7 @@ void aggregateWithTailSamplerAction(final int testPercent) throws InterruptedExc countDownLatch.countDown(); }); } - Thread.sleep(GROUP_DURATION_FOR_ONLY_SINGLE_CONCLUDE * 1000); + Thread.sleep(GROUP_DURATION_FOR_ONLY_SINGLE_CONCLUDE * 2000); boolean allThreadsFinished = countDownLatch.await(5L, TimeUnit.SECONDS); assertThat(allThreadsFinished, equalTo(true)); List errorEventList = eventBatch.stream().map(Record::getData).filter(event -> { diff --git a/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateProcessorITWithAcks.java b/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateProcessorITWithAcks.java new file mode 100644 index 0000000000..1534c65de3 --- /dev/null +++ b/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateProcessorITWithAcks.java @@ -0,0 +1,621 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.processor.aggregate; + +import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSet; +import org.opensearch.dataprepper.model.buffer.Buffer; +import static org.opensearch.dataprepper.test.helper.ReflectivelySetField.setField; +import org.opensearch.dataprepper.expression.ExpressionEvaluator; +import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.configuration.PluginModel; +import org.opensearch.dataprepper.model.configuration.PluginSetting; +import org.opensearch.dataprepper.model.plugin.PluginFactory; +import org.opensearch.dataprepper.model.event.AggregateEventHandle; +import org.opensearch.dataprepper.model.CheckpointState; +import org.opensearch.dataprepper.core.pipeline.common.FutureHelper; +import org.opensearch.dataprepper.core.pipeline.Pipeline; +import org.opensearch.dataprepper.core.pipeline.common.FutureHelperResult; +import org.opensearch.dataprepper.core.acknowledgements.DefaultAcknowledgementSet; +import org.opensearch.dataprepper.core.acknowledgements.DefaultAcknowledgementSetMetrics; +import org.opensearch.dataprepper.core.pipeline.ProcessWorker; +import org.opensearch.dataprepper.plugins.processor.aggregate.actions.AppendAggregateAction; +import org.opensearch.dataprepper.plugins.processor.aggregate.actions.AppendAggregateActionConfig; +import org.opensearch.dataprepper.plugins.processor.aggregate.actions.CountAggregateAction; +import org.opensearch.dataprepper.plugins.processor.aggregate.actions.CountAggregateActionConfig; +import org.opensearch.dataprepper.plugins.processor.aggregate.actions.HistogramAggregateAction; +import org.opensearch.dataprepper.plugins.processor.aggregate.actions.HistogramAggregateActionConfig; +import org.opensearch.dataprepper.plugins.processor.aggregate.actions.RateLimiterMode; +import org.opensearch.dataprepper.plugins.processor.aggregate.actions.RateLimiterAggregateAction; +import org.opensearch.dataprepper.plugins.processor.aggregate.actions.RateLimiterAggregateActionConfig; +import org.opensearch.dataprepper.plugins.processor.aggregate.actions.PercentSamplerAggregateAction; +import org.opensearch.dataprepper.plugins.processor.aggregate.actions.PercentSamplerAggregateActionConfig; +import org.opensearch.dataprepper.plugins.processor.aggregate.actions.TailSamplerAggregateActionConfig; +import org.opensearch.dataprepper.plugins.processor.aggregate.actions.TailSamplerAggregateAction; +import org.opensearch.dataprepper.plugins.processor.aggregate.actions.RemoveDuplicatesAggregateAction; +import org.opensearch.dataprepper.plugins.processor.aggregate.actions.PutAllAggregateAction; +import org.opensearch.dataprepper.plugins.processor.aggregate.actions.OutputFormat; +import org.opensearch.dataprepper.model.processor.Processor; +import org.opensearch.dataprepper.model.event.JacksonEvent; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.model.source.Source; + +import static org.awaitility.Awaitility.await; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.hamcrest.MatcherAssert.assertThat; +import org.mockito.Mock; +import org.mockito.MockedStatic; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.mockStatic; +import static org.mockito.Mockito.when; +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.Matchers.greaterThanOrEqualTo; +import static org.hamcrest.Matchers.lessThan; + + +import org.apache.commons.lang3.RandomStringUtils; +import java.time.Duration; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.concurrent.Future; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.Executors; + + +public class AggregateProcessorITWithAcks { + private static final int testValue = 1; + private static final int GROUP_DURATION_FOR_ONLY_SINGLE_CONCLUDE = 2; + private static final int NUM_UNIQUE_EVENTS_PER_BATCH = 8; + private static final int NUM_EVENTS_PER_BATCH = 5; + private static final Duration TEST_TIMEOUT = Duration.ofSeconds(5); + + @Mock + private Pipeline pipeline; + @Mock + private Buffer buffer; + @Mock + private Source source; + @Mock + private PluginFactory pluginFactory; + @Mock + private ExpressionEvaluator expressionEvaluator; + @Mock + private CheckpointState checkpointState; + @Mock + private PluginModel actionConfiguration; + @Mock + private AggregateProcessorConfig aggregateProcessorConfig; + private int callCount; + private boolean aggregatedResultReceived; + List> records; + private String testKey; + + private PluginMetrics pluginMetrics; + private List processors; + private List> sinkFutures; + AcknowledgementSet acknowledgementSet; + ScheduledExecutorService scheduledExecutorService; + List> aggregatedResults; + + @BeforeEach + void setup() { + testKey = UUID.randomUUID().toString(); + pluginMetrics = PluginMetrics.fromNames(UUID.randomUUID().toString(), UUID.randomUUID().toString()); + scheduledExecutorService = Executors.newScheduledThreadPool(3); + acknowledgementSet = new DefaultAcknowledgementSet(scheduledExecutorService, (result) -> {}, Duration.ofSeconds(10), new DefaultAcknowledgementSetMetrics(pluginMetrics)); + final List identificationKeys = new ArrayList<>(); + identificationKeys.add("firstRandomNumber"); + identificationKeys.add("secondRandomNumber"); + identificationKeys.add("thirdRandomNumber"); + callCount = 0; + aggregatedResultReceived = false; + aggregatedResults = new ArrayList<>(); + + pipeline = mock(Pipeline.class); + source = mock(Source.class); + buffer = mock(Buffer.class); + processors = List.of(); + aggregateProcessorConfig = mock(AggregateProcessorConfig.class); + actionConfiguration = mock(PluginModel.class); + pluginFactory = mock(PluginFactory.class); + expressionEvaluator = mock(ExpressionEvaluator.class); + when(pipeline.isStopRequested()).thenReturn(false).thenReturn(true); + when(source.areAcknowledgementsEnabled()).thenReturn(true); + when(pipeline.getSource()).thenReturn(source); + when(buffer.isEmpty()).thenReturn(true); + when(pipeline.getPeerForwarderDrainTimeout()).thenReturn(Duration.ofMillis(100)); + when(pipeline.getReadBatchTimeoutInMillis()).thenReturn(500); + when(aggregateProcessorConfig.getOutputUnaggregatedEvents()).thenReturn(false); + when(aggregateProcessorConfig.getIdentificationKeys()).thenReturn(identificationKeys); + when(aggregateProcessorConfig.getWhenCondition()).thenReturn(null); + + records = getRecords(testKey, testValue, acknowledgementSet); + acknowledgementSet.complete(); + checkpointState = mock(CheckpointState.class); + final Map.Entry readResult = Map.entry(records, checkpointState); + + when(buffer.read(pipeline.getReadBatchTimeoutInMillis())).thenAnswer(a -> { + if (callCount == 0) { + callCount++; + return Map.entry(records, checkpointState); + } else { + return Map.entry(List.of(), checkpointState); + } + }); + + final Future sinkFuture = mock(Future.class); + sinkFutures = List.of(sinkFuture); + doAnswer( a -> { + List> receivedRecords = (List>)a.getArgument(0); + if (receivedRecords.size() > 0) { + aggregatedResults = receivedRecords; + for (Record record: receivedRecords) { + if (record.getData().getEventHandle() instanceof AggregateEventHandle) { + aggregatedResultReceived = true; + } + record.getData().getEventHandle().release(true); + } + } + + return sinkFutures; + }).when(pipeline).publishToSinks(any()); + when(aggregateProcessorConfig.getAggregateAction()).thenReturn(actionConfiguration); + when(actionConfiguration.getPluginName()).thenReturn(UUID.randomUUID().toString()); + when(actionConfiguration.getPluginSettings()).thenReturn(Collections.emptyMap()); + } + + @Test + public void testHistogramAggregation() throws Exception { + HistogramAggregateActionConfig histogramAggregateActionConfig = new HistogramAggregateActionConfig(); + setField(HistogramAggregateActionConfig.class, histogramAggregateActionConfig, "outputFormat", OutputFormat.RAW); + setField(HistogramAggregateActionConfig.class, histogramAggregateActionConfig, "key", testKey); + final String testKeyPrefix = RandomStringUtils.randomAlphabetic(4)+"_"; + setField(HistogramAggregateActionConfig.class, histogramAggregateActionConfig, "generatedKeyPrefix", testKeyPrefix); + final String testUnits = RandomStringUtils.randomAlphabetic(3); + setField(HistogramAggregateActionConfig.class, histogramAggregateActionConfig, "units", testUnits); + setField(HistogramAggregateActionConfig.class, histogramAggregateActionConfig, "recordMinMax", true); + List testBuckets = new ArrayList(); + testBuckets.add(10.0); + setField(HistogramAggregateActionConfig.class, histogramAggregateActionConfig, "buckets", testBuckets); + AggregateAction aggregateAction = new HistogramAggregateAction(histogramAggregateActionConfig); + when(aggregateProcessorConfig.getGroupDuration()).thenReturn(Duration.ofMillis(GROUP_DURATION_FOR_ONLY_SINGLE_CONCLUDE)); + when(pluginFactory.loadPlugin(eq(AggregateAction.class), any(PluginSetting.class))) + .thenReturn(aggregateAction); + final Processor processor = new AggregateProcessor(aggregateProcessorConfig, pluginMetrics, pluginFactory, expressionEvaluator); + processors = List.of(processor); + + final FutureHelperResult futureHelperResult = mock(FutureHelperResult.class); + when(futureHelperResult.getFailedReasons()).thenReturn(Collections.emptyList()); + try (final MockedStatic futureHelperMockedStatic = mockStatic(FutureHelper.class)) { + futureHelperMockedStatic.when(() -> FutureHelper.awaitFuturesIndefinitely(sinkFutures)) + .thenReturn(futureHelperResult); + + final ProcessWorker processWorker = new ProcessWorker(buffer, processors, pipeline); + + processWorker.run(); + } + await().atMost(TEST_TIMEOUT) + .untilAsserted(() -> { + assertTrue(aggregatedResultReceived); + assertThat(aggregatedResults.size(), equalTo(1)); + assertTrue(((DefaultAcknowledgementSet)acknowledgementSet).isDone()); + }); + } + + @Test + public void testPercentSamplerAggregation() throws Exception { + double testPercent = 50.0; + PercentSamplerAggregateActionConfig percentSamplerAggregateActionConfig = new PercentSamplerAggregateActionConfig(); + setField(PercentSamplerAggregateActionConfig.class, percentSamplerAggregateActionConfig, "percent", testPercent); + AggregateAction aggregateAction = new PercentSamplerAggregateAction(percentSamplerAggregateActionConfig); + when(pluginFactory.loadPlugin(eq(AggregateAction.class), any(PluginSetting.class))) + .thenReturn(aggregateAction); + when(aggregateProcessorConfig.getGroupDuration()).thenReturn(Duration.ofSeconds(GROUP_DURATION_FOR_ONLY_SINGLE_CONCLUDE)); + + final Processor processor = new AggregateProcessor(aggregateProcessorConfig, pluginMetrics, pluginFactory, expressionEvaluator); + processors = List.of(processor); + final FutureHelperResult futureHelperResult = mock(FutureHelperResult.class); + when(futureHelperResult.getFailedReasons()).thenReturn(Collections.emptyList()); + try (final MockedStatic futureHelperMockedStatic = mockStatic(FutureHelper.class)) { + futureHelperMockedStatic.when(() -> FutureHelper.awaitFuturesIndefinitely(sinkFutures)) + .thenReturn(futureHelperResult); + + final ProcessWorker processWorker = new ProcessWorker(buffer, processors, pipeline); + + processWorker.run(); + } + await().atMost(TEST_TIMEOUT) + .untilAsserted(() -> { + assertFalse(aggregatedResultReceived); + assertThat(aggregatedResults.size(), greaterThanOrEqualTo(1)); + assertThat(aggregatedResults.size(), lessThan(5)); + assertTrue(((DefaultAcknowledgementSet)acknowledgementSet).isDone()); + }); + } + + + @Test + public void testPutAllAggregation() throws Exception { + AggregateAction aggregateAction = new PutAllAggregateAction(); + when(pluginFactory.loadPlugin(eq(AggregateAction.class), any(PluginSetting.class))) + .thenReturn(aggregateAction); + when(aggregateProcessorConfig.getGroupDuration()).thenReturn(Duration.ofSeconds(GROUP_DURATION_FOR_ONLY_SINGLE_CONCLUDE)); + when(pluginFactory.loadPlugin(eq(AggregateAction.class), any(PluginSetting.class))) + .thenReturn(aggregateAction); + when(aggregateProcessorConfig.getGroupDuration()).thenReturn(Duration.ofSeconds(GROUP_DURATION_FOR_ONLY_SINGLE_CONCLUDE)); + final Processor processor = new AggregateProcessor(aggregateProcessorConfig, pluginMetrics, pluginFactory, expressionEvaluator); + processors = List.of(processor); + final FutureHelperResult futureHelperResult = mock(FutureHelperResult.class); + when(futureHelperResult.getFailedReasons()).thenReturn(Collections.emptyList()); + try (final MockedStatic futureHelperMockedStatic = mockStatic(FutureHelper.class)) { + futureHelperMockedStatic.when(() -> FutureHelper.awaitFuturesIndefinitely(sinkFutures)) + .thenReturn(futureHelperResult); + + final ProcessWorker processWorker = new ProcessWorker(buffer, processors, pipeline); + + processWorker.run(); + } + await().atMost(TEST_TIMEOUT) + .untilAsserted(() -> { + assertTrue(aggregatedResultReceived); + assertThat(aggregatedResults.size(), equalTo(1)); + assertTrue(((DefaultAcknowledgementSet)acknowledgementSet).isDone()); + }); + } + + + @Test + public void testRateLimiterDropAggregation() throws Exception { + RateLimiterAggregateActionConfig rateLimiterAggregateActionConfig = mock(RateLimiterAggregateActionConfig.class); + final int eventsPerSecond = 1; + when(rateLimiterAggregateActionConfig.getEventsPerSecond()).thenReturn(eventsPerSecond); + when(rateLimiterAggregateActionConfig.getWhenExceeds()).thenReturn(RateLimiterMode.DROP); + AggregateAction aggregateAction = new RateLimiterAggregateAction(rateLimiterAggregateActionConfig); + when(pluginFactory.loadPlugin(eq(AggregateAction.class), any(PluginSetting.class))) + .thenReturn(aggregateAction); + when(aggregateProcessorConfig.getGroupDuration()).thenReturn(Duration.ofSeconds(GROUP_DURATION_FOR_ONLY_SINGLE_CONCLUDE)); + final Processor processor = new AggregateProcessor(aggregateProcessorConfig, pluginMetrics, pluginFactory, expressionEvaluator); + processors = List.of(processor); + final FutureHelperResult futureHelperResult = mock(FutureHelperResult.class); + when(futureHelperResult.getFailedReasons()).thenReturn(Collections.emptyList()); + try (final MockedStatic futureHelperMockedStatic = mockStatic(FutureHelper.class)) { + futureHelperMockedStatic.when(() -> FutureHelper.awaitFuturesIndefinitely(sinkFutures)) + .thenReturn(futureHelperResult); + + final ProcessWorker processWorker = new ProcessWorker(buffer, processors, pipeline); + + processWorker.run(); + } + await().atMost(TEST_TIMEOUT) + .untilAsserted(() -> { + assertFalse(aggregatedResultReceived); + assertThat(aggregatedResults.size(), equalTo(1)); + assertTrue(((DefaultAcknowledgementSet)acknowledgementSet).isDone()); + }); + } + + @Test + public void testRemoveDuplicatesAggregation() { + AggregateAction aggregateAction = new RemoveDuplicatesAggregateAction(); + when(pluginFactory.loadPlugin(eq(AggregateAction.class), any(PluginSetting.class))) + .thenReturn(aggregateAction); + when(aggregateProcessorConfig.getGroupDuration()).thenReturn(Duration.ofSeconds(GROUP_DURATION_FOR_ONLY_SINGLE_CONCLUDE)); + final Processor processor = new AggregateProcessor(aggregateProcessorConfig, pluginMetrics, pluginFactory, expressionEvaluator); + processors = List.of(processor); + final FutureHelperResult futureHelperResult = mock(FutureHelperResult.class); + when(futureHelperResult.getFailedReasons()).thenReturn(Collections.emptyList()); + try (final MockedStatic futureHelperMockedStatic = mockStatic(FutureHelper.class)) { + futureHelperMockedStatic.when(() -> FutureHelper.awaitFuturesIndefinitely(sinkFutures)) + .thenReturn(futureHelperResult); + + final ProcessWorker processWorker = new ProcessWorker(buffer, processors, pipeline); + + processWorker.run(); + } + await().atMost(TEST_TIMEOUT) + .untilAsserted(() -> { + assertFalse(aggregatedResultReceived); + assertThat(aggregatedResults.size(), equalTo(1)); + assertTrue(((DefaultAcknowledgementSet)acknowledgementSet).isDone()); + }); + } + + @Test + public void testRateLimiterNoDropAggregation() throws Exception { + RateLimiterAggregateActionConfig rateLimiterAggregateActionConfig = mock(RateLimiterAggregateActionConfig.class); + final int eventsPerSecond = 50; + when(rateLimiterAggregateActionConfig.getEventsPerSecond()).thenReturn(eventsPerSecond); + when(rateLimiterAggregateActionConfig.getWhenExceeds()).thenReturn(RateLimiterMode.BLOCK); + AggregateAction aggregateAction = new RateLimiterAggregateAction(rateLimiterAggregateActionConfig); + when(pluginFactory.loadPlugin(eq(AggregateAction.class), any(PluginSetting.class))) + .thenReturn(aggregateAction); + when(aggregateProcessorConfig.getGroupDuration()).thenReturn(Duration.ofSeconds(GROUP_DURATION_FOR_ONLY_SINGLE_CONCLUDE)); + final Processor processor = new AggregateProcessor(aggregateProcessorConfig, pluginMetrics, pluginFactory, expressionEvaluator); + processors = List.of(processor); + final FutureHelperResult futureHelperResult = mock(FutureHelperResult.class); + when(futureHelperResult.getFailedReasons()).thenReturn(Collections.emptyList()); + try (final MockedStatic futureHelperMockedStatic = mockStatic(FutureHelper.class)) { + futureHelperMockedStatic.when(() -> FutureHelper.awaitFuturesIndefinitely(sinkFutures)) + .thenReturn(futureHelperResult); + + final ProcessWorker processWorker = new ProcessWorker(buffer, processors, pipeline); + + processWorker.run(); + } + await().atMost(TEST_TIMEOUT) + .untilAsserted(() -> { + assertFalse(aggregatedResultReceived); + assertThat(aggregatedResults.size(), equalTo(5)); + assertTrue(((DefaultAcknowledgementSet)acknowledgementSet).isDone()); + }); + } + + + @Test + public void testRateLimiterNoDropAggregationWithMultipleAcknowledgementSets() throws Exception { + AcknowledgementSet acknowledgementSet2 = new DefaultAcknowledgementSet(scheduledExecutorService, (result) -> {}, Duration.ofSeconds(10), new DefaultAcknowledgementSetMetrics(pluginMetrics)); + AcknowledgementSet acknowledgementSet3 = new DefaultAcknowledgementSet(scheduledExecutorService, (result) -> {}, Duration.ofSeconds(10), new DefaultAcknowledgementSetMetrics(pluginMetrics)); + final List> records2 = getRecords(testKey, 1, acknowledgementSet2); + acknowledgementSet2.complete(); + final List> records3 = getRecords(testKey, 1, acknowledgementSet3); + acknowledgementSet3.complete(); + when(aggregateProcessorConfig.getIdentificationKeys()).thenReturn(List.of(testKey)); + + RateLimiterAggregateActionConfig rateLimiterAggregateActionConfig = mock(RateLimiterAggregateActionConfig.class); + final int eventsPerSecond = 50; + when(rateLimiterAggregateActionConfig.getEventsPerSecond()).thenReturn(eventsPerSecond); + when(rateLimiterAggregateActionConfig.getWhenExceeds()).thenReturn(RateLimiterMode.BLOCK); + AggregateAction aggregateAction = new RateLimiterAggregateAction(rateLimiterAggregateActionConfig); + when(aggregateProcessorConfig.getGroupDuration()).thenReturn(Duration.ofSeconds(GROUP_DURATION_FOR_ONLY_SINGLE_CONCLUDE)); + when(pluginFactory.loadPlugin(eq(AggregateAction.class), any(PluginSetting.class))) + .thenReturn(aggregateAction); + buffer = mock(Buffer.class); + when(buffer.isEmpty()).thenReturn(true); + when(buffer.read(pipeline.getReadBatchTimeoutInMillis())).thenAnswer(a -> { + if (callCount == 0) { + callCount++; + return Map.entry(records, checkpointState); + } else if (callCount == 1) { + callCount++; + return Map.entry(records2, checkpointState); + } else if (callCount == 2) { + callCount++; + return Map.entry(records3, checkpointState); + } else { + return Map.entry(List.of(), checkpointState); + } + }); + + final Processor processor = new AggregateProcessor(aggregateProcessorConfig, pluginMetrics, pluginFactory, expressionEvaluator); + processors = List.of(processor); + final FutureHelperResult futureHelperResult = mock(FutureHelperResult.class); + when(futureHelperResult.getFailedReasons()).thenReturn(Collections.emptyList()); + try (final MockedStatic futureHelperMockedStatic = mockStatic(FutureHelper.class)) { + futureHelperMockedStatic.when(() -> FutureHelper.awaitFuturesIndefinitely(sinkFutures)) + .thenReturn(futureHelperResult); + + final ProcessWorker processWorker = new ProcessWorker(buffer, processors, pipeline); + + processWorker.run(); + } + await().atMost(TEST_TIMEOUT) + .untilAsserted(() -> { + assertFalse(aggregatedResultReceived); + assertThat(aggregatedResults.size(), equalTo(5)); + assertTrue(((DefaultAcknowledgementSet)acknowledgementSet).isDone()); + }); + } + + + @Test + public void testCountAggregationWithMultipleAcknowledgementSets() throws Exception { + AcknowledgementSet acknowledgementSet2 = new DefaultAcknowledgementSet(scheduledExecutorService, (result) -> {}, Duration.ofSeconds(10), new DefaultAcknowledgementSetMetrics(pluginMetrics)); + AcknowledgementSet acknowledgementSet3 = new DefaultAcknowledgementSet(scheduledExecutorService, (result) -> {}, Duration.ofSeconds(10), new DefaultAcknowledgementSetMetrics(pluginMetrics)); + final List> records2 = getRecords(testKey, 1, acknowledgementSet2); + acknowledgementSet2.complete(); + final List> records3 = getRecords(testKey, 1, acknowledgementSet3); + acknowledgementSet3.complete(); + when(aggregateProcessorConfig.getIdentificationKeys()).thenReturn(List.of(testKey)); + + CountAggregateActionConfig countAggregateActionConfig = new CountAggregateActionConfig(); + setField(CountAggregateActionConfig.class, countAggregateActionConfig, "outputFormat", OutputFormat.RAW); + AggregateAction aggregateAction = new CountAggregateAction(countAggregateActionConfig); + when(aggregateProcessorConfig.getGroupDuration()).thenReturn(Duration.ofSeconds(GROUP_DURATION_FOR_ONLY_SINGLE_CONCLUDE)); + when(pluginFactory.loadPlugin(eq(AggregateAction.class), any(PluginSetting.class))) + .thenReturn(aggregateAction); + callCount = 0; + buffer = mock(Buffer.class); + when(buffer.isEmpty()).thenReturn(true); + when(buffer.read(pipeline.getReadBatchTimeoutInMillis())).thenAnswer(a -> { + if (callCount == 0) { + callCount++; + return Map.entry(records, checkpointState); + } else if (callCount == 1) { + callCount++; + return Map.entry(records2, checkpointState); + } else if (callCount == 2) { + callCount++; + return Map.entry(records3, checkpointState); + } else { + return Map.entry(List.of(), checkpointState); + } + }); + final Processor processor = new AggregateProcessor(aggregateProcessorConfig, pluginMetrics, pluginFactory, expressionEvaluator); + processors = List.of(processor); + + final FutureHelperResult futureHelperResult = mock(FutureHelperResult.class); + when(futureHelperResult.getFailedReasons()).thenReturn(Collections.emptyList()); + try (final MockedStatic futureHelperMockedStatic = mockStatic(FutureHelper.class)) { + futureHelperMockedStatic.when(() -> FutureHelper.awaitFuturesIndefinitely(sinkFutures)) + .thenReturn(futureHelperResult); + + final ProcessWorker processWorker = new ProcessWorker(buffer, processors, pipeline); + + processWorker.run(); + } + await().atMost(TEST_TIMEOUT) + .untilAsserted(() -> { + assertTrue(aggregatedResultReceived); + assertThat(aggregatedResults.size(), equalTo(1)); + assertTrue(((DefaultAcknowledgementSet)acknowledgementSet).isDone()); + assertTrue(((DefaultAcknowledgementSet)acknowledgementSet2).isDone()); + assertTrue(((DefaultAcknowledgementSet)acknowledgementSet3).isDone()); + }); + } + + @Test + public void testCountAggregation() throws Exception { + CountAggregateActionConfig countAggregateActionConfig = new CountAggregateActionConfig(); + setField(CountAggregateActionConfig.class, countAggregateActionConfig, "outputFormat", OutputFormat.RAW); + AggregateAction aggregateAction = new CountAggregateAction(countAggregateActionConfig); + when(aggregateProcessorConfig.getGroupDuration()).thenReturn(Duration.ofMillis(GROUP_DURATION_FOR_ONLY_SINGLE_CONCLUDE)); + when(pluginFactory.loadPlugin(eq(AggregateAction.class), any(PluginSetting.class))) + .thenReturn(aggregateAction); + final Processor processor = new AggregateProcessor(aggregateProcessorConfig, pluginMetrics, pluginFactory, expressionEvaluator); + processors = List.of(processor); + + final FutureHelperResult futureHelperResult = mock(FutureHelperResult.class); + when(futureHelperResult.getFailedReasons()).thenReturn(Collections.emptyList()); + try (final MockedStatic futureHelperMockedStatic = mockStatic(FutureHelper.class)) { + futureHelperMockedStatic.when(() -> FutureHelper.awaitFuturesIndefinitely(sinkFutures)) + .thenReturn(futureHelperResult); + + final ProcessWorker processWorker = new ProcessWorker(buffer, processors, pipeline); + + processWorker.run(); + } + await().atMost(TEST_TIMEOUT) + .untilAsserted(() -> { + assertTrue(aggregatedResultReceived); + assertThat(aggregatedResults.size(), equalTo(1)); + assertTrue(((DefaultAcknowledgementSet)acknowledgementSet).isDone()); + }); + } + + @Test + public void testTailSamplerAggregationWithNoErrors() throws Exception { + TailSamplerAggregateActionConfig tailSamplerAggregateActionConfig = mock(TailSamplerAggregateActionConfig.class); + final Duration testWaitPeriod = Duration.ofMillis(1); + final String testCondition = "/status == 2"; + when(tailSamplerAggregateActionConfig.getPercent()).thenReturn(100); + when(tailSamplerAggregateActionConfig.getWaitPeriod()).thenReturn(testWaitPeriod); + when(tailSamplerAggregateActionConfig.getCondition()).thenReturn(testCondition); + when(expressionEvaluator.evaluateConditional(eq(testCondition), any(Event.class))).thenReturn(false); + AggregateAction aggregateAction = new TailSamplerAggregateAction(tailSamplerAggregateActionConfig, expressionEvaluator); + when(pluginFactory.loadPlugin(eq(AggregateAction.class), any(PluginSetting.class))).thenReturn(aggregateAction); + when(aggregateProcessorConfig.getGroupDuration()).thenReturn(Duration.ofSeconds(GROUP_DURATION_FOR_ONLY_SINGLE_CONCLUDE)); + final Processor processor = new AggregateProcessor(aggregateProcessorConfig, pluginMetrics, pluginFactory, expressionEvaluator); + processors = List.of(processor); + + final FutureHelperResult futureHelperResult = mock(FutureHelperResult.class); + when(futureHelperResult.getFailedReasons()).thenReturn(Collections.emptyList()); + try (final MockedStatic futureHelperMockedStatic = mockStatic(FutureHelper.class)) { + futureHelperMockedStatic.when(() -> FutureHelper.awaitFuturesIndefinitely(sinkFutures)) + .thenReturn(futureHelperResult); + + final ProcessWorker processWorker = new ProcessWorker(buffer, processors, pipeline); + + processWorker.run(); + } + await().atMost(TEST_TIMEOUT) + .untilAsserted(() -> { + assertFalse(aggregatedResultReceived); + assertThat(aggregatedResults.size(), equalTo(5)); + assertTrue(((DefaultAcknowledgementSet)acknowledgementSet).isDone()); + }); + } + + + + @Test + public void testTailSamplerAggregation() throws Exception { + TailSamplerAggregateActionConfig tailSamplerAggregateActionConfig = mock(TailSamplerAggregateActionConfig.class); + final Duration testWaitPeriod = Duration.ofMillis(1); + final String testCondition = "/status == 2"; + when(tailSamplerAggregateActionConfig.getPercent()).thenReturn(50); + when(tailSamplerAggregateActionConfig.getWaitPeriod()).thenReturn(testWaitPeriod); + when(tailSamplerAggregateActionConfig.getCondition()).thenReturn(testCondition); + when(expressionEvaluator.evaluateConditional(eq(testCondition), any(Event.class))).thenReturn(true); + AggregateAction aggregateAction = new TailSamplerAggregateAction(tailSamplerAggregateActionConfig, expressionEvaluator); + when(pluginFactory.loadPlugin(eq(AggregateAction.class), any(PluginSetting.class))).thenReturn(aggregateAction); + when(aggregateProcessorConfig.getGroupDuration()).thenReturn(Duration.ofSeconds(GROUP_DURATION_FOR_ONLY_SINGLE_CONCLUDE)); + final Processor processor = new AggregateProcessor(aggregateProcessorConfig, pluginMetrics, pluginFactory, expressionEvaluator); + processors = List.of(processor); + + final FutureHelperResult futureHelperResult = mock(FutureHelperResult.class); + when(futureHelperResult.getFailedReasons()).thenReturn(Collections.emptyList()); + try (final MockedStatic futureHelperMockedStatic = mockStatic(FutureHelper.class)) { + futureHelperMockedStatic.when(() -> FutureHelper.awaitFuturesIndefinitely(sinkFutures)) + .thenReturn(futureHelperResult); + + final ProcessWorker processWorker = new ProcessWorker(buffer, processors, pipeline); + + processWorker.run(); + } + await().atMost(TEST_TIMEOUT) + .untilAsserted(() -> { + assertFalse(aggregatedResultReceived); + assertThat(aggregatedResults.size(), equalTo(5)); + assertTrue(((DefaultAcknowledgementSet)acknowledgementSet).isDone()); + }); + } + + @Test + public void testAppendAggregation() throws Exception { + AppendAggregateActionConfig appendAggregateActionConfig = mock(AppendAggregateActionConfig.class); + when(appendAggregateActionConfig.getKeysToAppend()).thenReturn(List.of(testKey)); + AggregateAction aggregateAction = new AppendAggregateAction(appendAggregateActionConfig); + when(pluginFactory.loadPlugin(eq(AggregateAction.class), any(PluginSetting.class))).thenReturn(aggregateAction); + when(aggregateProcessorConfig.getGroupDuration()).thenReturn(Duration.ofSeconds(GROUP_DURATION_FOR_ONLY_SINGLE_CONCLUDE)); + final Processor processor = new AggregateProcessor(aggregateProcessorConfig, pluginMetrics, pluginFactory, expressionEvaluator); + processors = List.of(processor); + + final FutureHelperResult futureHelperResult = mock(FutureHelperResult.class); + when(futureHelperResult.getFailedReasons()).thenReturn(Collections.emptyList()); + try (final MockedStatic futureHelperMockedStatic = mockStatic(FutureHelper.class)) { + futureHelperMockedStatic.when(() -> FutureHelper.awaitFuturesIndefinitely(sinkFutures)) + .thenReturn(futureHelperResult); + + final ProcessWorker processWorker = new ProcessWorker(buffer, processors, pipeline); + + processWorker.run(); + } + await().atMost(TEST_TIMEOUT) + .untilAsserted(() -> { + assertTrue(aggregatedResultReceived); + assertThat(aggregatedResults.size(), equalTo(1)); + assertTrue(((DefaultAcknowledgementSet)acknowledgementSet).isDone()); + }); + } + + + private List> getRecords(String key, int value, AcknowledgementSet ackSet) { + final List> events = new ArrayList<>(); + final Map eventMap = Map.of(key, value); + + for (int i = 0; i < NUM_EVENTS_PER_BATCH; i++) { + final Event event = JacksonEvent.builder() + .withEventType("event") + .withData(eventMap) + .build(); + events.add(new Record<>(event)); + ackSet.add(event); + } + return events; + } + +} + 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 69d4f8c8a4..578e22a351 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 @@ -23,17 +23,17 @@ @JsonClassDescription("The anomaly detector processor takes structured data and runs anomaly detection algorithms " + "on fields that you can configure in that data.") public class AnomalyDetectorProcessorConfig { + @JsonPropertyDescription("A non-ordered List that is used as input to the ML algorithm to detect anomalies in the values of the keys in the list. At least one key is required.") + @JsonProperty("keys") + @NotEmpty + private List keys; + @JsonPropertyDescription("The ML algorithm (or model) used to detect anomalies. You must provide a mode. See random_cut_forest mode.") @JsonProperty("mode") @NotNull @UsesDataPrepperPlugin(pluginType = AnomalyDetectorMode.class) private PluginModel detectorMode; - @JsonPropertyDescription("A non-ordered List that is used as input to the ML algorithm to detect anomalies in the values of the keys in the list. At least one key is required.") - @JsonProperty("keys") - @NotEmpty - private List keys; - @JsonPropertyDescription("If provided, anomalies will be detected within each unique instance of these keys. For example, if you provide the ip field, anomalies will be detected separately for each unique IP address.") @JsonProperty("identification_keys") @ExampleValues({ diff --git a/data-prepper-plugins/anomaly-detector-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/anomalydetector/modes/RandomCutForestModeConfig.java b/data-prepper-plugins/anomaly-detector-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/anomalydetector/modes/RandomCutForestModeConfig.java index c477746253..74c90fca50 100644 --- a/data-prepper-plugins/anomaly-detector-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/anomalydetector/modes/RandomCutForestModeConfig.java +++ b/data-prepper-plugins/anomaly-detector-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/anomalydetector/modes/RandomCutForestModeConfig.java @@ -13,6 +13,7 @@ import java.util.HashSet; public class RandomCutForestModeConfig { + public static final String DEFAULT_TYPE = "metrics"; public static final int DEFAULT_SHINGLE_SIZE = 4; private static final int MIN_SHINGLE_SIZE = 1; public static final int MAX_SHINGLE_SIZE = 60; @@ -27,31 +28,31 @@ public class RandomCutForestModeConfig { public static final String VERSION_1_0 = "1.0"; @JsonPropertyDescription("The algorithm version number. Default is 1.0.") - @JsonProperty("version") + @JsonProperty(value = "version", defaultValue = VERSION_1_0) private String version = VERSION_1_0; public static final Set validVersions = new HashSet<>(Set.of(VERSION_1_0)); @JsonPropertyDescription("The type of data sent to the algorithm. Default is metrics type") - @JsonProperty("type") + @JsonProperty(value = "type", defaultValue = DEFAULT_TYPE) private String type = RandomCutForestType.METRICS.toString(); public static final Set validTypes = new HashSet<>(Set.of(RandomCutForestType.METRICS.toString())); - @JsonPropertyDescription("The shingle size used in the ML algorithm. Default is 60.") - @JsonProperty("shingle_size") + @JsonPropertyDescription("The shingle size used in the ML algorithm. Default is 4.") + @JsonProperty(value = "shingle_size", defaultValue = "" + DEFAULT_SHINGLE_SIZE) private int shingleSize = DEFAULT_SHINGLE_SIZE; @JsonPropertyDescription("The sample size used in the ML algorithm. Default is 256.") - @JsonProperty("sample_size") + @JsonProperty(value = "sample_size", defaultValue = "" + DEFAULT_SAMPLE_SIZE) private int sampleSize = DEFAULT_SAMPLE_SIZE; @JsonPropertyDescription("The time decay value used in the ML algorithm. Used as the mathematical expression timeDecay divided by SampleSize in the ML algorithm. Default is 0.1") - @JsonProperty("time_decay") + @JsonProperty(value = "time_decay", defaultValue = "" + DEFAULT_TIME_DECAY) private double timeDecay = DEFAULT_TIME_DECAY; - @JsonPropertyDescription("Output after indicates the number of events to consume before outputting anamolies. Default is 32.") - @JsonProperty("output_after") + @JsonPropertyDescription("Output after indicates the number of events to consume before outputting anomalies. Default is 32.") + @JsonProperty(value = "output_after", defaultValue = "" + DEFAULT_OUTPUT_AFTER) private int outputAfter = DEFAULT_OUTPUT_AFTER; @AssertTrue(message = "Value of output_after must be less than or equal to the value of sample_size") diff --git a/data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/processor/LambdaProcessorConfig.java b/data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/processor/LambdaProcessorConfig.java index cfa7417896..c9675ed931 100644 --- a/data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/processor/LambdaProcessorConfig.java +++ b/data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/processor/LambdaProcessorConfig.java @@ -17,7 +17,7 @@ import org.opensearch.dataprepper.plugins.lambda.common.config.LambdaCommonConfig; @JsonPropertyOrder -@JsonClassDescription("The aws_lambda processor enables invocation of an AWS Lambda function within your Data Prepper pipeline in order to process events." + +@JsonClassDescription("The aws_lambda processor enables invocation of an AWS Lambda function within your Data Prepper pipeline in order to process events." + "It supports both synchronous and asynchronous invocations based on your use case.") public class LambdaProcessorConfig extends LambdaCommonConfig { static final String DEFAULT_INVOCATION_TYPE = "request-response"; diff --git a/data-prepper-plugins/aws-plugin-api/build.gradle b/data-prepper-plugins/aws-plugin-api/build.gradle index 9383c8c9f7..1042ffadc9 100644 --- a/data-prepper-plugins/aws-plugin-api/build.gradle +++ b/data-prepper-plugins/aws-plugin-api/build.gradle @@ -3,7 +3,7 @@ dependencies { implementation 'software.amazon.awssdk:auth' implementation 'software.amazon.awssdk:apache-client' implementation 'org.apache.httpcomponents.client5:httpclient5:5.3.1' - testImplementation 'org.hibernate.validator:hibernate-validator:8.0.1.Final' + testImplementation 'org.hibernate.validator:hibernate-validator:8.0.2.Final' } test { diff --git a/data-prepper-plugins/aws-plugin/build.gradle b/data-prepper-plugins/aws-plugin/build.gradle index 710ce5b7b4..6915278023 100644 --- a/data-prepper-plugins/aws-plugin/build.gradle +++ b/data-prepper-plugins/aws-plugin/build.gradle @@ -8,7 +8,7 @@ dependencies { implementation 'software.amazon.awssdk:secretsmanager' implementation 'software.amazon.awssdk:sts' implementation 'software.amazon.awssdk:arns' - implementation 'org.hibernate.validator:hibernate-validator:8.0.1.Final' + implementation 'org.hibernate.validator:hibernate-validator:8.0.2.Final' testImplementation 'com.fasterxml.jackson.dataformat:jackson-dataformat-yaml' testImplementation 'com.fasterxml.jackson.datatype:jackson-datatype-jsr310' } diff --git a/data-prepper-plugins/date-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/date/DateProcessorConfig.java b/data-prepper-plugins/date-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/date/DateProcessorConfig.java index f813a8fa1f..2e3114d979 100644 --- a/data-prepper-plugins/date-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/date/DateProcessorConfig.java +++ b/data-prepper-plugins/date-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/date/DateProcessorConfig.java @@ -142,7 +142,7 @@ public static boolean isValidPattern(final String pattern) { }) private List match; - @JsonProperty("destination") + @JsonProperty(value = "destination", defaultValue = DEFAULT_DESTINATION) @JsonPropertyDescription("The field used to store the timestamp parsed by the date processor. " + "Can be used with both match and from_time_received. Default is @timestamp.") private String destination = DEFAULT_DESTINATION; diff --git a/data-prepper-plugins/http-source/build.gradle b/data-prepper-plugins/http-source/build.gradle index 2d5c5ceceb..fcfb4ad67f 100644 --- a/data-prepper-plugins/http-source/build.gradle +++ b/data-prepper-plugins/http-source/build.gradle @@ -19,7 +19,7 @@ dependencies { implementation 'software.amazon.awssdk:acm' implementation 'software.amazon.awssdk:s3' implementation 'software.amazon.awssdk:apache-client' - testImplementation 'org.assertj:assertj-core:3.25.3' + testImplementation 'org.assertj:assertj-core:3.27.0' testImplementation project(':data-prepper-api').sourceSets.test.output testImplementation project(':data-prepper-test-common') } diff --git a/data-prepper-plugins/kafka-plugins/build.gradle b/data-prepper-plugins/kafka-plugins/build.gradle index 0ccb88b395..60c0472387 100644 --- a/data-prepper-plugins/kafka-plugins/build.gradle +++ b/data-prepper-plugins/kafka-plugins/build.gradle @@ -64,7 +64,7 @@ dependencies { testImplementation project(':data-prepper-pipeline-parser') testImplementation 'org.apache.kafka:kafka_2.13:3.6.1' testImplementation 'org.apache.kafka:kafka_2.13:3.6.1:test' - testImplementation 'org.apache.curator:curator-test:5.5.0' + testImplementation 'org.apache.curator:curator-test:5.7.1' testImplementation('com.kjetland:mbknor-jackson-jsonschema_2.13:1.0.39') testImplementation project(':data-prepper-plugins:otel-metrics-source') testImplementation project(':data-prepper-plugins:otel-proto-common') diff --git a/data-prepper-plugins/kinesis-source/src/integrationTest/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisSourceIT.java b/data-prepper-plugins/kinesis-source/src/integrationTest/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisSourceIT.java index 028f73ad9f..76081c5525 100644 --- a/data-prepper-plugins/kinesis-source/src/integrationTest/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisSourceIT.java +++ b/data-prepper-plugins/kinesis-source/src/integrationTest/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisSourceIT.java @@ -11,6 +11,7 @@ package org.opensearch.dataprepper.plugins.kinesis.source; import io.micrometer.core.instrument.Counter; +import io.micrometer.core.instrument.DistributionSummary; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -26,6 +27,7 @@ import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.model.plugin.PluginFactory; import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.plugins.codec.CompressionOption; import org.opensearch.dataprepper.plugins.codec.json.NdjsonInputCodec; import org.opensearch.dataprepper.plugins.codec.json.NdjsonInputConfig; import org.opensearch.dataprepper.plugins.kinesis.extension.KinesisLeaseConfig; @@ -70,9 +72,11 @@ import static org.mockito.Mockito.when; import static org.opensearch.dataprepper.plugins.kinesis.source.processor.KinesisRecordProcessor.ACKNOWLEDGEMENT_SET_FAILURES_METRIC_NAME; import static org.opensearch.dataprepper.plugins.kinesis.source.processor.KinesisRecordProcessor.ACKNOWLEDGEMENT_SET_SUCCESS_METRIC_NAME; -import static org.opensearch.dataprepper.plugins.kinesis.source.processor.KinesisRecordProcessor.KINESIS_RECORD_PROCESSED; -import static org.opensearch.dataprepper.plugins.kinesis.source.processor.KinesisRecordProcessor.KINESIS_RECORD_PROCESSING_ERRORS; +import static org.opensearch.dataprepper.plugins.kinesis.source.processor.KinesisRecordProcessor.KINESIS_RECORD_BYTES_PROCESSED_METRIC_NAME; +import static org.opensearch.dataprepper.plugins.kinesis.source.processor.KinesisRecordProcessor.KINESIS_RECORD_BYTES_RECEIVED_METRIC_NAME; +import static org.opensearch.dataprepper.plugins.kinesis.source.processor.KinesisRecordProcessor.KINESIS_RECORD_PROCESSED_METRIC_NAME; import static org.opensearch.dataprepper.plugins.kinesis.source.processor.KinesisRecordProcessor.KINESIS_CHECKPOINT_FAILURES; +import static org.opensearch.dataprepper.plugins.kinesis.source.processor.KinesisRecordProcessor.KINESIS_RECORD_PROCESSING_ERRORS_METRIC_NAME; import static org.opensearch.dataprepper.plugins.kinesis.source.processor.KinesisRecordProcessor.KINESIS_STREAM_TAG_KEY; public class KinesisSourceIT { @@ -83,6 +87,8 @@ public class KinesisSourceIT { private static final String codec_plugin_name = "ndjson"; private static final String LEASE_TABLE_PREFIX = "kinesis-lease-table"; private static final int NUMBER_OF_RECORDS_TO_ACCUMULATE = 10; + private static final int MAX_INITIALIZATION_ATTEMPTS = 3; + private static final Duration BUFFER_TIMEOUT = Duration.ofMillis(1); @Mock private AcknowledgementSetManager acknowledgementSetManager; @@ -141,6 +147,12 @@ public class KinesisSourceIT { @Mock private Counter checkpointFailures; + @Mock + private DistributionSummary bytesReceivedSummary; + + @Mock + private DistributionSummary bytesProcessedSummary; + private KinesisClient kinesisClient; private DynamoDbClient dynamoDbClient; @@ -165,6 +177,7 @@ void setup() { when(kinesisStreamConfig.getName()).thenReturn(streamName); when(kinesisStreamConfig.getCheckPointInterval()).thenReturn(CHECKPOINT_INTERVAL); when(kinesisStreamConfig.getInitialPosition()).thenReturn(InitialPositionInStream.TRIM_HORIZON); + when(kinesisStreamConfig.getCompression()).thenReturn(CompressionOption.NONE); when(kinesisSourceConfig.getConsumerStrategy()).thenReturn(ConsumerStrategy.ENHANCED_FAN_OUT); when(kinesisSourceConfig.getStreams()).thenReturn(List.of(kinesisStreamConfig)); when(kinesisLeaseConfig.getLeaseCoordinationTable()).thenReturn(kinesisLeaseCoordinationTableConfig); @@ -179,7 +192,8 @@ void setup() { when(kinesisSourceConfig.getCodec()).thenReturn(pluginModel); when(kinesisSourceConfig.isAcknowledgments()).thenReturn(false); when(kinesisSourceConfig.getNumberOfRecordsToAccumulate()).thenReturn(NUMBER_OF_RECORDS_TO_ACCUMULATE); - when(kinesisSourceConfig.getBufferTimeout()).thenReturn(Duration.ofMillis(1)); + when(kinesisSourceConfig.getBufferTimeout()).thenReturn(BUFFER_TIMEOUT); + when(kinesisSourceConfig.getMaxInitializationAttempts()).thenReturn(MAX_INITIALIZATION_ATTEMPTS); kinesisClientFactory = mock(KinesisClientFactory.class); when(kinesisClientFactory.buildDynamoDBClient(kinesisLeaseCoordinationTableConfig.getAwsRegion())).thenReturn(DynamoDbAsyncClient.builder() @@ -204,10 +218,10 @@ void setup() { when(pluginMetrics.counterWithTags(ACKNOWLEDGEMENT_SET_FAILURES_METRIC_NAME, KINESIS_STREAM_TAG_KEY, streamName)) .thenReturn(acknowledgementSetFailures); - when(pluginMetrics.counterWithTags(KINESIS_RECORD_PROCESSED, KINESIS_STREAM_TAG_KEY, streamName)) + when(pluginMetrics.counterWithTags(KINESIS_RECORD_PROCESSED_METRIC_NAME, KINESIS_STREAM_TAG_KEY, streamName)) .thenReturn(recordsProcessed); - when(pluginMetrics.counterWithTags(KINESIS_RECORD_PROCESSING_ERRORS, KINESIS_STREAM_TAG_KEY, streamName)) + when(pluginMetrics.counterWithTags(KINESIS_RECORD_PROCESSING_ERRORS_METRIC_NAME, KINESIS_STREAM_TAG_KEY, streamName)) .thenReturn(recordProcessingErrors); when(pluginMetrics.counterWithTags(KINESIS_CHECKPOINT_FAILURES, KINESIS_STREAM_TAG_KEY, streamName)) @@ -215,6 +229,9 @@ void setup() { kinesisClient = KinesisClient.builder().region(Region.of(System.getProperty(AWS_REGION))).build(); dynamoDbClient = DynamoDbClient.builder().region(Region.of(System.getProperty(AWS_REGION))).build(); + when(pluginMetrics.summary(KINESIS_RECORD_BYTES_RECEIVED_METRIC_NAME)).thenReturn(bytesReceivedSummary); + + when(pluginMetrics.summary(KINESIS_RECORD_BYTES_PROCESSED_METRIC_NAME)).thenReturn(bytesProcessedSummary); kinesisIngester = new KinesisIngester(kinesisClient, streamName, dynamoDbClient, leaseTableName); kinesisIngester.createStream(); diff --git a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/converter/KinesisRecordConverter.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/converter/KinesisRecordConverter.java index 7e1af1a41d..3514670097 100644 --- a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/converter/KinesisRecordConverter.java +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/converter/KinesisRecordConverter.java @@ -15,6 +15,7 @@ import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.model.event.EventMetadata; import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.plugins.kinesis.source.processor.KinesisInputOutputRecord; import software.amazon.kinesis.retrieval.KinesisClientRecord; import java.io.ByteArrayInputStream; @@ -32,13 +33,12 @@ public KinesisRecordConverter(final InputCodec codec) { this.codec = codec; } - public List> convert(final DecompressionEngine decompressionEngine, - List kinesisClientRecords, - final String streamName) throws IOException { - List> records = new ArrayList<>(); + public List convert(final DecompressionEngine decompressionEngine, + List kinesisClientRecords, + final String streamName) throws IOException { + List records = new ArrayList<>(); for (KinesisClientRecord kinesisClientRecord : kinesisClientRecords) { processRecord(decompressionEngine, kinesisClientRecord, record -> { - records.add(record); Event event = record.getData(); EventMetadata eventMetadata = event.getMetadata(); eventMetadata.setAttribute(MetadataKeyAttributes.KINESIS_STREAM_NAME_METADATA_ATTRIBUTE, @@ -49,6 +49,9 @@ public List> convert(final DecompressionEngine decompressionEngine final Instant externalOriginationTime = kinesisClientRecord.approximateArrivalTimestamp(); event.getEventHandle().setExternalOriginationTime(externalOriginationTime); event.getMetadata().setExternalOriginationTime(externalOriginationTime); + records.add(KinesisInputOutputRecord.builder() + .withIncomingRecordSizeBytes(kinesisClientRecord.data().position()) + .withDataPrepperRecord(record).build()); }); } return records; diff --git a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisInputOutputRecord.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisInputOutputRecord.java new file mode 100644 index 0000000000..782e74ba21 --- /dev/null +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisInputOutputRecord.java @@ -0,0 +1,25 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + +package org.opensearch.dataprepper.plugins.kinesis.source.processor; + +import lombok.AllArgsConstructor; +import lombok.Builder; +import lombok.Getter; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.record.Record; + +@Builder(setterPrefix = "with") +@Getter +@AllArgsConstructor +public class KinesisInputOutputRecord { + private Record dataPrepperRecord; + private long incomingRecordSizeBytes; +} diff --git a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisRecordProcessor.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisRecordProcessor.java index 9c216aa9a5..2b087c5139 100644 --- a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisRecordProcessor.java +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisRecordProcessor.java @@ -12,6 +12,7 @@ import com.google.common.annotations.VisibleForTesting; import io.micrometer.core.instrument.Counter; +import io.micrometer.core.instrument.DistributionSummary; import org.opensearch.dataprepper.buffer.common.BufferAccumulator; import org.opensearch.dataprepper.common.concurrent.BackgroundThreadFactory; import org.opensearch.dataprepper.metrics.PluginMetrics; @@ -70,10 +71,14 @@ public class KinesisRecordProcessor implements ShardRecordProcessor { private final Counter recordsProcessed; private final Counter recordProcessingErrors; private final Counter checkpointFailures; + private final DistributionSummary bytesReceivedSummary; + private final DistributionSummary bytesProcessedSummary; public static final String ACKNOWLEDGEMENT_SET_SUCCESS_METRIC_NAME = "acknowledgementSetSuccesses"; public static final String ACKNOWLEDGEMENT_SET_FAILURES_METRIC_NAME = "acknowledgementSetFailures"; - public static final String KINESIS_RECORD_PROCESSED = "recordProcessed"; - public static final String KINESIS_RECORD_PROCESSING_ERRORS = "recordProcessingErrors"; + public static final String KINESIS_RECORD_PROCESSED_METRIC_NAME = "recordProcessed"; + public static final String KINESIS_RECORD_PROCESSING_ERRORS_METRIC_NAME = "recordProcessingErrors"; + public static final String KINESIS_RECORD_BYTES_RECEIVED_METRIC_NAME = "bytesReceived"; + public static final String KINESIS_RECORD_BYTES_PROCESSED_METRIC_NAME = "bytesProcessed"; public static final String KINESIS_CHECKPOINT_FAILURES = "checkpointFailures"; public static final String KINESIS_STREAM_TAG_KEY = "stream"; private AtomicBoolean isStopRequested; @@ -93,9 +98,11 @@ public KinesisRecordProcessor(final BufferAccumulator> bufferAccum this.acknowledgementSetManager = acknowledgementSetManager; this.acknowledgementSetSuccesses = pluginMetrics.counterWithTags(ACKNOWLEDGEMENT_SET_SUCCESS_METRIC_NAME, KINESIS_STREAM_TAG_KEY, streamIdentifier.streamName()); this.acknowledgementSetFailures = pluginMetrics.counterWithTags(ACKNOWLEDGEMENT_SET_FAILURES_METRIC_NAME, KINESIS_STREAM_TAG_KEY, streamIdentifier.streamName()); - this.recordsProcessed = pluginMetrics.counterWithTags(KINESIS_RECORD_PROCESSED, KINESIS_STREAM_TAG_KEY, streamIdentifier.streamName()); - this.recordProcessingErrors = pluginMetrics.counterWithTags(KINESIS_RECORD_PROCESSING_ERRORS, KINESIS_STREAM_TAG_KEY, streamIdentifier.streamName()); + this.recordsProcessed = pluginMetrics.counterWithTags(KINESIS_RECORD_PROCESSED_METRIC_NAME, KINESIS_STREAM_TAG_KEY, streamIdentifier.streamName()); + this.recordProcessingErrors = pluginMetrics.counterWithTags(KINESIS_RECORD_PROCESSING_ERRORS_METRIC_NAME, KINESIS_STREAM_TAG_KEY, streamIdentifier.streamName()); this.checkpointFailures = pluginMetrics.counterWithTags(KINESIS_CHECKPOINT_FAILURES, KINESIS_STREAM_TAG_KEY, streamIdentifier.streamName()); + this.bytesReceivedSummary = pluginMetrics.summary(KINESIS_RECORD_BYTES_RECEIVED_METRIC_NAME); + this.bytesProcessedSummary = pluginMetrics.summary(KINESIS_RECORD_BYTES_PROCESSED_METRIC_NAME); this.checkpointInterval = kinesisStreamConfig.getCheckPointInterval(); this.bufferAccumulator = bufferAccumulator; this.kinesisCheckpointerTracker = kinesisCheckpointerTracker; @@ -161,16 +168,21 @@ public void processRecords(ProcessRecordsInput processRecordsInput) { // Track the records for checkpoint purpose kinesisCheckpointerTracker.addRecordForCheckpoint(extendedSequenceNumber, processRecordsInput.checkpointer()); - List> records = kinesisRecordConverter.convert( + + List kinesisOutputRecords = kinesisRecordConverter.convert( kinesisStreamConfig.getCompression().getDecompressionEngine(), processRecordsInput.records(), streamIdentifier.streamName()); int eventCount = 0; - for (Record record: records) { - Event event = record.getData(); + for (KinesisInputOutputRecord kinesisInputOutputRecord: kinesisOutputRecords) { + Record dataPrepperRecord = kinesisInputOutputRecord.getDataPrepperRecord(); + long incomingRecordSizeBytes = kinesisInputOutputRecord.getIncomingRecordSizeBytes(); + bytesReceivedSummary.record(incomingRecordSizeBytes); + Event event = dataPrepperRecord.getData(); acknowledgementSetOpt.ifPresent(acknowledgementSet -> acknowledgementSet.add(event)); - bufferAccumulator.add(record); + bufferAccumulator.add(dataPrepperRecord); + bytesProcessedSummary.record(incomingRecordSizeBytes); eventCount++; } diff --git a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisInputOutputRecordTest.java b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisInputOutputRecordTest.java new file mode 100644 index 0000000000..643a57c0fe --- /dev/null +++ b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisInputOutputRecordTest.java @@ -0,0 +1,57 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + +package org.opensearch.dataprepper.plugins.kinesis.source; + +import org.junit.jupiter.api.Test; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.event.JacksonEvent; +import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.plugins.kinesis.source.converter.MetadataKeyAttributes; +import org.opensearch.dataprepper.plugins.kinesis.source.processor.KinesisInputOutputRecord; +import software.amazon.kinesis.retrieval.KinesisClientRecord; + +import java.nio.ByteBuffer; +import java.util.UUID; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertNull; + +public class KinesisInputOutputRecordTest { + + @Test + void builder_defaultCreatesObjectCorrectly() { + + KinesisInputOutputRecord kinesisInputOutputRecord = KinesisInputOutputRecord.builder().build(); + + assertEquals(0L, kinesisInputOutputRecord.getIncomingRecordSizeBytes()); + assertNull(kinesisInputOutputRecord.getDataPrepperRecord()); + } + + @Test + void builder_createsObjectCorrectly() { + Event event = JacksonEvent.fromMessage(UUID.randomUUID().toString()); + event.getMetadata().setAttribute(MetadataKeyAttributes.KINESIS_STREAM_NAME_METADATA_ATTRIBUTE, UUID.randomUUID().toString()); + Record record = new Record<>(event); + KinesisClientRecord kinesisClientRecord = KinesisClientRecord.builder() + .data(ByteBuffer.wrap(event.toJsonString().getBytes())) + .sequenceNumber(Integer.toString(1000)).subSequenceNumber(1).build(); + + KinesisInputOutputRecord kinesisInputOutputRecord = KinesisInputOutputRecord.builder() + .withIncomingRecordSizeBytes(100L) + .withDataPrepperRecord(record) + .build(); + + assertNotNull(kinesisInputOutputRecord.getDataPrepperRecord()); + assertEquals(kinesisInputOutputRecord.getDataPrepperRecord(), record); + assertEquals(100L, kinesisInputOutputRecord.getIncomingRecordSizeBytes()); + } +} diff --git a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/converter/KinesisRecordConverterTest.java b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/converter/KinesisRecordConverterTest.java index 95ecc10d41..0f9081455e 100644 --- a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/converter/KinesisRecordConverterTest.java +++ b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/converter/KinesisRecordConverterTest.java @@ -15,11 +15,10 @@ import org.opensearch.dataprepper.event.TestEventFactory; import org.opensearch.dataprepper.model.codec.DecompressionEngine; import org.opensearch.dataprepper.model.codec.InputCodec; -import org.opensearch.dataprepper.model.event.Event; -import org.opensearch.dataprepper.model.record.Record; import org.opensearch.dataprepper.plugins.codec.CompressionOption; import org.opensearch.dataprepper.plugins.codec.json.NdjsonInputCodec; import org.opensearch.dataprepper.plugins.codec.json.NdjsonInputConfig; +import org.opensearch.dataprepper.plugins.kinesis.source.processor.KinesisInputOutputRecord; import software.amazon.kinesis.retrieval.KinesisClientRecord; import java.io.ByteArrayInputStream; @@ -99,13 +98,13 @@ public void testRecordConverterWithNdJsonInputCodec() throws IOException { InputStream inputStream = new ByteArrayInputStream(writer.toString().getBytes()); when(decompressionEngine.createInputStream(any(InputStream.class))).thenReturn(inputStream); - List> events = kinesisRecordConverter.convert(decompressionEngine, List.of(kinesisClientRecord), streamId); + List kinesisOutputRecords = kinesisRecordConverter.convert(decompressionEngine, List.of(kinesisClientRecord), streamId); - assertEquals(events.size(), numRecords); - events.forEach(eventRecord -> { - assertEquals(eventRecord.getData().getMetadata().getAttribute(MetadataKeyAttributes.KINESIS_PARTITION_KEY_METADATA_ATTRIBUTE), partitionKey); - assertEquals(eventRecord.getData().getMetadata().getAttribute(MetadataKeyAttributes.KINESIS_SEQUENCE_NUMBER_METADATA_ATTRIBUTE), sequenceNumber); - assertEquals(eventRecord.getData().getMetadata().getAttribute(MetadataKeyAttributes.KINESIS_SUB_SEQUENCE_NUMBER_METADATA_ATTRIBUTE), subsequenceNumber); + assertEquals(kinesisOutputRecords.size(), numRecords); + kinesisOutputRecords.forEach(KinesisInputOutputRecord -> { + assertEquals(KinesisInputOutputRecord.getDataPrepperRecord().getData().getMetadata().getAttribute(MetadataKeyAttributes.KINESIS_PARTITION_KEY_METADATA_ATTRIBUTE), partitionKey); + assertEquals(KinesisInputOutputRecord.getDataPrepperRecord().getData().getMetadata().getAttribute(MetadataKeyAttributes.KINESIS_SEQUENCE_NUMBER_METADATA_ATTRIBUTE), sequenceNumber); + assertEquals(KinesisInputOutputRecord.getDataPrepperRecord().getData().getMetadata().getAttribute(MetadataKeyAttributes.KINESIS_SUB_SEQUENCE_NUMBER_METADATA_ATTRIBUTE), subsequenceNumber); }); } diff --git a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisRecordProcessorTest.java b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisRecordProcessorTest.java index e044676799..b5037f2ad3 100644 --- a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisRecordProcessorTest.java +++ b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisRecordProcessorTest.java @@ -11,6 +11,7 @@ package org.opensearch.dataprepper.plugins.kinesis.source.processor; import io.micrometer.core.instrument.Counter; +import io.micrometer.core.instrument.DistributionSummary; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; @@ -75,8 +76,10 @@ import static org.opensearch.dataprepper.plugins.kinesis.source.processor.KinesisRecordProcessor.ACKNOWLEDGEMENT_SET_FAILURES_METRIC_NAME; import static org.opensearch.dataprepper.plugins.kinesis.source.processor.KinesisRecordProcessor.ACKNOWLEDGEMENT_SET_SUCCESS_METRIC_NAME; import static org.opensearch.dataprepper.plugins.kinesis.source.processor.KinesisRecordProcessor.KINESIS_CHECKPOINT_FAILURES; -import static org.opensearch.dataprepper.plugins.kinesis.source.processor.KinesisRecordProcessor.KINESIS_RECORD_PROCESSED; -import static org.opensearch.dataprepper.plugins.kinesis.source.processor.KinesisRecordProcessor.KINESIS_RECORD_PROCESSING_ERRORS; +import static org.opensearch.dataprepper.plugins.kinesis.source.processor.KinesisRecordProcessor.KINESIS_RECORD_BYTES_PROCESSED_METRIC_NAME; +import static org.opensearch.dataprepper.plugins.kinesis.source.processor.KinesisRecordProcessor.KINESIS_RECORD_BYTES_RECEIVED_METRIC_NAME; +import static org.opensearch.dataprepper.plugins.kinesis.source.processor.KinesisRecordProcessor.KINESIS_RECORD_PROCESSED_METRIC_NAME; +import static org.opensearch.dataprepper.plugins.kinesis.source.processor.KinesisRecordProcessor.KINESIS_RECORD_PROCESSING_ERRORS_METRIC_NAME; import static org.opensearch.dataprepper.plugins.kinesis.source.processor.KinesisRecordProcessor.KINESIS_STREAM_TAG_KEY; public class KinesisRecordProcessorTest { @@ -126,6 +129,12 @@ public class KinesisRecordProcessorTest { @Mock private Counter recordProcessingErrors; + @Mock + private DistributionSummary bytesReceivedSummary; + + @Mock + private DistributionSummary bytesProcessedSummary; + @Mock private Counter checkpointFailures; @@ -187,26 +196,45 @@ public void setup() throws IOException { when(compressionOption.getDecompressionEngine()).thenReturn(decompressionEngine); recordProcessed = mock(Counter.class); - when(pluginMetrics.counterWithTags(KINESIS_RECORD_PROCESSED, KINESIS_STREAM_TAG_KEY, streamIdentifier.streamName())).thenReturn(recordProcessed); + when(pluginMetrics.counterWithTags(KINESIS_RECORD_PROCESSED_METRIC_NAME, KINESIS_STREAM_TAG_KEY, streamIdentifier.streamName())).thenReturn(recordProcessed); recordProcessingErrors = mock(Counter.class); - when(pluginMetrics.counterWithTags(KINESIS_RECORD_PROCESSING_ERRORS, KINESIS_STREAM_TAG_KEY, streamIdentifier.streamName())).thenReturn(recordProcessingErrors); + when(pluginMetrics.counterWithTags(KINESIS_RECORD_PROCESSING_ERRORS_METRIC_NAME, KINESIS_STREAM_TAG_KEY, streamIdentifier.streamName())).thenReturn(recordProcessingErrors); + + bytesReceivedSummary = mock(DistributionSummary.class); + when(pluginMetrics.summary(KINESIS_RECORD_BYTES_RECEIVED_METRIC_NAME)).thenReturn(bytesReceivedSummary); + + bytesProcessedSummary = mock(DistributionSummary.class); + when(pluginMetrics.summary(KINESIS_RECORD_BYTES_PROCESSED_METRIC_NAME)).thenReturn(bytesProcessedSummary); } @Test void testProcessRecordsWithoutAcknowledgementsWithCheckpointApplied() throws Exception { - List kinesisClientRecords = createInputKinesisClientRecords(); - when(processRecordsInput.records()).thenReturn(kinesisClientRecords); when(kinesisSourceConfig.isAcknowledgments()).thenReturn(false); when(kinesisStreamConfig.getCheckPointInterval()).thenReturn(Duration.ofMillis(0)); when(acknowledgementSetManager.create(any(), any(Duration.class))).thenReturn(acknowledgementSet); - List> records = new ArrayList<>(); + List records = new ArrayList<>(); Event event = JacksonEvent.fromMessage(UUID.randomUUID().toString()); event.getMetadata().setAttribute(MetadataKeyAttributes.KINESIS_STREAM_NAME_METADATA_ATTRIBUTE, streamId); Record record = new Record<>(event); - records.add(record); + KinesisClientRecord kinesisClientRecord = KinesisClientRecord.builder() + .data(ByteBuffer.wrap(event.toJsonString().getBytes())) + .sequenceNumber(Integer.toString(1000)).subSequenceNumber(1).build(); + List kinesisClientRecords = new ArrayList<>(); + kinesisClientRecords.add(kinesisClientRecord); + final long recordsSize = kinesisClientRecords.stream() + .map(kclRecord -> kclRecord.data().position()) + .mapToLong(Integer::longValue).sum(); + + records.add(KinesisInputOutputRecord.builder() + .withDataPrepperRecord(record) + .withIncomingRecordSizeBytes(recordsSize).build() + ); + when(processRecordsInput.records()).thenReturn(kinesisClientRecords); + + InputStream inputStream = mock(InputStream.class); when(decompressionEngine.createInputStream(inputStream)).thenReturn(inputStream); when(kinesisRecordConverter.convert(eq(decompressionEngine), eq(kinesisClientRecords), eq(streamId))).thenReturn(records); @@ -240,22 +268,35 @@ void testProcessRecordsWithoutAcknowledgementsWithCheckpointApplied() verify(acknowledgementSetManager, times(0)).create(any(), any(Duration.class)); verify(recordProcessed, times(1)).increment(anyDouble()); + verify(bytesReceivedSummary, times(1)).record(eq((double) recordsSize)); + verify(bytesProcessedSummary, times(1)).record(eq((double) recordsSize)); } @Test public void testProcessRecordsWithoutAcknowledgementsEnabled() throws Exception { - List kinesisClientRecords = createInputKinesisClientRecords(); - when(processRecordsInput.records()).thenReturn(kinesisClientRecords); when(kinesisSourceConfig.isAcknowledgments()).thenReturn(false); when(kinesisStreamConfig.getCheckPointInterval()).thenReturn(Duration.ofMillis(0)); when(acknowledgementSetManager.create(any(), any(Duration.class))).thenReturn(acknowledgementSet); - List> records = new ArrayList<>(); + List records = new ArrayList<>(); Event event = JacksonEvent.fromMessage(UUID.randomUUID().toString()); event.getMetadata().setAttribute(MetadataKeyAttributes.KINESIS_STREAM_NAME_METADATA_ATTRIBUTE, streamId); Record record = new Record<>(event); - records.add(record); + KinesisClientRecord kinesisClientRecord = KinesisClientRecord.builder() + .data(ByteBuffer.wrap(event.toJsonString().getBytes())) + .sequenceNumber(Integer.toString(1000)).subSequenceNumber(1).build(); + List kinesisClientRecords = new ArrayList<>(); + kinesisClientRecords.add(kinesisClientRecord); + final long recordsSize = kinesisClientRecords.stream() + .map(kclRecord -> kclRecord.data().position()) + .mapToLong(Integer::longValue).sum(); + records.add(KinesisInputOutputRecord.builder() + .withDataPrepperRecord(record) + .withIncomingRecordSizeBytes(recordsSize).build() + ); + + when(processRecordsInput.records()).thenReturn(kinesisClientRecords); when(kinesisRecordConverter.convert(eq(decompressionEngine), eq(kinesisClientRecords), eq(streamId))).thenReturn(records); kinesisRecordProcessor = new KinesisRecordProcessor(bufferAccumulator, kinesisSourceConfig, @@ -281,13 +322,13 @@ public void testProcessRecordsWithoutAcknowledgementsEnabled() verify(acknowledgementSetManager, times(0)).create(any(), any(Duration.class)); verify(recordProcessed, times(1)).increment(anyDouble()); + verify(bytesReceivedSummary, times(1)).record(eq((double) recordsSize)); + verify(bytesProcessedSummary, times(1)).record(eq((double) recordsSize)); } @Test void testProcessRecordsWithAcknowledgementsEnabled() throws Exception { - List kinesisClientRecords = createInputKinesisClientRecords(); - when(processRecordsInput.records()).thenReturn(kinesisClientRecords); when(kinesisSourceConfig.isAcknowledgments()).thenReturn(true); when(kinesisStreamConfig.getCheckPointInterval()).thenReturn(Duration.ofMillis(0)); AtomicReference numEventsAdded = new AtomicReference<>(0); @@ -302,11 +343,23 @@ void testProcessRecordsWithAcknowledgementsEnabled() return acknowledgementSet; }).when(acknowledgementSetManager).create(any(Consumer.class), any(Duration.class)); - List> records = new ArrayList<>(); + List records = new ArrayList<>(); Event event = JacksonEvent.fromMessage(UUID.randomUUID().toString()); event.getMetadata().setAttribute(MetadataKeyAttributes.KINESIS_STREAM_NAME_METADATA_ATTRIBUTE, streamId); Record record = new Record<>(event); - records.add(record); + KinesisClientRecord kinesisClientRecord = KinesisClientRecord.builder() + .data(ByteBuffer.wrap(event.toJsonString().getBytes())) + .sequenceNumber(Integer.toString(1000)).subSequenceNumber(1).build(); + List kinesisClientRecords = new ArrayList<>(); + when(processRecordsInput.records()).thenReturn(kinesisClientRecords); + kinesisClientRecords.add(kinesisClientRecord); + final long recordsSize = kinesisClientRecords.stream() + .map(kclRecord -> kclRecord.data().position()) + .mapToLong(Integer::longValue).sum(); + records.add(KinesisInputOutputRecord.builder() + .withDataPrepperRecord(record) + .withIncomingRecordSizeBytes(recordsSize).build() + ); when(kinesisRecordConverter.convert(eq(decompressionEngine), eq(kinesisClientRecords), eq(streamId))).thenReturn(records); kinesisRecordProcessor = new KinesisRecordProcessor(bufferAccumulator, kinesisSourceConfig, @@ -337,13 +390,13 @@ void testProcessRecordsWithAcknowledgementsEnabled() verify(acknowledgementSetSuccesses, atLeastOnce()).increment(); verify(recordProcessed, times(1)).increment(anyDouble()); verifyNoInteractions(recordProcessingErrors); + verify(bytesReceivedSummary, times(1)).record(eq((double) recordsSize)); + verify(bytesProcessedSummary, times(1)).record(eq((double) recordsSize)); } @Test void testProcessRecordsWithNDJsonInputCodec() throws Exception { - List kinesisClientRecords = createInputKinesisClientRecords(); - when(processRecordsInput.records()).thenReturn(kinesisClientRecords); when(kinesisSourceConfig.isAcknowledgments()).thenReturn(false); when(kinesisStreamConfig.getCheckPointInterval()).thenReturn(Duration.ofMillis(0)); @@ -357,11 +410,23 @@ void testProcessRecordsWithNDJsonInputCodec() when(acknowledgementSetManager.create(any(), any(Duration.class))).thenReturn(acknowledgementSet); - List> records = new ArrayList<>(); + List records = new ArrayList<>(); Event event = JacksonEvent.fromMessage(UUID.randomUUID().toString()); event.getMetadata().setAttribute(MetadataKeyAttributes.KINESIS_STREAM_NAME_METADATA_ATTRIBUTE, streamId); Record record = new Record<>(event); - records.add(record); + KinesisClientRecord kinesisClientRecord = KinesisClientRecord.builder() + .data(ByteBuffer.wrap(event.toJsonString().getBytes())) + .sequenceNumber(Integer.toString(1000)).subSequenceNumber(1).build(); + List kinesisClientRecords = new ArrayList<>(); + when(processRecordsInput.records()).thenReturn(kinesisClientRecords); + kinesisClientRecords.add(kinesisClientRecord); + final long recordsSize = kinesisClientRecords.stream() + .map(kclRecord -> kclRecord.data().position()) + .mapToLong(Integer::longValue).sum(); + records.add(KinesisInputOutputRecord.builder() + .withDataPrepperRecord(record) + .withIncomingRecordSizeBytes(recordsSize).build() + ); when(kinesisRecordConverter.convert(eq(decompressionEngine), eq(kinesisClientRecords), eq(streamId))).thenReturn(records); kinesisRecordProcessor = new KinesisRecordProcessor(bufferAccumulator, kinesisSourceConfig, @@ -391,20 +456,32 @@ void testProcessRecordsWithNDJsonInputCodec() verify(acknowledgementSetManager, times(0)).create(any(), any(Duration.class)); verify(recordProcessed, times(1)).increment(anyDouble()); + verify(bytesReceivedSummary, times(1)).record(eq((double) recordsSize)); + verify(bytesProcessedSummary, times(1)).record(eq((double) recordsSize)); } @Test void testProcessRecordsNoThrowException() throws Exception { - List kinesisClientRecords = createInputKinesisClientRecords(); - when(processRecordsInput.records()).thenReturn(kinesisClientRecords); when(kinesisSourceConfig.isAcknowledgments()).thenReturn(false); - List> records = new ArrayList<>(); + List records = new ArrayList<>(); Event event = JacksonEvent.fromMessage(UUID.randomUUID().toString()); event.getMetadata().setAttribute(MetadataKeyAttributes.KINESIS_STREAM_NAME_METADATA_ATTRIBUTE, streamId); Record record = new Record<>(event); - records.add(record); + List kinesisClientRecords = new ArrayList<>(); + when(processRecordsInput.records()).thenReturn(kinesisClientRecords); + KinesisClientRecord kinesisClientRecord = KinesisClientRecord.builder() + .data(ByteBuffer.wrap(event.toJsonString().getBytes())) + .sequenceNumber(Integer.toString(1000)).subSequenceNumber(1).build(); + kinesisClientRecords.add(kinesisClientRecord); + final long recordsSize = kinesisClientRecords.stream() + .map(kclRecord -> kclRecord.data().position()) + .mapToLong(Integer::longValue).sum(); + records.add(KinesisInputOutputRecord.builder() + .withDataPrepperRecord(record) + .withIncomingRecordSizeBytes(recordsSize).build() + ); when(kinesisRecordConverter.convert(eq(decompressionEngine), eq(kinesisClientRecords), eq(streamId))).thenReturn(records); final Throwable exception = mock(RuntimeException.class); doThrow(exception).when(bufferAccumulator).add(any(Record.class)); @@ -416,20 +493,33 @@ void testProcessRecordsNoThrowException() assertDoesNotThrow(() -> kinesisRecordProcessor.processRecords(processRecordsInput)); verify(recordProcessingErrors, times(1)).increment(); verify(recordProcessed, times(0)).increment(anyDouble()); + verify(bytesReceivedSummary, times(1)).record(eq((double) recordsSize)); + verify(bytesProcessedSummary, times(0)).record(eq((double) recordsSize)); } @Test void testProcessRecordsBufferFlushNoThrowException() throws Exception { - List kinesisClientRecords = createInputKinesisClientRecords(); - when(processRecordsInput.records()).thenReturn(kinesisClientRecords); when(kinesisSourceConfig.isAcknowledgments()).thenReturn(false); - List> records = new ArrayList<>(); + List records = new ArrayList<>(); Event event = JacksonEvent.fromMessage(UUID.randomUUID().toString()); event.getMetadata().setAttribute(MetadataKeyAttributes.KINESIS_STREAM_NAME_METADATA_ATTRIBUTE, streamId); Record record = new Record<>(event); - records.add(record); + KinesisClientRecord kinesisClientRecord = KinesisClientRecord.builder() + .data(ByteBuffer.wrap(event.toJsonString().getBytes())) + .sequenceNumber(Integer.toString(1000)).subSequenceNumber(1).build(); + List kinesisClientRecords = new ArrayList<>(); + when(processRecordsInput.records()).thenReturn(kinesisClientRecords); + kinesisClientRecords.add(kinesisClientRecord); + final long recordsSize = kinesisClientRecords.stream() + .map(kclRecord -> kclRecord.data().position()) + .mapToLong(Integer::longValue).sum(); + records.add(KinesisInputOutputRecord.builder() + .withDataPrepperRecord(record) + .withIncomingRecordSizeBytes(recordsSize).build() + ); + when(kinesisRecordConverter.convert(eq(decompressionEngine), eq(kinesisClientRecords), eq(streamId))).thenReturn(records); final Throwable exception = mock(RuntimeException.class); doThrow(exception).when(bufferAccumulator).flush(); @@ -441,7 +531,8 @@ void testProcessRecordsBufferFlushNoThrowException() assertDoesNotThrow(() -> kinesisRecordProcessor.processRecords(processRecordsInput)); verify(recordProcessingErrors, times(1)).increment(); verify(recordProcessed, times(0)).increment(anyDouble()); - + verify(bytesReceivedSummary, times(1)).record(eq((double) recordsSize)); + verify(bytesProcessedSummary, times(1)).record(eq((double) recordsSize)); } @Test diff --git a/data-prepper-plugins/mongodb/src/main/java/org/opensearch/dataprepper/plugins/mongo/stream/StreamAcknowledgementManager.java b/data-prepper-plugins/mongodb/src/main/java/org/opensearch/dataprepper/plugins/mongo/stream/StreamAcknowledgementManager.java index 38d4fc9794..b8e3026fee 100644 --- a/data-prepper-plugins/mongodb/src/main/java/org/opensearch/dataprepper/plugins/mongo/stream/StreamAcknowledgementManager.java +++ b/data-prepper-plugins/mongodb/src/main/java/org/opensearch/dataprepper/plugins/mongo/stream/StreamAcknowledgementManager.java @@ -18,9 +18,6 @@ import java.util.concurrent.Executors; import java.util.function.Consumer; -import static org.opensearch.dataprepper.logging.DataPrepperMarkers.NOISY; - - public class StreamAcknowledgementManager { private static final Logger LOG = LoggerFactory.getLogger(StreamAcknowledgementManager.class); private static final int CHECKPOINT_RECORD_INTERVAL = 50; @@ -111,7 +108,6 @@ private void monitorAcknowledgment(final ExecutorService executorService, final } } else { if (System.currentTimeMillis() - lastCheckpointTime >= checkPointIntervalInMs) { - LOG.info(NOISY, "No records processed. Extend the lease of the partition worker."); partitionCheckpoint.extendLease(); this.noDataExtendLeaseCount.increment(); lastCheckpointTime = System.currentTimeMillis(); diff --git a/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/CopyValueProcessorConfig.java b/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/CopyValueProcessorConfig.java index c7ec7683ef..e44c1da74e 100644 --- a/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/CopyValueProcessorConfig.java +++ b/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/CopyValueProcessorConfig.java @@ -30,13 +30,15 @@ public static class Entry { @NotEmpty @NotNull @JsonProperty("from_key") - @JsonPropertyDescription("The key of the entry to be copied.") + @JsonPropertyDescription("The key of the entry to be copied. Either from_key and " + + "to_key or from_list and to_list must be defined.") private String fromKey; @NotEmpty @NotNull @JsonProperty("to_key") - @JsonPropertyDescription("The key of the new entry to be added.") + @JsonPropertyDescription("The key of the new entry to be added. Either from_key and " + + "to_key or from_list and to_list must be defined.") private String toKey; @JsonProperty("overwrite_if_to_key_exists") @@ -86,14 +88,16 @@ public Entry() { private List entries; @JsonProperty(FROM_LIST_KEY) - @JsonPropertyDescription("The key of the list of objects to be copied. to_list must also be defined.") + @JsonPropertyDescription("The key of the list of objects to be copied. Either from_key and " + + "to_key or from_list and to_list must be defined.") @AlsoRequired(values = { @AlsoRequired.Required(name = TO_LIST_KEY) }) private String fromList; @JsonProperty(TO_LIST_KEY) - @JsonPropertyDescription("The key of the new list to be added. from_list must also be defined.") + @JsonPropertyDescription("The key of the new list to be added. Either from_key and " + + "to_key or from_list and to_list must be defined.") @AlsoRequired(values = { @AlsoRequired.Required(name = FROM_LIST_KEY) }) diff --git a/data-prepper-plugins/obfuscate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/obfuscation/ObfuscationProcessorConfig.java b/data-prepper-plugins/obfuscate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/obfuscation/ObfuscationProcessorConfig.java index c8a5ef4e07..fa0ddf355c 100644 --- a/data-prepper-plugins/obfuscate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/obfuscation/ObfuscationProcessorConfig.java +++ b/data-prepper-plugins/obfuscate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/obfuscation/ObfuscationProcessorConfig.java @@ -22,7 +22,7 @@ import java.util.List; @JsonPropertyOrder -@JsonClassDescription("The obfuscate process enables obfuscation of fields inside your documents in order to " + +@JsonClassDescription("The obfuscate processor enables obfuscation of fields inside your documents in order to " + "protect sensitive data.") public class ObfuscationProcessorConfig { diff --git a/data-prepper-plugins/opensearch/build.gradle b/data-prepper-plugins/opensearch/build.gradle index 5e7879d8d1..1cd2f54335 100644 --- a/data-prepper-plugins/opensearch/build.gradle +++ b/data-prepper-plugins/opensearch/build.gradle @@ -36,13 +36,13 @@ dependencies { implementation 'software.amazon.awssdk:apache-client' implementation 'software.amazon.awssdk:netty-nio-client' implementation 'co.elastic.clients:elasticsearch-java:7.17.0' - implementation('org.apache.maven:maven-artifact:3.9.8') { + implementation('org.apache.maven:maven-artifact:3.9.9') { exclude group: 'org.codehaus.plexus' } testImplementation testLibs.junit.vintage testImplementation libs.commons.io - testImplementation 'net.bytebuddy:byte-buddy:1.14.17' - testImplementation 'net.bytebuddy:byte-buddy-agent:1.14.17' + testImplementation 'net.bytebuddy:byte-buddy:1.15.11' + testImplementation 'net.bytebuddy:byte-buddy-agent:1.15.11' testImplementation testLibs.slf4j.simple } diff --git a/data-prepper-plugins/otel-logs-source/build.gradle b/data-prepper-plugins/otel-logs-source/build.gradle index 822e945ba9..b1bf2857e9 100644 --- a/data-prepper-plugins/otel-logs-source/build.gradle +++ b/data-prepper-plugins/otel-logs-source/build.gradle @@ -30,7 +30,7 @@ dependencies { implementation libs.commons.lang3 implementation libs.bouncycastle.bcprov implementation libs.bouncycastle.bcpkix - testImplementation 'org.assertj:assertj-core:3.25.3' + testImplementation 'org.assertj:assertj-core:3.27.0' testImplementation libs.commons.io } diff --git a/data-prepper-plugins/otel-metrics-raw-processor/build.gradle b/data-prepper-plugins/otel-metrics-raw-processor/build.gradle index a4316fca16..261b8b29b4 100644 --- a/data-prepper-plugins/otel-metrics-raw-processor/build.gradle +++ b/data-prepper-plugins/otel-metrics-raw-processor/build.gradle @@ -21,7 +21,7 @@ dependencies { implementation 'com.fasterxml.jackson.core:jackson-databind' implementation 'com.fasterxml.jackson.dataformat:jackson-dataformat-yaml' implementation libs.guava.core - testImplementation 'org.assertj:assertj-core:3.25.3' + testImplementation 'org.assertj:assertj-core:3.27.0' } jacocoTestCoverageVerification { diff --git a/data-prepper-plugins/otel-metrics-source/build.gradle b/data-prepper-plugins/otel-metrics-source/build.gradle index 96d250d67d..6c5a95d795 100644 --- a/data-prepper-plugins/otel-metrics-source/build.gradle +++ b/data-prepper-plugins/otel-metrics-source/build.gradle @@ -30,7 +30,7 @@ dependencies { implementation libs.commons.lang3 implementation libs.bouncycastle.bcprov implementation libs.bouncycastle.bcpkix - testImplementation 'org.assertj:assertj-core:3.25.3' + testImplementation 'org.assertj:assertj-core:3.27.0' testImplementation libs.commons.io } diff --git a/data-prepper-plugins/otel-proto-common/build.gradle b/data-prepper-plugins/otel-proto-common/build.gradle index 03bafff809..657d9a8bd9 100644 --- a/data-prepper-plugins/otel-proto-common/build.gradle +++ b/data-prepper-plugins/otel-proto-common/build.gradle @@ -15,5 +15,5 @@ dependencies { implementation 'com.fasterxml.jackson.dataformat:jackson-dataformat-yaml' implementation libs.commons.lang3 implementation libs.commons.codec - testImplementation 'org.assertj:assertj-core:3.25.3' + testImplementation 'org.assertj:assertj-core:3.27.0' } diff --git a/data-prepper-plugins/otel-trace-source/build.gradle b/data-prepper-plugins/otel-trace-source/build.gradle index 3e86224f4d..4685b3212a 100644 --- a/data-prepper-plugins/otel-trace-source/build.gradle +++ b/data-prepper-plugins/otel-trace-source/build.gradle @@ -28,7 +28,7 @@ dependencies { implementation libs.commons.lang3 implementation libs.bouncycastle.bcprov implementation libs.bouncycastle.bcpkix - testImplementation 'org.assertj:assertj-core:3.25.3' + testImplementation 'org.assertj:assertj-core:3.27.0' testImplementation testLibs.slf4j.simple testImplementation libs.commons.io testImplementation 'com.jayway.jsonpath:json-path-assert:2.6.0' diff --git a/data-prepper-plugins/parquet-codecs/build.gradle b/data-prepper-plugins/parquet-codecs/build.gradle index c402fb6741..4c3b252fb9 100644 --- a/data-prepper-plugins/parquet-codecs/build.gradle +++ b/data-prepper-plugins/parquet-codecs/build.gradle @@ -7,7 +7,7 @@ dependencies { implementation project(':data-prepper-api') implementation project(':data-prepper-plugins:common') implementation libs.avro.core - implementation 'org.apache.commons:commons-text:1.11.0' + implementation 'org.apache.commons:commons-text:1.13.0' implementation libs.parquet.avro implementation libs.parquet.column implementation libs.parquet.common diff --git a/data-prepper-plugins/s3-sink/build.gradle b/data-prepper-plugins/s3-sink/build.gradle index 8dd6b5783d..e298b6cbd7 100644 --- a/data-prepper-plugins/s3-sink/build.gradle +++ b/data-prepper-plugins/s3-sink/build.gradle @@ -11,7 +11,7 @@ dependencies { implementation 'com.fasterxml.jackson.core:jackson-core' implementation 'com.fasterxml.jackson.core:jackson-databind' implementation libs.commons.compress - implementation 'joda-time:joda-time:2.12.7' + implementation 'joda-time:joda-time:2.13.0' implementation 'org.hibernate.validator:hibernate-validator:8.0.1.Final' implementation 'com.fasterxml.jackson.dataformat:jackson-dataformat-csv' implementation 'software.amazon.awssdk:netty-nio-client' diff --git a/data-prepper-plugins/s3-source/build.gradle b/data-prepper-plugins/s3-source/build.gradle index 1a3f213237..f4afbfbfe3 100644 --- a/data-prepper-plugins/s3-source/build.gradle +++ b/data-prepper-plugins/s3-source/build.gradle @@ -22,7 +22,7 @@ dependencies { implementation 'com.fasterxml.jackson.core:jackson-databind' implementation libs.commons.io implementation libs.commons.compress - implementation 'joda-time:joda-time:2.12.7' + implementation 'joda-time:joda-time:2.13.0' implementation 'org.hibernate.validator:hibernate-validator:8.0.1.Final' implementation 'com.fasterxml.jackson.dataformat:jackson-dataformat-csv' implementation 'com.fasterxml.jackson.datatype:jackson-datatype-jsr310' @@ -31,7 +31,7 @@ dependencies { implementation 'dev.failsafe:failsafe:3.3.2' implementation 'org.apache.httpcomponents:httpcore:4.4.16' testImplementation libs.commons.lang3 - testImplementation 'org.wiremock:wiremock:3.8.0' + testImplementation 'org.wiremock:wiremock:3.10.0' testImplementation 'org.eclipse.jetty:jetty-bom:11.0.20' testImplementation 'com.fasterxml.jackson.dataformat:jackson-dataformat-yaml' testImplementation testLibs.junit.vintage diff --git a/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/JiraClient.java b/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/JiraClient.java index 7e463f4534..3c809fa798 100644 --- a/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/JiraClient.java +++ b/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/JiraClient.java @@ -1,3 +1,13 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + package org.opensearch.dataprepper.plugins.source.jira; import com.fasterxml.jackson.core.JsonProcessingException; diff --git a/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/JiraItemInfo.java b/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/JiraItemInfo.java index 0952f20a2a..349ab7bccb 100644 --- a/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/JiraItemInfo.java +++ b/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/JiraItemInfo.java @@ -1,3 +1,13 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + package org.opensearch.dataprepper.plugins.source.jira; import lombok.Getter; diff --git a/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/JiraIterator.java b/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/JiraIterator.java index 3027b9ec99..5ae040cc7f 100644 --- a/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/JiraIterator.java +++ b/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/JiraIterator.java @@ -1,3 +1,13 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + package org.opensearch.dataprepper.plugins.source.jira; diff --git a/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/JiraService.java b/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/JiraService.java index 38eb9eed40..4600d1bdeb 100644 --- a/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/JiraService.java +++ b/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/JiraService.java @@ -1,3 +1,13 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + package org.opensearch.dataprepper.plugins.source.jira; import io.micrometer.core.instrument.Counter; @@ -14,8 +24,10 @@ import javax.inject.Named; import java.time.Instant; import java.util.ArrayList; +import java.util.HashSet; import java.util.List; import java.util.Queue; +import java.util.Set; import java.util.regex.Matcher; import java.util.regex.Pattern; import java.util.stream.Collectors; @@ -25,9 +37,12 @@ import static org.opensearch.dataprepper.plugins.source.jira.utils.JqlConstants.DELIMITER; import static org.opensearch.dataprepper.plugins.source.jira.utils.JqlConstants.GREATER_THAN_EQUALS; import static org.opensearch.dataprepper.plugins.source.jira.utils.JqlConstants.ISSUE_TYPE_IN; +import static org.opensearch.dataprepper.plugins.source.jira.utils.JqlConstants.ISSUE_TYPE_NOT_IN; import static org.opensearch.dataprepper.plugins.source.jira.utils.JqlConstants.PREFIX; import static org.opensearch.dataprepper.plugins.source.jira.utils.JqlConstants.PROJECT_IN; +import static org.opensearch.dataprepper.plugins.source.jira.utils.JqlConstants.PROJECT_NOT_IN; import static org.opensearch.dataprepper.plugins.source.jira.utils.JqlConstants.STATUS_IN; +import static org.opensearch.dataprepper.plugins.source.jira.utils.JqlConstants.STATUS_NOT_IN; import static org.opensearch.dataprepper.plugins.source.jira.utils.JqlConstants.SUFFIX; @@ -117,26 +132,41 @@ private void addItemsToQueue(List issueList, Queue itemInfo private StringBuilder createIssueFilterCriteria(JiraSourceConfig configuration, Instant ts) { log.info("Creating issue filter criteria"); - if (!CollectionUtils.isEmpty(JiraConfigHelper.getProjectKeyFilter(configuration))) { + if (!CollectionUtils.isEmpty(JiraConfigHelper.getProjectNameIncludeFilter(configuration)) || !CollectionUtils.isEmpty(JiraConfigHelper.getProjectNameExcludeFilter(configuration)) ) { validateProjectFilters(configuration); } StringBuilder jiraQl = new StringBuilder(UPDATED + GREATER_THAN_EQUALS + ts.toEpochMilli()); - if (!CollectionUtils.isEmpty(JiraConfigHelper.getProjectKeyFilter(configuration))) { - jiraQl.append(PROJECT_IN).append(JiraConfigHelper.getProjectKeyFilter(configuration).stream() + if (!CollectionUtils.isEmpty(JiraConfigHelper.getProjectNameIncludeFilter(configuration))) { + jiraQl.append(PROJECT_IN).append(JiraConfigHelper.getProjectNameIncludeFilter(configuration).stream() + .collect(Collectors.joining(DELIMITER, PREFIX, SUFFIX))) + .append(CLOSING_ROUND_BRACKET); + } + if (!CollectionUtils.isEmpty(JiraConfigHelper.getProjectNameExcludeFilter(configuration))) { + jiraQl.append(PROJECT_NOT_IN).append(JiraConfigHelper.getProjectNameExcludeFilter(configuration).stream() + .collect(Collectors.joining(DELIMITER, PREFIX, SUFFIX))) + .append(CLOSING_ROUND_BRACKET); + } + if (!CollectionUtils.isEmpty(JiraConfigHelper.getIssueTypeIncludeFilter(configuration))) { + jiraQl.append(ISSUE_TYPE_IN).append(JiraConfigHelper.getIssueTypeIncludeFilter(configuration).stream() + .collect(Collectors.joining(DELIMITER, PREFIX, SUFFIX))) + .append(CLOSING_ROUND_BRACKET); + } + if (!CollectionUtils.isEmpty(JiraConfigHelper.getIssueTypeExcludeFilter(configuration))) { + jiraQl.append(ISSUE_TYPE_NOT_IN).append(JiraConfigHelper.getIssueTypeExcludeFilter(configuration).stream() .collect(Collectors.joining(DELIMITER, PREFIX, SUFFIX))) .append(CLOSING_ROUND_BRACKET); } - if (!CollectionUtils.isEmpty(JiraConfigHelper.getIssueTypeFilter(configuration))) { - jiraQl.append(ISSUE_TYPE_IN).append(JiraConfigHelper.getIssueTypeFilter(configuration).stream() + if (!CollectionUtils.isEmpty(JiraConfigHelper.getIssueStatusIncludeFilter(configuration))) { + jiraQl.append(STATUS_IN).append(JiraConfigHelper.getIssueStatusIncludeFilter(configuration).stream() .collect(Collectors.joining(DELIMITER, PREFIX, SUFFIX))) .append(CLOSING_ROUND_BRACKET); } - if (!CollectionUtils.isEmpty(JiraConfigHelper.getIssueStatusFilter(configuration))) { - jiraQl.append(STATUS_IN).append(JiraConfigHelper.getIssueStatusFilter(configuration).stream() + if (!CollectionUtils.isEmpty(JiraConfigHelper.getIssueStatusExcludeFilter(configuration))) { + jiraQl.append(STATUS_NOT_IN).append(JiraConfigHelper.getIssueStatusExcludeFilter(configuration).stream() .collect(Collectors.joining(DELIMITER, PREFIX, SUFFIX))) .append(CLOSING_ROUND_BRACKET); } - log.trace("Created issue filter criteria JiraQl query: {}", jiraQl); + log.error("Created issue filter criteria JiraQl query: {}", jiraQl); return jiraQl; } @@ -148,9 +178,21 @@ private StringBuilder createIssueFilterCriteria(JiraSourceConfig configuration, private void validateProjectFilters(JiraSourceConfig configuration) { log.trace("Validating project filters"); List badFilters = new ArrayList<>(); + Set includedProjects = new HashSet<>(); + List includedAndExcludedProjects = new ArrayList<>(); Pattern regex = Pattern.compile("[^A-Z0-9]"); - JiraConfigHelper.getProjectKeyFilter(configuration).forEach(projectFilter -> { + JiraConfigHelper.getProjectNameIncludeFilter(configuration).forEach(projectFilter -> { Matcher matcher = regex.matcher(projectFilter); + includedProjects.add(projectFilter); + if (matcher.find() || projectFilter.length() <= 1 || projectFilter.length() > 10) { + badFilters.add(projectFilter); + } + }); + JiraConfigHelper.getProjectNameExcludeFilter(configuration).forEach(projectFilter -> { + Matcher matcher = regex.matcher(projectFilter); + if (includedProjects.contains(projectFilter)) { + includedAndExcludedProjects.add(projectFilter); + } if (matcher.find() || projectFilter.length() <= 1 || projectFilter.length() > 10) { badFilters.add(projectFilter); } @@ -162,6 +204,14 @@ private void validateProjectFilters(JiraSourceConfig configuration) { "Invalid project key found in filter configuration for " + filters); } + if (!includedAndExcludedProjects.isEmpty()) { + String filters = String.join("\"" + includedAndExcludedProjects + "\"", ", "); + log.error("One or more project keys found in both include and exclude: {}", includedAndExcludedProjects); + throw new BadRequestException("Bad request exception occurred " + + "Project filters is invalid because the following projects are listed in both include and exclude" + + filters); + } + } } \ No newline at end of file diff --git a/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/JiraSource.java b/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/JiraSource.java index 2641ab60ad..2ffc7b3b53 100644 --- a/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/JiraSource.java +++ b/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/JiraSource.java @@ -1,3 +1,13 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + package org.opensearch.dataprepper.plugins.source.jira; diff --git a/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/JiraSourceConfig.java b/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/JiraSourceConfig.java index f37434cf9d..df5cd70f0b 100644 --- a/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/JiraSourceConfig.java +++ b/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/JiraSourceConfig.java @@ -1,16 +1,24 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + package org.opensearch.dataprepper.plugins.source.jira; import com.fasterxml.jackson.annotation.JsonProperty; -import jakarta.validation.constraints.Size; +import jakarta.validation.Valid; import lombok.Getter; +import org.opensearch.dataprepper.plugins.source.jira.configuration.AuthenticationConfig; +import org.opensearch.dataprepper.plugins.source.jira.configuration.FilterConfig; import org.opensearch.dataprepper.plugins.source.source_crawler.base.CrawlerSourceConfig; import java.time.Duration; -import java.util.ArrayList; import java.util.List; -import java.util.Map; - -import static org.opensearch.dataprepper.plugins.source.jira.utils.Constants.OAUTH2; @Getter public class JiraSourceConfig implements CrawlerSourceConfig { @@ -20,50 +28,23 @@ public class JiraSourceConfig implements CrawlerSourceConfig { /** * Jira account url */ - @JsonProperty("account_url") - private String accountUrl; + @JsonProperty("hosts") + private List hosts; /** - * A map of connector credentials specific to this source + * Authentication Config to Access Jira */ - @JsonProperty("connector_credentials") - private Map connectorCredentials; + @JsonProperty("authentication") + @Valid + private AuthenticationConfig authenticationConfig; - /** - * List of projects to ingest - */ - @JsonProperty("projects") - @Size(max = 1000, message = "Project type filter should not be more than 1000") - private List project = new ArrayList<>(); /** - * List of specific issue types to ingest. - * Ex: Story, Epic, Task etc + * Filter Config to filter what tickets get ingested */ - @JsonProperty("issue_types") - @Size(max = 1000, message = "Issue type filter should be less than 1000") - private List issueType = new ArrayList<>(); + @JsonProperty("filter") + private FilterConfig filterConfig; - /** - * Optional Inclusion patterns for filtering some tickets - */ - @JsonProperty("inclusion_patterns") - @Size(max = 100, message = "inclusion pattern filters should not be more than 1000") - private List inclusionPatterns; - - /** - * Optional Exclusion patterns for excluding some tickets - */ - @JsonProperty("exclusion_patterns") - @Size(max = 1000, message = "exclusion pattern filter should be less than 1000") - private List exclusionPatterns; - - /** - * Optional Status filter to ingest the tickets - */ - @JsonProperty("statuses") - @Size(max = 1000, message = "Status filter should be less than 1000") - private List status = new ArrayList<>(); /** * Number of worker threads to spawn to parallel source fetching @@ -78,43 +59,11 @@ public class JiraSourceConfig implements CrawlerSourceConfig { @JsonProperty("backoff_time") private Duration backOff = DEFAULT_BACKOFF_MILLIS; - public String getJiraId() { - return this.getConnectorCredentials().get("jira_id"); - } - - public String getJiraCredential() { - return this.getConnectorCredentials().get("jira_credential"); + public String getAccountUrl() { + return this.getHosts().get(0); } public String getAuthType() { - return this.getConnectorCredentials().get("auth_type"); - } - - public String getAccessToken() { - return fetchGivenOAuthAttribute("access_token"); - } - - public String getRefreshToken() { - return fetchGivenOAuthAttribute("refresh_token"); - } - - public String getClientId() { - return fetchGivenOAuthAttribute("client_id"); - } - - public String getClientSecret() { - return fetchGivenOAuthAttribute("client_secret"); + return this.getAuthenticationConfig().getAuthType(); } - - private String fetchGivenOAuthAttribute(String givenAttribute) { - if (!OAUTH2.equals(getAuthType())) { - throw new RuntimeException("Authentication Type is not OAuth2."); - } - String attributeValue = this.getConnectorCredentials().get(givenAttribute); - if (attributeValue == null || attributeValue.isEmpty()) { - throw new RuntimeException(String.format("%s is required for OAuth2 AuthType", givenAttribute)); - } - return attributeValue; - } - } diff --git a/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/configuration/AuthenticationConfig.java b/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/configuration/AuthenticationConfig.java new file mode 100644 index 0000000000..25cfd4185a --- /dev/null +++ b/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/configuration/AuthenticationConfig.java @@ -0,0 +1,45 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + +package org.opensearch.dataprepper.plugins.source.jira.configuration; + +import com.fasterxml.jackson.annotation.JsonProperty; +import jakarta.validation.Valid; +import jakarta.validation.constraints.AssertTrue; +import lombok.Getter; + +import static org.opensearch.dataprepper.plugins.source.jira.utils.Constants.BASIC; +import static org.opensearch.dataprepper.plugins.source.jira.utils.Constants.OAUTH2; + +@Getter +public class AuthenticationConfig { + @JsonProperty("basic") + @Valid + private BasicConfig basicConfig; + + @JsonProperty("oauth2") + @Valid + private Oauth2Config oauth2Config; + + @AssertTrue(message = "Authentication config should have either basic or oauth2") + private boolean isValidAuthenticationConfig() { + boolean hasBasic = basicConfig != null; + boolean hasOauth = oauth2Config != null; + return hasBasic ^ hasOauth; + } + + public String getAuthType() { + if (basicConfig != null) { + return BASIC; + } else { + return OAUTH2; + } + } +} diff --git a/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/configuration/BasicConfig.java b/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/configuration/BasicConfig.java new file mode 100644 index 0000000000..b3a261f13a --- /dev/null +++ b/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/configuration/BasicConfig.java @@ -0,0 +1,29 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + +package org.opensearch.dataprepper.plugins.source.jira.configuration; + +import com.fasterxml.jackson.annotation.JsonProperty; +import jakarta.validation.constraints.AssertTrue; +import lombok.Getter; + +@Getter +public class BasicConfig { + @JsonProperty("username") + private String username; + + @JsonProperty("password") + private String password; + + @AssertTrue(message = "Username and Password are both required for Basic Auth") + private boolean isBasicConfigValid() { + return username != null && password != null; + } +} diff --git a/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/configuration/FilterConfig.java b/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/configuration/FilterConfig.java new file mode 100644 index 0000000000..987fa545e7 --- /dev/null +++ b/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/configuration/FilterConfig.java @@ -0,0 +1,26 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + +package org.opensearch.dataprepper.plugins.source.jira.configuration; + +import com.fasterxml.jackson.annotation.JsonProperty; +import lombok.Getter; + +@Getter +public class FilterConfig { + @JsonProperty("project") + private ProjectConfig projectConfig; + + @JsonProperty("status") + private StatusConfig statusConfig; + + @JsonProperty("issue_type") + private IssueTypeConfig issueTypeConfig; +} diff --git a/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/configuration/IssueTypeConfig.java b/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/configuration/IssueTypeConfig.java new file mode 100644 index 0000000000..977d9a3a9e --- /dev/null +++ b/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/configuration/IssueTypeConfig.java @@ -0,0 +1,29 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + +package org.opensearch.dataprepper.plugins.source.jira.configuration; + +import com.fasterxml.jackson.annotation.JsonProperty; +import jakarta.validation.constraints.Size; +import lombok.Getter; + +import java.util.ArrayList; +import java.util.List; + +@Getter +public class IssueTypeConfig { + @JsonProperty("include") + @Size(max = 1000, message = "Issue type filter should not be more than 1000") + private List include = new ArrayList<>(); + + @JsonProperty("exclude") + @Size(max = 1000, message = "Issue type filter should not be more than 1000") + private List exclude = new ArrayList<>(); +} diff --git a/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/configuration/NameConfig.java b/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/configuration/NameConfig.java new file mode 100644 index 0000000000..3df833d072 --- /dev/null +++ b/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/configuration/NameConfig.java @@ -0,0 +1,30 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + + +package org.opensearch.dataprepper.plugins.source.jira.configuration; + +import com.fasterxml.jackson.annotation.JsonProperty; +import jakarta.validation.constraints.Size; +import lombok.Getter; + +import java.util.ArrayList; +import java.util.List; + +@Getter +public class NameConfig { + @JsonProperty("include") + @Size(max = 1000, message = "Project name type filter should not be more than 1000") + private List include = new ArrayList<>(); + + @JsonProperty("exclude") + @Size(max = 1000, message = "Project name type filter should not be more than 1000") + private List exclude = new ArrayList<>(); +} diff --git a/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/configuration/Oauth2Config.java b/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/configuration/Oauth2Config.java new file mode 100644 index 0000000000..3282e7b38f --- /dev/null +++ b/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/configuration/Oauth2Config.java @@ -0,0 +1,35 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + +package org.opensearch.dataprepper.plugins.source.jira.configuration; + +import com.fasterxml.jackson.annotation.JsonProperty; +import jakarta.validation.constraints.AssertTrue; +import lombok.Getter; + +@Getter +public class Oauth2Config { + @JsonProperty("client_id") + private String clientId; + + @JsonProperty("client_secret") + private String clientSecret; + + @JsonProperty("access_token") + private String accessToken; + + @JsonProperty("refresh_token") + private String refreshToken; + + @AssertTrue(message = "Client ID, Client Secret, Access Token, and Refresh Token are both required for Oauth2") + private boolean isOauth2ConfigValid() { + return clientId != null && clientSecret != null && accessToken != null && refreshToken != null; + } +} diff --git a/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/configuration/ProjectConfig.java b/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/configuration/ProjectConfig.java new file mode 100644 index 0000000000..19bec8ad05 --- /dev/null +++ b/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/configuration/ProjectConfig.java @@ -0,0 +1,20 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + +package org.opensearch.dataprepper.plugins.source.jira.configuration; + +import com.fasterxml.jackson.annotation.JsonProperty; +import lombok.Getter; + +@Getter +public class ProjectConfig { + @JsonProperty("key") + private NameConfig nameConfig; +} diff --git a/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/configuration/StatusConfig.java b/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/configuration/StatusConfig.java new file mode 100644 index 0000000000..288d138f31 --- /dev/null +++ b/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/configuration/StatusConfig.java @@ -0,0 +1,29 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + +package org.opensearch.dataprepper.plugins.source.jira.configuration; + +import com.fasterxml.jackson.annotation.JsonProperty; +import jakarta.validation.constraints.Size; +import lombok.Getter; + +import java.util.ArrayList; +import java.util.List; + +@Getter +public class StatusConfig { + @JsonProperty("include") + @Size(max = 1000, message = "status type filter should not be more than 1000") + private List include = new ArrayList<>(); + + @JsonProperty("exclude") + @Size(max = 1000, message = "status type filter should not be more than 1000") + private List exclude = new ArrayList<>(); +} diff --git a/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/exception/BadRequestException.java b/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/exception/BadRequestException.java index 589e8626ab..69c0cca638 100644 --- a/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/exception/BadRequestException.java +++ b/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/exception/BadRequestException.java @@ -1,3 +1,13 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + package org.opensearch.dataprepper.plugins.source.jira.exception; /** diff --git a/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/exception/UnAuthorizedException.java b/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/exception/UnAuthorizedException.java index 1efa30e032..441f966bd8 100644 --- a/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/exception/UnAuthorizedException.java +++ b/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/exception/UnAuthorizedException.java @@ -1,3 +1,13 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + package org.opensearch.dataprepper.plugins.source.jira.exception; /** diff --git a/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/models/IssueBean.java b/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/models/IssueBean.java index ec437ac25d..5f2769883f 100644 --- a/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/models/IssueBean.java +++ b/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/models/IssueBean.java @@ -1,3 +1,13 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + package org.opensearch.dataprepper.plugins.source.jira.models; import com.fasterxml.jackson.annotation.JsonIgnore; diff --git a/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/models/SearchResults.java b/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/models/SearchResults.java index 96bc445cdb..677967a340 100644 --- a/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/models/SearchResults.java +++ b/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/models/SearchResults.java @@ -1,3 +1,14 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + + package org.opensearch.dataprepper.plugins.source.jira.models; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/rest/BasicAuthInterceptor.java b/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/rest/BasicAuthInterceptor.java index 57dbb65f94..e3a1b992f0 100644 --- a/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/rest/BasicAuthInterceptor.java +++ b/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/rest/BasicAuthInterceptor.java @@ -1,3 +1,13 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + package org.opensearch.dataprepper.plugins.source.jira.rest; import org.opensearch.dataprepper.plugins.source.jira.JiraSourceConfig; @@ -17,8 +27,8 @@ public class BasicAuthInterceptor implements ClientHttpRequestInterceptor { private final String password; public BasicAuthInterceptor(JiraSourceConfig config) { - this.username = config.getJiraId(); - this.password = config.getJiraCredential(); + this.username = config.getAuthenticationConfig().getBasicConfig().getUsername(); + this.password = config.getAuthenticationConfig().getBasicConfig().getPassword();; } @Override diff --git a/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/rest/CustomRestTemplateConfig.java b/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/rest/CustomRestTemplateConfig.java index 231351bcee..70cd189267 100644 --- a/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/rest/CustomRestTemplateConfig.java +++ b/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/rest/CustomRestTemplateConfig.java @@ -1,3 +1,13 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + package org.opensearch.dataprepper.plugins.source.jira.rest; diff --git a/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/rest/JiraRestClient.java b/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/rest/JiraRestClient.java index 6b71a032b8..caf5d84ee7 100644 --- a/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/rest/JiraRestClient.java +++ b/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/rest/JiraRestClient.java @@ -1,3 +1,13 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + package org.opensearch.dataprepper.plugins.source.jira.rest; import com.google.common.annotations.VisibleForTesting; @@ -34,17 +44,20 @@ public class JiraRestClient { public static final String REST_API_SEARCH = "rest/api/3/search"; public static final String REST_API_FETCH_ISSUE = "rest/api/3/issue"; + public static final String REST_API_PROJECTS = "/rest/api/3/project/search"; public static final String FIFTY = "50"; public static final String START_AT = "startAt"; public static final String MAX_RESULT = "maxResults"; public static final List RETRY_ATTEMPT_SLEEP_TIME = List.of(1, 2, 5, 10, 20, 40); private static final String TICKET_FETCH_LATENCY_TIMER = "ticketFetchLatency"; private static final String SEARCH_CALL_LATENCY_TIMER = "searchCallLatency"; + private static final String PROJECTS_FETCH_LATENCY_TIMER = "projectFetchLatency"; private static final String ISSUES_REQUESTED = "issuesRequested"; private final RestTemplate restTemplate; private final JiraAuthConfig authConfig; private final Timer ticketFetchLatencyTimer; private final Timer searchCallLatencyTimer; + private final Timer projectFetchLatencyTimer; private final Counter issuesRequestedCounter; private final PluginMetrics jiraPluginMetrics = PluginMetrics.fromNames("jiraRestClient", "aws"); private int sleepTimeMultiplier = 1000; @@ -55,6 +68,8 @@ public JiraRestClient(RestTemplate restTemplate, JiraAuthConfig authConfig) { ticketFetchLatencyTimer = jiraPluginMetrics.timer(TICKET_FETCH_LATENCY_TIMER); searchCallLatencyTimer = jiraPluginMetrics.timer(SEARCH_CALL_LATENCY_TIMER); + projectFetchLatencyTimer = jiraPluginMetrics.timer(PROJECTS_FETCH_LATENCY_TIMER); + issuesRequestedCounter = jiraPluginMetrics.counter(ISSUES_REQUESTED); } diff --git a/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/rest/OAuth2RequestInterceptor.java b/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/rest/OAuth2RequestInterceptor.java index bf748ceb26..be71df7ac7 100644 --- a/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/rest/OAuth2RequestInterceptor.java +++ b/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/rest/OAuth2RequestInterceptor.java @@ -1,3 +1,13 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + package org.opensearch.dataprepper.plugins.source.jira.rest; import org.opensearch.dataprepper.plugins.source.jira.rest.auth.JiraAuthConfig; diff --git a/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/rest/auth/JiraAuthConfig.java b/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/rest/auth/JiraAuthConfig.java index 274ce8b5d7..cae78da954 100644 --- a/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/rest/auth/JiraAuthConfig.java +++ b/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/rest/auth/JiraAuthConfig.java @@ -1,3 +1,13 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + package org.opensearch.dataprepper.plugins.source.jira.rest.auth; /** diff --git a/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/rest/auth/JiraAuthFactory.java b/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/rest/auth/JiraAuthFactory.java index 1ba1d9717f..f178d56812 100644 --- a/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/rest/auth/JiraAuthFactory.java +++ b/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/rest/auth/JiraAuthFactory.java @@ -1,3 +1,13 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + package org.opensearch.dataprepper.plugins.source.jira.rest.auth; import org.opensearch.dataprepper.plugins.source.jira.JiraSourceConfig; diff --git a/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/rest/auth/JiraBasicAuthConfig.java b/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/rest/auth/JiraBasicAuthConfig.java index 233cbf9f49..32509a2229 100644 --- a/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/rest/auth/JiraBasicAuthConfig.java +++ b/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/rest/auth/JiraBasicAuthConfig.java @@ -1,3 +1,13 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + package org.opensearch.dataprepper.plugins.source.jira.rest.auth; diff --git a/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/rest/auth/JiraOauthConfig.java b/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/rest/auth/JiraOauthConfig.java index f7f4e8493f..791c9e6ff4 100644 --- a/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/rest/auth/JiraOauthConfig.java +++ b/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/rest/auth/JiraOauthConfig.java @@ -1,3 +1,13 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + package org.opensearch.dataprepper.plugins.source.jira.rest.auth; import lombok.Getter; @@ -55,10 +65,10 @@ public class JiraOauthConfig implements JiraAuthConfig { public JiraOauthConfig(JiraSourceConfig jiraSourceConfig) { this.jiraSourceConfig = jiraSourceConfig; - this.accessToken = jiraSourceConfig.getAccessToken(); - this.refreshToken = jiraSourceConfig.getRefreshToken(); - this.clientId = jiraSourceConfig.getClientId(); - this.clientSecret = jiraSourceConfig.getClientSecret(); + this.accessToken = jiraSourceConfig.getAuthenticationConfig().getOauth2Config().getAccessToken(); + this.refreshToken = jiraSourceConfig.getAuthenticationConfig().getOauth2Config().getRefreshToken(); + this.clientId = jiraSourceConfig.getAuthenticationConfig().getOauth2Config().getClientId(); + this.clientSecret = jiraSourceConfig.getAuthenticationConfig().getOauth2Config().getClientSecret(); } public String getJiraAccountCloudId() { diff --git a/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/utils/AddressValidation.java b/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/utils/AddressValidation.java index 0fc1c379f5..d6cc166226 100644 --- a/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/utils/AddressValidation.java +++ b/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/utils/AddressValidation.java @@ -1,3 +1,13 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + package org.opensearch.dataprepper.plugins.source.jira.utils; import lombok.NonNull; diff --git a/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/utils/Constants.java b/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/utils/Constants.java index ff6a780bfd..067380def6 100644 --- a/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/utils/Constants.java +++ b/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/utils/Constants.java @@ -1,3 +1,13 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + package org.opensearch.dataprepper.plugins.source.jira.utils; /** diff --git a/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/utils/JiraConfigHelper.java b/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/utils/JiraConfigHelper.java index 07b1e1a213..dd815bd607 100644 --- a/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/utils/JiraConfigHelper.java +++ b/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/utils/JiraConfigHelper.java @@ -1,9 +1,20 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + package org.opensearch.dataprepper.plugins.source.jira.utils; import lombok.extern.slf4j.Slf4j; import org.opensearch.dataprepper.plugins.source.jira.JiraSourceConfig; +import java.util.ArrayList; import java.util.List; import static org.opensearch.dataprepper.plugins.source.jira.utils.Constants.BASIC; @@ -14,18 +25,23 @@ */ @Slf4j public class JiraConfigHelper { - - public static final String ISSUE_STATUS_FILTER = "status"; - public static final String ISSUE_TYPE_FILTER = "issuetype"; - - /** * Get Issue Status Filter from repository configuration. * * @return List Issue Status Filter. */ - public static List getIssueStatusFilter(JiraSourceConfig repositoryConfiguration) { - return repositoryConfiguration.getStatus(); + public static List getIssueStatusIncludeFilter(JiraSourceConfig repositoryConfiguration) { + if (repositoryConfiguration.getFilterConfig() == null || repositoryConfiguration.getFilterConfig().getStatusConfig() == null) { + return new ArrayList<>(); + } + return repositoryConfiguration.getFilterConfig().getStatusConfig().getInclude(); + } + + public static List getIssueStatusExcludeFilter(JiraSourceConfig repositoryConfiguration) { + if (repositoryConfiguration.getFilterConfig() == null || repositoryConfiguration.getFilterConfig().getStatusConfig() == null) { + return new ArrayList<>(); + } + return repositoryConfiguration.getFilterConfig().getStatusConfig().getExclude(); } /** @@ -33,8 +49,18 @@ public static List getIssueStatusFilter(JiraSourceConfig repositoryConfi * * @return List Issue Type Filter. */ - public static List getIssueTypeFilter(JiraSourceConfig repositoryConfiguration) { - return repositoryConfiguration.getIssueType(); + public static List getIssueTypeIncludeFilter(JiraSourceConfig repositoryConfiguration) { + if (repositoryConfiguration.getFilterConfig() == null || repositoryConfiguration.getFilterConfig().getIssueTypeConfig() == null) { + return new ArrayList<>(); + } + return repositoryConfiguration.getFilterConfig().getIssueTypeConfig().getInclude(); + } + + public static List getIssueTypeExcludeFilter(JiraSourceConfig repositoryConfiguration) { + if (repositoryConfiguration.getFilterConfig() == null || repositoryConfiguration.getFilterConfig().getIssueTypeConfig() == null) { + return new ArrayList<>(); + } + return repositoryConfiguration.getFilterConfig().getIssueTypeConfig().getExclude(); } /** @@ -43,8 +69,22 @@ public static List getIssueTypeFilter(JiraSourceConfig repositoryConfigu * * @return List Project Filter. */ - public static List getProjectKeyFilter(JiraSourceConfig repositoryConfiguration) { - return repositoryConfiguration.getProject(); + public static List getProjectNameIncludeFilter(JiraSourceConfig repositoryConfiguration) { + if (repositoryConfiguration.getFilterConfig() == null || + repositoryConfiguration.getFilterConfig().getProjectConfig() == null || + repositoryConfiguration.getFilterConfig().getProjectConfig().getNameConfig() == null) { + return new ArrayList<>(); + } + return repositoryConfiguration.getFilterConfig().getProjectConfig().getNameConfig().getInclude(); + } + + public static List getProjectNameExcludeFilter(JiraSourceConfig repositoryConfiguration) { + if (repositoryConfiguration.getFilterConfig() == null || + repositoryConfiguration.getFilterConfig().getProjectConfig() == null || + repositoryConfiguration.getFilterConfig().getProjectConfig().getNameConfig() == null) { + return new ArrayList<>(); + } + return repositoryConfiguration.getFilterConfig().getProjectConfig().getNameConfig().getExclude(); } @@ -62,13 +102,13 @@ public static boolean validateConfig(JiraSourceConfig config) { } if (BASIC.equals(authType)) { - if (config.getJiraId() == null || config.getJiraCredential() == null) { + if (config.getAuthenticationConfig().getBasicConfig().getUsername() == null || config.getAuthenticationConfig().getBasicConfig().getPassword() == null) { throw new RuntimeException("Jira ID or Credential are required for Basic AuthType"); } } if (OAUTH2.equals(authType)) { - if (config.getAccessToken() == null || config.getRefreshToken() == null) { + if (config.getAuthenticationConfig().getOauth2Config().getAccessToken() == null || config.getAuthenticationConfig().getOauth2Config().getRefreshToken() == null) { throw new RuntimeException("Access Token or Refresh Token are required for OAuth2 AuthType"); } } diff --git a/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/utils/JiraContentType.java b/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/utils/JiraContentType.java index 9d37a6e8ca..3f43c0e01e 100644 --- a/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/utils/JiraContentType.java +++ b/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/utils/JiraContentType.java @@ -1,3 +1,13 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + package org.opensearch.dataprepper.plugins.source.jira.utils; import lombok.AllArgsConstructor; diff --git a/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/utils/JqlConstants.java b/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/utils/JqlConstants.java index 5b88208f74..99474024e1 100644 --- a/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/utils/JqlConstants.java +++ b/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/utils/JqlConstants.java @@ -1,3 +1,13 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + package org.opensearch.dataprepper.plugins.source.jira.utils; public class JqlConstants { @@ -7,10 +17,13 @@ public class JqlConstants { public static final String SLASH = "/"; public static final String PROJECT_IN = " AND project in ("; public static final String STATUS_IN = " AND status in ("; + public static final String PROJECT_NOT_IN = " AND project not in ("; + public static final String STATUS_NOT_IN = " AND status not in ("; public static final String DELIMITER = "\",\""; public static final String PREFIX = "\""; public static final String SUFFIX = "\""; public static final String ISSUE_TYPE_IN = " AND issueType in ("; + public static final String ISSUE_TYPE_NOT_IN = " AND issueType not in ("; public static final String JQL_FIELD = "jql"; public static final String EXPAND_FIELD = "expand"; public static final String EXPAND_VALUE = "all"; diff --git a/data-prepper-plugins/saas-source-plugins/jira-source/src/test/java/org/opensearch/dataprepper/plugins/source/jira/JiraClientTest.java b/data-prepper-plugins/saas-source-plugins/jira-source/src/test/java/org/opensearch/dataprepper/plugins/source/jira/JiraClientTest.java index 4720ffc89b..78531afd61 100644 --- a/data-prepper-plugins/saas-source-plugins/jira-source/src/test/java/org/opensearch/dataprepper/plugins/source/jira/JiraClientTest.java +++ b/data-prepper-plugins/saas-source-plugins/jira-source/src/test/java/org/opensearch/dataprepper/plugins/source/jira/JiraClientTest.java @@ -1,3 +1,13 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + package org.opensearch.dataprepper.plugins.source.jira; import com.fasterxml.jackson.core.JsonProcessingException; diff --git a/data-prepper-plugins/saas-source-plugins/jira-source/src/test/java/org/opensearch/dataprepper/plugins/source/jira/JiraConfigHelperTest.java b/data-prepper-plugins/saas-source-plugins/jira-source/src/test/java/org/opensearch/dataprepper/plugins/source/jira/JiraConfigHelperTest.java index ac26864a25..eeb9c03941 100644 --- a/data-prepper-plugins/saas-source-plugins/jira-source/src/test/java/org/opensearch/dataprepper/plugins/source/jira/JiraConfigHelperTest.java +++ b/data-prepper-plugins/saas-source-plugins/jira-source/src/test/java/org/opensearch/dataprepper/plugins/source/jira/JiraConfigHelperTest.java @@ -1,9 +1,27 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + package org.opensearch.dataprepper.plugins.source.jira; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; import org.mockito.Mock; import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.dataprepper.plugins.source.jira.configuration.AuthenticationConfig; +import org.opensearch.dataprepper.plugins.source.jira.configuration.BasicConfig; +import org.opensearch.dataprepper.plugins.source.jira.configuration.FilterConfig; +import org.opensearch.dataprepper.plugins.source.jira.configuration.IssueTypeConfig; +import org.opensearch.dataprepper.plugins.source.jira.configuration.NameConfig; +import org.opensearch.dataprepper.plugins.source.jira.configuration.Oauth2Config; +import org.opensearch.dataprepper.plugins.source.jira.configuration.ProjectConfig; +import org.opensearch.dataprepper.plugins.source.jira.configuration.StatusConfig; import org.opensearch.dataprepper.plugins.source.jira.utils.JiraConfigHelper; import java.util.List; @@ -23,6 +41,30 @@ public class JiraConfigHelperTest { @Mock JiraSourceConfig jiraSourceConfig; + @Mock + FilterConfig filterConfig; + + @Mock + StatusConfig statusConfig; + + @Mock + IssueTypeConfig issueTypeConfig; + + @Mock + ProjectConfig projectConfig; + + @Mock + NameConfig nameConfig; + + @Mock + AuthenticationConfig authenticationConfig; + + @Mock + BasicConfig basicConfig; + + @Mock + Oauth2Config oauth2Config; + @Test void testInitialization() { JiraConfigHelper jiraConfigHelper = new JiraConfigHelper(); @@ -31,28 +73,48 @@ void testInitialization() { @Test void testGetIssueStatusFilter() { - assertTrue(JiraConfigHelper.getIssueStatusFilter(jiraSourceConfig).isEmpty()); + when(jiraSourceConfig.getFilterConfig()).thenReturn(filterConfig); + when(filterConfig.getStatusConfig()).thenReturn(statusConfig); + assertTrue(JiraConfigHelper.getIssueStatusIncludeFilter(jiraSourceConfig).isEmpty()); + assertTrue(JiraConfigHelper.getIssueStatusExcludeFilter(jiraSourceConfig).isEmpty()); List issueStatusFilter = List.of("Done", "In Progress"); - when(jiraSourceConfig.getProject()).thenReturn(issueStatusFilter); - assertEquals(issueStatusFilter, JiraConfigHelper.getProjectKeyFilter(jiraSourceConfig)); + List issueStatusExcludeFilter = List.of("Done2", "In Progress2"); + when(statusConfig.getInclude()).thenReturn(issueStatusFilter); + when(statusConfig.getExclude()).thenReturn(issueStatusExcludeFilter); + assertEquals(issueStatusFilter, JiraConfigHelper.getIssueStatusIncludeFilter(jiraSourceConfig)); + assertEquals(issueStatusExcludeFilter, JiraConfigHelper.getIssueStatusExcludeFilter(jiraSourceConfig)); } @Test void testGetIssueTypeFilter() { - assertTrue(JiraConfigHelper.getProjectKeyFilter(jiraSourceConfig).isEmpty()); + when(jiraSourceConfig.getFilterConfig()).thenReturn(filterConfig); + when(filterConfig.getIssueTypeConfig()).thenReturn(issueTypeConfig); + assertTrue(JiraConfigHelper.getIssueTypeIncludeFilter(jiraSourceConfig).isEmpty()); + assertTrue(JiraConfigHelper.getIssueTypeExcludeFilter(jiraSourceConfig).isEmpty()); List issueTypeFilter = List.of("Bug", "Story"); - when(jiraSourceConfig.getProject()).thenReturn(issueTypeFilter); - assertEquals(issueTypeFilter, JiraConfigHelper.getProjectKeyFilter(jiraSourceConfig)); + List issueTypeExcludeFilter = List.of("Bug2", "Story2"); + when(issueTypeConfig.getInclude()).thenReturn(issueTypeFilter); + when(issueTypeConfig.getExclude()).thenReturn(issueTypeExcludeFilter); + assertEquals(issueTypeFilter, JiraConfigHelper.getIssueTypeIncludeFilter(jiraSourceConfig)); + assertEquals(issueTypeExcludeFilter, JiraConfigHelper.getIssueTypeExcludeFilter(jiraSourceConfig)); } @Test - void testGetProjectKeyFilter() { - assertTrue(JiraConfigHelper.getProjectKeyFilter(jiraSourceConfig).isEmpty()); - List projectKeyFilter = List.of("TEST", "TEST2"); - when(jiraSourceConfig.getProject()).thenReturn(projectKeyFilter); - assertEquals(projectKeyFilter, JiraConfigHelper.getProjectKeyFilter(jiraSourceConfig)); + void testGetProjectNameFilter() { + when(jiraSourceConfig.getFilterConfig()).thenReturn(filterConfig); + when(filterConfig.getProjectConfig()).thenReturn(projectConfig); + when(projectConfig.getNameConfig()).thenReturn(nameConfig); + assertTrue(JiraConfigHelper.getProjectNameIncludeFilter(jiraSourceConfig).isEmpty()); + assertTrue(JiraConfigHelper.getProjectNameExcludeFilter(jiraSourceConfig).isEmpty()); + List projectNameFilter = List.of("TEST", "TEST2"); + List projectNameExcludeFilter = List.of("TEST3", "TEST4"); + when(nameConfig.getInclude()).thenReturn(projectNameFilter); + when(nameConfig.getExclude()).thenReturn(projectNameExcludeFilter); + assertEquals(projectNameFilter, JiraConfigHelper.getProjectNameIncludeFilter(jiraSourceConfig)); + assertEquals(projectNameExcludeFilter, JiraConfigHelper.getProjectNameExcludeFilter(jiraSourceConfig)); } + @Test void testValidateConfig() { assertThrows(RuntimeException.class, () -> JiraConfigHelper.validateConfig(jiraSourceConfig)); @@ -68,16 +130,18 @@ void testValidateConfig() { void testValidateConfigBasic() { when(jiraSourceConfig.getAccountUrl()).thenReturn("https://test.com"); when(jiraSourceConfig.getAuthType()).thenReturn(BASIC); + when(jiraSourceConfig.getAuthenticationConfig()).thenReturn(authenticationConfig); + when(authenticationConfig.getBasicConfig()).thenReturn(basicConfig); assertThrows(RuntimeException.class, () -> JiraConfigHelper.validateConfig(jiraSourceConfig)); - when(jiraSourceConfig.getJiraId()).thenReturn("id"); + when(basicConfig.getUsername()).thenReturn("id"); assertThrows(RuntimeException.class, () -> JiraConfigHelper.validateConfig(jiraSourceConfig)); - when(jiraSourceConfig.getJiraCredential()).thenReturn("credential"); - when(jiraSourceConfig.getJiraId()).thenReturn(null); + when(basicConfig.getPassword()).thenReturn("credential"); + when(basicConfig.getUsername()).thenReturn(null); assertThrows(RuntimeException.class, () -> JiraConfigHelper.validateConfig(jiraSourceConfig)); - when(jiraSourceConfig.getJiraId()).thenReturn("id"); + when(basicConfig.getUsername()).thenReturn("id"); assertDoesNotThrow(() -> JiraConfigHelper.validateConfig(jiraSourceConfig)); } @@ -85,16 +149,18 @@ void testValidateConfigBasic() { void testValidateConfigOauth2() { when(jiraSourceConfig.getAccountUrl()).thenReturn("https://test.com"); when(jiraSourceConfig.getAuthType()).thenReturn(OAUTH2); + when(jiraSourceConfig.getAuthenticationConfig()).thenReturn(authenticationConfig); + when(authenticationConfig.getOauth2Config()).thenReturn(oauth2Config); assertThrows(RuntimeException.class, () -> JiraConfigHelper.validateConfig(jiraSourceConfig)); - when(jiraSourceConfig.getAccessToken()).thenReturn("id"); + when(oauth2Config.getAccessToken()).thenReturn("id"); assertThrows(RuntimeException.class, () -> JiraConfigHelper.validateConfig(jiraSourceConfig)); - when(jiraSourceConfig.getRefreshToken()).thenReturn("credential"); - when(jiraSourceConfig.getAccessToken()).thenReturn(null); + when(oauth2Config.getRefreshToken()).thenReturn("credential"); + when(oauth2Config.getAccessToken()).thenReturn(null); assertThrows(RuntimeException.class, () -> JiraConfigHelper.validateConfig(jiraSourceConfig)); - when(jiraSourceConfig.getAccessToken()).thenReturn("id"); + when(oauth2Config.getAccessToken()).thenReturn("id"); assertDoesNotThrow(() -> JiraConfigHelper.validateConfig(jiraSourceConfig)); } } diff --git a/data-prepper-plugins/saas-source-plugins/jira-source/src/test/java/org/opensearch/dataprepper/plugins/source/jira/JiraItemInfoTest.java b/data-prepper-plugins/saas-source-plugins/jira-source/src/test/java/org/opensearch/dataprepper/plugins/source/jira/JiraItemInfoTest.java index 49dc6873bd..f5e6b0906b 100644 --- a/data-prepper-plugins/saas-source-plugins/jira-source/src/test/java/org/opensearch/dataprepper/plugins/source/jira/JiraItemInfoTest.java +++ b/data-prepper-plugins/saas-source-plugins/jira-source/src/test/java/org/opensearch/dataprepper/plugins/source/jira/JiraItemInfoTest.java @@ -1,3 +1,13 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + package org.opensearch.dataprepper.plugins.source.jira; import org.junit.jupiter.api.BeforeEach; diff --git a/data-prepper-plugins/saas-source-plugins/jira-source/src/test/java/org/opensearch/dataprepper/plugins/source/jira/JiraIteratorTest.java b/data-prepper-plugins/saas-source-plugins/jira-source/src/test/java/org/opensearch/dataprepper/plugins/source/jira/JiraIteratorTest.java index 8d9dc85869..73e184330a 100644 --- a/data-prepper-plugins/saas-source-plugins/jira-source/src/test/java/org/opensearch/dataprepper/plugins/source/jira/JiraIteratorTest.java +++ b/data-prepper-plugins/saas-source-plugins/jira-source/src/test/java/org/opensearch/dataprepper/plugins/source/jira/JiraIteratorTest.java @@ -1,3 +1,13 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + package org.opensearch.dataprepper.plugins.source.jira; import org.junit.jupiter.api.BeforeEach; diff --git a/data-prepper-plugins/saas-source-plugins/jira-source/src/test/java/org/opensearch/dataprepper/plugins/source/jira/JiraServiceTest.java b/data-prepper-plugins/saas-source-plugins/jira-source/src/test/java/org/opensearch/dataprepper/plugins/source/jira/JiraServiceTest.java index af07ab6f0d..0de0ea47dc 100644 --- a/data-prepper-plugins/saas-source-plugins/jira-source/src/test/java/org/opensearch/dataprepper/plugins/source/jira/JiraServiceTest.java +++ b/data-prepper-plugins/saas-source-plugins/jira-source/src/test/java/org/opensearch/dataprepper/plugins/source/jira/JiraServiceTest.java @@ -1,3 +1,13 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + package org.opensearch.dataprepper.plugins.source.jira; import com.fasterxml.jackson.core.JsonProcessingException; @@ -45,6 +55,7 @@ import static org.opensearch.dataprepper.plugins.source.jira.utils.Constants.CREATED; import static org.opensearch.dataprepper.plugins.source.jira.utils.Constants.KEY; import static org.opensearch.dataprepper.plugins.source.jira.utils.Constants.NAME; +import static org.opensearch.dataprepper.plugins.source.jira.utils.Constants.OAUTH2; import static org.opensearch.dataprepper.plugins.source.jira.utils.Constants.PROJECT; import static org.opensearch.dataprepper.plugins.source.jira.utils.Constants.UPDATED; @@ -85,16 +96,45 @@ public static JiraSourceConfig createJiraConfiguration(String auth_type, List issueStatus, List projectKey) throws JsonProcessingException { ObjectMapper objectMapper = new ObjectMapper(); - Map connectorCredentialsMap = new HashMap<>(); - connectorCredentialsMap.put("auth_type", auth_type); + Map authenticationMap = new HashMap<>(); + Map basicMap = new HashMap<>(); + Map oauth2Map = new HashMap<>(); + if (auth_type.equals(BASIC)) { + basicMap.put("username", "test_username"); + basicMap.put("password", "test_password"); + authenticationMap.put("basic", basicMap); + } else if (auth_type.equals(OAUTH2)) { + oauth2Map.put("client_id", "test-client-id"); + oauth2Map.put("client_secret", "test-client-secret"); + oauth2Map.put("access_token", "test-access-token"); + oauth2Map.put("refresh_token", "test-refresh-token"); + authenticationMap.put("oauth2", oauth2Map); + } Map jiraSourceConfigMap = new HashMap<>(); - jiraSourceConfigMap.put("account_url", ACCESSIBLE_RESOURCES); - jiraSourceConfigMap.put("connector_credentials", connectorCredentialsMap); - jiraSourceConfigMap.put("issue_types", issueType); - jiraSourceConfigMap.put("statuses", issueStatus); - jiraSourceConfigMap.put("projects", projectKey); + List hosts = new ArrayList<>(); + hosts.add(ACCESSIBLE_RESOURCES); + + Map filterMap = new HashMap<>(); + Map projectMap = new HashMap<>(); + Map issueTypeMap = new HashMap<>(); + Map statusMap = new HashMap<>(); + + issueTypeMap.put("include", issueType); + filterMap.put("issue_type", issueTypeMap); + + statusMap.put("include", issueStatus); + filterMap.put("status", statusMap); + + Map nameMap = new HashMap<>(); + nameMap.put("include", projectKey); + projectMap.put("key", nameMap); + filterMap.put("project", projectMap); + + jiraSourceConfigMap.put("hosts", hosts); + jiraSourceConfigMap.put("authentication", authenticationMap); + jiraSourceConfigMap.put("filter", filterMap); String jiraSourceConfigJsonString = objectMapper.writeValueAsString(jiraSourceConfigMap); return objectMapper.readValue(jiraSourceConfigJsonString, JiraSourceConfig.class); diff --git a/data-prepper-plugins/saas-source-plugins/jira-source/src/test/java/org/opensearch/dataprepper/plugins/source/jira/JiraSourceConfigTest.java b/data-prepper-plugins/saas-source-plugins/jira-source/src/test/java/org/opensearch/dataprepper/plugins/source/jira/JiraSourceConfigTest.java index 9da7011d0d..35a2450fdb 100644 --- a/data-prepper-plugins/saas-source-plugins/jira-source/src/test/java/org/opensearch/dataprepper/plugins/source/jira/JiraSourceConfigTest.java +++ b/data-prepper-plugins/saas-source-plugins/jira-source/src/test/java/org/opensearch/dataprepper/plugins/source/jira/JiraSourceConfigTest.java @@ -1,3 +1,13 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + package org.opensearch.dataprepper.plugins.source.jira; import com.fasterxml.jackson.core.JsonProcessingException; @@ -21,54 +31,69 @@ public class JiraSourceConfigTest { private final String refreshToken = "refresh token test"; private final String clientId = "client id test"; private final String clientSecret = "client secret test"; - private final String jiraCredential = "test Jira Credential"; - private final String jiraId = "test Jira Id"; + private final String password = "test Jira Credential"; + private final String username = "test Jira Id"; private final String accountUrl = "https://example.atlassian.net"; private List projectList = new ArrayList<>(); private List issueTypeList = new ArrayList<>(); - private List inclusionPatternList = new ArrayList<>(); - private List exclusionPatternList = new ArrayList<>(); private List statusList = new ArrayList<>(); - private Map connectorCredentialMap = new HashMap<>(); private JiraSourceConfig jiraSourceConfig; private JiraSourceConfig createJiraSourceConfig(String authtype, boolean hasToken) throws JsonProcessingException { Map configMap = new HashMap<>(); - configMap.put("account_url", accountUrl); - - connectorCredentialMap.put("auth_type", authtype); - if (hasToken) { - connectorCredentialMap.put("access_token", accessToken); - connectorCredentialMap.put("refresh_token", refreshToken); - } else { - connectorCredentialMap.put("refresh_token", ""); + List hosts = new ArrayList<>(); + hosts.add(accountUrl); + + configMap.put("hosts", hosts); + + Map authenticationMap = new HashMap<>(); + Map basicMap = new HashMap<>(); + Map oauth2Map = new HashMap<>(); + if (authtype.equals(BASIC)) { + basicMap.put("username", username); + basicMap.put("password", password); + authenticationMap.put("basic", basicMap); + } else if (authtype.equals(OAUTH2)) { + if (hasToken) { + oauth2Map.put("access_token", accessToken); + oauth2Map.put("refresh_token", refreshToken); + } else { + oauth2Map.put("access_token", null); + oauth2Map.put("refresh_token", null); + } + oauth2Map.put("client_id", clientId); + oauth2Map.put("client_secret", clientSecret); + authenticationMap.put("oauth2", oauth2Map); } - connectorCredentialMap.put("jira_id", jiraId); - connectorCredentialMap.put("jira_credential", jiraCredential); - connectorCredentialMap.put("client_id", clientId); - connectorCredentialMap.put("client_secret", clientSecret); - configMap.put("connector_credentials", connectorCredentialMap); + configMap.put("authentication", authenticationMap); projectList.add("project1"); projectList.add("project2"); - configMap.put("projects", projectList); issueTypeList.add("issue type 1"); issueTypeList.add("issue type 2"); - configMap.put("issue_types", issueTypeList); - - inclusionPatternList.add("pattern 1"); - inclusionPatternList.add("pattern 2"); - configMap.put("inclusion_patterns", inclusionPatternList); - - exclusionPatternList.add("pattern 3"); - exclusionPatternList.add("pattern 4"); - configMap.put("exclusion_patterns", exclusionPatternList); statusList.add("status 1"); statusList.add("status 2"); - configMap.put("statuses", statusList); + + Map filterMap = new HashMap<>(); + Map projectMap = new HashMap<>(); + Map issueTypeMap = new HashMap<>(); + Map statusMap = new HashMap<>(); + + issueTypeMap.put("include", issueTypeList); + filterMap.put("issue_type", issueTypeMap); + + statusMap.put("include", statusList); + filterMap.put("status", statusMap); + + Map nameMap = new HashMap<>(); + nameMap.put("include", projectList); + projectMap.put("key", nameMap); + filterMap.put("project", projectMap); + + configMap.put("filter", filterMap); ObjectMapper objectMapper = new ObjectMapper(); String jsonConfig = objectMapper.writeValueAsString(configMap); @@ -79,40 +104,29 @@ private JiraSourceConfig createJiraSourceConfig(String authtype, boolean hasToke @Test void testGetters() throws JsonProcessingException { jiraSourceConfig = createJiraSourceConfig(BASIC, false); - assertEquals(jiraSourceConfig.getInclusionPatterns(), inclusionPatternList); - assertEquals(jiraSourceConfig.getIssueType(), issueTypeList); - assertEquals(jiraSourceConfig.getExclusionPatterns(), exclusionPatternList); + assertEquals(jiraSourceConfig.getFilterConfig().getIssueTypeConfig().getInclude(), issueTypeList); assertEquals(jiraSourceConfig.getNumWorkers(), DEFAULT_NUMBER_OF_WORKERS); - assertEquals(jiraSourceConfig.getProject(), projectList); - assertEquals(jiraSourceConfig.getStatus(), statusList); - assertEquals(jiraSourceConfig.getConnectorCredentials(), connectorCredentialMap); + assertEquals(jiraSourceConfig.getFilterConfig().getProjectConfig().getNameConfig().getInclude(), projectList); + assertEquals(jiraSourceConfig.getFilterConfig().getStatusConfig().getInclude(), statusList); assertEquals(jiraSourceConfig.getAccountUrl(), accountUrl); assertNotNull(jiraSourceConfig.getBackOff()); - assertEquals(jiraSourceConfig.getJiraCredential(), jiraCredential); - assertEquals(jiraSourceConfig.getJiraId(), jiraId); + assertEquals(jiraSourceConfig.getAuthenticationConfig().getBasicConfig().getPassword(), password); + assertEquals(jiraSourceConfig.getAuthenticationConfig().getBasicConfig().getUsername(), username); } @Test void testFetchGivenOauthAttributeWrongAuthType() throws JsonProcessingException { jiraSourceConfig = createJiraSourceConfig(BASIC, true); - assertThrows(RuntimeException.class, () -> jiraSourceConfig.getAccessToken()); + assertThrows(RuntimeException.class, () -> jiraSourceConfig.getAuthenticationConfig().getOauth2Config().getAccessToken()); } @Test void testFetchGivenOauthAtrribute() throws JsonProcessingException { jiraSourceConfig = createJiraSourceConfig(OAUTH2, true); - assertEquals(accessToken, jiraSourceConfig.getAccessToken()); - assertEquals(refreshToken, jiraSourceConfig.getRefreshToken()); - assertEquals(clientId, jiraSourceConfig.getClientId()); - assertEquals(clientSecret, jiraSourceConfig.getClientSecret()); - } - - @Test - void testFetchGivenOauthAtrributeMissing() throws JsonProcessingException { - jiraSourceConfig = createJiraSourceConfig(OAUTH2, false); - assertThrows(RuntimeException.class, () -> jiraSourceConfig.getAccessToken()); - assertThrows(RuntimeException.class, () -> jiraSourceConfig.getRefreshToken()); - + assertEquals(accessToken, jiraSourceConfig.getAuthenticationConfig().getOauth2Config().getAccessToken()); + assertEquals(refreshToken, jiraSourceConfig.getAuthenticationConfig().getOauth2Config().getRefreshToken()); + assertEquals(clientId, jiraSourceConfig.getAuthenticationConfig().getOauth2Config().getClientId()); + assertEquals(clientSecret, jiraSourceConfig.getAuthenticationConfig().getOauth2Config().getClientSecret()); } } diff --git a/data-prepper-plugins/saas-source-plugins/jira-source/src/test/java/org/opensearch/dataprepper/plugins/source/jira/JiraSourceTest.java b/data-prepper-plugins/saas-source-plugins/jira-source/src/test/java/org/opensearch/dataprepper/plugins/source/jira/JiraSourceTest.java index 46cf58b7a9..725c91b66e 100644 --- a/data-prepper-plugins/saas-source-plugins/jira-source/src/test/java/org/opensearch/dataprepper/plugins/source/jira/JiraSourceTest.java +++ b/data-prepper-plugins/saas-source-plugins/jira-source/src/test/java/org/opensearch/dataprepper/plugins/source/jira/JiraSourceTest.java @@ -1,3 +1,13 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + package org.opensearch.dataprepper.plugins.source.jira; import org.junit.jupiter.api.Test; @@ -11,6 +21,8 @@ import org.opensearch.dataprepper.model.plugin.PluginFactory; import org.opensearch.dataprepper.model.record.Record; import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourceCoordinator; +import org.opensearch.dataprepper.plugins.source.jira.configuration.AuthenticationConfig; +import org.opensearch.dataprepper.plugins.source.jira.configuration.BasicConfig; import org.opensearch.dataprepper.plugins.source.jira.rest.auth.JiraAuthConfig; import org.opensearch.dataprepper.plugins.source.source_crawler.base.Crawler; import org.opensearch.dataprepper.plugins.source.source_crawler.base.PluginExecutorServiceProvider; @@ -58,7 +70,12 @@ public class JiraSourceTest { @Mock private ExecutorService executorService; -// = new PluginExecutorServiceProvider(); + + @Mock + AuthenticationConfig authenticationConfig; + + @Mock + BasicConfig basicConfig; @Test void initialization() { @@ -73,8 +90,10 @@ void testStart() { JiraSource source = new JiraSource(pluginMetrics, jiraSourceConfig, jiraOauthConfig, pluginFactory, acknowledgementSetManager, crawler, executorServiceProvider); when(jiraSourceConfig.getAccountUrl()).thenReturn(ACCESSIBLE_RESOURCES); when(jiraSourceConfig.getAuthType()).thenReturn(BASIC); - when(jiraSourceConfig.getJiraId()).thenReturn("Test Id"); - when(jiraSourceConfig.getJiraCredential()).thenReturn("Test Credential"); + when(jiraSourceConfig.getAuthenticationConfig()).thenReturn(authenticationConfig); + when(authenticationConfig.getBasicConfig()).thenReturn(basicConfig); + when(basicConfig.getUsername()).thenReturn("Test Id"); + when(basicConfig.getPassword()).thenReturn("Test Credential"); source.setEnhancedSourceCoordinator(sourceCooridinator); source.start(buffer); @@ -87,8 +106,10 @@ void testStop() { JiraSource source = new JiraSource(pluginMetrics, jiraSourceConfig, jiraOauthConfig, pluginFactory, acknowledgementSetManager, crawler, executorServiceProvider); when(jiraSourceConfig.getAccountUrl()).thenReturn(ACCESSIBLE_RESOURCES); when(jiraSourceConfig.getAuthType()).thenReturn(BASIC); - when(jiraSourceConfig.getJiraId()).thenReturn("Test Id"); - when(jiraSourceConfig.getJiraCredential()).thenReturn("Test Credential"); + when(jiraSourceConfig.getAuthenticationConfig()).thenReturn(authenticationConfig); + when(authenticationConfig.getBasicConfig()).thenReturn(basicConfig); + when(basicConfig.getUsername()).thenReturn("Test Id"); + when(basicConfig.getPassword()).thenReturn("Test Credential"); source.setEnhancedSourceCoordinator(sourceCooridinator); source.start(buffer); diff --git a/data-prepper-plugins/saas-source-plugins/jira-source/src/test/java/org/opensearch/dataprepper/plugins/source/jira/exception/BadRequestExceptionTest.java b/data-prepper-plugins/saas-source-plugins/jira-source/src/test/java/org/opensearch/dataprepper/plugins/source/jira/exception/BadRequestExceptionTest.java index 74f5873af2..91578ec810 100644 --- a/data-prepper-plugins/saas-source-plugins/jira-source/src/test/java/org/opensearch/dataprepper/plugins/source/jira/exception/BadRequestExceptionTest.java +++ b/data-prepper-plugins/saas-source-plugins/jira-source/src/test/java/org/opensearch/dataprepper/plugins/source/jira/exception/BadRequestExceptionTest.java @@ -1,3 +1,13 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + package org.opensearch.dataprepper.plugins.source.jira.exception; import org.junit.jupiter.api.BeforeEach; diff --git a/data-prepper-plugins/saas-source-plugins/jira-source/src/test/java/org/opensearch/dataprepper/plugins/source/jira/exception/UnAuthorizedExceptionTest.java b/data-prepper-plugins/saas-source-plugins/jira-source/src/test/java/org/opensearch/dataprepper/plugins/source/jira/exception/UnAuthorizedExceptionTest.java index ecedff2d60..87c06051e1 100644 --- a/data-prepper-plugins/saas-source-plugins/jira-source/src/test/java/org/opensearch/dataprepper/plugins/source/jira/exception/UnAuthorizedExceptionTest.java +++ b/data-prepper-plugins/saas-source-plugins/jira-source/src/test/java/org/opensearch/dataprepper/plugins/source/jira/exception/UnAuthorizedExceptionTest.java @@ -1,3 +1,13 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + package org.opensearch.dataprepper.plugins.source.jira.exception; import org.junit.jupiter.api.BeforeEach; diff --git a/data-prepper-plugins/saas-source-plugins/jira-source/src/test/java/org/opensearch/dataprepper/plugins/source/jira/models/IssueBeanTest.java b/data-prepper-plugins/saas-source-plugins/jira-source/src/test/java/org/opensearch/dataprepper/plugins/source/jira/models/IssueBeanTest.java index 0f99ddccbf..812c59c1f7 100644 --- a/data-prepper-plugins/saas-source-plugins/jira-source/src/test/java/org/opensearch/dataprepper/plugins/source/jira/models/IssueBeanTest.java +++ b/data-prepper-plugins/saas-source-plugins/jira-source/src/test/java/org/opensearch/dataprepper/plugins/source/jira/models/IssueBeanTest.java @@ -1,3 +1,13 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + package org.opensearch.dataprepper.plugins.source.jira.models; import org.junit.jupiter.api.BeforeEach; diff --git a/data-prepper-plugins/saas-source-plugins/jira-source/src/test/java/org/opensearch/dataprepper/plugins/source/jira/models/SearchResultsTest.java b/data-prepper-plugins/saas-source-plugins/jira-source/src/test/java/org/opensearch/dataprepper/plugins/source/jira/models/SearchResultsTest.java index 3a569ca39d..bd0d54e2bf 100644 --- a/data-prepper-plugins/saas-source-plugins/jira-source/src/test/java/org/opensearch/dataprepper/plugins/source/jira/models/SearchResultsTest.java +++ b/data-prepper-plugins/saas-source-plugins/jira-source/src/test/java/org/opensearch/dataprepper/plugins/source/jira/models/SearchResultsTest.java @@ -1,3 +1,13 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + package org.opensearch.dataprepper.plugins.source.jira.models; import com.fasterxml.jackson.core.JsonProcessingException; diff --git a/data-prepper-plugins/saas-source-plugins/jira-source/src/test/java/org/opensearch/dataprepper/plugins/source/jira/rest/BasicAuthInterceptorTest.java b/data-prepper-plugins/saas-source-plugins/jira-source/src/test/java/org/opensearch/dataprepper/plugins/source/jira/rest/BasicAuthInterceptorTest.java index 18c94cb426..a57b879d6d 100644 --- a/data-prepper-plugins/saas-source-plugins/jira-source/src/test/java/org/opensearch/dataprepper/plugins/source/jira/rest/BasicAuthInterceptorTest.java +++ b/data-prepper-plugins/saas-source-plugins/jira-source/src/test/java/org/opensearch/dataprepper/plugins/source/jira/rest/BasicAuthInterceptorTest.java @@ -1,3 +1,13 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + package org.opensearch.dataprepper.plugins.source.jira.rest; import org.junit.jupiter.api.BeforeEach; @@ -6,6 +16,8 @@ import org.mockito.Mock; import org.mockito.junit.jupiter.MockitoExtension; import org.opensearch.dataprepper.plugins.source.jira.JiraSourceConfig; +import org.opensearch.dataprepper.plugins.source.jira.configuration.AuthenticationConfig; +import org.opensearch.dataprepper.plugins.source.jira.configuration.BasicConfig; import org.springframework.http.HttpHeaders; import org.springframework.http.HttpRequest; import org.springframework.http.client.ClientHttpRequestExecution; @@ -36,6 +48,12 @@ public class BasicAuthInterceptorTest { @Mock private JiraSourceConfig mockConfig; + @Mock + AuthenticationConfig authenticationConfig; + + @Mock + BasicConfig basicConfig; + @Mock private HttpHeaders mockHeaders; @@ -43,8 +61,10 @@ public class BasicAuthInterceptorTest { @BeforeEach void setUp() { - when(mockConfig.getJiraId()).thenReturn("testUser"); - when(mockConfig.getJiraCredential()).thenReturn("testPassword"); + when(mockConfig.getAuthenticationConfig()).thenReturn(authenticationConfig); + when(authenticationConfig.getBasicConfig()).thenReturn(basicConfig); + when(basicConfig.getUsername()).thenReturn("testUser"); + when(basicConfig.getPassword()).thenReturn("testPassword"); when(mockRequest.getHeaders()).thenReturn(mockHeaders); interceptor = new BasicAuthInterceptor(mockConfig); } diff --git a/data-prepper-plugins/saas-source-plugins/jira-source/src/test/java/org/opensearch/dataprepper/plugins/source/jira/rest/CustomRestTemplateConfigTest.java b/data-prepper-plugins/saas-source-plugins/jira-source/src/test/java/org/opensearch/dataprepper/plugins/source/jira/rest/CustomRestTemplateConfigTest.java index ce0675f22a..11d29c78c0 100644 --- a/data-prepper-plugins/saas-source-plugins/jira-source/src/test/java/org/opensearch/dataprepper/plugins/source/jira/rest/CustomRestTemplateConfigTest.java +++ b/data-prepper-plugins/saas-source-plugins/jira-source/src/test/java/org/opensearch/dataprepper/plugins/source/jira/rest/CustomRestTemplateConfigTest.java @@ -1,3 +1,13 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + package org.opensearch.dataprepper.plugins.source.jira.rest; import org.junit.jupiter.api.BeforeEach; @@ -8,6 +18,9 @@ import org.mockito.Mock; import org.mockito.junit.jupiter.MockitoExtension; import org.opensearch.dataprepper.plugins.source.jira.JiraSourceConfig; +import org.opensearch.dataprepper.plugins.source.jira.configuration.AuthenticationConfig; +import org.opensearch.dataprepper.plugins.source.jira.configuration.BasicConfig; +import org.opensearch.dataprepper.plugins.source.jira.configuration.Oauth2Config; import org.opensearch.dataprepper.plugins.source.jira.rest.auth.JiraAuthConfig; import org.springframework.http.client.ClientHttpRequestInterceptor; import org.springframework.http.client.InterceptingClientHttpRequestFactory; @@ -19,6 +32,7 @@ import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertInstanceOf; import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.mockito.Mockito.lenient; import static org.mockito.Mockito.when; import static org.opensearch.dataprepper.plugins.source.jira.utils.Constants.BASIC; import static org.opensearch.dataprepper.plugins.source.jira.utils.Constants.OAUTH2; @@ -34,6 +48,15 @@ class CustomRestTemplateConfigTest { @Mock private JiraAuthConfig mockAuthConfig; + @Mock + private BasicConfig mockBasicConfig; + + @Mock + private Oauth2Config mockOauth2Config; + + @Mock + private AuthenticationConfig mockAuthenticationConfig; + private static Stream provideAuthTypeAndExpectedInterceptorType() { return Stream.of( Arguments.of(OAUTH2, OAuth2RequestInterceptor.class), @@ -52,6 +75,16 @@ void setUp() { @MethodSource("provideAuthTypeAndExpectedInterceptorType") void testBasicAuthRestTemplateWithOAuth2(String authType, Class interceptorClassType) { when(mockSourceConfig.getAuthType()).thenReturn(authType); + lenient().when(mockSourceConfig.getAuthenticationConfig()).thenReturn(mockAuthenticationConfig); + lenient().when(mockAuthenticationConfig.getOauth2Config()).thenReturn(mockOauth2Config); + lenient().when(mockOauth2Config.getAccessToken()).thenReturn("accessToken"); + lenient().when(mockOauth2Config.getRefreshToken()).thenReturn("refreshToken"); + lenient().when(mockOauth2Config.getClientId()).thenReturn("clientId"); + lenient().when(mockOauth2Config.getClientSecret()).thenReturn("clientSecret"); + lenient().when(mockAuthenticationConfig.getBasicConfig()).thenReturn(mockBasicConfig); + lenient().when(mockBasicConfig.getUsername()).thenReturn("username"); + lenient().when(mockBasicConfig.getPassword()).thenReturn("password"); + RestTemplate restTemplate = config.basicAuthRestTemplate(mockSourceConfig, mockAuthConfig); assertNotNull(restTemplate); assertInstanceOf(InterceptingClientHttpRequestFactory.class, restTemplate.getRequestFactory()); diff --git a/data-prepper-plugins/saas-source-plugins/jira-source/src/test/java/org/opensearch/dataprepper/plugins/source/jira/rest/JiraRestClientTest.java b/data-prepper-plugins/saas-source-plugins/jira-source/src/test/java/org/opensearch/dataprepper/plugins/source/jira/rest/JiraRestClientTest.java index d294b4f599..3fe0640700 100644 --- a/data-prepper-plugins/saas-source-plugins/jira-source/src/test/java/org/opensearch/dataprepper/plugins/source/jira/rest/JiraRestClientTest.java +++ b/data-prepper-plugins/saas-source-plugins/jira-source/src/test/java/org/opensearch/dataprepper/plugins/source/jira/rest/JiraRestClientTest.java @@ -1,3 +1,13 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + package org.opensearch.dataprepper.plugins.source.jira.rest; import com.fasterxml.jackson.core.JsonProcessingException; diff --git a/data-prepper-plugins/saas-source-plugins/jira-source/src/test/java/org/opensearch/dataprepper/plugins/source/jira/rest/OAuth2RequestInterceptorTest.java b/data-prepper-plugins/saas-source-plugins/jira-source/src/test/java/org/opensearch/dataprepper/plugins/source/jira/rest/OAuth2RequestInterceptorTest.java index 277c42ca2e..01e850fec9 100644 --- a/data-prepper-plugins/saas-source-plugins/jira-source/src/test/java/org/opensearch/dataprepper/plugins/source/jira/rest/OAuth2RequestInterceptorTest.java +++ b/data-prepper-plugins/saas-source-plugins/jira-source/src/test/java/org/opensearch/dataprepper/plugins/source/jira/rest/OAuth2RequestInterceptorTest.java @@ -1,3 +1,13 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + package org.opensearch.dataprepper.plugins.source.jira.rest; import org.junit.jupiter.api.BeforeEach; diff --git a/data-prepper-plugins/saas-source-plugins/jira-source/src/test/java/org/opensearch/dataprepper/plugins/source/jira/rest/auth/JiraAuthFactoryTest.java b/data-prepper-plugins/saas-source-plugins/jira-source/src/test/java/org/opensearch/dataprepper/plugins/source/jira/rest/auth/JiraAuthFactoryTest.java index 566b3ae8ed..5106bd8ad1 100644 --- a/data-prepper-plugins/saas-source-plugins/jira-source/src/test/java/org/opensearch/dataprepper/plugins/source/jira/rest/auth/JiraAuthFactoryTest.java +++ b/data-prepper-plugins/saas-source-plugins/jira-source/src/test/java/org/opensearch/dataprepper/plugins/source/jira/rest/auth/JiraAuthFactoryTest.java @@ -1,3 +1,13 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + package org.opensearch.dataprepper.plugins.source.jira.rest.auth; import org.junit.jupiter.api.BeforeEach; @@ -6,6 +16,8 @@ import org.mockito.Mock; import org.mockito.junit.jupiter.MockitoExtension; import org.opensearch.dataprepper.plugins.source.jira.JiraSourceConfig; +import org.opensearch.dataprepper.plugins.source.jira.configuration.AuthenticationConfig; +import org.opensearch.dataprepper.plugins.source.jira.configuration.Oauth2Config; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertInstanceOf; @@ -18,6 +30,12 @@ public class JiraAuthFactoryTest { @Mock private JiraSourceConfig sourceConfig; + @Mock + private AuthenticationConfig authenticationConfig; + + @Mock + private Oauth2Config oauth2Config; + private JiraAuthFactory jiraAuthFactory; @BeforeEach @@ -28,6 +46,8 @@ void setUp() { @Test void testGetObjectOauth2() { when(sourceConfig.getAuthType()).thenReturn(OAUTH2); + when(sourceConfig.getAuthenticationConfig()).thenReturn(authenticationConfig); + when(authenticationConfig.getOauth2Config()).thenReturn(oauth2Config); assertInstanceOf(JiraOauthConfig.class, jiraAuthFactory.getObject()); } diff --git a/data-prepper-plugins/saas-source-plugins/jira-source/src/test/java/org/opensearch/dataprepper/plugins/source/jira/rest/auth/JiraBasicAuthConfigTest.java b/data-prepper-plugins/saas-source-plugins/jira-source/src/test/java/org/opensearch/dataprepper/plugins/source/jira/rest/auth/JiraBasicAuthConfigTest.java index 60dad25507..8ab10eb0bc 100644 --- a/data-prepper-plugins/saas-source-plugins/jira-source/src/test/java/org/opensearch/dataprepper/plugins/source/jira/rest/auth/JiraBasicAuthConfigTest.java +++ b/data-prepper-plugins/saas-source-plugins/jira-source/src/test/java/org/opensearch/dataprepper/plugins/source/jira/rest/auth/JiraBasicAuthConfigTest.java @@ -1,3 +1,13 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + package org.opensearch.dataprepper.plugins.source.jira.rest.auth; import org.junit.jupiter.api.BeforeEach; diff --git a/data-prepper-plugins/saas-source-plugins/jira-source/src/test/java/org/opensearch/dataprepper/plugins/source/jira/rest/auth/JiraOauthConfigTest.java b/data-prepper-plugins/saas-source-plugins/jira-source/src/test/java/org/opensearch/dataprepper/plugins/source/jira/rest/auth/JiraOauthConfigTest.java index 1b648ab404..bafe995801 100644 --- a/data-prepper-plugins/saas-source-plugins/jira-source/src/test/java/org/opensearch/dataprepper/plugins/source/jira/rest/auth/JiraOauthConfigTest.java +++ b/data-prepper-plugins/saas-source-plugins/jira-source/src/test/java/org/opensearch/dataprepper/plugins/source/jira/rest/auth/JiraOauthConfigTest.java @@ -1,5 +1,16 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + package org.opensearch.dataprepper.plugins.source.jira.rest.auth; +import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; import org.mockito.Mock; @@ -38,7 +49,13 @@ public class JiraOauthConfigTest { @Mock RestTemplate restTemplateMock; - JiraSourceConfig jiraSourceConfig = createJiraConfigurationFromYaml("oauth2-auth-jira-pipeline.yaml"); + + JiraSourceConfig jiraSourceConfig; + + @BeforeEach + void setUp() { + jiraSourceConfig = createJiraConfigurationFromYaml("oauth2-auth-jira-pipeline.yaml"); + } @Test void testRenewToken() throws InterruptedException { diff --git a/data-prepper-plugins/saas-source-plugins/jira-source/src/test/java/org/opensearch/dataprepper/plugins/source/jira/utils/AddressValidationTest.java b/data-prepper-plugins/saas-source-plugins/jira-source/src/test/java/org/opensearch/dataprepper/plugins/source/jira/utils/AddressValidationTest.java index b6f56a11ee..3c346c8e0e 100644 --- a/data-prepper-plugins/saas-source-plugins/jira-source/src/test/java/org/opensearch/dataprepper/plugins/source/jira/utils/AddressValidationTest.java +++ b/data-prepper-plugins/saas-source-plugins/jira-source/src/test/java/org/opensearch/dataprepper/plugins/source/jira/utils/AddressValidationTest.java @@ -1,3 +1,13 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + package org.opensearch.dataprepper.plugins.source.jira.utils; import org.junit.jupiter.api.Test; diff --git a/data-prepper-plugins/saas-source-plugins/jira-source/src/test/java/org/opensearch/dataprepper/plugins/source/jira/utils/JiraContentTypeTest.java b/data-prepper-plugins/saas-source-plugins/jira-source/src/test/java/org/opensearch/dataprepper/plugins/source/jira/utils/JiraContentTypeTest.java index c342e9a3cd..3437ab9319 100644 --- a/data-prepper-plugins/saas-source-plugins/jira-source/src/test/java/org/opensearch/dataprepper/plugins/source/jira/utils/JiraContentTypeTest.java +++ b/data-prepper-plugins/saas-source-plugins/jira-source/src/test/java/org/opensearch/dataprepper/plugins/source/jira/utils/JiraContentTypeTest.java @@ -1,3 +1,13 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + package org.opensearch.dataprepper.plugins.source.jira.utils; import org.junit.jupiter.api.Test; diff --git a/data-prepper-plugins/saas-source-plugins/jira-source/src/test/resources/basic-auth-jira-pipeline.yaml b/data-prepper-plugins/saas-source-plugins/jira-source/src/test/resources/basic-auth-jira-pipeline.yaml index 09b15d40d6..0bfa6384e8 100644 --- a/data-prepper-plugins/saas-source-plugins/jira-source/src/test/resources/basic-auth-jira-pipeline.yaml +++ b/data-prepper-plugins/saas-source-plugins/jira-source/src/test/resources/basic-auth-jira-pipeline.yaml @@ -1,6 +1,6 @@ -account_url: "https://jira.com/" -connector_credentials: - auth_type: "Basic" - jira_id: "jiraId" - jira_credential: "jiraApiKey" +hosts: ["https://jira.com/"] +authentication: + basic: + username: "jiraId" + password: "jiraApiKey" diff --git a/data-prepper-plugins/saas-source-plugins/jira-source/src/test/resources/oauth2-auth-jira-pipeline.yaml b/data-prepper-plugins/saas-source-plugins/jira-source/src/test/resources/oauth2-auth-jira-pipeline.yaml index ae1f0b508d..09c9e9f2c5 100644 --- a/data-prepper-plugins/saas-source-plugins/jira-source/src/test/resources/oauth2-auth-jira-pipeline.yaml +++ b/data-prepper-plugins/saas-source-plugins/jira-source/src/test/resources/oauth2-auth-jira-pipeline.yaml @@ -1,9 +1,8 @@ -account_url: "https://jira.com/" -connector_credentials: - auth_type: "OAuth2" - jira_id: "jira_id" - client_id: "client_id" - client_secret: "client_secret" - access_token: "access_token" - refresh_token: "refresh_token" +hosts: ["https://jira.com/"] +authentication: + oauth2: + client_id: "client_id" + client_secret: "client_secret" + access_token: "access_token" + refresh_token: "refresh_token" diff --git a/data-prepper-plugins/sqs-source/README.md b/data-prepper-plugins/sqs-source/README.md index 708389f769..ff4313605f 100644 --- a/data-prepper-plugins/sqs-source/README.md +++ b/data-prepper-plugins/sqs-source/README.md @@ -1,70 +1,22 @@ -# SQS Source +# SQS Source -This source allows Data Prepper to use SQS as a source. It uses SQS for notifications -of which data are new and loads those messages to push out events. +This source allows Data Prepper to use SQS as a source. It reads messages from specified SQS queues and processes them into events. -### Example: +## Example Configuration -The following configuration shows a minimum configuration for reading and Sqs messages and push out events. - -``` +```yaml sqs-pipeline: source: sqs: - acknowledgments: true - queue_urls: - - https://sqs.us-east-1.amazonaws.com/895099421235/MyQueue-1 - - https://sqs.us-east-1.amazonaws.com/895099421235/MyQueue-2 - - https://sqs.us-east-1.amazonaws.com/895099421235/MyQueue-3 - number_of_threads : 1 - batch_size : 10 - visibility_timeout: PT30S - wait_time : PT20S + queues: + - url: + batch_size: 10 + workers: 1 + - url: + batch_size: 10 + workers: 1 aws: - sts_region: us-east-1 - sts_role_arn: arn:aws:iam::895099421235:role/test-arn -``` - -## Configuration Options - -All Duration values are a string that represents a duration. They support ISO_8601 notation string ("PT20.345S", "PT15M", etc.) as well as simple notation Strings for seconds ("60s") and milliseconds ("1500ms"). - -* `queue_url or queue_urls` (Required) : The SQS configuration. See [SQS Configuration](#sqs_configuration) for details. - -* `aws` (Optional) : AWS configurations. See [AWS Configuration](#aws_configuration) for details. - -* `acknowledgments` (Optional) : Enables End-to-end acknowledgments. If set to `true`, sqs message is deleted only after all events from the sqs message are successfully acknowledged by all sinks. Default value `false`. - -### SQS Configuration - -* `number_of_threads` (Optional) : define no of threads for sqs queue processing. default to 1. -* `batch_size` (Optional) : define batch size for sqs messages processing. default to 10. -* `polling_frequency` (Optional) : Duration - A delay to place between reading and processing a batch of SQS messages and making a subsequent request. Defaults to 0 seconds. -* `visibility_timeout` (Optional) : Duration - The visibility timeout to apply to messages read from the SQS queue. Defaults to null -* `wait_time` (Optional) : Duration - The time to wait for long-polling on the SQS API. Defaults to null. - -### AWS Configuration - -The AWS configuration is the same for both SQS. - -* `sts_region` (Optional) : The AWS region to use for credentials. Defaults to [standard SDK behavior to determine the region](https://docs.aws.amazon.com/sdk-for-java/latest/developer-guide/region-selection.html). -* `sts_role_arn` (Optional) : The AWS STS role to assume for requests to SQS. Defaults to null, which will use the [standard SDK behavior for credentials](https://docs.aws.amazon.com/sdk-for-java/latest/developer-guide/credentials.html). -* `sts_header_overrides` (Optional): A map of header overrides to make when assuming the IAM role for the sink plugin. - -## Metrics - -* `sqsMessagesReceived` - The number of SQS messages received from the queue by the SQS Source. -* `sqsMessagesDeleted` - The number of SQS messages deleted from the queue by the SQS Source. -* `sqsMessagesFailed` - The number of SQS messages that the SQS Source failed to parse. -* `sqsMessagesDeleteFailed` - The number of SQS messages that the SQS Source failed to delete from the SQS queue. -* `acknowledgementSetCallbackCounter` - The number of SQS messages processed by SQS Source and successfully acknowledge by sink. - -## Developer Guide - -The integration tests for this plugin do not run as part of the Data Prepper build. - -The following command runs the integration tests: - -``` -./gradlew :data-prepper-plugins:sqs-source:integrationTest -Dtests.sqs.source.aws.region= -Dtests.sqs.source.queue.url= -``` + region: + sts_role_arn: + sink: + - stdout: diff --git a/data-prepper-plugins/sqs-source/build.gradle b/data-prepper-plugins/sqs-source/build.gradle index 722d5afb7f..b4ffbc8e5e 100644 --- a/data-prepper-plugins/sqs-source/build.gradle +++ b/data-prepper-plugins/sqs-source/build.gradle @@ -1,8 +1,17 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +plugins { + id 'java' +} + dependencies { implementation project(':data-prepper-api') - implementation libs.armeria.core - implementation project(':data-prepper-plugins:aws-sqs-common') implementation project(':data-prepper-plugins:buffer-common') + implementation project(':data-prepper-plugins:common') + implementation libs.armeria.core implementation project(':data-prepper-plugins:aws-plugin-api') implementation 'software.amazon.awssdk:sqs' implementation 'software.amazon.awssdk:arns' @@ -10,40 +19,10 @@ dependencies { implementation 'io.micrometer:micrometer-core' implementation 'com.fasterxml.jackson.core:jackson-annotations' implementation 'com.fasterxml.jackson.datatype:jackson-datatype-jsr310' + implementation 'org.hibernate.validator:hibernate-validator:8.0.1.Final' testImplementation 'com.fasterxml.jackson.dataformat:jackson-dataformat-yaml' testImplementation project(':data-prepper-plugins:blocking-buffer') } test { useJUnitPlatform() } - -sourceSets { - integrationTest { - java { - compileClasspath += main.output + test.output - runtimeClasspath += main.output + test.output - srcDir file('src/integrationTest/java') - } - resources.srcDir file('src/integrationTest/resources') - } -} - -configurations { - integrationTestImplementation.extendsFrom testImplementation - integrationTestRuntime.extendsFrom testRuntime -} - -task integrationTest(type: Test) { - group = 'verification' - testClassesDirs = sourceSets.integrationTest.output.classesDirs - - useJUnitPlatform() - - classpath = sourceSets.integrationTest.runtimeClasspath - systemProperty 'tests.sqs.source.aws.region', System.getProperty('tests.sqs.source.aws.region') - systemProperty 'tests.sqs.source.queue.url', System.getProperty('tests.sqs.source.queue.url') - - filter { - includeTestsMatching '*IT' - } -} \ No newline at end of file diff --git a/data-prepper-plugins/sqs-source/src/integrationTest/java/org/opensearch/dataprepper/plugins/source/sqssource/RecordsGenerator.java b/data-prepper-plugins/sqs-source/src/integrationTest/java/org/opensearch/dataprepper/plugins/source/sqssource/RecordsGenerator.java deleted file mode 100644 index a0d4a24d68..0000000000 --- a/data-prepper-plugins/sqs-source/src/integrationTest/java/org/opensearch/dataprepper/plugins/source/sqssource/RecordsGenerator.java +++ /dev/null @@ -1,11 +0,0 @@ -/* - * Copyright OpenSearch Contributors - * SPDX-License-Identifier: Apache-2.0 - */ -package org.opensearch.dataprepper.plugins.source.sqssource; - -import java.util.List; - -public interface RecordsGenerator { - void pushMessages(final List messages, final String queueUrl); -} diff --git a/data-prepper-plugins/sqs-source/src/integrationTest/java/org/opensearch/dataprepper/plugins/source/sqssource/SqsRecordsGenerator.java b/data-prepper-plugins/sqs-source/src/integrationTest/java/org/opensearch/dataprepper/plugins/source/sqssource/SqsRecordsGenerator.java deleted file mode 100644 index 18e4ea2980..0000000000 --- a/data-prepper-plugins/sqs-source/src/integrationTest/java/org/opensearch/dataprepper/plugins/source/sqssource/SqsRecordsGenerator.java +++ /dev/null @@ -1,47 +0,0 @@ -/* - * Copyright OpenSearch Contributors - * SPDX-License-Identifier: Apache-2.0 - */ -package org.opensearch.dataprepper.plugins.source.sqssource; - -import software.amazon.awssdk.services.sqs.SqsClient; -import software.amazon.awssdk.services.sqs.model.SendMessageBatchRequest; -import software.amazon.awssdk.services.sqs.model.SendMessageBatchRequestEntry; - -import java.util.ArrayList; -import java.util.List; -import java.util.UUID; - -public class SqsRecordsGenerator implements RecordsGenerator { - - private final SqsClient sqsClient; - - public SqsRecordsGenerator(final SqsClient sqsClient){ - this.sqsClient = sqsClient; - } - - @Override - public void pushMessages(final List messages, String queueUrl) { - final List> batches = splitIntoBatches(messages, 10); - batches.forEach(batch -> { - List entries = new ArrayList<>(); - batch.forEach(msg -> entries.add(SendMessageBatchRequestEntry.builder() - .id(UUID.randomUUID() + "-" + UUID.randomUUID()).messageBody(msg).build())); - sqsClient.sendMessageBatch(SendMessageBatchRequest.builder().queueUrl(queueUrl).entries(entries).build()); - }); - } - - private static List> splitIntoBatches(List messages, int batchSize) { - List> batches = new ArrayList<>(); - int totalRecords = messages.size(); - int numBatches = (int) Math.ceil((double) totalRecords / batchSize); - - for (int i = 0; i < numBatches; i++) { - int startIndex = i * batchSize; - int endIndex = Math.min(startIndex + batchSize, totalRecords); - List batch = messages.subList(startIndex, endIndex); - batches.add(batch); - } - return batches; - } -} diff --git a/data-prepper-plugins/sqs-source/src/integrationTest/java/org/opensearch/dataprepper/plugins/source/sqssource/SqsSourceTaskIT.java b/data-prepper-plugins/sqs-source/src/integrationTest/java/org/opensearch/dataprepper/plugins/source/sqssource/SqsSourceTaskIT.java deleted file mode 100644 index 31e12db422..0000000000 --- a/data-prepper-plugins/sqs-source/src/integrationTest/java/org/opensearch/dataprepper/plugins/source/sqssource/SqsSourceTaskIT.java +++ /dev/null @@ -1,147 +0,0 @@ -/* - * Copyright OpenSearch Contributors - * SPDX-License-Identifier: Apache-2.0 - */ -package org.opensearch.dataprepper.plugins.source.sqssource; - -import com.linecorp.armeria.client.retry.Backoff; -import io.micrometer.core.instrument.Counter; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.CsvSource; -import org.mockito.Mock; -import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSetManager; -import org.opensearch.dataprepper.model.configuration.PluginSetting; -import org.opensearch.dataprepper.model.event.Event; -import org.opensearch.dataprepper.model.record.Record; -import org.opensearch.dataprepper.plugins.aws.sqs.common.SqsService; -import org.opensearch.dataprepper.plugins.aws.sqs.common.handler.SqsMessageHandler; -import org.opensearch.dataprepper.plugins.aws.sqs.common.metrics.SqsMetrics; -import org.opensearch.dataprepper.plugins.aws.sqs.common.model.SqsOptions; -import org.opensearch.dataprepper.plugins.buffer.blockingbuffer.BlockingBuffer; -import org.opensearch.dataprepper.plugins.source.sqssource.handler.RawSqsMessageHandler; -import software.amazon.awssdk.regions.Region; -import software.amazon.awssdk.services.sqs.SqsClient; - -import java.time.Duration; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.concurrent.Executors; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.TimeUnit; -import java.util.stream.Collectors; - -import static org.hamcrest.CoreMatchers.equalTo; -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.collection.IsIterableContainingInAnyOrder.containsInAnyOrder; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; - -public class SqsSourceTaskIT { - - static final long INITIAL_DELAY = Duration.ofSeconds(20).toMillis(); - - static final long MAXIMUM_DELAY = Duration.ofMinutes(5).toMillis(); - - static final double JITTER_RATE = 0.20; - - private static final String TEST_PIPELINE_NAME = "pipeline"; - - private static final String MESSAGE = "message"; - - private static final String JSON_MESSAGE = "{\"array\":[{\"name\":\"abc\",\"test\":[{\"company\":\"xyz\"}]},{\"number\":1}]}"; - - private static final String LOG_MESSAGE = "2023-06-14T11:59:54,350 [main] INFO Test - Application started Successfully\n"; - - private static final String AWS_SQS_QUEUE_URL = "tests.sqs.source.queue.url"; - - private static final String AWS_REGION = "tests.sqs.source.aws.region"; - public static final Duration BUFFER_TIMEOUT = Duration.ofSeconds(10); - public static final int RECORDS_TO_ACCUMULATE = 100; - - @Mock - private AcknowledgementSetManager acknowledgementSetManager; - - private BlockingBuffer> buffer; - - private Counter messageReceivedCounter; - - private Counter messageDeletedCounter; - - private Backoff backoff; - - private SqsClient sqsClient; - - private SqsMetrics sqsMetrics; - - private ScheduledExecutorService executorService; - - @ParameterizedTest - @CsvSource({"2,1","10,2","50,4","100,5","200,7","500,10","1000,15","2000,24"}) - public void process_sqs_messages(int messageLoad,int threadSleepTime){ - final SqsRecordsGenerator sqsRecordsGenerator = new SqsRecordsGenerator(sqsClient); - final String queueUrl = System.getProperty(AWS_SQS_QUEUE_URL); - - List inputDataList = pushMessagesToQueue(sqsRecordsGenerator, queueUrl,messageLoad); - this.buffer = getBuffer(inputDataList.size()); - - SqsOptions sqsOptions = new SqsOptions.Builder().setSqsUrl(queueUrl).setMaximumMessages(10).build(); - executorService.scheduleAtFixedRate(createObjectUnderTest(sqsOptions),0,1, TimeUnit.MILLISECONDS); - try { - Thread.sleep(Duration.ofSeconds(threadSleepTime).toMillis()); - } catch (InterruptedException e) { - } - executorService.shutdown(); - final List> bufferEvents = new ArrayList<>(buffer.read((int) Duration.ofSeconds(10).toMillis()).getKey()); - final List bufferData = bufferEvents.stream().map(obj -> obj.getData().get(MESSAGE, String.class)).collect(Collectors.toList()); - assertThat(bufferData, containsInAnyOrder(inputDataList.toArray())); - assertThat(bufferData.size(),equalTo(inputDataList.size())); - } - - @BeforeEach - public void setup(){ - this.acknowledgementSetManager = mock(AcknowledgementSetManager.class); - this.messageReceivedCounter = mock(Counter.class); - this.messageDeletedCounter = mock(Counter.class); - this.sqsMetrics = mock(SqsMetrics.class); - when(sqsMetrics.getSqsMessagesReceivedCounter()).thenReturn(messageReceivedCounter); - when(sqsMetrics.getSqsMessagesDeletedCounter()).thenReturn(messageDeletedCounter); - this.backoff = Backoff.exponential(INITIAL_DELAY, MAXIMUM_DELAY).withJitter(JITTER_RATE) - .withMaxAttempts(Integer.MAX_VALUE); - this.sqsClient = SqsClient.builder().region(Region.of(System.getProperty(AWS_REGION))).build(); - executorService = Executors.newSingleThreadScheduledExecutor(); - } - - public SqsSourceTask createObjectUnderTest(final SqsOptions sqsOptions){ - SqsService sqsService = new SqsService(sqsMetrics,sqsClient,backoff); - SqsMessageHandler sqsHandler = new RawSqsMessageHandler(sqsService); - return new SqsSourceTask(buffer, RECORDS_TO_ACCUMULATE, BUFFER_TIMEOUT - ,sqsService,sqsOptions,sqsMetrics, - acknowledgementSetManager,Boolean.FALSE,sqsHandler); - } - - private static List pushMessagesToQueue(SqsRecordsGenerator sqsRecordsGenerator, String queueUrl,final int load) { - List inputDataList = new ArrayList<>(); - for(int msgCount = 0; msgCount < load/2; msgCount++) - generateMessagesForSqsPush().forEach(obj -> inputDataList.add(obj)); - sqsRecordsGenerator.pushMessages(inputDataList, queueUrl); - return inputDataList; - } - - private static List generateMessagesForSqsPush(){ - List messages = new ArrayList<>(2); - messages.add(JSON_MESSAGE); - messages.add(LOG_MESSAGE); - return messages; - } - - private BlockingBuffer> getBuffer(final int bufferSize) { - final HashMap integerHashMap = new HashMap<>(); - integerHashMap.put("buffer_size", bufferSize); - integerHashMap.put("batch_size", bufferSize); - final PluginSetting pluginSetting = new PluginSetting("blocking_buffer", integerHashMap); - pluginSetting.setPipelineName(TEST_PIPELINE_NAME); - return new BlockingBuffer<>(pluginSetting); - } -} diff --git a/data-prepper-plugins/sqs-source/src/main/java/org/opensearch/dataprepper/plugins/source/sqs/AwsAuthenticationAdapter.java b/data-prepper-plugins/sqs-source/src/main/java/org/opensearch/dataprepper/plugins/source/sqs/AwsAuthenticationAdapter.java new file mode 100644 index 0000000000..08600cba13 --- /dev/null +++ b/data-prepper-plugins/sqs-source/src/main/java/org/opensearch/dataprepper/plugins/source/sqs/AwsAuthenticationAdapter.java @@ -0,0 +1,36 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.sqs; + +import org.opensearch.dataprepper.aws.api.AwsCredentialsOptions; +import org.opensearch.dataprepper.aws.api.AwsCredentialsSupplier; +import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; + +class AwsAuthenticationAdapter { + private final AwsCredentialsSupplier awsCredentialsSupplier; + private final SqsSourceConfig sqsSourceConfig; + + + AwsAuthenticationAdapter( + final AwsCredentialsSupplier awsCredentialsSupplier, + final SqsSourceConfig sqsSourceConfig) { + this.awsCredentialsSupplier = awsCredentialsSupplier; + this.sqsSourceConfig = sqsSourceConfig; + } + + AwsCredentialsProvider getCredentialsProvider() { + final AwsAuthenticationOptions awsAuthenticationOptions = sqsSourceConfig.getAwsAuthenticationOptions(); + + final AwsCredentialsOptions options = AwsCredentialsOptions.builder() + .withStsRoleArn(awsAuthenticationOptions.getAwsStsRoleArn()) + .withRegion(awsAuthenticationOptions.getAwsRegion()) + .withStsHeaderOverrides(awsAuthenticationOptions.getAwsStsHeaderOverrides()) + .withStsExternalId(awsAuthenticationOptions.getAwsStsExternalId()) + .build(); + + return awsCredentialsSupplier.getProvider(options); + } +} diff --git a/data-prepper-plugins/sqs-source/src/main/java/org/opensearch/dataprepper/plugins/source/sqs/AwsAuthenticationOptions.java b/data-prepper-plugins/sqs-source/src/main/java/org/opensearch/dataprepper/plugins/source/sqs/AwsAuthenticationOptions.java new file mode 100644 index 0000000000..99da366de3 --- /dev/null +++ b/data-prepper-plugins/sqs-source/src/main/java/org/opensearch/dataprepper/plugins/source/sqs/AwsAuthenticationOptions.java @@ -0,0 +1,69 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + + package org.opensearch.dataprepper.plugins.source.sqs; + + import com.fasterxml.jackson.annotation.JsonProperty; + import jakarta.validation.constraints.Size; + import software.amazon.awssdk.arns.Arn; + import software.amazon.awssdk.regions.Region; + + import java.util.Map; + import java.util.Optional; + + public class AwsAuthenticationOptions { + private static final String AWS_IAM_ROLE = "role"; + private static final String AWS_IAM = "iam"; + + @JsonProperty("region") + @Size(min = 1, message = "Region cannot be empty string") + private String awsRegion; + + @JsonProperty("sts_role_arn") + @Size(min = 20, max = 2048, message = "awsStsRoleArn length should be between 1 and 2048 characters") + private String awsStsRoleArn; + + @JsonProperty("sts_external_id") + @Size(min = 2, max = 1224, message = "awsStsExternalId length should be between 2 and 1224 characters") + private String awsStsExternalId; + + @JsonProperty("sts_header_overrides") + @Size(max = 5, message = "sts_header_overrides supports a maximum of 5 headers to override") + private Map awsStsHeaderOverrides; + + void validateStsRoleArn() { + final Arn arn = getArn(); + if (!AWS_IAM.equals(arn.service())) { + throw new IllegalArgumentException("sts_role_arn must be an IAM Role"); + } + final Optional resourceType = arn.resource().resourceType(); + if (resourceType.isEmpty() || !resourceType.get().equals(AWS_IAM_ROLE)) { + throw new IllegalArgumentException("sts_role_arn must be an IAM Role"); + } + } + + private Arn getArn() { + try { + return Arn.fromString(awsStsRoleArn); + } catch (final Exception e) { + throw new IllegalArgumentException(String.format("The value provided for sts_role_arn is not a valid AWS ARN. Provided value: %s", awsStsRoleArn)); } + } + + public String getAwsStsRoleArn() { + return awsStsRoleArn; + } + + public String getAwsStsExternalId() { + return awsStsExternalId; + } + + public Region getAwsRegion() { + return awsRegion != null ? Region.of(awsRegion) : null; + } + + public Map getAwsStsHeaderOverrides() { + return awsStsHeaderOverrides; + } + } \ No newline at end of file diff --git a/data-prepper-plugins/sqs-source/src/main/java/org/opensearch/dataprepper/plugins/source/sqs/QueueConfig.java b/data-prepper-plugins/sqs-source/src/main/java/org/opensearch/dataprepper/plugins/source/sqs/QueueConfig.java new file mode 100644 index 0000000000..ca5566d6cd --- /dev/null +++ b/data-prepper-plugins/sqs-source/src/main/java/org/opensearch/dataprepper/plugins/source/sqs/QueueConfig.java @@ -0,0 +1,97 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.sqs; + +import com.fasterxml.jackson.annotation.JsonProperty; + +import jakarta.validation.Valid; +import jakarta.validation.constraints.Max; +import jakarta.validation.constraints.Min; +import jakarta.validation.constraints.NotNull; +import java.time.Duration; + +import org.hibernate.validator.constraints.time.DurationMax; +import org.hibernate.validator.constraints.time.DurationMin; + +public class QueueConfig { + + private static final Integer DEFAULT_MAXIMUM_MESSAGES = null; + private static final boolean DEFAULT_VISIBILITY_DUPLICATE_PROTECTION = false; + private static final Duration DEFAULT_VISIBILITY_TIMEOUT_SECONDS = null; + private static final Duration DEFAULT_VISIBILITY_DUPLICATE_PROTECTION_TIMEOUT = Duration.ofHours(2); + private static final Duration DEFAULT_WAIT_TIME_SECONDS = null; + private static final Duration DEFAULT_POLL_DELAY_SECONDS = Duration.ofSeconds(0); + static final int DEFAULT_NUMBER_OF_WORKERS = 1; + + @JsonProperty("url") + @NotNull + private String url; + + @JsonProperty("workers") + @Valid + private int numWorkers = DEFAULT_NUMBER_OF_WORKERS; + + @JsonProperty("maximum_messages") + @Min(1) + @Max(10) + private Integer maximumMessages = DEFAULT_MAXIMUM_MESSAGES; + + @JsonProperty("poll_delay") + @DurationMin(seconds = 0) + private Duration pollDelay = DEFAULT_POLL_DELAY_SECONDS; + + @JsonProperty("visibility_timeout") + @DurationMin(seconds = 0) + @DurationMax(seconds = 43200) + private Duration visibilityTimeout = DEFAULT_VISIBILITY_TIMEOUT_SECONDS; + + @JsonProperty("visibility_duplication_protection") + @NotNull + private boolean visibilityDuplicateProtection = DEFAULT_VISIBILITY_DUPLICATE_PROTECTION; + + @JsonProperty("visibility_duplicate_protection_timeout") + @DurationMin(seconds = 30) + @DurationMax(hours = 24) + private Duration visibilityDuplicateProtectionTimeout = DEFAULT_VISIBILITY_DUPLICATE_PROTECTION_TIMEOUT; + + @JsonProperty("wait_time") + @DurationMin(seconds = 0) + @DurationMax(seconds = 20) + private Duration waitTime = DEFAULT_WAIT_TIME_SECONDS; + + public String getUrl() { + return url; + } + + public Integer getMaximumMessages() { + return maximumMessages; + } + + public int getNumWorkers() { + return numWorkers; + } + + public Duration getVisibilityTimeout() { + return visibilityTimeout; + } + + public boolean getVisibilityDuplicateProtection() { + return visibilityDuplicateProtection; + } + + public Duration getVisibilityDuplicateProtectionTimeout() { + return visibilityDuplicateProtectionTimeout; + } + + public Duration getWaitTime() { + return waitTime; + } + + public Duration getPollDelay() { + return pollDelay; + } +} + diff --git a/data-prepper-plugins/sqs-source/src/main/java/org/opensearch/dataprepper/plugins/source/sqs/RawSqsMessageHandler.java b/data-prepper-plugins/sqs-source/src/main/java/org/opensearch/dataprepper/plugins/source/sqs/RawSqsMessageHandler.java new file mode 100644 index 0000000000..493b7ab7d7 --- /dev/null +++ b/data-prepper-plugins/sqs-source/src/main/java/org/opensearch/dataprepper/plugins/source/sqs/RawSqsMessageHandler.java @@ -0,0 +1,65 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.sqs; + +import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSet; +import org.opensearch.dataprepper.model.buffer.Buffer; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.event.EventMetadata; +import org.opensearch.dataprepper.model.event.JacksonEvent; +import org.opensearch.dataprepper.model.record.Record; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import software.amazon.awssdk.services.sqs.model.Message; +import software.amazon.awssdk.services.sqs.model.MessageAttributeValue; +import software.amazon.awssdk.services.sqs.model.MessageSystemAttributeName; + +import java.util.Collections; +import java.util.Map; + +public class RawSqsMessageHandler implements SqsMessageHandler { + + private static final Logger LOG = LoggerFactory.getLogger(RawSqsMessageHandler.class); + + @Override + public void handleMessage(final Message message, + final String url, + final Buffer> buffer, + final int bufferTimeoutMillis, + final AcknowledgementSet acknowledgementSet) { + try { + final Map systemAttributes = message.attributes(); + final Map customAttributes = message.messageAttributes(); + final Event event = JacksonEvent.builder() + .withEventType("DOCUMENT") + .withData(Collections.singletonMap("message", message.body())) + .build(); + + final EventMetadata eventMetadata = event.getMetadata(); + eventMetadata.setAttribute("queueUrl", url); + + for (Map.Entry entry : systemAttributes.entrySet()) { + String originalKey = entry.getKey().toString(); + String lowerCamelCaseKey = originalKey.substring(0, 1).toLowerCase() + originalKey.substring(1); + eventMetadata.setAttribute(lowerCamelCaseKey, entry.getValue()); + } + + for (Map.Entry entry : customAttributes.entrySet()) { + String originalKey = entry.getKey(); + String lowerCamelCaseKey = originalKey.substring(0, 1).toLowerCase() + originalKey.substring(1); + eventMetadata.setAttribute(lowerCamelCaseKey, entry.getValue().stringValue()); + } + + if (acknowledgementSet != null) { + acknowledgementSet.add(event); + } + buffer.write(new Record<>(event), bufferTimeoutMillis); + } catch (Exception e) { + LOG.error("Error processing SQS message: {}", e.getMessage(), e); + throw new RuntimeException(e); + } + } +} diff --git a/data-prepper-plugins/sqs-source/src/main/java/org/opensearch/dataprepper/plugins/source/sqs/SqsEventProcessor.java b/data-prepper-plugins/sqs-source/src/main/java/org/opensearch/dataprepper/plugins/source/sqs/SqsEventProcessor.java new file mode 100644 index 0000000000..a03c485c37 --- /dev/null +++ b/data-prepper-plugins/sqs-source/src/main/java/org/opensearch/dataprepper/plugins/source/sqs/SqsEventProcessor.java @@ -0,0 +1,29 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.sqs; + +import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSet; +import org.opensearch.dataprepper.model.buffer.Buffer; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.record.Record; +import software.amazon.awssdk.services.sqs.model.Message; +import java.io.IOException; + +public class SqsEventProcessor { + private final SqsMessageHandler sqsMessageHandler; + SqsEventProcessor(final SqsMessageHandler sqsMessageHandler) { + this.sqsMessageHandler= sqsMessageHandler; + } + + void addSqsObject(final Message message, + final String url, + final Buffer> buffer, + final int bufferTimeoutmillis, + final AcknowledgementSet acknowledgementSet) throws IOException { + sqsMessageHandler.handleMessage(message, url, buffer, bufferTimeoutmillis, acknowledgementSet); + } + +} \ No newline at end of file diff --git a/data-prepper-plugins/sqs-source/src/main/java/org/opensearch/dataprepper/plugins/source/sqs/SqsMessageHandler.java b/data-prepper-plugins/sqs-source/src/main/java/org/opensearch/dataprepper/plugins/source/sqs/SqsMessageHandler.java new file mode 100644 index 0000000000..79012b5e00 --- /dev/null +++ b/data-prepper-plugins/sqs-source/src/main/java/org/opensearch/dataprepper/plugins/source/sqs/SqsMessageHandler.java @@ -0,0 +1,20 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.source.sqs; + +import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSet; +import org.opensearch.dataprepper.model.buffer.Buffer; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.record.Record; +import software.amazon.awssdk.services.sqs.model.Message; +import java.io.IOException; + +public interface SqsMessageHandler { + void handleMessage(final Message message, + final String url, + final Buffer> buffer, + final int bufferTimeoutMillis, + final AcknowledgementSet acknowledgementSet) throws IOException ; +} diff --git a/data-prepper-plugins/sqs-source/src/main/java/org/opensearch/dataprepper/plugins/source/sqs/SqsRetriesExhaustedException.java b/data-prepper-plugins/sqs-source/src/main/java/org/opensearch/dataprepper/plugins/source/sqs/SqsRetriesExhaustedException.java new file mode 100644 index 0000000000..4e1f9507e6 --- /dev/null +++ b/data-prepper-plugins/sqs-source/src/main/java/org/opensearch/dataprepper/plugins/source/sqs/SqsRetriesExhaustedException.java @@ -0,0 +1,18 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.sqs; + +/** + * This exception is thrown when SQS retries are exhausted + * + * @since 2.1 + */ +public class SqsRetriesExhaustedException extends RuntimeException { + + public SqsRetriesExhaustedException(final String errorMessage) { + super(errorMessage); + } +} \ No newline at end of file diff --git a/data-prepper-plugins/sqs-source/src/main/java/org/opensearch/dataprepper/plugins/source/sqs/SqsService.java b/data-prepper-plugins/sqs-source/src/main/java/org/opensearch/dataprepper/plugins/source/sqs/SqsService.java new file mode 100644 index 0000000000..d53f269323 --- /dev/null +++ b/data-prepper-plugins/sqs-source/src/main/java/org/opensearch/dataprepper/plugins/source/sqs/SqsService.java @@ -0,0 +1,128 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + + package org.opensearch.dataprepper.plugins.source.sqs; + + import com.linecorp.armeria.client.retry.Backoff; + import org.opensearch.dataprepper.common.concurrent.BackgroundThreadFactory; + import org.opensearch.dataprepper.metrics.PluginMetrics; + import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSetManager; + import org.slf4j.Logger; + import org.slf4j.LoggerFactory; + import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; + import software.amazon.awssdk.core.client.config.ClientOverrideConfiguration; + import software.amazon.awssdk.core.retry.RetryPolicy; + import software.amazon.awssdk.services.sqs.SqsClient; + import org.opensearch.dataprepper.model.buffer.Buffer; + import org.opensearch.dataprepper.model.event.Event; + import org.opensearch.dataprepper.model.record.Record; + import java.time.Duration; + import java.util.ArrayList; + import java.util.List; + import java.util.concurrent.TimeUnit; + import java.util.concurrent.Executors; + import java.util.concurrent.ExecutorService; + import java.util.stream.Collectors; + import java.util.stream.IntStream; + + public class SqsService { + private static final Logger LOG = LoggerFactory.getLogger(SqsService.class); + static final long SHUTDOWN_TIMEOUT = 30L; + static final long INITIAL_DELAY = Duration.ofSeconds(20).toMillis(); + static final long MAXIMUM_DELAY = Duration.ofMinutes(5).toMillis(); + static final double JITTER_RATE = 0.20; + + private final SqsSourceConfig sqsSourceConfig; + private final SqsEventProcessor sqsEventProcessor; + private final SqsClient sqsClient; + private final PluginMetrics pluginMetrics; + private final AcknowledgementSetManager acknowledgementSetManager; + private final List allSqsUrlExecutorServices; + private final List sqsWorkers; + private final Buffer> buffer; + + public SqsService(final Buffer> buffer, + final AcknowledgementSetManager acknowledgementSetManager, + final SqsSourceConfig sqsSourceConfig, + final SqsEventProcessor sqsEventProcessor, + final PluginMetrics pluginMetrics, + final AwsCredentialsProvider credentialsProvider) { + + this.sqsSourceConfig = sqsSourceConfig; + this.sqsEventProcessor = sqsEventProcessor; + this.pluginMetrics = pluginMetrics; + this.acknowledgementSetManager = acknowledgementSetManager; + this.allSqsUrlExecutorServices = new ArrayList<>(); + this.sqsWorkers = new ArrayList<>(); + this.sqsClient = createSqsClient(credentialsProvider); + this.buffer = buffer; + } + + + public void start() { + final Backoff backoff = Backoff.exponential(INITIAL_DELAY, MAXIMUM_DELAY).withJitter(JITTER_RATE) + .withMaxAttempts(Integer.MAX_VALUE); + + LOG.info("Starting SqsService"); + + sqsSourceConfig.getQueues().forEach(queueConfig -> { + String queueUrl = queueConfig.getUrl(); + String queueName = queueUrl.substring(queueUrl.lastIndexOf('/') + 1); + + int numWorkers = queueConfig.getNumWorkers(); + ExecutorService executorService = Executors.newFixedThreadPool( + numWorkers, BackgroundThreadFactory.defaultExecutorThreadFactory("sqs-source" + queueName)); + allSqsUrlExecutorServices.add(executorService); + List workers = IntStream.range(0, numWorkers) + .mapToObj(i -> new SqsWorker( + buffer, + acknowledgementSetManager, + sqsClient, + sqsEventProcessor, + sqsSourceConfig, + queueConfig, + pluginMetrics, + backoff)) + .collect(Collectors.toList()); + + sqsWorkers.addAll(workers); + workers.forEach(executorService::submit); + LOG.info("Started SQS workers for queue {} with {} workers", queueUrl, numWorkers); + }); + } + + SqsClient createSqsClient(final AwsCredentialsProvider credentialsProvider) { + LOG.debug("Creating SQS client"); + return SqsClient.builder() + .region(sqsSourceConfig.getAwsAuthenticationOptions().getAwsRegion()) + .credentialsProvider(credentialsProvider) + .overrideConfiguration(ClientOverrideConfiguration.builder() + .retryPolicy(RetryPolicy.builder().numRetries(5).build()) + .build()) + .build(); + } + + public void stop() { + allSqsUrlExecutorServices.forEach(ExecutorService::shutdown); + sqsWorkers.forEach(SqsWorker::stop); + allSqsUrlExecutorServices.forEach(executorService -> { + try { + if (!executorService.awaitTermination(SHUTDOWN_TIMEOUT, TimeUnit.SECONDS)) { + LOG.warn("Failed to terminate SqsWorkers"); + executorService.shutdownNow(); + } + } catch (InterruptedException e) { + LOG.error("Interrupted during shutdown, exiting uncleanly...", e); + executorService.shutdownNow(); + Thread.currentThread().interrupt(); + } + }); + + sqsClient.close(); + LOG.info("SqsService shutdown completed."); + } + + } + \ No newline at end of file diff --git a/data-prepper-plugins/sqs-source/src/main/java/org/opensearch/dataprepper/plugins/source/sqs/SqsSource.java b/data-prepper-plugins/sqs-source/src/main/java/org/opensearch/dataprepper/plugins/source/sqs/SqsSource.java new file mode 100644 index 0000000000..980e59048b --- /dev/null +++ b/data-prepper-plugins/sqs-source/src/main/java/org/opensearch/dataprepper/plugins/source/sqs/SqsSource.java @@ -0,0 +1,69 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.sqs; + +import org.opensearch.dataprepper.aws.api.AwsCredentialsSupplier; +import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSetManager; +import org.opensearch.dataprepper.model.annotations.DataPrepperPlugin; +import org.opensearch.dataprepper.model.annotations.DataPrepperPluginConstructor; +import org.opensearch.dataprepper.model.buffer.Buffer; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.model.source.Source; +import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; +import java.util.Objects; + +@DataPrepperPlugin(name = "sqs", pluginType = Source.class,pluginConfigurationType = SqsSourceConfig.class) +public class SqsSource implements Source> { + + private final PluginMetrics pluginMetrics; + private final SqsSourceConfig sqsSourceConfig; + private SqsService sqsService; + private final AcknowledgementSetManager acknowledgementSetManager; + private final AwsCredentialsSupplier awsCredentialsSupplier; + private final boolean acknowledgementsEnabled; + + + @DataPrepperPluginConstructor + public SqsSource(final PluginMetrics pluginMetrics, + final SqsSourceConfig sqsSourceConfig, + final AcknowledgementSetManager acknowledgementSetManager, + final AwsCredentialsSupplier awsCredentialsSupplier) { + + this.pluginMetrics = pluginMetrics; + this.sqsSourceConfig = sqsSourceConfig; + this.acknowledgementsEnabled = sqsSourceConfig.getAcknowledgements(); + this.acknowledgementSetManager = acknowledgementSetManager; + this.awsCredentialsSupplier = awsCredentialsSupplier; + + } + + @Override + public void start(Buffer> buffer) { + if (buffer == null) { + throw new IllegalStateException("Buffer is null"); + } + final AwsAuthenticationAdapter awsAuthenticationAdapter = new AwsAuthenticationAdapter(awsCredentialsSupplier, sqsSourceConfig); + final AwsCredentialsProvider credentialsProvider = awsAuthenticationAdapter.getCredentialsProvider(); + final SqsMessageHandler rawSqsMessageHandler = new RawSqsMessageHandler(); + final SqsEventProcessor sqsEventProcessor = new SqsEventProcessor(rawSqsMessageHandler); + sqsService = new SqsService(buffer, acknowledgementSetManager, sqsSourceConfig, sqsEventProcessor, pluginMetrics, credentialsProvider); + sqsService.start(); + } + + @Override + public boolean areAcknowledgementsEnabled() { + return acknowledgementsEnabled; + } + + @Override + public void stop() { + if (Objects.nonNull(sqsService)) { + sqsService.stop(); + } + } +} diff --git a/data-prepper-plugins/sqs-source/src/main/java/org/opensearch/dataprepper/plugins/source/sqs/SqsSourceConfig.java b/data-prepper-plugins/sqs-source/src/main/java/org/opensearch/dataprepper/plugins/source/sqs/SqsSourceConfig.java new file mode 100644 index 0000000000..c84a3a3d69 --- /dev/null +++ b/data-prepper-plugins/sqs-source/src/main/java/org/opensearch/dataprepper/plugins/source/sqs/SqsSourceConfig.java @@ -0,0 +1,50 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.sqs; + +import com.fasterxml.jackson.annotation.JsonProperty; +import jakarta.validation.Valid; +import jakarta.validation.constraints.NotNull; +import java.time.Duration; +import java.util.List; + +public class SqsSourceConfig { + + static final Duration DEFAULT_BUFFER_TIMEOUT = Duration.ofSeconds(10); + static final int DEFAULT_NUMBER_OF_RECORDS_TO_ACCUMULATE = 100; + + @JsonProperty("aws") + @NotNull + @Valid + private AwsAuthenticationOptions awsAuthenticationOptions; + + @JsonProperty("acknowledgments") + private boolean acknowledgments = false; + + @JsonProperty("buffer_timeout") + private Duration bufferTimeout = DEFAULT_BUFFER_TIMEOUT; + + @JsonProperty("queues") + @NotNull + @Valid + private List queues; + + public AwsAuthenticationOptions getAwsAuthenticationOptions() { + return awsAuthenticationOptions; + } + + public boolean getAcknowledgements() { + return acknowledgments; + } + + public Duration getBufferTimeout() { + return bufferTimeout; + } + + public List getQueues() { + return queues; + } +} diff --git a/data-prepper-plugins/sqs-source/src/main/java/org/opensearch/dataprepper/plugins/source/sqs/SqsWorker.java b/data-prepper-plugins/sqs-source/src/main/java/org/opensearch/dataprepper/plugins/source/sqs/SqsWorker.java new file mode 100644 index 0000000000..3f58906b33 --- /dev/null +++ b/data-prepper-plugins/sqs-source/src/main/java/org/opensearch/dataprepper/plugins/source/sqs/SqsWorker.java @@ -0,0 +1,326 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.sqs; + +import com.linecorp.armeria.client.retry.Backoff; +import io.micrometer.core.instrument.Counter; +import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSet; +import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSetManager; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.record.Record; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import software.amazon.awssdk.core.exception.SdkException; +import software.amazon.awssdk.services.sqs.SqsClient; +import software.amazon.awssdk.services.sqs.model.ChangeMessageVisibilityRequest; +import software.amazon.awssdk.services.sqs.model.DeleteMessageBatchRequest; +import software.amazon.awssdk.services.sqs.model.DeleteMessageBatchRequestEntry; +import software.amazon.awssdk.services.sqs.model.DeleteMessageBatchResponse; +import software.amazon.awssdk.services.sqs.model.Message; +import software.amazon.awssdk.services.sqs.model.ReceiveMessageRequest; +import software.amazon.awssdk.services.sqs.model.ReceiveMessageResponse; +import software.amazon.awssdk.services.sqs.model.SqsException; +import software.amazon.awssdk.services.sts.model.StsException; +import org.opensearch.dataprepper.model.buffer.Buffer; + +import java.time.Duration; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; + + +public class SqsWorker implements Runnable { + private static final Logger LOG = LoggerFactory.getLogger(SqsWorker.class); + static final String SQS_MESSAGES_RECEIVED_METRIC_NAME = "sqsMessagesReceived"; + static final String SQS_MESSAGES_DELETED_METRIC_NAME = "sqsMessagesDeleted"; + static final String SQS_MESSAGES_FAILED_METRIC_NAME = "sqsMessagesFailed"; + static final String SQS_MESSAGES_DELETE_FAILED_METRIC_NAME = "sqsMessagesDeleteFailed"; + static final String SQS_VISIBILITY_TIMEOUT_CHANGED_COUNT_METRIC_NAME = "sqsVisibilityTimeoutChangedCount"; + static final String SQS_VISIBILITY_TIMEOUT_CHANGE_FAILED_COUNT_METRIC_NAME = "sqsVisibilityTimeoutChangeFailedCount"; + static final String ACKNOWLEDGEMENT_SET_CALLACK_METRIC_NAME = "acknowledgementSetCallbackCounter"; + + private final SqsClient sqsClient; + private final SqsEventProcessor sqsEventProcessor; + private final Counter sqsMessagesReceivedCounter; + private final Counter sqsMessagesDeletedCounter; + private final Counter sqsMessagesFailedCounter; + private final Counter sqsMessagesDeleteFailedCounter; + private final Counter acknowledgementSetCallbackCounter; + private final Counter sqsVisibilityTimeoutChangedCount; + private final Counter sqsVisibilityTimeoutChangeFailedCount; + private final Backoff standardBackoff; + private final QueueConfig queueConfig; + private int failedAttemptCount; + private final boolean endToEndAcknowledgementsEnabled; + private final AcknowledgementSetManager acknowledgementSetManager; + private volatile boolean isStopped = false; + private final Buffer> buffer; + private final int bufferTimeoutMillis; + private Map messageVisibilityTimesMap; + + public SqsWorker(final Buffer> buffer, + final AcknowledgementSetManager acknowledgementSetManager, + final SqsClient sqsClient, + final SqsEventProcessor sqsEventProcessor, + final SqsSourceConfig sqsSourceConfig, + final QueueConfig queueConfig, + final PluginMetrics pluginMetrics, + final Backoff backoff) { + + this.sqsClient = sqsClient; + this.sqsEventProcessor = sqsEventProcessor; + this.queueConfig = queueConfig; + this.acknowledgementSetManager = acknowledgementSetManager; + this.standardBackoff = backoff; + this.endToEndAcknowledgementsEnabled = sqsSourceConfig.getAcknowledgements(); + this.buffer = buffer; + this.bufferTimeoutMillis = (int) sqsSourceConfig.getBufferTimeout().toMillis(); + + messageVisibilityTimesMap = new HashMap<>(); + failedAttemptCount = 0; + sqsMessagesReceivedCounter = pluginMetrics.counter(SQS_MESSAGES_RECEIVED_METRIC_NAME); + sqsMessagesDeletedCounter = pluginMetrics.counter(SQS_MESSAGES_DELETED_METRIC_NAME); + sqsMessagesFailedCounter = pluginMetrics.counter(SQS_MESSAGES_FAILED_METRIC_NAME); + sqsMessagesDeleteFailedCounter = pluginMetrics.counter(SQS_MESSAGES_DELETE_FAILED_METRIC_NAME); + acknowledgementSetCallbackCounter = pluginMetrics.counter(ACKNOWLEDGEMENT_SET_CALLACK_METRIC_NAME); + sqsVisibilityTimeoutChangedCount = pluginMetrics.counter(SQS_VISIBILITY_TIMEOUT_CHANGED_COUNT_METRIC_NAME); + sqsVisibilityTimeoutChangeFailedCount = pluginMetrics.counter(SQS_VISIBILITY_TIMEOUT_CHANGE_FAILED_COUNT_METRIC_NAME); + } + + @Override + public void run() { + while (!isStopped) { + int messagesProcessed = 0; + try { + messagesProcessed = processSqsMessages(); + + } catch (final Exception e) { + LOG.error("Unable to process SQS messages. Processing error due to: {}", e.getMessage()); + applyBackoff(); + } + + if (messagesProcessed > 0 && queueConfig.getPollDelay().toMillis() > 0) { + try { + Thread.sleep(queueConfig.getPollDelay().toMillis()); + } catch (final InterruptedException e) { + LOG.error("Thread is interrupted while polling SQS.", e); + } + } + } + } + + int processSqsMessages() { + final List messages = getMessagesFromSqs(); + if (!messages.isEmpty()) { + sqsMessagesReceivedCounter.increment(messages.size()); + final List deleteMessageBatchRequestEntries = processSqsEvents(messages); + if (!deleteMessageBatchRequestEntries.isEmpty()) { + deleteSqsMessages(deleteMessageBatchRequestEntries); + } + } + return messages.size(); + } + + private List getMessagesFromSqs() { + try { + final ReceiveMessageRequest request = createReceiveMessageRequest(); + final ReceiveMessageResponse response = sqsClient.receiveMessage(request); + List messages = response.messages(); + failedAttemptCount = 0; + return messages; + + } catch (final SqsException | StsException e) { + LOG.error("Error reading from SQS: {}. Retrying with exponential backoff.", e.getMessage()); + applyBackoff(); + return Collections.emptyList(); + } + } + + private void applyBackoff() { + final long delayMillis = standardBackoff.nextDelayMillis(++failedAttemptCount); + if (delayMillis < 0) { + Thread.currentThread().interrupt(); + throw new SqsRetriesExhaustedException("SQS retries exhausted. Make sure that SQS configuration is valid, SQS queue exists, and IAM role has required permissions."); + } + final Duration delayDuration = Duration.ofMillis(delayMillis); + LOG.info("Pausing SQS processing for {}.{} seconds due to an error in processing.", + delayDuration.getSeconds(), delayDuration.toMillisPart()); + try { + Thread.sleep(delayMillis); + } catch (final InterruptedException e){ + LOG.error("Thread is interrupted while polling SQS with retry.", e); + } + } + + private ReceiveMessageRequest createReceiveMessageRequest() { + ReceiveMessageRequest.Builder requestBuilder = ReceiveMessageRequest.builder() + .queueUrl(queueConfig.getUrl()) + .attributeNamesWithStrings("All") + .messageAttributeNames("All"); + + if (queueConfig.getWaitTime() != null) { + requestBuilder.waitTimeSeconds((int) queueConfig.getWaitTime().getSeconds()); + } + if (queueConfig.getMaximumMessages() != null) { + requestBuilder.maxNumberOfMessages(queueConfig.getMaximumMessages()); + } + if (queueConfig.getVisibilityTimeout() != null) { + requestBuilder.visibilityTimeout((int) queueConfig.getVisibilityTimeout().getSeconds()); + } + return requestBuilder.build(); + } + + private List processSqsEvents(final List messages) { + final List deleteMessageBatchRequestEntryCollection = new ArrayList<>(); + final Map messageAcknowledgementSetMap = new HashMap<>(); + final Map> messageWaitingForAcknowledgementsMap = new HashMap<>(); + + for (Message message : messages) { + List waitingForAcknowledgements = new ArrayList<>(); + AcknowledgementSet acknowledgementSet = null; + + final int visibilityTimeout; + if (queueConfig.getVisibilityTimeout() != null) { + visibilityTimeout = (int) queueConfig.getVisibilityTimeout().getSeconds(); + } else { + visibilityTimeout = (int) Duration.ofSeconds(30).getSeconds(); + + } + + final int maxVisibilityTimeout = (int)queueConfig.getVisibilityDuplicateProtectionTimeout().getSeconds(); + final int progressCheckInterval = visibilityTimeout/2 - 1; + if (endToEndAcknowledgementsEnabled) { + int expiryTimeout = visibilityTimeout - 2; + final boolean visibilityDuplicateProtectionEnabled = queueConfig.getVisibilityDuplicateProtection(); + if (visibilityDuplicateProtectionEnabled) { + expiryTimeout = maxVisibilityTimeout; + } + acknowledgementSet = acknowledgementSetManager.create( + (result) -> { + acknowledgementSetCallbackCounter.increment(); + // Delete only if this is positive acknowledgement + if (visibilityDuplicateProtectionEnabled) { + messageVisibilityTimesMap.remove(message); + } + if (result) { + deleteSqsMessages(waitingForAcknowledgements); + } + }, + Duration.ofSeconds(expiryTimeout)); + if (visibilityDuplicateProtectionEnabled) { + acknowledgementSet.addProgressCheck( + (ratio) -> { + int newValue = messageVisibilityTimesMap.getOrDefault(message, visibilityTimeout) + progressCheckInterval; + if (newValue >= maxVisibilityTimeout) { + return; + } + messageVisibilityTimesMap.put(message, newValue); + final int newVisibilityTimeoutSeconds = visibilityTimeout; + increaseVisibilityTimeout(message, newVisibilityTimeoutSeconds); + }, + Duration.ofSeconds(progressCheckInterval)); + } + messageAcknowledgementSetMap.put(message, acknowledgementSet); + messageWaitingForAcknowledgementsMap.put(message, waitingForAcknowledgements); + } + } + + if (endToEndAcknowledgementsEnabled) { + LOG.debug("Created acknowledgement sets for {} messages.", messages.size()); + } + + for (Message message : messages) { + final AcknowledgementSet acknowledgementSet = messageAcknowledgementSetMap.get(message); + final List waitingForAcknowledgements = messageWaitingForAcknowledgementsMap.get(message); + final Optional deleteMessageBatchRequestEntry = processSqsObject(message, acknowledgementSet); + if (endToEndAcknowledgementsEnabled) { + deleteMessageBatchRequestEntry.ifPresent(waitingForAcknowledgements::add); + acknowledgementSet.complete(); + } else { + deleteMessageBatchRequestEntry.ifPresent(deleteMessageBatchRequestEntryCollection::add); + } + } + + return deleteMessageBatchRequestEntryCollection; + } + + + private Optional processSqsObject( + final Message message, + final AcknowledgementSet acknowledgementSet) { + try { + sqsEventProcessor.addSqsObject(message, queueConfig.getUrl(), buffer, bufferTimeoutMillis, acknowledgementSet); + return Optional.of(buildDeleteMessageBatchRequestEntry(message)); + } catch (final Exception e) { + sqsMessagesFailedCounter.increment(); + LOG.error("Error processing from SQS: {}. Retrying with exponential backoff.", e.getMessage()); + applyBackoff(); + return Optional.empty(); + } + } + + private void increaseVisibilityTimeout(final Message message, final int newVisibilityTimeoutSeconds) { + if(isStopped) { + LOG.info("Some messages are pending completion of acknowledgments. Data Prepper will not increase the visibility timeout because it is shutting down. {}", message); + return; + } + final ChangeMessageVisibilityRequest changeMessageVisibilityRequest = ChangeMessageVisibilityRequest.builder() + .visibilityTimeout(newVisibilityTimeoutSeconds) + .queueUrl(queueConfig.getUrl()) + .receiptHandle(message.receiptHandle()) + .build(); + + try { + sqsClient.changeMessageVisibility(changeMessageVisibilityRequest); + sqsVisibilityTimeoutChangedCount.increment(); + LOG.debug("Set visibility timeout for message {} to {}", message.messageId(), newVisibilityTimeoutSeconds); + } catch (Exception e) { + LOG.error("Failed to set visibility timeout for message {} to {}", message.messageId(), newVisibilityTimeoutSeconds, e); + sqsVisibilityTimeoutChangeFailedCount.increment(); + } + } + + + private DeleteMessageBatchRequestEntry buildDeleteMessageBatchRequestEntry(Message message) { + return DeleteMessageBatchRequestEntry.builder() + .id(message.messageId()) + .receiptHandle(message.receiptHandle()) + .build(); + } + + private void deleteSqsMessages(final List deleteEntries) { + if (deleteEntries.isEmpty()) return; + + try { + DeleteMessageBatchRequest deleteRequest = DeleteMessageBatchRequest.builder() + .queueUrl(queueConfig.getUrl()) + .entries(deleteEntries) + .build(); + DeleteMessageBatchResponse response = sqsClient.deleteMessageBatch(deleteRequest); + + if (response.hasSuccessful()) { + int successfulDeletes = response.successful().size(); + sqsMessagesDeletedCounter.increment(successfulDeletes); + } + if (response.hasFailed()) { + int failedDeletes = response.failed().size(); + sqsMessagesDeleteFailedCounter.increment(failedDeletes); + LOG.error("Failed to delete {} messages from SQS.", failedDeletes); + } + } catch (SdkException e) { + LOG.error("Failed to delete messages from SQS: {}", e.getMessage()); + sqsMessagesDeleteFailedCounter.increment(deleteEntries.size()); + } + } + + void stop() { + isStopped = true; + } +} \ No newline at end of file diff --git a/data-prepper-plugins/sqs-source/src/main/java/org/opensearch/dataprepper/plugins/source/sqssource/SqsSource.java b/data-prepper-plugins/sqs-source/src/main/java/org/opensearch/dataprepper/plugins/source/sqssource/SqsSource.java deleted file mode 100644 index 21888e756b..0000000000 --- a/data-prepper-plugins/sqs-source/src/main/java/org/opensearch/dataprepper/plugins/source/sqssource/SqsSource.java +++ /dev/null @@ -1,115 +0,0 @@ -/* - * Copyright OpenSearch Contributors - * SPDX-License-Identifier: Apache-2.0 - */ -package org.opensearch.dataprepper.plugins.source.sqssource; - -import com.linecorp.armeria.client.retry.Backoff; -import org.opensearch.dataprepper.aws.api.AwsCredentialsSupplier; -import org.opensearch.dataprepper.metrics.PluginMetrics; -import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSetManager; -import org.opensearch.dataprepper.model.annotations.DataPrepperPlugin; -import org.opensearch.dataprepper.model.annotations.DataPrepperPluginConstructor; -import org.opensearch.dataprepper.model.buffer.Buffer; -import org.opensearch.dataprepper.model.event.Event; -import org.opensearch.dataprepper.model.record.Record; -import org.opensearch.dataprepper.model.source.Source; -import org.opensearch.dataprepper.plugins.aws.sqs.common.ClientFactory; -import org.opensearch.dataprepper.plugins.aws.sqs.common.SqsService; -import org.opensearch.dataprepper.plugins.aws.sqs.common.handler.SqsMessageHandler; -import org.opensearch.dataprepper.plugins.aws.sqs.common.metrics.SqsMetrics; -import org.opensearch.dataprepper.plugins.aws.sqs.common.model.SqsOptions; -import org.opensearch.dataprepper.plugins.source.sqssource.config.SqsSourceConfig; -import org.opensearch.dataprepper.plugins.source.sqssource.handler.RawSqsMessageHandler; -import software.amazon.awssdk.services.sqs.SqsClient; - -import java.time.Duration; -import java.util.concurrent.Executors; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.TimeUnit; - -@DataPrepperPlugin(name = "sqs", pluginType = Source.class,pluginConfigurationType = SqsSourceConfig.class) -public class SqsSource implements Source> { - - static final long INITIAL_DELAY = Duration.ofSeconds(20).toMillis(); - - static final long MAXIMUM_DELAY = Duration.ofMinutes(5).toMillis(); - - static final Duration BUFFER_TIMEOUT = Duration.ofSeconds(10); - - static final int NO_OF_RECORDS_TO_ACCUMULATE = 100; - - static final double JITTER_RATE = 0.20; - - private final SqsSourceConfig sqsSourceConfig; - - private final AcknowledgementSetManager acknowledgementSetManager; - - private final PluginMetrics pluginMetrics; - - private final boolean acknowledgementsEnabled; - - private final AwsCredentialsSupplier awsCredentialsSupplier; - - private final ScheduledExecutorService scheduledExecutorService; - - @DataPrepperPluginConstructor - public SqsSource(final PluginMetrics pluginMetrics, - final SqsSourceConfig sqsSourceConfig, - final AcknowledgementSetManager acknowledgementSetManager, - final AwsCredentialsSupplier awsCredentialsSupplier) { - this.sqsSourceConfig = sqsSourceConfig; - this.acknowledgementSetManager = acknowledgementSetManager; - this.acknowledgementsEnabled = sqsSourceConfig.getAcknowledgements(); - this.pluginMetrics = pluginMetrics; - this.awsCredentialsSupplier = awsCredentialsSupplier; - this.scheduledExecutorService = Executors.newScheduledThreadPool(sqsSourceConfig.getNumberOfThreads()); - } - - @Override - public void start(Buffer> buffer) { - if (buffer == null) { - throw new IllegalStateException("Buffer is null"); - } - - final SqsMetrics sqsMetrics = new SqsMetrics(pluginMetrics); - - final SqsClient sqsClient = ClientFactory.createSqsClient(sqsSourceConfig.getAws().getAwsRegion(), - sqsSourceConfig.getAws().getAwsStsRoleArn(), - sqsSourceConfig.getAws().getAwsStsHeaderOverrides(), - awsCredentialsSupplier); - - final Backoff backoff = Backoff.exponential(INITIAL_DELAY, MAXIMUM_DELAY).withJitter(JITTER_RATE) - .withMaxAttempts(Integer.MAX_VALUE); - final SqsService sqsService = new SqsService(sqsMetrics,sqsClient,backoff); - - final SqsMessageHandler sqsHandler = new RawSqsMessageHandler(sqsService); - final SqsOptions.Builder sqsOptionsBuilder = new SqsOptions.Builder() - .setPollDelay(sqsSourceConfig.getPollingFrequency()) - .setVisibilityTimeout(sqsSourceConfig.getVisibilityTimeout()) - .setWaitTime(sqsSourceConfig.getWaitTime()) - .setMaximumMessages(sqsSourceConfig.getBatchSize()); - final long pollingFrequencyInMillis = sqsSourceConfig.getPollingFrequency().toMillis(); - sqsSourceConfig.getUrls().forEach(url -> { - scheduledExecutorService.scheduleAtFixedRate(new SqsSourceTask(buffer,NO_OF_RECORDS_TO_ACCUMULATE,BUFFER_TIMEOUT, - sqsService, - sqsOptionsBuilder.setSqsUrl(url).build(), - sqsMetrics, - acknowledgementSetManager, - sqsSourceConfig.getAcknowledgements(), - sqsHandler) - ,0, pollingFrequencyInMillis == 0 ? 1 : pollingFrequencyInMillis, - TimeUnit.MILLISECONDS); - }); - } - - @Override - public boolean areAcknowledgementsEnabled() { - return acknowledgementsEnabled; - } - - @Override - public void stop() { - scheduledExecutorService.shutdown(); - } -} diff --git a/data-prepper-plugins/sqs-source/src/main/java/org/opensearch/dataprepper/plugins/source/sqssource/SqsSourceTask.java b/data-prepper-plugins/sqs-source/src/main/java/org/opensearch/dataprepper/plugins/source/sqssource/SqsSourceTask.java deleted file mode 100644 index 0d23b69172..0000000000 --- a/data-prepper-plugins/sqs-source/src/main/java/org/opensearch/dataprepper/plugins/source/sqssource/SqsSourceTask.java +++ /dev/null @@ -1,112 +0,0 @@ -/* - * Copyright OpenSearch Contributors - * SPDX-License-Identifier: Apache-2.0 - */ -package org.opensearch.dataprepper.plugins.source.sqssource; - -import org.opensearch.dataprepper.buffer.common.BufferAccumulator; -import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSet; -import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSetManager; -import org.opensearch.dataprepper.model.buffer.Buffer; -import org.opensearch.dataprepper.model.event.Event; -import org.opensearch.dataprepper.model.record.Record; -import org.opensearch.dataprepper.plugins.aws.sqs.common.SqsService; -import org.opensearch.dataprepper.plugins.aws.sqs.common.handler.SqsMessageHandler; -import org.opensearch.dataprepper.plugins.aws.sqs.common.metrics.SqsMetrics; -import org.opensearch.dataprepper.plugins.aws.sqs.common.model.SqsOptions; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import software.amazon.awssdk.services.sqs.model.DeleteMessageBatchRequestEntry; -import software.amazon.awssdk.services.sqs.model.Message; - -import java.time.Duration; -import java.util.ArrayList; -import java.util.List; - -/** - * Class responsible for processing the sqs message with the help of SqsMessageHandler - * - */ -public class SqsSourceTask implements Runnable{ - - private static final Logger LOG = LoggerFactory.getLogger(SqsSourceTask.class); - - private final SqsService sqsService; - - private final SqsOptions sqsOptions; - - private final SqsMetrics sqsMetrics; - - private final AcknowledgementSetManager acknowledgementSetManager; - - private final boolean endToEndAcknowledgementsEnabled; - - private final SqsMessageHandler sqsHandler; - - private final BufferAccumulator> bufferAccumulator; - - public SqsSourceTask(final Buffer> buffer, - final int noOfRecordsToAccumulate, - final Duration bufferTimeout, - final SqsService sqsService, - final SqsOptions sqsOptions, - final SqsMetrics sqsMetrics, - final AcknowledgementSetManager acknowledgementSetManager, - final boolean endToEndAcknowledgementsEnabled, - final SqsMessageHandler sqsHandler) { - this.sqsService = sqsService; - this.sqsOptions = sqsOptions; - this.sqsMetrics = sqsMetrics; - this.acknowledgementSetManager = acknowledgementSetManager; - this.endToEndAcknowledgementsEnabled = endToEndAcknowledgementsEnabled; - this.sqsHandler = sqsHandler; - this.bufferAccumulator = BufferAccumulator.create(buffer,noOfRecordsToAccumulate,bufferTimeout); - } - - /** - * read the messages from sqs queue and push the message into buffer in a loop. - */ - @Override - public void run() { - try { - processSqsMessages(); - } catch (final Exception e) { - LOG.error("Unable to process SQS messages. Processing error due to: {}", e.getMessage()); - sqsService.applyBackoff(); - } - } - - /** - * read the messages from sqs queue and push the message into buffer and finally will delete - * the sqs message from queue after successful buffer push. - */ - void processSqsMessages() { - AcknowledgementSet acknowledgementSet = null; - List deleteMessageBatchRequestEntries = null; - final List waitingForAcknowledgements = new ArrayList<>(); - - if(endToEndAcknowledgementsEnabled) - acknowledgementSet = sqsService.createAcknowledgementSet(sqsOptions.getSqsUrl(), - acknowledgementSetManager, - waitingForAcknowledgements); - - final List messages = sqsService.getMessagesFromSqs(sqsOptions); - - if(!messages.isEmpty()) { - LOG.info("Thread Name : {} , messages processed: {}",Thread.currentThread().getName(),messages.size()); - sqsMetrics.getSqsMessagesReceivedCounter().increment(); - try { - deleteMessageBatchRequestEntries = sqsHandler.handleMessages(messages, bufferAccumulator, acknowledgementSet); - } catch(final Exception e) { - LOG.error("Error while processing handleMessages : ",e); - sqsService.applyBackoff(); - } - if(deleteMessageBatchRequestEntries != null) { - if (endToEndAcknowledgementsEnabled) - waitingForAcknowledgements.addAll(deleteMessageBatchRequestEntries); - else - sqsService.deleteMessagesFromQueue(deleteMessageBatchRequestEntries, sqsOptions.getSqsUrl()); - } - } - } -} diff --git a/data-prepper-plugins/sqs-source/src/main/java/org/opensearch/dataprepper/plugins/source/sqssource/config/AwsAuthenticationOptions.java b/data-prepper-plugins/sqs-source/src/main/java/org/opensearch/dataprepper/plugins/source/sqssource/config/AwsAuthenticationOptions.java deleted file mode 100644 index c5c71f585d..0000000000 --- a/data-prepper-plugins/sqs-source/src/main/java/org/opensearch/dataprepper/plugins/source/sqssource/config/AwsAuthenticationOptions.java +++ /dev/null @@ -1,70 +0,0 @@ -/* - * Copyright OpenSearch Contributors - * SPDX-License-Identifier: Apache-2.0 - */ -package org.opensearch.dataprepper.plugins.source.sqssource.config; - -import com.fasterxml.jackson.annotation.JsonProperty; -import jakarta.validation.constraints.AssertTrue; -import jakarta.validation.constraints.Size; -import software.amazon.awssdk.arns.Arn; -import software.amazon.awssdk.regions.Region; - -import java.util.Map; -import java.util.Optional; - - -public class AwsAuthenticationOptions { - - private static final String AWS_IAM_ROLE = "role"; - - private static final String AWS_IAM = "iam"; - - @JsonProperty("sts_region") - @Size(min = 1, message = "Region cannot be empty string") - private String awsRegion; - - @JsonProperty("sts_role_arn") - @Size(min = 20, max = 2048, message = "awsStsRoleArn length should be between 1 and 2048 characters") - private String awsStsRoleArn; - - @JsonProperty("sts_header_overrides") - @Size(max = 5, message = "sts_header_overrides supports a maximum of 5 headers to override") - private Map awsStsHeaderOverrides; - - - - @AssertTrue(message = "sts_role_arn must be an IAM Role") - boolean isValidStsRoleArn() { - final Arn arn = getArn(); - boolean status = true; - if (!AWS_IAM.equals(arn.service())) { - status = false; - } - final Optional resourceType = arn.resource().resourceType(); - if (resourceType.isEmpty() || !resourceType.get().equals(AWS_IAM_ROLE)) { - status = false; - } - return status; - } - - private Arn getArn() { - try { - return Arn.fromString(awsStsRoleArn); - } catch (final Exception e) { - throw new IllegalArgumentException(String.format("Invalid ARN format for awsStsRoleArn. Check the format of %s", awsStsRoleArn)); - } - } - - public String getAwsStsRoleArn() { - return awsStsRoleArn; - } - - public Region getAwsRegion() { - return awsRegion != null ? Region.of(awsRegion) : null; - } - - public Map getAwsStsHeaderOverrides() { - return awsStsHeaderOverrides; - } -} \ No newline at end of file diff --git a/data-prepper-plugins/sqs-source/src/main/java/org/opensearch/dataprepper/plugins/source/sqssource/config/SqsSourceConfig.java b/data-prepper-plugins/sqs-source/src/main/java/org/opensearch/dataprepper/plugins/source/sqssource/config/SqsSourceConfig.java deleted file mode 100644 index 8ff8e531f7..0000000000 --- a/data-prepper-plugins/sqs-source/src/main/java/org/opensearch/dataprepper/plugins/source/sqssource/config/SqsSourceConfig.java +++ /dev/null @@ -1,77 +0,0 @@ -/* - * Copyright OpenSearch Contributors - * SPDX-License-Identifier: Apache-2.0 - */ -package org.opensearch.dataprepper.plugins.source.sqssource.config; - -import com.fasterxml.jackson.annotation.JsonAlias; -import com.fasterxml.jackson.annotation.JsonProperty; -import jakarta.validation.Valid; -import jakarta.validation.constraints.Max; -import jakarta.validation.constraints.NotNull; -import jakarta.validation.constraints.Positive; - -import java.time.Duration; -import java.util.List; - -public class SqsSourceConfig { - @Valid - @JsonProperty("aws") - private AwsAuthenticationOptions aws; - - @JsonProperty("acknowledgments") - private boolean acknowledgments = false; - - @JsonProperty("visibility_timeout") - private Duration visibilityTimeout; - - @JsonProperty("wait_time") - private Duration waitTime; - - @JsonAlias({"queue_url", "queue_urls"}) - @NotNull - private List urls; - - @JsonProperty("polling_frequency") - private Duration pollingFrequency = Duration.ZERO; - - @JsonProperty("batch_size") - @Max(10) - private Integer batchSize = 10; - - @JsonProperty("number_of_threads") - @Positive(message = "number_of_threads should be unsigned value") - private Integer numberOfThreads = 1; - - public List getUrls() { - return urls; - } - - public Duration getPollingFrequency() { - return pollingFrequency; - } - - public Integer getBatchSize() { - return batchSize; - } - - public Integer getNumberOfThreads() { - return numberOfThreads; - } - - public AwsAuthenticationOptions getAws() { - return aws; - } - - public boolean getAcknowledgements() { - return acknowledgments; - } - - public Duration getVisibilityTimeout() { - return visibilityTimeout; - } - - public Duration getWaitTime() { - return waitTime; - } -} diff --git a/data-prepper-plugins/sqs-source/src/main/java/org/opensearch/dataprepper/plugins/source/sqssource/handler/RawSqsMessageHandler.java b/data-prepper-plugins/sqs-source/src/main/java/org/opensearch/dataprepper/plugins/source/sqssource/handler/RawSqsMessageHandler.java deleted file mode 100644 index 844eecb97b..0000000000 --- a/data-prepper-plugins/sqs-source/src/main/java/org/opensearch/dataprepper/plugins/source/sqssource/handler/RawSqsMessageHandler.java +++ /dev/null @@ -1,67 +0,0 @@ -/* - * Copyright OpenSearch Contributors - * SPDX-License-Identifier: Apache-2.0 - */ -package org.opensearch.dataprepper.plugins.source.sqssource.handler; - -import org.opensearch.dataprepper.buffer.common.BufferAccumulator; -import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSet; -import org.opensearch.dataprepper.model.event.Event; -import org.opensearch.dataprepper.model.event.JacksonEvent; -import org.opensearch.dataprepper.model.record.Record; -import org.opensearch.dataprepper.plugins.aws.sqs.common.SqsService; -import org.opensearch.dataprepper.plugins.aws.sqs.common.handler.SqsMessageHandler; -import software.amazon.awssdk.services.sqs.model.DeleteMessageBatchRequestEntry; -import software.amazon.awssdk.services.sqs.model.Message; - -import java.util.List; -import java.util.Objects; - -/** - * implements the SqsMessageHandler to read and parse the sqs message and push to buffer. - * - */ -public class RawSqsMessageHandler implements SqsMessageHandler { - - private final SqsService sqsService; - - public RawSqsMessageHandler(final SqsService sqsService){ - this.sqsService = sqsService; - } - - /** - * helps to send end to end acknowledgements after successful processing. - * - * @param messages - list of sqs messages for processing - * @return AcknowledgementSet - will generate the AcknowledgementSet if endToEndAcknowledgementsEnabled is true else null - */ - @Override - public List handleMessages(final List messages, - final BufferAccumulator> bufferAccumulator, - final AcknowledgementSet acknowledgementSet) { - messages.forEach(message -> { - final Record eventRecord = new Record(JacksonEvent.fromMessage(message.body())); - try { - // Always add record to acknowledgementSet before adding to - // buffer because another thread may take and process - // buffer contents before the event record is added - // to acknowledgement set - if(Objects.nonNull(acknowledgementSet)){ - acknowledgementSet.add(eventRecord.getData()); - } - bufferAccumulator.add(eventRecord); - } catch (Exception e) { - // Exception may occur when we failed to flush. In which - // case, not sending acknowledgement would be correct because - // we need to retry - throw new RuntimeException(e); - } - }); - try { - bufferAccumulator.flush(); - } catch (final Exception e) { - throw new RuntimeException(e); - } - return sqsService.getDeleteMessageBatchRequestEntryList(messages); - } -} diff --git a/data-prepper-plugins/sqs-source/src/test/java/org/opensearch/dataprepper/plugins/source/sqs/AwsAuthenticationAdapterTest.java b/data-prepper-plugins/sqs-source/src/test/java/org/opensearch/dataprepper/plugins/source/sqs/AwsAuthenticationAdapterTest.java new file mode 100644 index 0000000000..04806ff4d3 --- /dev/null +++ b/data-prepper-plugins/sqs-source/src/test/java/org/opensearch/dataprepper/plugins/source/sqs/AwsAuthenticationAdapterTest.java @@ -0,0 +1,90 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.sqs; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; +import org.mockito.ArgumentCaptor; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.dataprepper.aws.api.AwsCredentialsOptions; +import org.opensearch.dataprepper.aws.api.AwsCredentialsSupplier; +import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; +import software.amazon.awssdk.regions.Region; + +import java.util.Collections; +import java.util.Map; +import java.util.UUID; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.notNullValue; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; +import static org.mockito.Mockito.mock; + + +@ExtendWith(MockitoExtension.class) +class AwsAuthenticationAdapterTest { + @Mock + private AwsCredentialsSupplier awsCredentialsSupplier; + @Mock + private SqsSourceConfig sqsSourceConfig; + + @Mock + private AwsAuthenticationOptions awsAuthenticationOptions; + private String stsRoleArn; + + @BeforeEach + void setUp() { + when(sqsSourceConfig.getAwsAuthenticationOptions()).thenReturn(awsAuthenticationOptions); + + stsRoleArn = UUID.randomUUID().toString(); + when(awsAuthenticationOptions.getAwsStsRoleArn()).thenReturn(stsRoleArn); + } + + private AwsAuthenticationAdapter createObjectUnderTest() { + return new AwsAuthenticationAdapter(awsCredentialsSupplier, sqsSourceConfig); + } + + @Test + void getCredentialsProvider_returns_AwsCredentialsProvider_from_AwsCredentialsSupplier() { + final AwsCredentialsProvider expectedProvider = mock(AwsCredentialsProvider.class); + when(awsCredentialsSupplier.getProvider(any(AwsCredentialsOptions.class))) + .thenReturn(expectedProvider); + + assertThat(createObjectUnderTest().getCredentialsProvider(), equalTo(expectedProvider)); + } + + @ParameterizedTest + @ValueSource(strings = {"us-east-1", "eu-west-1"}) + void getCredentialsProvider_creates_expected_AwsCredentialsOptions(final String regionString) { + final String externalId = UUID.randomUUID().toString(); + final Region region = Region.of(regionString); + + final Map headerOverrides = Collections.singletonMap(UUID.randomUUID().toString(), UUID.randomUUID().toString()); + when(awsAuthenticationOptions.getAwsStsExternalId()).thenReturn(externalId); + when(awsAuthenticationOptions.getAwsRegion()).thenReturn(region); + when(awsAuthenticationOptions.getAwsStsHeaderOverrides()).thenReturn(headerOverrides); + + createObjectUnderTest().getCredentialsProvider(); + + final ArgumentCaptor credentialsOptionsArgumentCaptor = ArgumentCaptor.forClass(AwsCredentialsOptions.class); + verify(awsCredentialsSupplier).getProvider(credentialsOptionsArgumentCaptor.capture()); + + final AwsCredentialsOptions actualOptions = credentialsOptionsArgumentCaptor.getValue(); + + assertThat(actualOptions, notNullValue()); + assertThat(actualOptions.getStsRoleArn(), equalTo(stsRoleArn)); + assertThat(actualOptions.getStsExternalId(), equalTo(externalId)); + assertThat(actualOptions.getRegion(), equalTo(region)); + assertThat(actualOptions.getStsHeaderOverrides(), equalTo(headerOverrides)); + } +} \ No newline at end of file diff --git a/data-prepper-plugins/sqs-source/src/test/java/org/opensearch/dataprepper/plugins/source/sqs/AwsAuthenticationOptionsTest.java b/data-prepper-plugins/sqs-source/src/test/java/org/opensearch/dataprepper/plugins/source/sqs/AwsAuthenticationOptionsTest.java new file mode 100644 index 0000000000..77eeeb519a --- /dev/null +++ b/data-prepper-plugins/sqs-source/src/test/java/org/opensearch/dataprepper/plugins/source/sqs/AwsAuthenticationOptionsTest.java @@ -0,0 +1,136 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.sqs; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.mockito.MockedStatic; +import software.amazon.awssdk.arns.Arn; +import software.amazon.awssdk.regions.Region; + +import java.lang.reflect.Field; +import java.util.HashMap; +import java.util.Map; +import java.util.UUID; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.nullValue; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.mockStatic; + +class AwsAuthenticationOptionsTest { + + private AwsAuthenticationOptions awsAuthenticationOptions; + + @BeforeEach + void setUp() { + awsAuthenticationOptions = new AwsAuthenticationOptions(); + } + + @Test + void getAwsRegion_returns_Region_of() throws NoSuchFieldException, IllegalAccessException { + final String regionString = UUID.randomUUID().toString(); + final Region expectedRegionObject = mock(Region.class); + reflectivelySetField(awsAuthenticationOptions, "awsRegion", regionString); + final Region actualRegion; + try (final MockedStatic regionMockedStatic = mockStatic(Region.class)) { + regionMockedStatic.when(() -> Region.of(regionString)).thenReturn(expectedRegionObject); + actualRegion = awsAuthenticationOptions.getAwsRegion(); + } + assertThat(actualRegion, equalTo(expectedRegionObject)); + } + + @Test + void getAwsRegion_returns_null_when_region_is_null() throws NoSuchFieldException, IllegalAccessException { + reflectivelySetField(awsAuthenticationOptions, "awsRegion", null); + assertThat(awsAuthenticationOptions.getAwsRegion(), nullValue()); + } + + @Test + void getStsExternalId_notNull() throws NoSuchFieldException, IllegalAccessException { + final String externalId = UUID.randomUUID().toString(); + reflectivelySetField(awsAuthenticationOptions, "awsStsExternalId", externalId); + assertThat(awsAuthenticationOptions.getAwsStsExternalId(), equalTo(externalId)); + } + + @Test + void getStsExternalId_Null() throws NoSuchFieldException, IllegalAccessException { + reflectivelySetField(awsAuthenticationOptions, "awsStsExternalId", null); + assertThat(awsAuthenticationOptions.getAwsStsExternalId(), nullValue()); + } + + @Test + void getAwsStsRoleArn_returns_correct_arn() throws NoSuchFieldException, IllegalAccessException { + final String stsRoleArn = "arn:aws:iam::123456789012:role/SampleRole"; + reflectivelySetField(awsAuthenticationOptions, "awsStsRoleArn", stsRoleArn); + assertThat(awsAuthenticationOptions.getAwsStsRoleArn(), equalTo(stsRoleArn)); + } + + @Test + void getAwsStsHeaderOverrides_returns_correct_map() throws NoSuchFieldException, IllegalAccessException { + Map headerOverrides = new HashMap<>(); + headerOverrides.put("header1", "value1"); + headerOverrides.put("header2", "value2"); + reflectivelySetField(awsAuthenticationOptions, "awsStsHeaderOverrides", headerOverrides); + assertThat(awsAuthenticationOptions.getAwsStsHeaderOverrides(), equalTo(headerOverrides)); + } + + @Test + void validateStsRoleArn_with_invalid_format_throws_exception() throws NoSuchFieldException, IllegalAccessException { + final String invalidFormatArn = "invalid-arn-format"; + reflectivelySetField(awsAuthenticationOptions, "awsStsRoleArn", invalidFormatArn); + + try (final MockedStatic arnMockedStatic = mockStatic(Arn.class)) { + arnMockedStatic.when(() -> Arn.fromString(invalidFormatArn)) + .thenThrow(new IllegalArgumentException("The value provided for sts_role_arn is not a valid AWS ARN. Provided value: " + invalidFormatArn)); + + IllegalArgumentException exception = assertThrows(IllegalArgumentException.class, () -> { + awsAuthenticationOptions.validateStsRoleArn(); + }); + assertThat(exception.getMessage(), equalTo("The value provided for sts_role_arn is not a valid AWS ARN. Provided value: " + invalidFormatArn)); + } + } + + @Test + void validateStsRoleArn_does_not_throw_for_valid_role_Arn() throws NoSuchFieldException, IllegalAccessException { + final String validRoleArn = "arn:aws:iam::123456789012:role/SampleRole"; + reflectivelySetField(awsAuthenticationOptions, "awsStsRoleArn", validRoleArn); + try { + awsAuthenticationOptions.validateStsRoleArn(); + } catch (Exception e) { + throw new AssertionError("Exception should not be thrown for a valid role ARN", e); + } + } + + @Test + void validateStsRoleArn_throws_exception_for_non_role_resource() throws NoSuchFieldException, IllegalAccessException { + final String nonRoleResourceArn = "arn:aws:iam::123456789012:group/MyGroup"; + reflectivelySetField(awsAuthenticationOptions, "awsStsRoleArn", nonRoleResourceArn); + Exception exception = assertThrows(IllegalArgumentException.class, () -> awsAuthenticationOptions.validateStsRoleArn()); + assertThat(exception.getMessage(), equalTo("sts_role_arn must be an IAM Role")); + } + + @Test + void validateStsRoleArn_throws_exception_when_service_is_not_iam() throws NoSuchFieldException, IllegalAccessException { + final String invalidServiceArn = "arn:aws:s3::123456789012:role/SampleRole"; + reflectivelySetField(awsAuthenticationOptions, "awsStsRoleArn", invalidServiceArn); + Exception exception = assertThrows(IllegalArgumentException.class, () -> awsAuthenticationOptions.validateStsRoleArn()); + assertThat(exception.getMessage(), equalTo("sts_role_arn must be an IAM Role")); + } + + + private void reflectivelySetField(final AwsAuthenticationOptions awsAuthenticationOptions, final String fieldName, final Object value) throws NoSuchFieldException, IllegalAccessException { + final Field field = AwsAuthenticationOptions.class.getDeclaredField(fieldName); + try { + field.setAccessible(true); + field.set(awsAuthenticationOptions, value); + } finally { + field.setAccessible(false); + } + } +} diff --git a/data-prepper-plugins/sqs-source/src/test/java/org/opensearch/dataprepper/plugins/source/sqs/QueueConfigTest.java b/data-prepper-plugins/sqs-source/src/test/java/org/opensearch/dataprepper/plugins/source/sqs/QueueConfigTest.java new file mode 100644 index 0000000000..f312d8abc6 --- /dev/null +++ b/data-prepper-plugins/sqs-source/src/test/java/org/opensearch/dataprepper/plugins/source/sqs/QueueConfigTest.java @@ -0,0 +1,31 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.sqs; + +import org.junit.jupiter.api.Test; +import java.time.Duration; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNull; + +public class QueueConfigTest { + + @Test + void testDefaultValues() { + final QueueConfig queueConfig = new QueueConfig(); + + assertNull(queueConfig.getUrl(), "URL should be null by default"); + assertEquals(1, queueConfig.getNumWorkers(), "Number of workers should default to 1"); + assertNull(queueConfig.getMaximumMessages(), "Maximum messages should be null by default"); + assertEquals(Duration.ofSeconds(0), queueConfig.getPollDelay(), "Poll delay should default to 0 seconds"); + assertNull(queueConfig.getVisibilityTimeout(), "Visibility timeout should be null by default"); + assertFalse(queueConfig.getVisibilityDuplicateProtection(), "Visibility duplicate protection should default to false"); + assertEquals(Duration.ofHours(2), queueConfig.getVisibilityDuplicateProtectionTimeout(), + "Visibility duplicate protection timeout should default to 2 hours"); + assertNull(queueConfig.getWaitTime(), "Wait time should default to null"); + } +} \ No newline at end of file diff --git a/data-prepper-plugins/sqs-source/src/test/java/org/opensearch/dataprepper/plugins/source/sqs/RawSqsMessageHandlerTest.java b/data-prepper-plugins/sqs-source/src/test/java/org/opensearch/dataprepper/plugins/source/sqs/RawSqsMessageHandlerTest.java new file mode 100644 index 0000000000..4606df45c6 --- /dev/null +++ b/data-prepper-plugins/sqs-source/src/test/java/org/opensearch/dataprepper/plugins/source/sqs/RawSqsMessageHandlerTest.java @@ -0,0 +1,44 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.sqs; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.mockito.ArgumentCaptor; +import org.opensearch.dataprepper.model.buffer.Buffer; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.record.Record; +import software.amazon.awssdk.services.sqs.model.Message; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; + +class RawSqsMessageHandlerTest { + + private final RawSqsMessageHandler rawSqsMessageHandler = new RawSqsMessageHandler(); + private Buffer> mockBuffer; + private int mockBufferTimeoutMillis; + + @BeforeEach + void setUp() { + mockBuffer = mock(Buffer.class); + mockBufferTimeoutMillis = 10000; + } + + @Test + void handleMessage_callsBufferWriteOnce() throws Exception { + Message message = Message.builder().body("{\"key\":\"value\"}").build(); + String queueUrl = "https://sqs.us-east-1.amazonaws.com/123456789012/test-queue"; + rawSqsMessageHandler.handleMessage(message, queueUrl, mockBuffer, mockBufferTimeoutMillis, null); + ArgumentCaptor> argumentCaptor = ArgumentCaptor.forClass(Record.class); + verify(mockBuffer, times(1)).write(argumentCaptor.capture(), eq(mockBufferTimeoutMillis)); + Record capturedRecord = argumentCaptor.getValue(); + assertEquals("DOCUMENT", capturedRecord.getData().getMetadata().getEventType(), "Event type should be 'DOCUMENT'"); + } +} diff --git a/data-prepper-plugins/sqs-source/src/test/java/org/opensearch/dataprepper/plugins/source/sqs/SqsEventProcessorTest.java b/data-prepper-plugins/sqs-source/src/test/java/org/opensearch/dataprepper/plugins/source/sqs/SqsEventProcessorTest.java new file mode 100644 index 0000000000..e10b8f471f --- /dev/null +++ b/data-prepper-plugins/sqs-source/src/test/java/org/opensearch/dataprepper/plugins/source/sqs/SqsEventProcessorTest.java @@ -0,0 +1,60 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.sqs; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.mockito.Mockito; +import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSet; +import org.opensearch.dataprepper.model.buffer.Buffer; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.record.Record; +import software.amazon.awssdk.services.sqs.model.Message; + +import java.io.IOException; + +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; + +class SqsEventProcessorTest { + + private SqsMessageHandler mockSqsMessageHandler; + private SqsEventProcessor sqsEventProcessor; + private Buffer> mockBuffer; + private int mockBufferTimeoutMillis; + private AcknowledgementSet mockAcknowledgementSet; + + @BeforeEach + void setUp() { + mockSqsMessageHandler = Mockito.mock(SqsMessageHandler.class); + mockBuffer = Mockito.mock(Buffer.class); + mockAcknowledgementSet = Mockito.mock(AcknowledgementSet.class); + mockBufferTimeoutMillis = 10000; + sqsEventProcessor = new SqsEventProcessor(mockSqsMessageHandler); + } + + @Test + void addSqsObject_callsHandleMessageWithCorrectParameters() throws IOException { + Message message = Message.builder().body("Test Message Body").build(); + String queueUrl = "https://sqs.us-east-1.amazonaws.com/123456789012/test-queue"; + sqsEventProcessor.addSqsObject(message, queueUrl, mockBuffer, mockBufferTimeoutMillis, mockAcknowledgementSet); + verify(mockSqsMessageHandler, times(1)).handleMessage(message, queueUrl, mockBuffer, mockBufferTimeoutMillis, mockAcknowledgementSet); + } + + @Test + void addSqsObject_propagatesIOExceptionThrownByHandleMessage() throws IOException { + Message message = Message.builder().body("Test Message Body").build(); + String queueUrl = "https://sqs.us-east-1.amazonaws.com/123456789012/test-queue"; + doThrow(new IOException("Handle message failed")).when(mockSqsMessageHandler).handleMessage(message, queueUrl, mockBuffer, mockBufferTimeoutMillis, mockAcknowledgementSet); + IOException thrownException = assertThrows(IOException.class, () -> + sqsEventProcessor.addSqsObject(message, queueUrl, mockBuffer, mockBufferTimeoutMillis, mockAcknowledgementSet) + ); + assert(thrownException.getMessage().equals("Handle message failed")); + verify(mockSqsMessageHandler, times(1)).handleMessage(message, queueUrl, mockBuffer, mockBufferTimeoutMillis, mockAcknowledgementSet); + } +} diff --git a/data-prepper-plugins/sqs-source/src/test/java/org/opensearch/dataprepper/plugins/source/sqs/SqsServiceTest.java b/data-prepper-plugins/sqs-source/src/test/java/org/opensearch/dataprepper/plugins/source/sqs/SqsServiceTest.java new file mode 100644 index 0000000000..3bbc44bbe6 --- /dev/null +++ b/data-prepper-plugins/sqs-source/src/test/java/org/opensearch/dataprepper/plugins/source/sqs/SqsServiceTest.java @@ -0,0 +1,81 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.sqs; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSetManager; +import org.opensearch.dataprepper.model.buffer.Buffer; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.record.Record; +import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; +import software.amazon.awssdk.regions.Region; +import software.amazon.awssdk.services.sqs.SqsClient; +import java.util.List; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; +import static org.mockito.Mockito.withSettings; + + +class SqsServiceTest { + private SqsSourceConfig sqsSourceConfig; + private SqsEventProcessor sqsEventProcessor; + private SqsClient sqsClient; + private PluginMetrics pluginMetrics; + private AcknowledgementSetManager acknowledgementSetManager; + private Buffer> buffer; + private AwsCredentialsProvider credentialsProvider; + + @BeforeEach + void setUp() { + sqsSourceConfig = mock(SqsSourceConfig.class); + sqsEventProcessor = mock(SqsEventProcessor.class); + sqsClient = mock(SqsClient.class, withSettings()); + pluginMetrics = mock(PluginMetrics.class); + acknowledgementSetManager = mock(AcknowledgementSetManager.class); + buffer = mock(Buffer.class); + credentialsProvider = mock(AwsCredentialsProvider.class); + + AwsAuthenticationOptions awsAuthenticationOptions = mock(AwsAuthenticationOptions.class); + when(awsAuthenticationOptions.getAwsRegion()).thenReturn(Region.US_EAST_1); + when(sqsSourceConfig.getAwsAuthenticationOptions()).thenReturn(awsAuthenticationOptions); + } + + @Test + void start_with_single_queue_starts_workers() { + QueueConfig queueConfig = mock(QueueConfig.class); + when(queueConfig.getUrl()).thenReturn("https://sqs.us-east-1.amazonaws.com/123456789012/MyQueue"); + when(queueConfig.getNumWorkers()).thenReturn(2); + when(sqsSourceConfig.getQueues()).thenReturn(List.of(queueConfig)); + SqsService sqsService = spy(new SqsService(buffer, acknowledgementSetManager, sqsSourceConfig, sqsEventProcessor, pluginMetrics, credentialsProvider)); + doReturn(sqsClient).when(sqsService).createSqsClient(credentialsProvider); + sqsService.start(); // if no exception is thrown here, then workers have been started + } + + @Test + void stop_should_shutdown_executors_and_workers_and_close_client() throws InterruptedException { + QueueConfig queueConfig = mock(QueueConfig.class); + when(queueConfig.getUrl()).thenReturn("MyQueue"); + when(queueConfig.getNumWorkers()).thenReturn(1); + when(sqsSourceConfig.getQueues()).thenReturn(List.of(queueConfig)); + SqsClient sqsClient = mock(SqsClient.class); + SqsService sqsService = new SqsService(buffer, acknowledgementSetManager, sqsSourceConfig, sqsEventProcessor, pluginMetrics, credentialsProvider) { + @Override + SqsClient createSqsClient(final AwsCredentialsProvider credentialsProvider) { + return sqsClient; + } + }; + sqsService.start(); + sqsService.stop(); + verify(sqsClient, times(1)).close(); + } + +} \ No newline at end of file diff --git a/data-prepper-plugins/sqs-source/src/test/java/org/opensearch/dataprepper/plugins/source/sqs/SqsSourceConfigTest.java b/data-prepper-plugins/sqs-source/src/test/java/org/opensearch/dataprepper/plugins/source/sqs/SqsSourceConfigTest.java new file mode 100644 index 0000000000..29f0443670 --- /dev/null +++ b/data-prepper-plugins/sqs-source/src/test/java/org/opensearch/dataprepper/plugins/source/sqs/SqsSourceConfigTest.java @@ -0,0 +1,27 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.sqs; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.junit.jupiter.api.Test; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNull; + +public class SqsSourceConfigTest { + + private final ObjectMapper objectMapper = new ObjectMapper(); + + @Test + void testDefaultValues() { + final SqsSourceConfig config = new SqsSourceConfig(); + assertNull(config.getAwsAuthenticationOptions(), "AWS Authentication Options should be null by default"); + assertFalse(config.getAcknowledgements(), "Acknowledgments should be false by default"); + assertEquals(SqsSourceConfig.DEFAULT_BUFFER_TIMEOUT, config.getBufferTimeout(), "Buffer timeout should default to 10 seconds"); + assertNull(config.getQueues(), "Queues should be null by default"); + } +} \ No newline at end of file diff --git a/data-prepper-plugins/sqs-source/src/test/java/org/opensearch/dataprepper/plugins/source/sqs/SqsSourceTest.java b/data-prepper-plugins/sqs-source/src/test/java/org/opensearch/dataprepper/plugins/source/sqs/SqsSourceTest.java new file mode 100644 index 0000000000..cf130c102e --- /dev/null +++ b/data-prepper-plugins/sqs-source/src/test/java/org/opensearch/dataprepper/plugins/source/sqs/SqsSourceTest.java @@ -0,0 +1,65 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.sqs; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.opensearch.dataprepper.aws.api.AwsCredentialsSupplier; +import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSetManager; +import org.opensearch.dataprepper.model.buffer.Buffer; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.record.Record; +import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; +import software.amazon.awssdk.regions.Region; + +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; + +class SqsSourceTest { + private final String PLUGIN_NAME = "sqs"; + private final String TEST_PIPELINE_NAME = "test_pipeline"; + private SqsSource sqsSource; + private PluginMetrics pluginMetrics; + private SqsSourceConfig sqsSourceConfig; + private AcknowledgementSetManager acknowledgementSetManager; + private AwsCredentialsSupplier awsCredentialsSupplier; + private Buffer> buffer; + + + @BeforeEach + void setUp() { + pluginMetrics = PluginMetrics.fromNames(PLUGIN_NAME, TEST_PIPELINE_NAME); + sqsSourceConfig = mock(SqsSourceConfig.class); + acknowledgementSetManager = mock(AcknowledgementSetManager.class); + awsCredentialsSupplier = mock(AwsCredentialsSupplier.class); + sqsSource = new SqsSource(pluginMetrics, sqsSourceConfig, acknowledgementSetManager, awsCredentialsSupplier); + buffer = mock(Buffer.class); + } + + @Test + void start_should_throw_IllegalStateException_when_buffer_is_null() { + assertThrows(IllegalStateException.class, () -> sqsSource.start(null)); + } + + @Test + void start_should_not_throw_when_buffer_is_not_null() { + AwsAuthenticationOptions awsAuthenticationOptions = mock(AwsAuthenticationOptions.class); + when(awsAuthenticationOptions.getAwsStsRoleArn()).thenReturn("arn:aws:iam::123456789012:role/example-role"); + when(sqsSourceConfig.getAwsAuthenticationOptions()).thenReturn(awsAuthenticationOptions); + when(awsCredentialsSupplier.getProvider(any())).thenReturn(mock(AwsCredentialsProvider.class)); + when(awsAuthenticationOptions.getAwsRegion()).thenReturn(Region.of("us-east-2")); + assertDoesNotThrow(() -> sqsSource.start(buffer)); + } + + @Test + void stop_should_not_throw_when_sqsService_is_null() { + sqsSource.stop(); + } +} \ No newline at end of file diff --git a/data-prepper-plugins/sqs-source/src/test/java/org/opensearch/dataprepper/plugins/source/sqs/SqsWorkerTest.java b/data-prepper-plugins/sqs-source/src/test/java/org/opensearch/dataprepper/plugins/source/sqs/SqsWorkerTest.java new file mode 100644 index 0000000000..7bb8e082cc --- /dev/null +++ b/data-prepper-plugins/sqs-source/src/test/java/org/opensearch/dataprepper/plugins/source/sqs/SqsWorkerTest.java @@ -0,0 +1,378 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.sqs; + +import com.linecorp.armeria.client.retry.Backoff; +import io.micrometer.core.instrument.Counter; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.ArgumentCaptor; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSet; +import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSetManager; +import org.opensearch.dataprepper.model.acknowledgements.ProgressCheck; +import org.opensearch.dataprepper.model.buffer.Buffer; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.record.Record; +import software.amazon.awssdk.services.sqs.SqsClient; +import software.amazon.awssdk.services.sqs.model.ChangeMessageVisibilityRequest; +import software.amazon.awssdk.services.sqs.model.DeleteMessageBatchRequest; +import software.amazon.awssdk.services.sqs.model.DeleteMessageBatchResponse; +import software.amazon.awssdk.services.sqs.model.DeleteMessageBatchResultEntry; +import software.amazon.awssdk.services.sqs.model.Message; +import software.amazon.awssdk.services.sqs.model.ReceiveMessageRequest; +import software.amazon.awssdk.services.sqs.model.ReceiveMessageResponse; +import software.amazon.awssdk.services.sqs.model.SqsException; +import software.amazon.awssdk.services.sqs.model.BatchResultErrorEntry; + +import java.io.IOException; +import java.time.Duration; +import java.util.Collections; +import java.util.UUID; +import java.util.function.Consumer; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyDouble; +import static org.mockito.ArgumentMatchers.anyInt; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.ArgumentMatchers.isNull; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.verifyNoInteractions; +import static org.mockito.Mockito.when; + +@ExtendWith(MockitoExtension.class) +class SqsWorkerTest { + + @Mock + private Buffer> buffer; + @Mock + private AcknowledgementSetManager acknowledgementSetManager; + @Mock + private SqsClient sqsClient; + @Mock + private SqsEventProcessor sqsEventProcessor; + @Mock + private SqsSourceConfig sqsSourceConfig; + @Mock + private QueueConfig queueConfig; + @Mock + private PluginMetrics pluginMetrics; + @Mock + private Backoff backoff; + @Mock + private Counter sqsMessagesReceivedCounter; + @Mock + private Counter sqsMessagesDeletedCounter; + @Mock + private Counter sqsMessagesFailedCounter; + @Mock + private Counter sqsMessagesDeleteFailedCounter; + @Mock + private Counter acknowledgementSetCallbackCounter; + @Mock + private Counter sqsVisibilityTimeoutChangedCount; + @Mock + private Counter sqsVisibilityTimeoutChangeFailedCount; + private int mockBufferTimeoutMillis = 10000; + + private SqsWorker createObjectUnderTest() { + return new SqsWorker( + buffer, + acknowledgementSetManager, + sqsClient, + sqsEventProcessor, + sqsSourceConfig, + queueConfig, + pluginMetrics, + backoff); + } + + @BeforeEach + void setUp() { + when(pluginMetrics.counter(SqsWorker.SQS_MESSAGES_RECEIVED_METRIC_NAME)).thenReturn(sqsMessagesReceivedCounter); + when(pluginMetrics.counter(SqsWorker.SQS_MESSAGES_DELETED_METRIC_NAME)).thenReturn(sqsMessagesDeletedCounter); + when(pluginMetrics.counter(SqsWorker.SQS_MESSAGES_FAILED_METRIC_NAME)).thenReturn(sqsMessagesFailedCounter); + when(pluginMetrics.counter(SqsWorker.SQS_MESSAGES_DELETE_FAILED_METRIC_NAME)).thenReturn(sqsMessagesDeleteFailedCounter); + when(pluginMetrics.counter(SqsWorker.ACKNOWLEDGEMENT_SET_CALLACK_METRIC_NAME)).thenReturn(acknowledgementSetCallbackCounter); + when(pluginMetrics.counter(SqsWorker.SQS_VISIBILITY_TIMEOUT_CHANGED_COUNT_METRIC_NAME)).thenReturn(sqsVisibilityTimeoutChangedCount); + when(pluginMetrics.counter(SqsWorker.SQS_VISIBILITY_TIMEOUT_CHANGE_FAILED_COUNT_METRIC_NAME)).thenReturn(sqsVisibilityTimeoutChangeFailedCount); + when(sqsSourceConfig.getAcknowledgements()).thenReturn(false); + when(sqsSourceConfig.getBufferTimeout()).thenReturn(Duration.ofSeconds(10)); + when(queueConfig.getUrl()).thenReturn("https://sqs.us-east-1.amazonaws.com/123456789012/MyQueue"); + when(queueConfig.getWaitTime()).thenReturn(Duration.ofSeconds(1)); + } + + @Test + void processSqsMessages_should_return_number_of_messages_processed_and_increment_counters() throws IOException { + final Message message = Message.builder() + .messageId(UUID.randomUUID().toString()) + .receiptHandle(UUID.randomUUID().toString()) + .body("{\"Records\":[{\"eventSource\":\"custom\",\"message\":\"Hello World\"}]}") + .build(); + + final ReceiveMessageResponse response = ReceiveMessageResponse.builder().messages(message).build(); + when(sqsClient.receiveMessage(any(ReceiveMessageRequest.class))).thenReturn(response); + + final DeleteMessageBatchResultEntry successfulDelete = DeleteMessageBatchResultEntry.builder().id(message.messageId()).build(); + final DeleteMessageBatchResponse deleteResponse = DeleteMessageBatchResponse.builder().successful(successfulDelete).build(); + when(sqsClient.deleteMessageBatch(any(DeleteMessageBatchRequest.class))).thenReturn(deleteResponse); + + int messagesProcessed = createObjectUnderTest().processSqsMessages(); + assertThat(messagesProcessed, equalTo(1)); + + verify(sqsMessagesReceivedCounter).increment(1); + verify(sqsMessagesDeletedCounter).increment(1); + verify(sqsMessagesDeleteFailedCounter, never()).increment(anyDouble()); + } + + @Test + void processSqsMessages_should_invoke_processSqsEvent_and_deleteSqsMessages_when_entries_non_empty() throws IOException { + final Message message = Message.builder() + .messageId(UUID.randomUUID().toString()) + .receiptHandle(UUID.randomUUID().toString()) + .body("{\"Records\":[{\"eventSource\":\"custom\",\"message\":\"Hello World\"}]}") + .build(); + + final ReceiveMessageResponse response = ReceiveMessageResponse.builder() + .messages(message) + .build(); + when(sqsClient.receiveMessage(any(ReceiveMessageRequest.class))).thenReturn(response); + + final DeleteMessageBatchResultEntry successfulDelete = DeleteMessageBatchResultEntry.builder() + .id(message.messageId()) + .build(); + final DeleteMessageBatchResponse deleteResponse = DeleteMessageBatchResponse.builder() + .successful(successfulDelete) + .build(); + when(sqsClient.deleteMessageBatch(any(DeleteMessageBatchRequest.class))).thenReturn(deleteResponse); + + SqsWorker sqsWorker = createObjectUnderTest(); + int messagesProcessed = sqsWorker.processSqsMessages(); + + assertThat(messagesProcessed, equalTo(1)); + verify(sqsEventProcessor, times(1)).addSqsObject(eq(message), eq("https://sqs.us-east-1.amazonaws.com/123456789012/MyQueue"), eq(buffer), eq(mockBufferTimeoutMillis), isNull()); + verify(sqsClient, times(1)).deleteMessageBatch(any(DeleteMessageBatchRequest.class)); + verify(sqsMessagesReceivedCounter).increment(1); + verify(sqsMessagesDeletedCounter).increment(1); + verify(sqsMessagesDeleteFailedCounter, never()).increment(anyDouble()); + } + + @Test + void processSqsMessages_should_not_invoke_processSqsEvent_and_deleteSqsMessages_when_entries_are_empty() throws IOException { + when(sqsClient.receiveMessage(any(ReceiveMessageRequest.class))) + .thenReturn(ReceiveMessageResponse.builder().messages(Collections.emptyList()).build()); + SqsWorker sqsWorker = createObjectUnderTest(); + int messagesProcessed = sqsWorker.processSqsMessages(); + assertThat(messagesProcessed, equalTo(0)); + verify(sqsEventProcessor, never()).addSqsObject(any(), anyString(), any(), anyInt(), any()); + verify(sqsClient, never()).deleteMessageBatch(any(DeleteMessageBatchRequest.class)); + verify(sqsMessagesReceivedCounter, never()).increment(anyDouble()); + verify(sqsMessagesDeletedCounter, never()).increment(anyDouble()); + } + + + @Test + void processSqsMessages_should_not_delete_messages_if_acknowledgements_enabled_until_acknowledged() throws IOException { + when(sqsSourceConfig.getAcknowledgements()).thenReturn(true); + AcknowledgementSet acknowledgementSet = mock(AcknowledgementSet.class); + when(acknowledgementSetManager.create(any(), any())).thenReturn(acknowledgementSet); + when(queueConfig.getUrl()).thenReturn("https://sqs.us-east-1.amazonaws.com/123456789012/MyQueue"); + + final Message message = Message.builder() + .messageId("msg-1") + .receiptHandle("rh-1") + .body("{\"Records\":[{\"eventSource\":\"custom\",\"message\":\"Hello World\"}]}") + .build(); + + final ReceiveMessageResponse response = ReceiveMessageResponse.builder().messages(message).build(); + when(sqsClient.receiveMessage(any(ReceiveMessageRequest.class))).thenReturn(response); + int messagesProcessed = createObjectUnderTest().processSqsMessages(); + assertThat(messagesProcessed, equalTo(1)); + verify(sqsEventProcessor, times(1)).addSqsObject(eq(message), + eq("https://sqs.us-east-1.amazonaws.com/123456789012/MyQueue"), + eq(buffer), + eq(mockBufferTimeoutMillis), + eq(acknowledgementSet)); + verify(sqsMessagesReceivedCounter).increment(1); + verifyNoInteractions(sqsMessagesDeletedCounter); + } + + @Test + void acknowledgementsEnabled_and_visibilityDuplicateProtectionEnabled_should_create_ack_sets_and_progress_check() { + when(sqsSourceConfig.getAcknowledgements()).thenReturn(true); + when(queueConfig.getVisibilityDuplicateProtection()).thenReturn(true); + + SqsWorker worker = new SqsWorker(buffer, acknowledgementSetManager, sqsClient, sqsEventProcessor, sqsSourceConfig, queueConfig, pluginMetrics, backoff); + Message message = Message.builder().messageId("msg-dup").receiptHandle("handle-dup").build(); + ReceiveMessageResponse response = ReceiveMessageResponse.builder().messages(message).build(); + when(sqsClient.receiveMessage((ReceiveMessageRequest) any())).thenReturn(response); + + AcknowledgementSet acknowledgementSet = mock(AcknowledgementSet.class); + when(acknowledgementSetManager.create(any(), any())).thenReturn(acknowledgementSet); + + int processed = worker.processSqsMessages(); + assertThat(processed, equalTo(1)); + + verify(acknowledgementSetManager).create(any(), any()); + verify(acknowledgementSet).addProgressCheck(any(), any()); + } + + @Test + void processSqsMessages_should_return_zero_messages_with_backoff_when_a_SqsException_is_thrown() { + when(sqsClient.receiveMessage(any(ReceiveMessageRequest.class))).thenThrow(SqsException.class); + final int messagesProcessed = createObjectUnderTest().processSqsMessages(); + verify(backoff).nextDelayMillis(1); + assertThat(messagesProcessed, equalTo(0)); + } + + @Test + void processSqsMessages_should_throw_when_a_SqsException_is_thrown_with_max_retries() { + when(sqsClient.receiveMessage(any(ReceiveMessageRequest.class))).thenThrow(SqsException.class); + when(backoff.nextDelayMillis(anyInt())).thenReturn((long) -1); + SqsWorker objectUnderTest = createObjectUnderTest(); + assertThrows(SqsRetriesExhaustedException.class, objectUnderTest::processSqsMessages); + } + + @Test + void processSqsMessages_should_update_visibility_timeout_when_progress_changes() throws IOException { + AcknowledgementSet acknowledgementSet = mock(AcknowledgementSet.class); + when(queueConfig.getVisibilityDuplicateProtection()).thenReturn(true); + when(queueConfig.getVisibilityTimeout()).thenReturn(Duration.ofMillis(1)); + when(acknowledgementSetManager.create(any(), any(Duration.class))).thenReturn(acknowledgementSet); + when(sqsSourceConfig.getAcknowledgements()).thenReturn(true); + final Message message = mock(Message.class); + final String testReceiptHandle = UUID.randomUUID().toString(); + when(message.messageId()).thenReturn(testReceiptHandle); + when(message.receiptHandle()).thenReturn(testReceiptHandle); + + final ReceiveMessageResponse receiveMessageResponse = mock(ReceiveMessageResponse.class); + when(sqsClient.receiveMessage(any(ReceiveMessageRequest.class))).thenReturn(receiveMessageResponse); + when(receiveMessageResponse.messages()).thenReturn(Collections.singletonList(message)); + + final int messagesProcessed = createObjectUnderTest().processSqsMessages(); + + assertThat(messagesProcessed, equalTo(1)); + verify(sqsEventProcessor).addSqsObject(any(), anyString(), any(), anyInt(), any()); + verify(acknowledgementSetManager).create(any(), any(Duration.class)); + + ArgumentCaptor> progressConsumerArgumentCaptor = ArgumentCaptor.forClass(Consumer.class); + verify(acknowledgementSet).addProgressCheck(progressConsumerArgumentCaptor.capture(), any(Duration.class)); + final Consumer actualConsumer = progressConsumerArgumentCaptor.getValue(); + final ProgressCheck progressCheck = mock(ProgressCheck.class); + actualConsumer.accept(progressCheck); + + ArgumentCaptor changeMessageVisibilityRequestArgumentCaptor = ArgumentCaptor.forClass(ChangeMessageVisibilityRequest.class); + verify(sqsClient).changeMessageVisibility(changeMessageVisibilityRequestArgumentCaptor.capture()); + ChangeMessageVisibilityRequest actualChangeVisibilityRequest = changeMessageVisibilityRequestArgumentCaptor.getValue(); + assertThat(actualChangeVisibilityRequest.queueUrl(), equalTo(queueConfig.getUrl())); + assertThat(actualChangeVisibilityRequest.receiptHandle(), equalTo(testReceiptHandle)); + verify(sqsMessagesReceivedCounter).increment(1); + } + @Test + void increaseVisibilityTimeout_doesNothing_whenIsStopped() throws IOException { + when(sqsSourceConfig.getAcknowledgements()).thenReturn(true); + when(queueConfig.getVisibilityDuplicateProtection()).thenReturn(false); + when(queueConfig.getVisibilityTimeout()).thenReturn(Duration.ofSeconds(30)); + AcknowledgementSet mockAcknowledgementSet = mock(AcknowledgementSet.class); + when(acknowledgementSetManager.create(any(), any())).thenReturn(mockAcknowledgementSet); + Message message = Message.builder() + .messageId(UUID.randomUUID().toString()) + .receiptHandle(UUID.randomUUID().toString()) + .body("{\"Records\":[{\"eventSource\":\"custom\",\"message\":\"Hello World\"}]}") + .build(); + ReceiveMessageResponse response = ReceiveMessageResponse.builder() + .messages(message) + .build(); + when(sqsClient.receiveMessage(any(ReceiveMessageRequest.class))).thenReturn(response); + SqsWorker sqsWorker = createObjectUnderTest(); + sqsWorker.stop(); + int messagesProcessed = sqsWorker.processSqsMessages(); + assertThat(messagesProcessed, equalTo(1)); + verify(sqsEventProcessor, times(1)).addSqsObject(eq(message), + eq("https://sqs.us-east-1.amazonaws.com/123456789012/MyQueue"), + eq(buffer), + eq(mockBufferTimeoutMillis), + eq(mockAcknowledgementSet)); + verify(sqsClient, never()).changeMessageVisibility(any(ChangeMessageVisibilityRequest.class)); + verify(sqsVisibilityTimeoutChangeFailedCount, never()).increment(); + } + + @Test + void deleteSqsMessages_incrementsFailedCounter_whenDeleteResponseHasFailedDeletes() throws IOException { + final Message message1 = Message.builder() + .messageId(UUID.randomUUID().toString()) + .receiptHandle(UUID.randomUUID().toString()) + .body("{\"Records\":[{\"eventSource\":\"custom\",\"message\":\"Hello World 1\"}]}") + .build(); + final Message message2 = Message.builder() + .messageId(UUID.randomUUID().toString()) + .receiptHandle(UUID.randomUUID().toString()) + .body("{\"Records\":[{\"eventSource\":\"custom\",\"message\":\"Hello World 2\"}]}") + .build(); + + final ReceiveMessageResponse receiveResponse = ReceiveMessageResponse.builder() + .messages(message1, message2) + .build(); + when(sqsClient.receiveMessage(any(ReceiveMessageRequest.class))).thenReturn(receiveResponse); + + DeleteMessageBatchResultEntry successfulDelete = DeleteMessageBatchResultEntry.builder() + .id(message1.messageId()) + .build(); + + BatchResultErrorEntry failedDelete = BatchResultErrorEntry.builder() + .id(message2.messageId()) + .code("ReceiptHandleIsInvalid") + .senderFault(true) + .message("Failed to delete message due to invalid receipt handle.") + .build(); + + DeleteMessageBatchResponse deleteResponse = DeleteMessageBatchResponse.builder() + .successful(successfulDelete) + .failed(failedDelete) + .build(); + + when(sqsClient.deleteMessageBatch(any(DeleteMessageBatchRequest.class))).thenReturn(deleteResponse); + SqsWorker sqsWorker = createObjectUnderTest(); + int messagesProcessed = sqsWorker.processSqsMessages(); + assertThat(messagesProcessed, equalTo(2)); + verify(sqsMessagesReceivedCounter).increment(2); + verify(sqsMessagesDeletedCounter).increment(1); + verify(sqsMessagesDeleteFailedCounter).increment(1); + } + @Test + void processSqsMessages_handlesException_correctly_when_addSqsObject_throwsException() throws IOException { + final Message message = Message.builder() + .messageId(UUID.randomUUID().toString()) + .receiptHandle(UUID.randomUUID().toString()) + .body("{\"Records\":[{\"eventSource\":\"custom\",\"message\":\"Hello World\"}]}") + .build(); + when(sqsClient.receiveMessage(any(ReceiveMessageRequest.class))).thenReturn( + ReceiveMessageResponse.builder().messages(message).build() + ); + doThrow(new RuntimeException("Processing failed")).when(sqsEventProcessor) + .addSqsObject(eq(message), eq("https://sqs.us-east-1.amazonaws.com/123456789012/MyQueue"), + any(), anyInt(), any()); + SqsWorker sqsWorker = createObjectUnderTest(); + int messagesProcessed = sqsWorker.processSqsMessages(); + assertThat(messagesProcessed, equalTo(1)); + verify(sqsMessagesReceivedCounter).increment(1); + verify(sqsMessagesFailedCounter).increment(); + verify(backoff).nextDelayMillis(anyInt()); + verify(sqsClient, never()).deleteMessageBatch(any(DeleteMessageBatchRequest.class)); + verify(sqsMessagesDeletedCounter, never()).increment(anyInt()); + } +} \ No newline at end of file diff --git a/data-prepper-plugins/sqs-source/src/test/java/org/opensearch/dataprepper/plugins/source/sqssource/SqsSourceTaskTest.java b/data-prepper-plugins/sqs-source/src/test/java/org/opensearch/dataprepper/plugins/source/sqssource/SqsSourceTaskTest.java deleted file mode 100644 index c4dac097a1..0000000000 --- a/data-prepper-plugins/sqs-source/src/test/java/org/opensearch/dataprepper/plugins/source/sqssource/SqsSourceTaskTest.java +++ /dev/null @@ -1,170 +0,0 @@ -package org.opensearch.dataprepper.plugins.source.sqssource; - -import com.linecorp.armeria.client.retry.Backoff; -import io.micrometer.core.instrument.Counter; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.extension.ExtendWith; -import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.ValueSource; -import org.mockito.Mock; -import org.mockito.junit.jupiter.MockitoExtension; -import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSet; -import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSetManager; -import org.opensearch.dataprepper.model.buffer.Buffer; -import org.opensearch.dataprepper.model.event.Event; -import org.opensearch.dataprepper.model.record.Record; -import org.opensearch.dataprepper.plugins.aws.sqs.common.SqsService; -import org.opensearch.dataprepper.plugins.aws.sqs.common.handler.SqsMessageHandler; -import org.opensearch.dataprepper.plugins.aws.sqs.common.metrics.SqsMetrics; -import org.opensearch.dataprepper.plugins.aws.sqs.common.model.SqsOptions; -import software.amazon.awssdk.services.sqs.SqsClient; -import software.amazon.awssdk.services.sqs.model.DeleteMessageBatchRequest; -import software.amazon.awssdk.services.sqs.model.DeleteMessageBatchResponse; -import software.amazon.awssdk.services.sqs.model.Message; -import software.amazon.awssdk.services.sqs.model.ReceiveMessageRequest; -import software.amazon.awssdk.services.sqs.model.ReceiveMessageResponse; -import software.amazon.awssdk.services.sqs.model.SqsException; -import software.amazon.awssdk.services.sts.model.StsException; - -import java.time.Duration; -import java.util.List; -import java.util.UUID; -import java.util.function.Consumer; - -import static org.mockito.ArgumentMatchers.any; -import static org.mockito.ArgumentMatchers.eq; -import static org.mockito.ArgumentMatchers.isNull; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.verifyNoInteractions; -import static org.mockito.Mockito.when; - - -@ExtendWith(MockitoExtension.class) -class SqsSourceTaskTest { - - private SqsService sqsService; - - private SqsOptions sqsOptions; - - @Mock - private SqsMetrics sqsMetrics; - - @Mock - private AcknowledgementSetManager acknowledgementSetManager; - - private boolean endToEndAcknowledgementsEnabled = false; - - @Mock - private SqsMessageHandler sqsHandler; - - @Mock - private SqsClient sqsClient; - - @Mock - private Buffer> buffer; - - @Mock - private Backoff backoff; - - private Counter messageReceivedCounter; - - private Counter messageDeletedCounter; - - private Counter sqsMessagesFailedCounter; - - private AcknowledgementSet acknowledgementSet; - - @BeforeEach - public void setup(){ - messageReceivedCounter = mock(Counter.class); - messageDeletedCounter = mock(Counter.class); - sqsMessagesFailedCounter = mock(Counter.class); - acknowledgementSet = mock(AcknowledgementSet.class); - } - - private SqsSourceTask createObjectUnderTest() { - sqsService = new SqsService(sqsMetrics,sqsClient,backoff); - sqsOptions = new SqsOptions.Builder() - .setSqsUrl("https://sqs.us-east-2.amazonaws.com/123456789012/MyQueue") - .setVisibilityTimeout(Duration.ofSeconds(30)) - .setWaitTime(Duration.ofSeconds(20)).build(); - return new SqsSourceTask(buffer,1,Duration.ofSeconds(10),sqsService, - sqsOptions, - sqsMetrics, - acknowledgementSetManager, - endToEndAcknowledgementsEnabled, - sqsHandler); - } - - @ParameterizedTest - @ValueSource(strings = { - "'{\"S.No\":\"1\",\"name\":\"data-prep\",\"country\":\"USA\"}'", - "Test Message", - "'2023-05-30T13:25:11,889 [main] INFO org.opensearch.dataprepper.pipeline.server.DataPrepperServer - Data Prepper server running at :4900'"}) - void processSqsMessages_test_with_different_types_of_messages(final String message) throws Exception { - when(sqsMetrics.getSqsMessagesReceivedCounter()).thenReturn(messageReceivedCounter); - when(sqsMetrics.getSqsMessagesDeletedCounter()).thenReturn(messageDeletedCounter); - - List messageList = List.of(Message.builder().body(message).messageId(UUID.randomUUID().toString()).receiptHandle(UUID.randomUUID().toString()).build()); - when(sqsClient.receiveMessage(any(ReceiveMessageRequest.class))).thenReturn(ReceiveMessageResponse.builder().messages(messageList).build()); - when(sqsMetrics.getSqsMessagesReceivedCounter()).thenReturn(messageReceivedCounter); - when(sqsMetrics.getSqsMessagesDeletedCounter()).thenReturn(messageDeletedCounter); - - when(sqsClient.deleteMessageBatch(any(DeleteMessageBatchRequest.class))). - thenReturn(DeleteMessageBatchResponse.builder().successful(builder -> builder.id(UUID.randomUUID().toString()).build()).build()); - final SqsSourceTask sqsSourceTask = createObjectUnderTest(); - sqsSourceTask.processSqsMessages(); - - verify(sqsHandler).handleMessages(eq(messageList), any(), isNull()); - - verify(sqsMetrics.getSqsMessagesReceivedCounter()).increment(); - verify(sqsMetrics.getSqsMessagesDeletedCounter()).increment(1); - } - - @Test - void processSqsMessages_should_return_zero_messages_with_backoff() { - when(sqsClient.receiveMessage(any(ReceiveMessageRequest.class))).thenThrow(SqsException.class); - when(sqsMetrics.getSqsReceiveMessagesFailedCounter()).thenReturn(sqsMessagesFailedCounter); - createObjectUnderTest().processSqsMessages(); - verify(backoff).nextDelayMillis(1); - verify(sqsMessagesFailedCounter).increment(); - } - - @Test - void processSqsMessages_should_return_one_message_with_buffer_write_fail_with_backoff() { - String message ="'{\"S.No\":\"1\",\"name\":\"data-prep\",\"country\":\"USA\"}'"; - List messageList = List.of(Message.builder().body(message).messageId(UUID.randomUUID().toString()).receiptHandle(UUID.randomUUID().toString()).build()); - when(sqsClient.receiveMessage(any(ReceiveMessageRequest.class))).thenReturn(ReceiveMessageResponse.builder().messages(messageList).build()); - when(sqsClient.deleteMessageBatch(any(DeleteMessageBatchRequest.class))).thenThrow(mock(StsException.class)); - when(sqsMetrics.getSqsMessagesReceivedCounter()).thenReturn(messageReceivedCounter); - when(sqsMetrics.getSqsMessagesDeleteFailedCounter()).thenReturn(messageDeletedCounter); - createObjectUnderTest().processSqsMessages(); - verify(backoff).nextDelayMillis(1); - verify(messageReceivedCounter).increment(); - } - - @Test - void processSqsMessages_test_with_different_types_of_messages_with_end_to_end_ack() throws Exception { - when(sqsMetrics.getSqsMessagesReceivedCounter()).thenReturn(messageReceivedCounter); - when(sqsMetrics.getSqsMessagesDeletedCounter()).thenReturn(messageDeletedCounter); - - endToEndAcknowledgementsEnabled = true; - - String message = "'{\"S.No\":\"1\",\"name\":\"data-prep\",\"country\":\"USA\"}'"; - when(acknowledgementSetManager.create(any( Consumer.class), any(Duration.class))).thenReturn(acknowledgementSet); - List messageList = List.of(Message.builder().body(message).messageId(UUID.randomUUID().toString()).receiptHandle(UUID.randomUUID().toString()).build()); - when(sqsClient.receiveMessage(any(ReceiveMessageRequest.class))).thenReturn(ReceiveMessageResponse.builder().messages(messageList).build()); - - createObjectUnderTest().processSqsMessages(); - - verify(sqsHandler).handleMessages(eq(messageList), any(), eq(acknowledgementSet)); - - verify(sqsMetrics.getSqsMessagesReceivedCounter()).increment(); - verify(acknowledgementSetManager).create(any(), any(Duration.class)); - verifyNoInteractions(sqsMetrics.getSqsMessagesDeletedCounter()); - } - -} - diff --git a/data-prepper-plugins/sqs-source/src/test/java/org/opensearch/dataprepper/plugins/source/sqssource/SqsSourceTest.java b/data-prepper-plugins/sqs-source/src/test/java/org/opensearch/dataprepper/plugins/source/sqssource/SqsSourceTest.java deleted file mode 100644 index 2d3400b7cc..0000000000 --- a/data-prepper-plugins/sqs-source/src/test/java/org/opensearch/dataprepper/plugins/source/sqssource/SqsSourceTest.java +++ /dev/null @@ -1,57 +0,0 @@ -package org.opensearch.dataprepper.plugins.source.sqssource; - -import io.micrometer.core.instrument.Counter; -import io.micrometer.core.instrument.Timer; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; -import org.mockito.Mock; -import org.opensearch.dataprepper.aws.api.AwsCredentialsSupplier; -import org.opensearch.dataprepper.metrics.PluginMetrics; -import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSetManager; -import org.opensearch.dataprepper.plugins.aws.sqs.common.metrics.SqsMetrics; -import org.opensearch.dataprepper.plugins.source.sqssource.config.SqsSourceConfig; - -import java.util.List; - -import static org.junit.jupiter.api.Assertions.assertThrows; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; - -class SqsSourceTest { - - @Mock - private PluginMetrics pluginMetrics; - - @Mock - private AcknowledgementSetManager acknowledgementSetManager; - - @Mock - private AwsCredentialsSupplier awsCredentialsSupplier; - - private SqsSource sqsSource; - - @BeforeEach - public void setup() { - SqsSourceConfig sqsSourceConfig = mock(SqsSourceConfig.class); - pluginMetrics = mock(PluginMetrics.class); - Timer timer = mock(Timer.class); - Counter counter = mock(Counter.class); - - awsCredentialsSupplier = mock(AwsCredentialsSupplier.class); - when(pluginMetrics.timer(SqsMetrics.SQS_MESSAGE_DELAY_METRIC_NAME)).thenReturn(timer); - when(pluginMetrics.counter(SqsMetrics.SQS_MESSAGES_RECEIVED_METRIC_NAME)).thenReturn(counter); - when(pluginMetrics.counter(SqsMetrics.SQS_MESSAGES_DELETED_METRIC_NAME)).thenReturn(counter); - when(pluginMetrics.counter(SqsMetrics.SQS_RECEIVE_MESSAGES_FAILED_METRIC_NAME)).thenReturn(counter); - when(pluginMetrics.counter(SqsMetrics.SQS_MESSAGES_DELETE_FAILED_METRIC_NAME)).thenReturn(counter); - when(pluginMetrics.counter(SqsMetrics.ACKNOWLEDGEMENT_SET_CALLBACK_METRIC_NAME)).thenReturn(counter); - when(sqsSourceConfig.getUrls()).thenReturn(List.of("https://sqs.us-east-1.amazonaws.com/123099425585/dp")); - this. sqsSource = - new SqsSource(pluginMetrics,sqsSourceConfig,acknowledgementSetManager,awsCredentialsSupplier); - } - - @Test - void start_should_throw_IllegalStateException_when_buffer_is_null(){ - assertThrows(IllegalStateException.class, () -> sqsSource.start(null)); - } - -} diff --git a/data-prepper-plugins/sqs-source/src/test/java/org/opensearch/dataprepper/plugins/source/sqssource/config/SqsSourceConfigTest.java b/data-prepper-plugins/sqs-source/src/test/java/org/opensearch/dataprepper/plugins/source/sqssource/config/SqsSourceConfigTest.java deleted file mode 100644 index ddecd8a346..0000000000 --- a/data-prepper-plugins/sqs-source/src/test/java/org/opensearch/dataprepper/plugins/source/sqssource/config/SqsSourceConfigTest.java +++ /dev/null @@ -1,47 +0,0 @@ -/* - * Copyright OpenSearch Contributors - * SPDX-License-Identifier: Apache-2.0 - */ -package org.opensearch.dataprepper.plugins.source.sqssource.config; - -import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.dataformat.yaml.YAMLFactory; -import com.fasterxml.jackson.dataformat.yaml.YAMLGenerator; -import org.junit.jupiter.api.Test; -import software.amazon.awssdk.regions.Region; - -import java.io.IOException; -import java.nio.file.Files; -import java.nio.file.Path; -import java.nio.file.Paths; -import java.time.Duration; - -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.nullValue; - -class SqsSourceConfigTest { - - private static final String SQS_CONFIGURATION_YAML = "/src/test/resources/pipeline.yaml"; - - private final ObjectMapper objectMapper = new ObjectMapper(new YAMLFactory().enable(YAMLGenerator.Feature.USE_PLATFORM_LINE_BREAKS)); - - @Test - void sqs_source_configuration_test() throws IOException { - final byte[] bytes = Files.readAllBytes(Path.of(Paths.get("").toAbsolutePath() + SQS_CONFIGURATION_YAML)); - final SqsSourceConfig sqsSourceConfig = objectMapper.readValue(bytes, SqsSourceConfig.class); - final AwsAuthenticationOptions aws = sqsSourceConfig.getAws(); - - assertThat(sqsSourceConfig.getAcknowledgements(),equalTo(false)); - assertThat(sqsSourceConfig.getPollingFrequency(),equalTo(Duration.ZERO)); - assertThat(sqsSourceConfig.getBatchSize(),equalTo(10)); - assertThat(sqsSourceConfig.getNumberOfThreads(),equalTo(5)); - assertThat(sqsSourceConfig.getVisibilityTimeout(),nullValue()); - assertThat(sqsSourceConfig.getWaitTime(),nullValue()); - assertThat(sqsSourceConfig.getUrls().get(0),equalTo("https://sqs.us-east-1.amazonaws.com/123099425585/dp")); - - assertThat(aws.getAwsRegion(),equalTo(Region.US_EAST_1)); - assertThat(aws.getAwsStsRoleArn(),equalTo("arn:aws:iam::278936200144:role/aos-role")); - assertThat(aws.getAwsStsHeaderOverrides().get("test"),equalTo("test")); - } -} diff --git a/data-prepper-plugins/sqs-source/src/test/java/org/opensearch/dataprepper/plugins/source/sqssource/handler/RawSqsMessageHandlerTest.java b/data-prepper-plugins/sqs-source/src/test/java/org/opensearch/dataprepper/plugins/source/sqssource/handler/RawSqsMessageHandlerTest.java deleted file mode 100644 index c71ae04204..0000000000 --- a/data-prepper-plugins/sqs-source/src/test/java/org/opensearch/dataprepper/plugins/source/sqssource/handler/RawSqsMessageHandlerTest.java +++ /dev/null @@ -1,132 +0,0 @@ -package org.opensearch.dataprepper.plugins.source.sqssource.handler; - -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Nested; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.extension.ExtendWith; -import org.mockito.ArgumentCaptor; -import org.mockito.InOrder; -import org.mockito.Mock; -import org.mockito.junit.jupiter.MockitoExtension; -import org.opensearch.dataprepper.buffer.common.BufferAccumulator; -import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSet; -import org.opensearch.dataprepper.model.event.Event; -import org.opensearch.dataprepper.model.record.Record; -import org.opensearch.dataprepper.plugins.aws.sqs.common.SqsService; -import software.amazon.awssdk.services.sqs.model.DeleteMessageBatchRequestEntry; -import software.amazon.awssdk.services.sqs.model.Message; - -import java.util.List; -import java.util.UUID; -import java.util.stream.Collectors; -import java.util.stream.IntStream; - -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.instanceOf; -import static org.mockito.Mockito.inOrder; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.times; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; - -@ExtendWith(MockitoExtension.class) -class RawSqsMessageHandlerTest { - @Mock - private SqsService sqsService; - - @Mock - private BufferAccumulator bufferAccumulator; - - private AcknowledgementSet acknowledgementSet; - private List messageBodies; - private List messages; - - @BeforeEach - void setUp() { - messageBodies = IntStream.range(0, 3).mapToObj(i -> UUID.randomUUID().toString()) - .collect(Collectors.toList()); - - messages = messageBodies.stream() - .map(body -> { - Message message = mock(Message.class); - when(message.body()).thenReturn(body); - return message; - }) - .collect(Collectors.toList()); - - - acknowledgementSet = null; - } - - private RawSqsMessageHandler createObjectUnderTest() { - return new RawSqsMessageHandler(sqsService); - } - - @Test - void handleMessages_writes_to_buffer_and_flushes() throws Exception { - createObjectUnderTest().handleMessages(messages, bufferAccumulator, acknowledgementSet); - - InOrder inOrder = inOrder(bufferAccumulator); - - ArgumentCaptor> recordArgumentCaptor = ArgumentCaptor.forClass(Record.class); - - inOrder.verify(bufferAccumulator, times(messages.size())).add(recordArgumentCaptor.capture()); - inOrder.verify(bufferAccumulator).flush(); - - List actualEventData = recordArgumentCaptor.getAllValues() - .stream() - .map(Record::getData) - .map(e -> e.get("message", Object.class)) - .collect(Collectors.toList()); - - assertThat(actualEventData.size(), equalTo(messages.size())); - - for (int i = 0; i < actualEventData.size(); i++){ - Object messageData = actualEventData.get(i); - assertThat(messageData, instanceOf(String.class)); - assertThat(messageData, equalTo(messageBodies.get(i))); - } - } - - @Test - void handleMessages_returns_deleteList() throws Exception { - List stubbedDeleteList = List.of(mock(DeleteMessageBatchRequestEntry.class)); - when(sqsService.getDeleteMessageBatchRequestEntryList(messages)) - .thenReturn(stubbedDeleteList); - - List actualList = createObjectUnderTest().handleMessages(messages, bufferAccumulator, acknowledgementSet); - - assertThat(actualList, equalTo(stubbedDeleteList)); - } - - @Nested - class WithAcknowledgementSet { - @BeforeEach - void setUp() { - acknowledgementSet = mock(AcknowledgementSet.class); - } - - @Test - void handleMessages_with_acknowledgementSet_adds_events() throws Exception { - createObjectUnderTest().handleMessages(messages, bufferAccumulator, acknowledgementSet); - - ArgumentCaptor eventArgumentCaptor = ArgumentCaptor.forClass(Event.class); - - verify(acknowledgementSet, times(messages.size())).add(eventArgumentCaptor.capture()); - - List actualEventData = eventArgumentCaptor.getAllValues() - .stream() - .map(e -> e.get("message", Object.class)) - .collect(Collectors.toList()); - - assertThat(actualEventData.size(), equalTo(messages.size())); - - for (int i = 0; i < actualEventData.size(); i++) { - Object messageData = actualEventData.get(i); - assertThat(messageData, instanceOf(String.class)); - assertThat(messageData, equalTo(messageBodies.get(i))); - } - } - } -} diff --git a/data-prepper-plugins/sqs-source/src/test/resources/pipeline.yaml b/data-prepper-plugins/sqs-source/src/test/resources/pipeline.yaml deleted file mode 100644 index 2e1e6c5500..0000000000 --- a/data-prepper-plugins/sqs-source/src/test/resources/pipeline.yaml +++ /dev/null @@ -1,10 +0,0 @@ -acknowledgments: false -queue_urls: - - https://sqs.us-east-1.amazonaws.com/123099425585/dp - - https://sqs.us-east-1.amazonaws.com/123099425585/dp1 -batch_size: 10 -number_of_threads: 5 -aws: - sts_region: us-east-1 - sts_role_arn: arn:aws:iam::278936200144:role/aos-role - sts_header_overrides: {"test":"test"} diff --git a/settings.gradle b/settings.gradle index d15b6d45e2..e2298a87a6 100644 --- a/settings.gradle +++ b/settings.gradle @@ -166,7 +166,7 @@ include 'data-prepper-plugins:obfuscate-processor' include 'data-prepper-plugins:parquet-codecs' include 'data-prepper-plugins:aws-sqs-common' include 'data-prepper-plugins:buffer-common' -//include 'data-prepper-plugins:sqs-source' +include 'data-prepper-plugins:sqs-source' //include 'data-prepper-plugins:cloudwatch-logs' //include 'data-prepper-plugins:http-sink' //include 'data-prepper-plugins:sns-sink'