callback, final Duration timeout);
-
- /**
- * Releases an event's reference
- *
- * @param eventHandle event handle
- * @param success indicates negative or positive acknowledgement
- *
- * @since 2.2
- */
- void releaseEventReference(final EventHandle eventHandle, boolean success);
-
- /**
- * Acquires an event's reference
- *
- * @param eventHandle event handle
- *
- * @since 2.2
- */
- void acquireEventReference(final EventHandle eventHandle);
-
- /**
- * Acquires an event's reference
- *
- * @param event event
- *
- * @since 2.2
- */
- void acquireEventReference(final Event event);
}
diff --git a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/configuration/PluginSetting.java b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/configuration/PluginSetting.java
index a8ea4a3ee1..61db9a3c7e 100644
--- a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/configuration/PluginSetting.java
+++ b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/configuration/PluginSetting.java
@@ -5,10 +5,22 @@
package org.opensearch.dataprepper.model.configuration;
+import org.opensearch.dataprepper.model.annotations.DataPrepperPlugin;
+
import java.util.Collections;
import java.util.List;
import java.util.Map;
+/**
+ * Deprecated class for getting plugin settings.
+ *
+ * Only projects within data-prepper-core should use this. It is currently used
+ * extensively in plugin framework to load plugins. In Data Prepper 3.0 this
+ * class will be moved into data-prepper-core and not exposed to plugins anymore.
+ *
+ * @deprecated Use {@link DataPrepperPlugin#pluginConfigurationType()} or {@link PipelineDescription} instead.
+ */
+@Deprecated
public class PluginSetting implements PipelineDescription {
private static final String UNEXPECTED_ATTRIBUTE_TYPE_MSG = "Unexpected type [%s] for attribute [%s]";
diff --git a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/AbstractEventHandle.java b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/AbstractEventHandle.java
new file mode 100644
index 0000000000..2ca40fbe59
--- /dev/null
+++ b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/AbstractEventHandle.java
@@ -0,0 +1,52 @@
+/*
+ * Copyright OpenSearch Contributors
+ * SPDX-License-Identifier: Apache-2.0
+ */
+
+package org.opensearch.dataprepper.model.event;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.time.Instant;
+import java.util.function.BiConsumer;
+
+abstract class AbstractEventHandle implements EventHandle, InternalEventHandle {
+ private Instant externalOriginationTime;
+ private final Instant internalOriginationTime;
+ private List> releaseConsumers;
+
+ AbstractEventHandle(final Instant internalOriginationTime) {
+ this.externalOriginationTime = null;
+ this.internalOriginationTime = internalOriginationTime;
+ this.releaseConsumers = new ArrayList<>();
+ }
+ @Override
+ public void setExternalOriginationTime(final Instant externalOriginationTime) {
+ this.externalOriginationTime = externalOriginationTime;
+ }
+
+ @Override
+ public Instant getInternalOriginationTime() {
+ return this.internalOriginationTime;
+ }
+
+ @Override
+ public Instant getExternalOriginationTime() {
+ return this.externalOriginationTime;
+ }
+
+ @Override
+ public void onRelease(BiConsumer releaseConsumer) {
+ synchronized (releaseConsumers) {
+ releaseConsumers.add(releaseConsumer);
+ }
+ }
+
+ public void notifyReleaseConsumers(boolean result) {
+ synchronized (releaseConsumers) {
+ for (final BiConsumer consumer: releaseConsumers) {
+ consumer.accept(this, result);
+ }
+ }
+ }
+}
diff --git a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/AggregateEventHandle.java b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/AggregateEventHandle.java
new file mode 100644
index 0000000000..921d689a3c
--- /dev/null
+++ b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/AggregateEventHandle.java
@@ -0,0 +1,77 @@
+/*
+ * Copyright OpenSearch Contributors
+ * SPDX-License-Identifier: Apache-2.0
+ */
+
+package org.opensearch.dataprepper.model.event;
+
+import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSet;
+import java.lang.ref.WeakReference;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.time.Instant;
+import java.io.Serializable;
+
+public class AggregateEventHandle extends AbstractEventHandle implements Serializable {
+ private List> acknowledgementSetRefList;
+ private Set acknowledgementSetHashes;
+
+ public AggregateEventHandle(final Instant internalOriginationTime) {
+ super(internalOriginationTime);
+ this.acknowledgementSetRefList = new ArrayList<>();
+ this.acknowledgementSetHashes = new HashSet<>();
+ }
+
+ @Override
+ public void addAcknowledgementSet(final AcknowledgementSet acknowledgementSet) {
+ int hashCode = acknowledgementSet.hashCode();
+ if (!acknowledgementSetHashes.contains(hashCode)) {
+ this.acknowledgementSetRefList.add(new WeakReference<>(acknowledgementSet));
+ acknowledgementSetHashes.add(hashCode);
+ }
+ }
+
+ @Override
+ public boolean hasAcknowledgementSet() {
+ return acknowledgementSetRefList.size() != 0;
+ }
+
+ @Override
+ public void acquireReference() {
+ synchronized (this) {
+ for (WeakReference acknowledgementSetRef: acknowledgementSetRefList) {;
+ AcknowledgementSet acknowledgementSet = acknowledgementSetRef.get();
+ if (acknowledgementSet != null) {
+ acknowledgementSet.acquire(this);
+ }
+ }
+ }
+ }
+
+ @Override
+ public boolean release(boolean result) {
+ notifyReleaseConsumers(result);
+ boolean returnValue = true;
+ synchronized (this) {
+ for (WeakReference acknowledgementSetRef: acknowledgementSetRefList) {
+ AcknowledgementSet acknowledgementSet = acknowledgementSetRef.get();
+ if (acknowledgementSet != null) {
+ acknowledgementSet.release(this, result);
+ } else {
+ returnValue = false;
+ }
+ }
+ }
+ return returnValue;
+ }
+
+ // For testing
+ List> getAcknowledgementSetRefs() {
+ return acknowledgementSetRefList;
+ }
+
+}
+
diff --git a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/DefaultEventHandle.java b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/DefaultEventHandle.java
index 743309bf75..340c104a14 100644
--- a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/DefaultEventHandle.java
+++ b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/DefaultEventHandle.java
@@ -8,35 +8,22 @@
import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSet;
import java.lang.ref.WeakReference;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.function.BiConsumer;
import java.time.Instant;
import java.io.Serializable;
-public class DefaultEventHandle implements EventHandle, InternalEventHandle, Serializable {
- private Instant externalOriginationTime;
- private final Instant internalOriginationTime;
+public class DefaultEventHandle extends AbstractEventHandle implements Serializable {
private WeakReference acknowledgementSetRef;
- private List> releaseConsumers;
public DefaultEventHandle(final Instant internalOriginationTime) {
+ super(internalOriginationTime);
this.acknowledgementSetRef = null;
- this.externalOriginationTime = null;
- this.internalOriginationTime = internalOriginationTime;
- this.releaseConsumers = new ArrayList<>();
}
@Override
- public void setAcknowledgementSet(final AcknowledgementSet acknowledgementSet) {
+ public void addAcknowledgementSet(final AcknowledgementSet acknowledgementSet) {
this.acknowledgementSetRef = new WeakReference<>(acknowledgementSet);
}
- @Override
- public void setExternalOriginationTime(final Instant externalOriginationTime) {
- this.externalOriginationTime = externalOriginationTime;
- }
-
public AcknowledgementSet getAcknowledgementSet() {
if (acknowledgementSetRef == null) {
return null;
@@ -45,32 +32,30 @@ public AcknowledgementSet getAcknowledgementSet() {
}
@Override
- public Instant getInternalOriginationTime() {
- return this.internalOriginationTime;
+ public boolean hasAcknowledgementSet() {
+ AcknowledgementSet acknowledgementSet = getAcknowledgementSet();
+ return acknowledgementSet != null;
}
@Override
- public Instant getExternalOriginationTime() {
- return this.externalOriginationTime;
+ public void acquireReference() {
+ synchronized (this) {
+ AcknowledgementSet acknowledgementSet = getAcknowledgementSet();
+ if (acknowledgementSet != null) {
+ acknowledgementSet.acquire(this);
+ }
+ }
}
@Override
- public void release(boolean result) {
- synchronized (releaseConsumers) {
- for (final BiConsumer consumer: releaseConsumers) {
- consumer.accept(this, result);
- }
- }
+ public boolean release(boolean result) {
+ notifyReleaseConsumers(result);
AcknowledgementSet acknowledgementSet = getAcknowledgementSet();
if (acknowledgementSet != null) {
acknowledgementSet.release(this, result);
+ return true;
}
+ return false;
}
- @Override
- public void onRelease(BiConsumer releaseConsumer) {
- synchronized (releaseConsumers) {
- releaseConsumers.add(releaseConsumer);
- }
- }
}
diff --git a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/Event.java b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/Event.java
index 740447ecc0..e0e36d9237 100644
--- a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/Event.java
+++ b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/Event.java
@@ -26,6 +26,15 @@
*/
public interface Event extends Serializable {
+ /**
+ * Adds or updates the key with a given value in the Event
+ *
+ * @param key where the value will be set
+ * @param value value to set the key to
+ * @since 2.8
+ */
+ void put(EventKey key, Object value);
+
/**
* Adds or updates the key with a given value in the Event
*
@@ -35,6 +44,17 @@ public interface Event extends Serializable {
*/
void put(String key, Object value);
+ /**
+ * Retrieves the given key from the Event
+ *
+ * @param key the value to retrieve from
+ * @param clazz the return type of the value
+ * @param The type
+ * @return T a clazz object from the key
+ * @since 2.8
+ */
+ T get(EventKey key, Class clazz);
+
/**
* Retrieves the given key from the Event
*
@@ -46,6 +66,17 @@ public interface Event extends Serializable {
*/
T get(String key, Class clazz);
+ /**
+ * Retrieves the given key from the Event as a List
+ *
+ * @param key the value to retrieve from
+ * @param clazz the return type of elements in the list
+ * @param The type
+ * @return {@literal List} a list of clazz elements
+ * @since 2.8
+ */
+ List getList(EventKey key, Class clazz);
+
/**
* Retrieves the given key from the Event as a List
*
@@ -57,6 +88,14 @@ public interface Event extends Serializable {
*/
List getList(String key, Class clazz);
+ /**
+ * Deletes the given key from the Event
+ *
+ * @param key the field to be deleted
+ * @since 2.8
+ */
+ void delete(EventKey key);
+
/**
* Deletes the given key from the Event
*
@@ -87,6 +126,15 @@ public interface Event extends Serializable {
*/
JsonNode getJsonNode();
+ /**
+ * Gets a serialized Json string of the specific key in the Event
+ *
+ * @param key the field to be returned
+ * @return Json string of the field
+ * @since 2.8
+ */
+ String getAsJsonString(EventKey key);
+
/**
* Gets a serialized Json string of the specific key in the Event
*
@@ -104,6 +152,15 @@ public interface Event extends Serializable {
*/
EventMetadata getMetadata();
+ /**
+ * Checks if the key exists.
+ *
+ * @param key name of the key to look for
+ * @return returns true if the key exists, otherwise false
+ * @since 2.8
+ */
+ boolean containsKey(EventKey key);
+
/**
* Checks if the key exists.
*
@@ -113,6 +170,15 @@ public interface Event extends Serializable {
*/
boolean containsKey(String key);
+ /**
+ * Checks if the value stored for the key is list
+ *
+ * @param key name of the key to look for
+ * @return returns true if the key is a list, otherwise false
+ * @since 2.8
+ */
+ boolean isValueAList(EventKey key);
+
/**
* Checks if the value stored for the key is list
*
diff --git a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/EventHandle.java b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/EventHandle.java
index d05dd8e36c..898384c32e 100644
--- a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/EventHandle.java
+++ b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/EventHandle.java
@@ -14,9 +14,10 @@ public interface EventHandle {
*
* @param result result to be used while releasing. This indicates if
* the operation on the event handle is success or not
+ * @return returns true if the event handle is released successful, false otherwise
* @since 2.2
*/
- void release(boolean result);
+ boolean release(boolean result);
/**
* sets external origination time
diff --git a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/EventKey.java b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/EventKey.java
new file mode 100644
index 0000000000..9086f0f641
--- /dev/null
+++ b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/EventKey.java
@@ -0,0 +1,21 @@
+/*
+ * Copyright OpenSearch Contributors
+ * SPDX-License-Identifier: Apache-2.0
+ */
+
+package org.opensearch.dataprepper.model.event;
+
+/**
+ * Model class to represent a key into a Data Prepper {@link Event}.
+ *
+ * @since 2.9
+ */
+public interface EventKey {
+ /**
+ * The original key provided as a string.
+ *
+ * @return The key as a string
+ * @since 2.9
+ */
+ String getKey();
+}
diff --git a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/EventKeyConfiguration.java b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/EventKeyConfiguration.java
new file mode 100644
index 0000000000..c35e8db38c
--- /dev/null
+++ b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/EventKeyConfiguration.java
@@ -0,0 +1,34 @@
+/*
+ * Copyright OpenSearch Contributors
+ * SPDX-License-Identifier: Apache-2.0
+ */
+
+package org.opensearch.dataprepper.model.event;
+
+import java.lang.annotation.Documented;
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+/**
+ * An annotation for an {@link EventKey} used in a Data Prepper pipeline configuration.
+ *
+ * Unless you need all actions on a configuration, you should use this annotation to
+ * provide the most appropriate validation.
+ *
+ * @since 2.9
+ */
+@Documented
+@Retention(RetentionPolicy.RUNTIME)
+@Target({ElementType.FIELD})
+public @interface EventKeyConfiguration {
+ /**
+ * Defines the {@link EventKeyFactory.EventAction}s to use when creating the {@link EventKey}
+ * for the configuration.
+ *
+ * @return The desired event actions.
+ * @since 2.9
+ */
+ EventKeyFactory.EventAction[] value();
+}
diff --git a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/EventKeyFactory.java b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/EventKeyFactory.java
new file mode 100644
index 0000000000..e7cbc25463
--- /dev/null
+++ b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/EventKeyFactory.java
@@ -0,0 +1,71 @@
+/*
+ * Copyright OpenSearch Contributors
+ * SPDX-License-Identifier: Apache-2.0
+ */
+
+package org.opensearch.dataprepper.model.event;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Set;
+
+/**
+ * A factory for producing {@link EventKey} objects.
+ *
+ * @since 2.9
+ */
+public interface EventKeyFactory {
+ /**
+ * Creates an {@link EventKey} with given actions.
+ *
+ * @param key The key
+ * @param forActions Actions to support
+ * @return The EventKey
+ * @since 2.9
+ */
+ EventKey createEventKey(String key, EventAction... forActions);
+
+ /**
+ * Creates an {@link EventKey} for the default actions, which are all.
+ *
+ * @param key The key
+ * @return The EventKey
+ * @since 2.9
+ */
+ default EventKey createEventKey(final String key) {
+ return createEventKey(key, EventAction.ALL);
+ }
+
+ /**
+ * An action on an Event.
+ *
+ * @since 2.9
+ */
+ enum EventAction {
+ GET,
+ DELETE,
+ PUT,
+ ALL(GET, DELETE, PUT);
+
+ private final List includedActions;
+
+ EventAction(EventAction... eventActions) {
+ includedActions = Arrays.asList(eventActions);
+
+ }
+
+ boolean isMutableAction() {
+ return this != GET;
+ }
+
+ Set getSupportedActions() {
+ final EnumSet supportedActions = EnumSet.noneOf(EventAction.class);
+ supportedActions.add(this);
+ supportedActions.addAll(includedActions);
+
+ return Collections.unmodifiableSet(supportedActions);
+ }
+ }
+}
diff --git a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/InternalEventHandle.java b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/InternalEventHandle.java
index 3817365f17..3ee88f698b 100644
--- a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/InternalEventHandle.java
+++ b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/InternalEventHandle.java
@@ -9,20 +9,27 @@
public interface InternalEventHandle {
/**
- * sets acknowledgement set
+ * adds acknowledgement set
*
* @param acknowledgementSet acknowledgementSet to be set in the event handle
- * @since 2.6
+ * @since 2.9
*/
- void setAcknowledgementSet(final AcknowledgementSet acknowledgementSet);
+ void addAcknowledgementSet(final AcknowledgementSet acknowledgementSet);
/**
- * gets acknowledgement set
+ * Indicates if the event handle has atleast one acknowledgement set
*
- * @return returns acknowledgementSet from the event handle
- * @since 2.6
+ * @return returns true if there is at least one acknowledgementSet in the event handle
+ * @since 2.9
*/
- AcknowledgementSet getAcknowledgementSet();
+ boolean hasAcknowledgementSet();
+
+ /**
+ * Acquires reference to acknowledgement set(s) in the event handle
+ *
+ * @since 2.9
+ */
+ void acquireReference();
}
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 9ef34bb82c..25ef31ec8b 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
@@ -28,8 +28,8 @@
import java.io.ObjectInputStream;
import java.time.Instant;
import java.util.ArrayList;
-import java.util.Arrays;
import java.util.Collections;
+import java.util.Deque;
import java.util.HashMap;
import java.util.Iterator;
import java.util.LinkedList;
@@ -38,8 +38,8 @@
import java.util.Objects;
import java.util.StringJoiner;
-import static com.google.common.base.Preconditions.checkArgument;
import static com.google.common.base.Preconditions.checkNotNull;
+import static org.opensearch.dataprepper.model.event.JacksonEventKey.trimTrailingSlashInKey;
/**
* A Jackson Implementation of {@link Event} interface. This implementation relies heavily on JsonNode to manage the keys of the event.
@@ -137,20 +137,15 @@ public JsonNode getJsonNode() {
return jsonNode;
}
- /**
- * Adds or updates the key with a given value in the Event.
- *
- * @param key where the value will be set
- * @param value value to set the key to
- * @since 1.2
- */
@Override
- public void put(final String key, final Object value) {
- checkArgument(!key.isEmpty(), "key cannot be an empty string for put method");
+ public void put(EventKey key, Object value) {
+ final JacksonEventKey jacksonEventKey = asJacksonEventKey(key);
- final String trimmedKey = checkAndTrimKey(key);
+ if(!jacksonEventKey.supports(EventKeyFactory.EventAction.PUT)) {
+ throw new IllegalArgumentException("key cannot be an empty string for put method");
+ }
- final LinkedList keys = new LinkedList<>(Arrays.asList(trimmedKey.split(SEPARATOR, -1)));
+ final Deque keys = new LinkedList<>(jacksonEventKey.getKeyPathList());
JsonNode parentNode = jsonNode;
@@ -166,6 +161,19 @@ public void put(final String key, final Object value) {
}
}
+ /**
+ * Adds or updates the key with a given value in the Event.
+ *
+ * @param key where the value will be set
+ * @param value value to set the key to
+ * @since 1.2
+ */
+ @Override
+ public void put(final String key, final Object value) {
+ final JacksonEventKey jacksonEventKey = new JacksonEventKey(key, true, EventKeyFactory.EventAction.PUT);
+ put(jacksonEventKey, value);
+ }
+
@Override
public EventHandle getEventHandle() {
return eventHandle;
@@ -189,6 +197,27 @@ private JsonNode getOrCreateNode(final JsonNode node, final String key) {
return childNode;
}
+ @Override
+ public T get(EventKey key, Class clazz) {
+ JacksonEventKey jacksonEventKey = asJacksonEventKey(key);
+
+ final JsonNode node = getNode(jacksonEventKey);
+ if (node.isMissingNode()) {
+ return null;
+ }
+
+ return mapNodeToObject(key.getKey(), node, clazz);
+ }
+
+ private static JacksonEventKey asJacksonEventKey(EventKey key) {
+ if(!(key instanceof JacksonEventKey)) {
+ throw new IllegalArgumentException("The key provided must be obtained through the EventKeyFactory.");
+ }
+
+ JacksonEventKey jacksonEventKey = (JacksonEventKey) key;
+ return jacksonEventKey;
+ }
+
/**
* Retrieves the value of type clazz from the key.
*
@@ -200,15 +229,8 @@ private JsonNode getOrCreateNode(final JsonNode node, final String key) {
*/
@Override
public T get(final String key, final Class clazz) {
-
- final String trimmedKey = checkAndTrimKey(key);
-
- final JsonNode node = getNode(trimmedKey);
- if (node.isMissingNode()) {
- return null;
- }
-
- return mapNodeToObject(key, node, clazz);
+ final JacksonEventKey jacksonEventKey = new JacksonEventKey(key, true, EventKeyFactory.EventAction.GET);
+ return get(jacksonEventKey, clazz);
}
private JsonNode getNode(final String key) {
@@ -216,6 +238,10 @@ private JsonNode getNode(final String key) {
return jsonNode.at(jsonPointer);
}
+ private JsonNode getNode(final JacksonEventKey key) {
+ return jsonNode.at(key.getJsonPointer());
+ }
+
private T mapNodeToObject(final String key, final JsonNode node, final Class clazz) {
try {
return mapper.treeToValue(node, clazz);
@@ -225,6 +251,18 @@ private T mapNodeToObject(final String key, final JsonNode node, final Class
}
}
+ @Override
+ public List getList(EventKey key, Class clazz) {
+ JacksonEventKey jacksonEventKey = asJacksonEventKey(key);
+
+ final JsonNode node = getNode(jacksonEventKey);
+ if (node.isMissingNode()) {
+ return null;
+ }
+
+ return mapNodeToList(jacksonEventKey.getKey(), node, clazz);
+ }
+
/**
* Retrieves the given key from the Event as a List
*
@@ -236,15 +274,8 @@ private T mapNodeToObject(final String key, final JsonNode node, final Class
*/
@Override
public List getList(final String key, final Class clazz) {
-
- final String trimmedKey = checkAndTrimKey(key);
-
- final JsonNode node = getNode(trimmedKey);
- if (node.isMissingNode()) {
- return null;
- }
-
- return mapNodeToList(key, node, clazz);
+ JacksonEventKey jacksonEventKey = new JacksonEventKey(key, true, EventKeyFactory.EventAction.GET);
+ return getList(jacksonEventKey, clazz);
}
private List mapNodeToList(final String key, final JsonNode node, final Class clazz) {
@@ -267,16 +298,15 @@ private JsonPointer toJsonPointer(final String key) {
return JsonPointer.compile(jsonPointerExpression);
}
- /**
- * Deletes the key from the event.
- *
- * @param key the field to be deleted
- */
@Override
- public void delete(final String key) {
+ public void delete(final EventKey key) {
+ final JacksonEventKey jacksonEventKey = asJacksonEventKey(key);
+
+ if(!jacksonEventKey.supports(EventKeyFactory.EventAction.DELETE)) {
+ throw new IllegalArgumentException("key cannot be an empty string for delete method");
+ }
- checkArgument(!key.isEmpty(), "key cannot be an empty string for delete method");
- final String trimmedKey = checkAndTrimKey(key);
+ final String trimmedKey = jacksonEventKey.getTrimmedKey();
final int index = trimmedKey.lastIndexOf(SEPARATOR);
JsonNode baseNode = jsonNode;
@@ -293,6 +323,17 @@ public void delete(final String key) {
}
}
+ /**
+ * Deletes the key from the event.
+ *
+ * @param key the field to be deleted
+ */
+ @Override
+ public void delete(final String key) {
+ final JacksonEventKey jacksonEventKey = new JacksonEventKey(key, true, EventKeyFactory.EventAction.DELETE);
+ delete(jacksonEventKey);
+ }
+
@Override
public void clear() {
// Delete all entries from the event
@@ -309,16 +350,22 @@ public String toJsonString() {
}
@Override
- public String getAsJsonString(final String key) {
- final String trimmedKey = checkAndTrimKey(key);
+ public String getAsJsonString(EventKey key) {
- final JsonNode node = getNode(trimmedKey);
+ JacksonEventKey jacksonEventKey = asJacksonEventKey(key);
+ final JsonNode node = getNode(jacksonEventKey);
if (node.isMissingNode()) {
return null;
}
return node.toString();
}
+ @Override
+ public String getAsJsonString(final String key) {
+ JacksonEventKey jacksonEventKey = new JacksonEventKey(key, true, EventKeyFactory.EventAction.GET);
+ return getAsJsonString(jacksonEventKey);
+ }
+
/**
* returns a string with formatted parts replaced by their values. The input
* string may contain parts with format "${.../.../...}" which are replaced
@@ -402,24 +449,35 @@ public EventMetadata getMetadata() {
}
@Override
- public boolean containsKey(final String key) {
-
- final String trimmedKey = checkAndTrimKey(key);
+ public boolean containsKey(EventKey key) {
+ JacksonEventKey jacksonEventKey = asJacksonEventKey(key);
- final JsonNode node = getNode(trimmedKey);
+ final JsonNode node = getNode(jacksonEventKey);
return !node.isMissingNode();
}
@Override
- public boolean isValueAList(final String key) {
- final String trimmedKey = checkAndTrimKey(key);
+ public boolean containsKey(final String key) {
+ JacksonEventKey jacksonEventKey = new JacksonEventKey(key, true, EventKeyFactory.EventAction.GET);
+ return containsKey(jacksonEventKey);
+ }
- final JsonNode node = getNode(trimmedKey);
+ @Override
+ public boolean isValueAList(EventKey key) {
+ JacksonEventKey jacksonEventKey = asJacksonEventKey(key);
+
+ final JsonNode node = getNode(jacksonEventKey);
return node.isArray();
}
+ @Override
+ public boolean isValueAList(final String key) {
+ JacksonEventKey jacksonEventKey = new JacksonEventKey(key, true, EventKeyFactory.EventAction.GET);
+ return isValueAList(jacksonEventKey);
+ }
+
@Override
public Map toMap() {
return mapper.convertValue(jsonNode, MAP_TYPE_REFERENCE);
@@ -427,30 +485,7 @@ public Map toMap() {
public static boolean isValidEventKey(final String key) {
- try {
- checkKey(key);
- return true;
- } catch (final Exception e) {
- return false;
- }
- }
- private String checkAndTrimKey(final String key) {
- checkKey(key);
- return trimTrailingSlashInKey(key);
- }
-
- private static void checkKey(final String key) {
- checkNotNull(key, "key cannot be null");
- if (key.isEmpty()) {
- // Empty string key is valid
- return;
- }
- if (key.length() > MAX_KEY_LENGTH) {
- throw new IllegalArgumentException("key cannot be longer than " + MAX_KEY_LENGTH + " characters");
- }
- if (!isValidKey(key)) {
- throw new IllegalArgumentException("key " + key + " must contain only alphanumeric chars with .-_@/ and must follow JsonPointer (ie. 'field/to/key')");
- }
+ return JacksonEventKey.isValidEventKey(key);
}
private String trimKey(final String key) {
@@ -459,31 +494,6 @@ private String trimKey(final String key) {
return trimTrailingSlashInKey(trimmedLeadingSlash);
}
- private String trimTrailingSlashInKey(final String key) {
- return key.length() > 1 && key.endsWith(SEPARATOR) ? key.substring(0, key.length() - 1) : key;
- }
-
- private static boolean isValidKey(final String key) {
- for (int i = 0; i < key.length(); i++) {
- char c = key.charAt(i);
-
- if (!(c >= 48 && c <= 57
- || c >= 65 && c <= 90
- || c >= 97 && c <= 122
- || c == '.'
- || c == '-'
- || c == '_'
- || c == '@'
- || c == '/'
- || c == '['
- || c == ']')) {
-
- return false;
- }
- }
- return true;
- }
-
/**
* Constructs an empty builder.
*
diff --git a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/JacksonEventKey.java b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/JacksonEventKey.java
new file mode 100644
index 0000000000..50d59a6585
--- /dev/null
+++ b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/JacksonEventKey.java
@@ -0,0 +1,194 @@
+/*
+ * Copyright OpenSearch Contributors
+ * SPDX-License-Identifier: Apache-2.0
+ */
+
+package org.opensearch.dataprepper.model.event;
+
+import com.fasterxml.jackson.core.JsonPointer;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+class JacksonEventKey implements EventKey {
+ private static final String SEPARATOR = "/";
+ private static final int MAX_KEY_LENGTH = 2048;
+ private final String key;
+ private final EventKeyFactory.EventAction[] eventActions;
+ private final String trimmedKey;
+ private List keyPathList;
+ private JsonPointer jsonPointer;
+ private final Set supportedActions;
+
+ /**
+ * Constructor for the JacksonEventKey which should only be used by implementation
+ * of {@link EventKeyFactory} in Data Prepper core.
+ *
+ * @param key The key
+ * @param eventActions Event actions to support
+ */
+ JacksonEventKey(final String key, final EventKeyFactory.EventAction... eventActions) {
+ this(key, false, eventActions);
+ }
+
+ /**
+ * Constructs a new JacksonEventKey.
+ *
+ * This overload should only be used by {@link JacksonEvent} directly. It allows for skipping creating
+ * some resources knowing they will not be needed. The {@link JacksonEvent} only needs a JSON pointer
+ * when performing GET event actions. So we can optimize PUT/DELETE actions when called with a string
+ * key instead of an EventKey by not creating the JSON Pointer at all.
+ *
+ * For EventKey's constructed through the factory, we should not perform lazy initialization since
+ * we may lose some possible validations.
+ *
+ * @param key the key
+ * @param lazy Use true to lazily initialize. This will not be thread-safe, however.
+ * @param eventActions Event actions to support
+ */
+ JacksonEventKey(final String key, final boolean lazy, final EventKeyFactory.EventAction... eventActions) {
+ this.key = Objects.requireNonNull(key, "Parameter key cannot be null for EventKey.");
+ this.eventActions = eventActions.length == 0 ? new EventKeyFactory.EventAction[] { EventKeyFactory.EventAction.ALL } : eventActions;
+
+ supportedActions = EnumSet.noneOf(EventKeyFactory.EventAction.class);
+ for (final EventKeyFactory.EventAction eventAction : this.eventActions) {
+ supportedActions.addAll(eventAction.getSupportedActions());
+ }
+
+ if(key.isEmpty()) {
+ for (final EventKeyFactory.EventAction action : this.eventActions) {
+ if (action.isMutableAction()) {
+ throw new IllegalArgumentException("Event key cannot be an empty string for " + action + " actions.");
+ }
+ }
+ }
+
+ trimmedKey = checkAndTrimKey(key);
+
+ if(!lazy) {
+ keyPathList = toKeyPathList();
+ jsonPointer = toJsonPointer(trimmedKey);
+ }
+ }
+
+ @Override
+ public String getKey() {
+ return key;
+ }
+
+ String getTrimmedKey() {
+ return trimmedKey;
+ }
+
+ List getKeyPathList() {
+ if(keyPathList == null) {
+ keyPathList = toKeyPathList();
+ }
+ return keyPathList;
+ }
+
+ JsonPointer getJsonPointer() {
+ if(jsonPointer == null) {
+ jsonPointer = toJsonPointer(trimmedKey);
+ }
+ return jsonPointer;
+ }
+
+ boolean supports(final EventKeyFactory.EventAction eventAction) {
+ return supportedActions.contains(eventAction);
+ }
+
+ @Override
+ public boolean equals(final Object other) {
+ if (this == other)
+ return true;
+ if (other == null || getClass() != other.getClass())
+ return false;
+ final JacksonEventKey that = (JacksonEventKey) other;
+ return Objects.equals(key, that.key) && Arrays.equals(eventActions, that.eventActions);
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hash(key, Arrays.hashCode(eventActions));
+ }
+
+ @Override
+ public String toString() {
+ return key;
+ }
+
+ private String checkAndTrimKey(final String key) {
+ checkKey(key);
+ return trimTrailingSlashInKey(key);
+ }
+
+ private static void checkKey(final String key) {
+ checkNotNull(key, "key cannot be null");
+ if (key.isEmpty()) {
+ // Empty string key is valid
+ return;
+ }
+ if (key.length() > MAX_KEY_LENGTH) {
+ throw new IllegalArgumentException("key cannot be longer than " + MAX_KEY_LENGTH + " characters");
+ }
+ if (!isValidKey(key)) {
+ throw new IllegalArgumentException("key " + key + " must contain only alphanumeric chars with .-_@/ and must follow JsonPointer (ie. 'field/to/key')");
+ }
+ }
+
+
+ static String trimTrailingSlashInKey(final String key) {
+ return key.length() > 1 && key.endsWith(SEPARATOR) ? key.substring(0, key.length() - 1) : key;
+ }
+
+ private static boolean isValidKey(final String key) {
+ for (int i = 0; i < key.length(); i++) {
+ char c = key.charAt(i);
+
+ if (!(c >= 48 && c <= 57
+ || c >= 65 && c <= 90
+ || c >= 97 && c <= 122
+ || c == '.'
+ || c == '-'
+ || c == '_'
+ || c == '@'
+ || c == '/'
+ || c == '['
+ || c == ']')) {
+
+ return false;
+ }
+ }
+ return true;
+ }
+
+ private List toKeyPathList() {
+ return Collections.unmodifiableList(Arrays.asList(trimmedKey.split(SEPARATOR, -1)));
+ }
+
+ private static JsonPointer toJsonPointer(final String key) {
+ final String jsonPointerExpression;
+ if (key.isEmpty() || key.startsWith("/")) {
+ jsonPointerExpression = key;
+ } else {
+ jsonPointerExpression = SEPARATOR + key;
+ }
+ return JsonPointer.compile(jsonPointerExpression);
+ }
+
+ static boolean isValidEventKey(final String key) {
+ try {
+ checkKey(key);
+ return true;
+ } catch (final Exception e) {
+ return false;
+ }
+ }
+}
diff --git a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/sink/AbstractSink.java b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/sink/AbstractSink.java
index 1c3e596265..26dd7e98a6 100644
--- a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/sink/AbstractSink.java
+++ b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/sink/AbstractSink.java
@@ -28,6 +28,7 @@ public abstract class AbstractSink> implements Sink {
private Thread retryThread;
private int maxRetries;
private int waitTimeMs;
+ private SinkThread sinkThread;
public AbstractSink(final PluginSetting pluginSetting, int numRetries, int waitTimeMs) {
this.pluginMetrics = PluginMetrics.fromPluginSetting(pluginSetting);
@@ -51,7 +52,8 @@ public void initialize() {
// the exceptions which are not retryable.
doInitialize();
if (!isReady() && retryThread == null) {
- retryThread = new Thread(new SinkThread(this, maxRetries, waitTimeMs));
+ sinkThread = new SinkThread(this, maxRetries, waitTimeMs);
+ retryThread = new Thread(sinkThread);
retryThread.start();
}
}
@@ -76,7 +78,7 @@ public void output(Collection records) {
@Override
public void shutdown() {
if (retryThread != null) {
- retryThread.stop();
+ sinkThread.stop();
}
}
diff --git a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/sink/SinkThread.java b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/sink/SinkThread.java
index c304de37af..451cef7dff 100644
--- a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/sink/SinkThread.java
+++ b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/sink/SinkThread.java
@@ -10,6 +10,8 @@ class SinkThread implements Runnable {
private int maxRetries;
private int waitTimeMs;
+ private volatile boolean isStopped = false;
+
public SinkThread(AbstractSink sink, int maxRetries, int waitTimeMs) {
this.sink = sink;
this.maxRetries = maxRetries;
@@ -19,11 +21,15 @@ public SinkThread(AbstractSink sink, int maxRetries, int waitTimeMs) {
@Override
public void run() {
int numRetries = 0;
- while (!sink.isReady() && numRetries++ < maxRetries) {
+ while (!sink.isReady() && numRetries++ < maxRetries && !isStopped) {
try {
Thread.sleep(waitTimeMs);
sink.doInitialize();
} catch (InterruptedException e){}
}
}
+
+ public void stop() {
+ isStopped = true;
+ }
}
diff --git a/data-prepper-api/src/test/java/org/opensearch/dataprepper/metrics/MetricsTestUtil.java b/data-prepper-api/src/test/java/org/opensearch/dataprepper/metrics/MetricsTestUtil.java
index a77d9de349..f6c0602f9e 100644
--- a/data-prepper-api/src/test/java/org/opensearch/dataprepper/metrics/MetricsTestUtil.java
+++ b/data-prepper-api/src/test/java/org/opensearch/dataprepper/metrics/MetricsTestUtil.java
@@ -6,25 +6,37 @@
package org.opensearch.dataprepper.metrics;
import io.micrometer.core.instrument.Measurement;
+import io.micrometer.core.instrument.Meter;
import io.micrometer.core.instrument.MeterRegistry;
import io.micrometer.core.instrument.Metrics;
import io.micrometer.core.instrument.Statistic;
import io.micrometer.core.instrument.simple.SimpleMeterRegistry;
+import java.util.ArrayList;
+import java.util.HashSet;
import java.util.List;
+import java.util.Set;
import java.util.stream.Collectors;
import java.util.stream.StreamSupport;
public class MetricsTestUtil {
- public static void initMetrics() {
- Metrics.globalRegistry.getRegistries().forEach(meterRegistry -> Metrics.globalRegistry.remove(meterRegistry));
- Metrics.globalRegistry.getMeters().forEach(meter -> Metrics.globalRegistry.remove(meter));
+ public static synchronized void initMetrics() {
+ final Set registries = new HashSet<>(Metrics.globalRegistry.getRegistries());
+ registries.forEach(Metrics.globalRegistry::remove);
+
+ final List meters = new ArrayList<>(Metrics.globalRegistry.getMeters());
+ meters.forEach(Metrics.globalRegistry::remove);
+
Metrics.addRegistry(new SimpleMeterRegistry());
}
- public static List getMeasurementList(final String meterName) {
- return StreamSupport.stream(getRegistry().find(meterName).meter().measure().spliterator(), false)
+ public static synchronized List getMeasurementList(final String meterName) {
+ final Meter meter = getRegistry().find(meterName).meter();
+ if(meter == null)
+ throw new RuntimeException("No metrics meter is available for " + meterName);
+
+ return StreamSupport.stream(meter.measure().spliterator(), false)
.collect(Collectors.toList());
}
diff --git a/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/event/AggregateEventHandleTests.java b/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/event/AggregateEventHandleTests.java
new file mode 100644
index 0000000000..9998d6eb6d
--- /dev/null
+++ b/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/event/AggregateEventHandleTests.java
@@ -0,0 +1,102 @@
+/*
+ * Copyright OpenSearch Contributors
+ * SPDX-License-Identifier: Apache-2.0
+ */
+
+package org.opensearch.dataprepper.model.event;
+
+import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSet;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.equalTo;
+import org.junit.jupiter.api.Test;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+import static org.mockito.Mockito.verify;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.times;
+import org.mockito.Mock;
+
+import java.lang.ref.WeakReference;
+import java.time.Instant;
+
+class AggregateEventHandleTests {
+ @Mock
+ private AcknowledgementSet acknowledgementSet1;
+ @Mock
+ private AcknowledgementSet acknowledgementSet2;
+ private int count;
+
+ @Test
+ void testBasic() {
+ Instant now = Instant.now();
+ AggregateEventHandle eventHandle = new AggregateEventHandle(now);
+ assertThat(eventHandle.getInternalOriginationTime(), equalTo(now));
+ assertThat(eventHandle.getExternalOriginationTime(), equalTo(null));
+ assertThat(eventHandle.hasAcknowledgementSet(), equalTo(false));
+ eventHandle.acquireReference();
+ eventHandle.release(true);
+ }
+
+ @Test
+ void testWithAcknowledgementSet() {
+ acknowledgementSet1 = mock(AcknowledgementSet.class);
+ acknowledgementSet2 = mock(AcknowledgementSet.class);
+ when(acknowledgementSet1.release(any(EventHandle.class), any(Boolean.class))).thenReturn(true);
+ when(acknowledgementSet2.release(any(EventHandle.class), any(Boolean.class))).thenReturn(true);
+ Instant now = Instant.now();
+ AggregateEventHandle eventHandle = new AggregateEventHandle(now);
+ assertThat(eventHandle.getInternalOriginationTime(), equalTo(now));
+ assertThat(eventHandle.getExternalOriginationTime(), equalTo(null));
+ eventHandle.addAcknowledgementSet(acknowledgementSet1);
+ // just do duplicate add
+ eventHandle.addAcknowledgementSet(acknowledgementSet1);
+ assertThat(eventHandle.hasAcknowledgementSet(), equalTo(true));
+ eventHandle.addAcknowledgementSet(acknowledgementSet2);
+ eventHandle.acquireReference();
+ verify(acknowledgementSet1).acquire(eventHandle);
+ verify(acknowledgementSet2).acquire(eventHandle);
+ eventHandle.release(true);
+ verify(acknowledgementSet1).release(eventHandle, true);
+ verify(acknowledgementSet2).release(eventHandle, true);
+ }
+
+ @Test
+ void testWithExternalOriginationTime() {
+ Instant now = Instant.now();
+ AggregateEventHandle eventHandle = new AggregateEventHandle(now);
+ assertThat(eventHandle.hasAcknowledgementSet(), equalTo(false));
+ assertThat(eventHandle.getInternalOriginationTime(), equalTo(now));
+ assertThat(eventHandle.getExternalOriginationTime(), equalTo(null));
+ eventHandle.setExternalOriginationTime(now.minusSeconds(60));
+ assertThat(eventHandle.getExternalOriginationTime(), equalTo(now.minusSeconds(60)));
+ eventHandle.release(true);
+ }
+
+ @Test
+ void testWithOnReleaseHandler() {
+ Instant now = Instant.now();
+ count = 0;
+ AggregateEventHandle eventHandle = new AggregateEventHandle(now);
+ acknowledgementSet1 = mock(AcknowledgementSet.class);
+ acknowledgementSet2 = mock(AcknowledgementSet.class);
+ eventHandle.onRelease((handle, result) -> {if (result) count++; });
+ eventHandle.addAcknowledgementSet(acknowledgementSet1);
+ assertThat(eventHandle.hasAcknowledgementSet(), equalTo(true));
+ eventHandle.addAcknowledgementSet(acknowledgementSet2);
+ // Simulate weak reference object not available for
+ // verification tests to pass 100%
+ for (WeakReference acknowledgementSetRef: eventHandle.getAcknowledgementSetRefs()) {
+ if (acknowledgementSetRef.get() == acknowledgementSet2 ) {
+ acknowledgementSetRef.clear();
+ break;
+ }
+ }
+ eventHandle.release(true);
+ assertThat(count, equalTo(1));
+ verify(acknowledgementSet1, times(1)).release(eventHandle, true);
+ verify(acknowledgementSet2, times(0)).release(eventHandle, true);
+
+ }
+
+}
+
diff --git a/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/event/DefaultEventHandleTests.java b/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/event/DefaultEventHandleTests.java
index b2a66b2d1d..f351febd11 100644
--- a/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/event/DefaultEventHandleTests.java
+++ b/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/event/DefaultEventHandleTests.java
@@ -13,6 +13,7 @@
import static org.mockito.Mockito.when;
import static org.mockito.Mockito.verify;
import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.doNothing;
import org.mockito.Mock;
import java.time.Instant;
@@ -29,6 +30,8 @@ void testBasic() {
assertThat(eventHandle.getAcknowledgementSet(), equalTo(null));
assertThat(eventHandle.getInternalOriginationTime(), equalTo(now));
assertThat(eventHandle.getExternalOriginationTime(), equalTo(null));
+ eventHandle.acquireReference();
+ assertThat(eventHandle.hasAcknowledgementSet(), equalTo(false));
eventHandle.release(true);
}
@@ -36,12 +39,16 @@ void testBasic() {
void testWithAcknowledgementSet() {
acknowledgementSet = mock(AcknowledgementSet.class);
when(acknowledgementSet.release(any(EventHandle.class), any(Boolean.class))).thenReturn(true);
+ doNothing().when(acknowledgementSet).acquire(any(EventHandle.class));
Instant now = Instant.now();
DefaultEventHandle eventHandle = new DefaultEventHandle(now);
assertThat(eventHandle.getAcknowledgementSet(), equalTo(null));
assertThat(eventHandle.getInternalOriginationTime(), equalTo(now));
assertThat(eventHandle.getExternalOriginationTime(), equalTo(null));
- eventHandle.setAcknowledgementSet(acknowledgementSet);
+ eventHandle.addAcknowledgementSet(acknowledgementSet);
+ assertThat(eventHandle.hasAcknowledgementSet(), equalTo(true));
+ eventHandle.acquireReference();
+ verify(acknowledgementSet).acquire(eventHandle);
eventHandle.release(true);
verify(acknowledgementSet).release(eventHandle, true);
}
diff --git a/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/event/EventActionTest.java b/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/event/EventActionTest.java
new file mode 100644
index 0000000000..edb63fa663
--- /dev/null
+++ b/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/event/EventActionTest.java
@@ -0,0 +1,69 @@
+/*
+ * Copyright OpenSearch Contributors
+ * SPDX-License-Identifier: Apache-2.0
+ */
+
+package org.opensearch.dataprepper.model.event;
+
+import org.junit.jupiter.api.extension.ExtensionContext;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.Arguments;
+import org.junit.jupiter.params.provider.ArgumentsProvider;
+import org.junit.jupiter.params.provider.ArgumentsSource;
+import org.junit.jupiter.params.provider.EnumSource;
+
+import java.util.stream.Stream;
+
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.hamcrest.CoreMatchers.hasItem;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.junit.jupiter.params.provider.Arguments.arguments;
+
+class EventActionTest {
+ @ParameterizedTest
+ @EnumSource(value = EventKeyFactory.EventAction.class, mode = EnumSource.Mode.EXCLUDE, names = {"GET"})
+ void isMutableAction_is_true_for_mutable_actions(final EventKeyFactory.EventAction eventAction) {
+ assertThat(eventAction.isMutableAction(), equalTo(true));
+ }
+
+ @ParameterizedTest
+ @EnumSource(value = EventKeyFactory.EventAction.class, mode = EnumSource.Mode.INCLUDE, names = {"GET"})
+ void isMutableAction_is_false_for_mutable_actions(final EventKeyFactory.EventAction eventAction) {
+ assertThat(eventAction.isMutableAction(), equalTo(false));
+ }
+
+ @ParameterizedTest
+ @EnumSource(value = EventKeyFactory.EventAction.class)
+ void getSupportedActions_includes_self(final EventKeyFactory.EventAction eventAction) {
+ assertThat(eventAction.getSupportedActions(), hasItem(eventAction));
+ }
+
+ @ParameterizedTest
+ @EnumSource(value = EventKeyFactory.EventAction.class)
+ void getSupportedActions_includes_for_all_actions_when_ALL(final EventKeyFactory.EventAction eventAction) {
+ assertThat(EventKeyFactory.EventAction.ALL.getSupportedActions(), hasItem(eventAction));
+ }
+
+ @ParameterizedTest
+ @ArgumentsSource(SupportsArgumentsProvider.class)
+ void supports_returns_expected_value(final EventKeyFactory.EventAction eventAction, final EventKeyFactory.EventAction otherAction, final boolean expectedSupports) {
+ assertThat(eventAction.getSupportedActions().contains(otherAction), equalTo(expectedSupports));
+ }
+
+ static class SupportsArgumentsProvider implements ArgumentsProvider {
+ @Override
+ public Stream extends Arguments> provideArguments(final ExtensionContext extensionContext) throws Exception {
+ return Stream.of(
+ arguments(EventKeyFactory.EventAction.GET, EventKeyFactory.EventAction.PUT, false),
+ arguments(EventKeyFactory.EventAction.GET, EventKeyFactory.EventAction.DELETE, false),
+ arguments(EventKeyFactory.EventAction.GET, EventKeyFactory.EventAction.ALL, false),
+ arguments(EventKeyFactory.EventAction.PUT, EventKeyFactory.EventAction.GET, false),
+ arguments(EventKeyFactory.EventAction.PUT, EventKeyFactory.EventAction.DELETE, false),
+ arguments(EventKeyFactory.EventAction.PUT, EventKeyFactory.EventAction.ALL, false),
+ arguments(EventKeyFactory.EventAction.DELETE, EventKeyFactory.EventAction.GET, false),
+ arguments(EventKeyFactory.EventAction.DELETE, EventKeyFactory.EventAction.PUT, false),
+ arguments(EventKeyFactory.EventAction.DELETE, EventKeyFactory.EventAction.ALL, false)
+ );
+ }
+ }
+}
\ No newline at end of file
diff --git a/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/event/EventKeyFactoryTest.java b/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/event/EventKeyFactoryTest.java
new file mode 100644
index 0000000000..c2ed2d56f3
--- /dev/null
+++ b/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/event/EventKeyFactoryTest.java
@@ -0,0 +1,47 @@
+/*
+ * Copyright OpenSearch Contributors
+ * SPDX-License-Identifier: Apache-2.0
+ */
+
+package org.opensearch.dataprepper.model.event;
+
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.mockito.Mock;
+import org.mockito.junit.jupiter.MockitoExtension;
+
+import java.util.UUID;
+
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.mockito.ArgumentMatchers.anyString;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+@ExtendWith(MockitoExtension.class)
+class EventKeyFactoryTest {
+
+ private String keyPath;
+
+ @Mock
+ private EventKey eventKey;
+
+ @BeforeEach
+ void setUp() {
+ keyPath = UUID.randomUUID().toString();
+ }
+
+ private EventKeyFactory createObjectUnderTest() {
+ return mock(EventKeyFactory.class);
+ }
+
+ @Test
+ void createEventKey_calls_with_ALL_action() {
+ final EventKeyFactory objectUnderTest = createObjectUnderTest();
+ when(objectUnderTest.createEventKey(anyString())).thenCallRealMethod();
+ when(objectUnderTest.createEventKey(keyPath, EventKeyFactory.EventAction.ALL)).thenReturn(eventKey);
+
+ assertThat(objectUnderTest.createEventKey(keyPath), equalTo(eventKey));
+ }
+}
\ No newline at end of file
diff --git a/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/event/JacksonEventKeyTest.java b/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/event/JacksonEventKeyTest.java
new file mode 100644
index 0000000000..5eb696a374
--- /dev/null
+++ b/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/event/JacksonEventKeyTest.java
@@ -0,0 +1,284 @@
+/*
+ * Copyright OpenSearch Contributors
+ * SPDX-License-Identifier: Apache-2.0
+ */
+
+package org.opensearch.dataprepper.model.event;
+
+import com.fasterxml.jackson.core.JsonPointer;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtensionContext;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.Arguments;
+import org.junit.jupiter.params.provider.ArgumentsProvider;
+import org.junit.jupiter.params.provider.ArgumentsSource;
+import org.junit.jupiter.params.provider.CsvSource;
+import org.junit.jupiter.params.provider.EnumSource;
+import org.junit.jupiter.params.provider.ValueSource;
+
+import java.util.List;
+import java.util.UUID;
+import java.util.stream.Stream;
+
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.hamcrest.CoreMatchers.not;
+import static org.hamcrest.CoreMatchers.notNullValue;
+import static org.hamcrest.CoreMatchers.sameInstance;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.params.provider.Arguments.arguments;
+
+class JacksonEventKeyTest {
+ @Test
+ void constructor_throws_with_null_key() {
+ assertThrows(NullPointerException.class, () -> new JacksonEventKey(null));
+ }
+
+ @Test
+ void getKey_with_empty_string_for_GET() {
+ final JacksonEventKey objectUnderTest = new JacksonEventKey("", EventKeyFactory.EventAction.GET);
+ assertThat(objectUnderTest.getKey(), equalTo(""));
+ assertThat(objectUnderTest.getTrimmedKey(), equalTo(""));
+ assertThat(objectUnderTest.getKeyPathList(), notNullValue());
+ assertThat(objectUnderTest.getKeyPathList(), equalTo(List.of("")));
+ assertThat(objectUnderTest.getJsonPointer(), notNullValue());
+ }
+
+ @ParameterizedTest
+ @EnumSource(value = EventKeyFactory.EventAction.class, mode = EnumSource.Mode.EXCLUDE, names = {"GET"})
+ void constructor_throws_with_empty_string_for_unsupported_actions(final EventKeyFactory.EventAction eventAction) {
+ assertThrows(IllegalArgumentException.class, () -> new JacksonEventKey("", eventAction));
+ }
+
+
+ @ParameterizedTest
+ @ValueSource(strings = {
+ "inv(alid",
+ "getMetadata(\"test_key\")"
+ })
+ void constructor_throws_with_invalid_key(final String key) {
+ assertThrows(IllegalArgumentException.class, () -> new JacksonEventKey(key));
+ }
+
+ @ParameterizedTest
+ @ValueSource(strings = {
+ "test_key",
+ "/test_key",
+ "key.with.dot",
+ "key-with-hyphen",
+ "key_with_underscore",
+ "key@with@at",
+ "key[with]brackets"
+ })
+ void getKey_returns_expected_result(final String key) {
+ assertThat(new JacksonEventKey(key).getKey(), equalTo(key));
+ }
+
+ @ParameterizedTest
+ @CsvSource(value = {
+ "test_key, test_key",
+ "/test_key, /test_key",
+ "/test_key/, /test_key",
+ "key.with.dot, key.with.dot",
+ "key-with-hyphen, key-with-hyphen",
+ "key_with_underscore, key_with_underscore",
+ "key@with@at, key@with@at",
+ "key[with]brackets, key[with]brackets"
+ })
+ void getTrimmedKey_returns_expected_result(final String key, final String expectedTrimmedKey) {
+ assertThat(new JacksonEventKey(key).getTrimmedKey(), equalTo(expectedTrimmedKey));
+ }
+
+ @ParameterizedTest
+ @ArgumentsSource(KeyPathListArgumentsProvider.class)
+ void getKeyPathList_returns_expected_value(final String key, final List expectedKeyPathList) {
+ assertThat(new JacksonEventKey(key).getKeyPathList(), equalTo(expectedKeyPathList));
+ }
+
+ @Test
+ void getJsonPointer_returns_a_valid_JsonPointer() {
+ final String testKey = UUID.randomUUID().toString();
+ final JacksonEventKey objectUnderTest = new JacksonEventKey(testKey);
+
+ final JsonPointer jsonPointer = objectUnderTest.getJsonPointer();
+ assertThat(jsonPointer, notNullValue());
+ assertThat(jsonPointer.toString(), equalTo("/" + testKey));
+ }
+
+ @Test
+ void getJsonPointer_returns_the_same_instance_for_multiple_calls() {
+ final String testKey = UUID.randomUUID().toString();
+ final JacksonEventKey objectUnderTest = new JacksonEventKey(testKey);
+
+ final JsonPointer jsonPointer = objectUnderTest.getJsonPointer();
+ assertThat(objectUnderTest.getJsonPointer(), sameInstance(jsonPointer));
+ assertThat(objectUnderTest.getJsonPointer(), sameInstance(jsonPointer));
+ }
+
+ @ParameterizedTest
+ @EnumSource(value = EventKeyFactory.EventAction.class)
+ void getJsonPointer_returns_valid_JsonPointer_when_constructed_with_fromJacksonEvent(final EventKeyFactory.EventAction eventAction) {
+ final String testKey = UUID.randomUUID().toString();
+ final JacksonEventKey objectUnderTest = new JacksonEventKey(testKey, true, eventAction);
+
+ final JsonPointer jsonPointer = objectUnderTest.getJsonPointer();
+ assertThat(jsonPointer, notNullValue());
+ assertThat(jsonPointer.toString(), equalTo("/" + testKey));
+ }
+
+ @ParameterizedTest
+ @ArgumentsSource(KeyPathListArgumentsProvider.class)
+ void getKeyPathList_returns_expected_value_when_constructed_with_fromJacksonEvent(final String key, final List expectedKeyPathList) {
+ assertThat(new JacksonEventKey(key, true).getKeyPathList(), equalTo(expectedKeyPathList));
+ }
+
+ @ParameterizedTest
+ @ArgumentsSource(SupportsArgumentsProvider.class)
+ void supports_returns_true_if_any_supports(final List eventActionsList, final EventKeyFactory.EventAction otherAction, final boolean expectedSupports) {
+ final String testKey = UUID.randomUUID().toString();
+ final EventKeyFactory.EventAction[] eventActions = new EventKeyFactory.EventAction[eventActionsList.size()];
+ eventActionsList.toArray(eventActions);
+ assertThat(new JacksonEventKey(testKey, eventActions).supports(otherAction), equalTo(expectedSupports));
+ }
+
+ @ParameterizedTest
+ @CsvSource(value = {
+ "test_key, true",
+ "/test_key, true",
+ "inv(alid, false",
+ "getMetadata(\"test_key\"), false",
+ "key.with.dot, true",
+ "key-with-hyphen, true",
+ "key_with_underscore, true",
+ "key@with@at, true",
+ "key[with]brackets, true"
+ })
+ void isValidEventKey_returns_expected_result(final String key, final boolean isValid) {
+ assertThat(JacksonEventKey.isValidEventKey(key), equalTo(isValid));
+ }
+
+
+ static class KeyPathListArgumentsProvider implements ArgumentsProvider {
+ @Override
+ public Stream extends Arguments> provideArguments(final ExtensionContext extensionContext) {
+ return Stream.of(
+ arguments("test_key", List.of("test_key")),
+ arguments("a/b", List.of("a", "b")),
+ arguments("a/b/", List.of("a", "b")),
+ arguments("a/b/c", List.of("a", "b", "c")),
+ arguments("a/b/c/", List.of("a", "b", "c"))
+ );
+ }
+ }
+
+ static class SupportsArgumentsProvider implements ArgumentsProvider {
+ @Override
+ public Stream extends Arguments> provideArguments(final ExtensionContext extensionContext) throws Exception {
+ return Stream.of(
+ arguments(List.of(), EventKeyFactory.EventAction.GET, true),
+ arguments(List.of(), EventKeyFactory.EventAction.PUT, true),
+ arguments(List.of(), EventKeyFactory.EventAction.DELETE, true),
+ arguments(List.of(), EventKeyFactory.EventAction.ALL, true),
+ arguments(List.of(EventKeyFactory.EventAction.GET), EventKeyFactory.EventAction.GET, true),
+ arguments(List.of(EventKeyFactory.EventAction.PUT), EventKeyFactory.EventAction.PUT, true),
+ arguments(List.of(EventKeyFactory.EventAction.DELETE), EventKeyFactory.EventAction.DELETE, true),
+ arguments(List.of(EventKeyFactory.EventAction.GET), EventKeyFactory.EventAction.PUT, false),
+ arguments(List.of(EventKeyFactory.EventAction.GET, EventKeyFactory.EventAction.PUT), EventKeyFactory.EventAction.PUT, true),
+ arguments(List.of(EventKeyFactory.EventAction.PUT, EventKeyFactory.EventAction.GET), EventKeyFactory.EventAction.PUT, true),
+ arguments(List.of(EventKeyFactory.EventAction.DELETE), EventKeyFactory.EventAction.PUT, false),
+ arguments(List.of(EventKeyFactory.EventAction.DELETE, EventKeyFactory.EventAction.GET), EventKeyFactory.EventAction.PUT, false),
+ arguments(List.of(EventKeyFactory.EventAction.DELETE, EventKeyFactory.EventAction.GET, EventKeyFactory.EventAction.PUT), EventKeyFactory.EventAction.PUT, true),
+ arguments(List.of(EventKeyFactory.EventAction.ALL), EventKeyFactory.EventAction.GET, true),
+ arguments(List.of(EventKeyFactory.EventAction.ALL), EventKeyFactory.EventAction.PUT, true),
+ arguments(List.of(EventKeyFactory.EventAction.ALL), EventKeyFactory.EventAction.DELETE, true)
+ );
+ }
+ }
+
+ @ParameterizedTest
+ @EnumSource(EventKeyFactory.EventAction.class)
+ void equals_returns_true_for_same_key_and_actions(final EventKeyFactory.EventAction eventAction) {
+ final String testKey = UUID.randomUUID().toString();
+ final JacksonEventKey objectUnderTest = new JacksonEventKey(testKey, eventAction);
+ final JacksonEventKey other = new JacksonEventKey(testKey, eventAction);
+
+ assertThat(objectUnderTest.equals(other), equalTo(true));
+ }
+
+ @Test
+ void equals_returns_true_for_same_instance() {
+ final JacksonEventKey objectUnderTest = new JacksonEventKey(UUID.randomUUID().toString(),
+ EventKeyFactory.EventAction.PUT);
+
+ assertThat(objectUnderTest.equals(objectUnderTest), equalTo(true));
+ }
+
+ @Test
+ void equals_returns_false_for_null() {
+ final JacksonEventKey objectUnderTest = new JacksonEventKey(UUID.randomUUID().toString(),
+ EventKeyFactory.EventAction.PUT);
+
+ assertThat(objectUnderTest.equals(null), equalTo(false));
+ }
+
+ @Test
+ void equals_returns_false_for_non_EventKey() {
+ final String testKey = UUID.randomUUID().toString();
+ final JacksonEventKey objectUnderTest = new JacksonEventKey(testKey,
+ EventKeyFactory.EventAction.PUT);
+
+ assertThat(objectUnderTest.equals(testKey), equalTo(false));
+ }
+
+ @Test
+ void equals_returns_false_for_same_key_but_different_actions() {
+ final String testKey = UUID.randomUUID().toString();
+ final JacksonEventKey objectUnderTest = new JacksonEventKey(testKey, EventKeyFactory.EventAction.PUT);
+ final JacksonEventKey other = new JacksonEventKey(testKey, EventKeyFactory.EventAction.GET);
+
+ assertThat(objectUnderTest.equals(other), equalTo(false));
+ }
+
+ @ParameterizedTest
+ @EnumSource(EventKeyFactory.EventAction.class)
+ void equals_returns_false_for_different_key_but_same_actions(final EventKeyFactory.EventAction eventAction) {
+ final JacksonEventKey objectUnderTest = new JacksonEventKey(UUID.randomUUID().toString(), eventAction);
+ final JacksonEventKey other = new JacksonEventKey(UUID.randomUUID().toString(), eventAction);
+
+ assertThat(objectUnderTest.equals(other), equalTo(false));
+ }
+
+ @ParameterizedTest
+ @EnumSource(EventKeyFactory.EventAction.class)
+ void hashCode_is_the_same_for_same_key_and_actions(final EventKeyFactory.EventAction eventAction) {
+ final String testKey = UUID.randomUUID().toString();
+ final JacksonEventKey objectUnderTest = new JacksonEventKey(testKey, eventAction);
+ final JacksonEventKey other = new JacksonEventKey(testKey, eventAction);
+
+ assertThat(objectUnderTest.hashCode(), equalTo(other.hashCode()));
+ }
+
+ @ParameterizedTest
+ @CsvSource({
+ "test, PUT, test2, PUT",
+ "test, PUT, test2, PUT",
+ "test, PUT, test, GET"
+ })
+ void hashCode_is_the_different_for_same_key_and_actions(
+ final String testKey, final EventKeyFactory.EventAction eventAction,
+ final String testKeyOther, final EventKeyFactory.EventAction eventActionOther) {
+ final JacksonEventKey objectUnderTest = new JacksonEventKey(testKey, eventAction);
+ final JacksonEventKey other = new JacksonEventKey(testKeyOther, eventActionOther);
+
+ assertThat(objectUnderTest.hashCode(), not(equalTo(other.hashCode())));
+ }
+
+ @ParameterizedTest
+ @EnumSource(EventKeyFactory.EventAction.class)
+ void toString_returns_the_key(final EventKeyFactory.EventAction eventAction) {
+ final String testKey = UUID.randomUUID().toString();
+ final JacksonEventKey objectUnderTest = new JacksonEventKey(testKey, eventAction);
+
+ assertThat(objectUnderTest.toString(), equalTo(testKey));
+ }
+}
\ No newline at end of file
diff --git a/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/event/JacksonEventTest.java b/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/event/JacksonEventTest.java
index 1a7efb7467..90645d2961 100644
--- a/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/event/JacksonEventTest.java
+++ b/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/event/JacksonEventTest.java
@@ -74,6 +74,53 @@ public void testPutAndGet_withRandomString() {
assertThat(result, is(equalTo(value)));
}
+ @Test
+ public void testPutAndGet_withRandomString_eventKey() {
+ final EventKey key = new JacksonEventKey("aRandomKey" + UUID.randomUUID());
+ final UUID value = UUID.randomUUID();
+
+ event.put(key, value);
+ final UUID result = event.get(key, UUID.class);
+
+ assertThat(result, is(notNullValue()));
+ assertThat(result, is(equalTo(value)));
+ }
+
+ @Test
+ public void testPutAndGet_withRandomString_eventKey_multiple_events() {
+ final EventKey key = new JacksonEventKey("aRandomKey" + UUID.randomUUID());
+ final UUID value = UUID.randomUUID();
+
+ for(int i = 0; i < 10; i++) {
+ event = JacksonEvent.builder()
+ .withEventType(eventType)
+ .build();
+
+ event.put(key, value);
+ final UUID result = event.get(key, UUID.class);
+
+ assertThat(result, is(notNullValue()));
+ assertThat(result, is(equalTo(value)));
+ }
+ }
+
+ @Test
+ public void testPutAndGet_eventKey_with_non_JacksonEventKey_throws() {
+ final EventKey key = mock(EventKey.class);
+ final UUID value = UUID.randomUUID();
+
+ assertThrows(IllegalArgumentException.class, () -> event.put(key, value));
+ assertThrows(IllegalArgumentException.class, () -> event.get(key, UUID.class));
+ }
+
+ @Test
+ public void testPut_eventKey_with_immutable_action() {
+ final EventKey key = new JacksonEventKey("aRandomKey" + UUID.randomUUID(), EventKeyFactory.EventAction.GET);
+ final UUID value = UUID.randomUUID();
+
+ assertThrows(IllegalArgumentException.class, () -> event.put(key, value));
+ }
+
@ParameterizedTest
@ValueSource(strings = {"/", "foo", "foo-bar", "foo_bar", "foo.bar", "/foo", "/foo/", "a1K.k3-01_02", "keyWithBrackets[]"})
void testPutAndGet_withStrings(final String key) {
@@ -86,6 +133,19 @@ void testPutAndGet_withStrings(final String key) {
assertThat(result, is(equalTo(value)));
}
+ @ParameterizedTest
+ @ValueSource(strings = {"/", "foo", "foo-bar", "foo_bar", "foo.bar", "/foo", "/foo/", "a1K.k3-01_02", "keyWithBrackets[]"})
+ void testPutAndGet_withStrings_eventKey(final String key) {
+ final UUID value = UUID.randomUUID();
+
+ final EventKey eventKey = new JacksonEventKey(key);
+ event.put(eventKey, value);
+ final UUID result = event.get(eventKey, UUID.class);
+
+ assertThat(result, is(notNullValue()));
+ assertThat(result, is(equalTo(value)));
+ }
+
@Test
public void testPutKeyCannotBeEmptyString() {
Throwable exception = assertThrows(IllegalArgumentException.class, () -> event.put("", "value"));
@@ -93,7 +153,7 @@ public void testPutKeyCannotBeEmptyString() {
}
@Test
- public void testPutAndGet_withMultLevelKey() {
+ public void testPutAndGet_withMultiLevelKey() {
final String key = "foo/bar";
final UUID value = UUID.randomUUID();
@@ -104,6 +164,18 @@ public void testPutAndGet_withMultLevelKey() {
assertThat(result, is(equalTo(value)));
}
+ @Test
+ public void testPutAndGet_withMultiLevelKey_eventKey() {
+ final EventKey key = new JacksonEventKey("foo/bar");
+ final UUID value = UUID.randomUUID();
+
+ event.put(key, value);
+ final UUID result = event.get(key, UUID.class);
+
+ assertThat(result, is(notNullValue()));
+ assertThat(result, is(equalTo(value)));
+ }
+
@Test
public void testPutAndGet_withMultiLevelKeyTwice() {
final String key = "foo/bar";
@@ -125,6 +197,27 @@ public void testPutAndGet_withMultiLevelKeyTwice() {
assertThat(result2, is(equalTo(value2)));
}
+ @Test
+ public void testPutAndGet_withMultiLevelKeyTwice_eventKey() {
+ final EventKey key = new JacksonEventKey("foo/bar");
+ final UUID value = UUID.randomUUID();
+
+ event.put(key, value);
+ final UUID result = event.get(key, UUID.class);
+
+ assertThat(result, is(notNullValue()));
+ assertThat(result, is(equalTo(value)));
+
+ final EventKey key2 = new JacksonEventKey("foo/fizz");
+ final UUID value2 = UUID.randomUUID();
+
+ event.put(key2, value2);
+ final UUID result2 = event.get(key2, UUID.class);
+
+ assertThat(result2, is(notNullValue()));
+ assertThat(result2, is(equalTo(value2)));
+ }
+
@Test
public void testPutAndGet_withMultiLevelKeyWithADash() {
final String key = "foo/bar-bar";
@@ -137,6 +230,18 @@ public void testPutAndGet_withMultiLevelKeyWithADash() {
assertThat(result, is(equalTo(value)));
}
+ @Test
+ public void testPutAndGet_withMultiLevelKeyWithADash_eventKey() {
+ final EventKey key = new JacksonEventKey("foo/bar-bar");
+ final UUID value = UUID.randomUUID();
+
+ event.put(key, value);
+ final UUID result = event.get(key, UUID.class);
+
+ assertThat(result, is(notNullValue()));
+ assertThat(result, is(equalTo(value)));
+ }
+
@ParameterizedTest
@ValueSource(strings = {"foo", "/foo", "/foo/", "foo/"})
void testGetAtRootLevel(final String key) {
@@ -148,6 +253,17 @@ void testGetAtRootLevel(final String key) {
assertThat(result, is(Map.of("foo", value)));
}
+ @ParameterizedTest
+ @ValueSource(strings = {"foo", "/foo", "/foo/", "foo/"})
+ void testGetAtRootLevel_eventKey(final String key) {
+ final String value = UUID.randomUUID().toString();
+
+ event.put(new JacksonEventKey(key), value);
+ final Map result = event.get(new JacksonEventKey("", EventKeyFactory.EventAction.GET), Map.class);
+
+ assertThat(result, is(Map.of("foo", value)));
+ }
+
@ParameterizedTest
@ValueSource(strings = {"/foo/bar", "foo/bar", "foo/bar/"})
void testGetAtRootLevelWithMultiLevelKey(final String key) {
@@ -159,6 +275,17 @@ void testGetAtRootLevelWithMultiLevelKey(final String key) {
assertThat(result, is(Map.of("foo", Map.of("bar", value))));
}
+ @ParameterizedTest
+ @ValueSource(strings = {"/foo/bar", "foo/bar", "foo/bar/"})
+ void testGetAtRootLevelWithMultiLevelKey_eventKey(final String key) {
+ final String value = UUID.randomUUID().toString();
+
+ event.put(new JacksonEventKey(key), value);
+ final Map result = event.get( new JacksonEventKey("", EventKeyFactory.EventAction.GET), Map.class);
+
+ assertThat(result, is(Map.of("foo", Map.of("bar", value))));
+ }
+
@Test
public void testPutUpdateAndGet_withPojo() {
final String key = "foo/bar";
@@ -293,6 +420,14 @@ public void testDeleteKey(final String key) {
assertThat(result, is(nullValue()));
}
+ @Test
+ public void testDelete_eventKey_with_immutable_action() {
+ final EventKey key = new JacksonEventKey("aRandomKey" + UUID.randomUUID(), EventKeyFactory.EventAction.GET);
+ final UUID value = UUID.randomUUID();
+
+ assertThrows(IllegalArgumentException.class, () -> event.delete(key));
+ }
+
@Test
public void testClear() {
event.put("key1", UUID.randomUUID());
diff --git a/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/event/JacksonEvent_JavaSerializationTest.java b/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/event/JacksonEvent_JavaSerializationTest.java
index b3ee46b55c..160f08d673 100644
--- a/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/event/JacksonEvent_JavaSerializationTest.java
+++ b/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/event/JacksonEvent_JavaSerializationTest.java
@@ -8,6 +8,7 @@
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSet;
+import static org.junit.jupiter.api.Assertions.assertFalse;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
@@ -20,7 +21,6 @@
import static org.hamcrest.MatcherAssert.assertThat;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.instanceOf;
-import static org.hamcrest.Matchers.nullValue;
import static org.mockito.Mockito.mock;
class JacksonEvent_JavaSerializationTest {
@@ -54,7 +54,7 @@ void serialize_without_acknowledgementSet_includes_data() throws IOException, Cl
assertThat(deserializedEvent.getMetadata(), equalTo(objectUnderTest.getMetadata()));
assertThat(deserializedEvent.getEventHandle(), instanceOf(InternalEventHandle.class));
- assertThat(((InternalEventHandle) deserializedEvent.getEventHandle()).getAcknowledgementSet(), nullValue());
+ assertFalse(((InternalEventHandle) deserializedEvent.getEventHandle()).hasAcknowledgementSet());
assertThat(deserializedEvent.getEventHandle().getInternalOriginationTime(), equalTo(objectUnderTest.getMetadata().getTimeReceived()));
}
@@ -63,7 +63,7 @@ void serialize_without_acknowledgementSet_includes_data() throws IOException, Cl
void serialize_with_acknowledgementSet_does_not_include_old_acknowledgement_set() throws IOException, ClassNotFoundException {
final JacksonEvent objectUnderTest = createObjectUnderTest();
final InternalEventHandle internalEventHandle = (InternalEventHandle) objectUnderTest.getEventHandle();
- internalEventHandle.setAcknowledgementSet(mock(AcknowledgementSet.class));
+ internalEventHandle.addAcknowledgementSet(mock(AcknowledgementSet.class));
final Object deserializedObject = serializeAndDeserialize(objectUnderTest);
@@ -74,7 +74,7 @@ void serialize_with_acknowledgementSet_does_not_include_old_acknowledgement_set(
assertThat(deserializedEvent.getMetadata(), equalTo(objectUnderTest.getMetadata()));
assertThat(deserializedEvent.getEventHandle(), instanceOf(InternalEventHandle.class));
- assertThat(((InternalEventHandle) deserializedEvent.getEventHandle()).getAcknowledgementSet(), nullValue());
+ assertFalse(((InternalEventHandle) deserializedEvent.getEventHandle()).hasAcknowledgementSet());
assertThat(deserializedEvent.getEventHandle().getInternalOriginationTime(), equalTo(objectUnderTest.getMetadata().getTimeReceived()));
}
@@ -84,4 +84,4 @@ private Object serializeAndDeserialize(final JacksonEvent objectUnderTest) throw
return objectInputStream.readObject();
}
-}
\ No newline at end of file
+}
diff --git a/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/sink/AbstractSinkTest.java b/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/sink/AbstractSinkTest.java
index 3b9fe7c007..8d1af7ea44 100644
--- a/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/sink/AbstractSinkTest.java
+++ b/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/sink/AbstractSinkTest.java
@@ -11,15 +11,10 @@
import org.opensearch.dataprepper.metrics.MetricNames;
import org.opensearch.dataprepper.metrics.MetricsTestUtil;
import org.opensearch.dataprepper.model.configuration.PluginSetting;
-import org.opensearch.dataprepper.model.record.Record;
import org.opensearch.dataprepper.model.event.Event;
-import org.opensearch.dataprepper.model.event.JacksonEvent;
import org.opensearch.dataprepper.model.event.EventHandle;
-
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertTrue;
-import static org.mockito.Mockito.when;
-import static org.mockito.Mockito.mock;
+import org.opensearch.dataprepper.model.event.JacksonEvent;
+import org.opensearch.dataprepper.model.record.Record;
import java.time.Duration;
import java.util.Arrays;
@@ -30,6 +25,12 @@
import java.util.UUID;
import static org.awaitility.Awaitility.await;
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
class AbstractSinkTest {
private int count;
@@ -71,13 +72,13 @@ void testMetrics() {
}
@Test
- void testSinkNotReady() {
+ void testSinkNotReady() throws InterruptedException {
final String sinkName = "testSink";
final String pipelineName = "pipelineName";
MetricsTestUtil.initMetrics();
PluginSetting pluginSetting = new PluginSetting(sinkName, Collections.emptyMap());
pluginSetting.setPipelineName(pipelineName);
- AbstractSink> abstractSink = new AbstractSinkNotReadyImpl(pluginSetting);
+ AbstractSinkNotReadyImpl abstractSink = new AbstractSinkNotReadyImpl(pluginSetting);
abstractSink.initialize();
assertEquals(abstractSink.isReady(), false);
assertEquals(abstractSink.getRetryThreadState(), Thread.State.RUNNABLE);
@@ -87,7 +88,10 @@ void testSinkNotReady() {
await().atMost(Duration.ofSeconds(5))
.until(abstractSink::isReady);
assertEquals(abstractSink.getRetryThreadState(), Thread.State.TERMINATED);
+ int initCountBeforeShutdown = abstractSink.initCount;
abstractSink.shutdown();
+ Thread.sleep(200);
+ assertThat(abstractSink.initCount, equalTo(initCountBeforeShutdown));
}
@Test
diff --git a/data-prepper-core/build.gradle b/data-prepper-core/build.gradle
index 429e07069c..c939129a1c 100644
--- a/data-prepper-core/build.gradle
+++ b/data-prepper-core/build.gradle
@@ -48,7 +48,6 @@ dependencies {
exclude group: 'commons-logging', module: 'commons-logging'
}
implementation 'software.amazon.cloudwatchlogs:aws-embedded-metrics:2.0.0-beta-1'
- testImplementation 'org.apache.logging.log4j:log4j-jpl:2.23.0'
testImplementation testLibs.spring.test
implementation libs.armeria.core
implementation libs.armeria.grpc
@@ -60,7 +59,6 @@ dependencies {
implementation 'software.amazon.awssdk:servicediscovery'
implementation 'com.fasterxml.jackson.datatype:jackson-datatype-jsr310'
testImplementation testLibs.junit.vintage
- testImplementation testLibs.mockito.inline
testImplementation libs.commons.lang3
testImplementation project(':data-prepper-test-event')
testImplementation project(':data-prepper-test-common')
@@ -90,8 +88,6 @@ task integrationTest(type: Test) {
classpath = sourceSets.integrationTest.runtimeClasspath
- systemProperty 'log4j.configurationFile', 'src/test/resources/log4j2.properties'
-
filter {
includeTestsMatching '*IT'
}
diff --git a/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/integration/ProcessorPipelineIT.java b/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/integration/ProcessorPipelineIT.java
new file mode 100644
index 0000000000..8673fd9f21
--- /dev/null
+++ b/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/integration/ProcessorPipelineIT.java
@@ -0,0 +1,121 @@
+/*
+ * Copyright OpenSearch Contributors
+ * SPDX-License-Identifier: Apache-2.0
+ */
+
+package org.opensearch.dataprepper.integration;
+
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+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.InMemorySinkAccessor;
+import org.opensearch.dataprepper.plugins.InMemorySourceAccessor;
+import org.opensearch.dataprepper.test.framework.DataPrepperTestRunner;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+import static org.awaitility.Awaitility.await;
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.hamcrest.CoreMatchers.not;
+import static org.hamcrest.CoreMatchers.notNullValue;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.empty;
+
+class ProcessorPipelineIT {
+ private static final Logger LOG = LoggerFactory.getLogger(ProcessorPipelineIT.class);
+ private static final String IN_MEMORY_IDENTIFIER = "ProcessorPipelineIT";
+ private static final String PIPELINE_CONFIGURATION_UNDER_TEST = "processor-pipeline.yaml";
+ private DataPrepperTestRunner dataPrepperTestRunner;
+ private InMemorySourceAccessor inMemorySourceAccessor;
+ private InMemorySinkAccessor inMemorySinkAccessor;
+
+ @BeforeEach
+ void setUp() {
+ dataPrepperTestRunner = DataPrepperTestRunner.builder()
+ .withPipelinesDirectoryOrFile(PIPELINE_CONFIGURATION_UNDER_TEST)
+ .build();
+
+ inMemorySourceAccessor = dataPrepperTestRunner.getInMemorySourceAccessor();
+ inMemorySinkAccessor = dataPrepperTestRunner.getInMemorySinkAccessor();
+ dataPrepperTestRunner.start();
+ LOG.info("Started test runner.");
+ }
+
+ @AfterEach
+ void tearDown() {
+ LOG.info("Test tear down. Stop the test runner.");
+ dataPrepperTestRunner.stop();
+ }
+
+ @Test
+ void run_with_single_record() {
+ final String messageValue = UUID.randomUUID().toString();
+ final Event event = JacksonEvent.fromMessage(messageValue);
+ final Record eventRecord = new Record<>(event);
+
+ LOG.info("Submitting a single record.");
+ inMemorySourceAccessor.submit(IN_MEMORY_IDENTIFIER, Collections.singletonList(eventRecord));
+
+ await().atMost(400, TimeUnit.MILLISECONDS)
+ .untilAsserted(() -> {
+ assertThat(inMemorySinkAccessor.get(IN_MEMORY_IDENTIFIER), not(empty()));
+ });
+
+ final List> records = inMemorySinkAccessor.get(IN_MEMORY_IDENTIFIER);
+
+ assertThat(records.size(), equalTo(1));
+
+ assertThat(records.get(0), notNullValue());
+ assertThat(records.get(0).getData(), notNullValue());
+ assertThat(records.get(0).getData().get("message", String.class), equalTo(messageValue));
+ assertThat(records.get(0).getData().get("test1", String.class), equalTo("knownPrefix10"));
+ assertThat(records.get(0).getData().get("test1_copy", String.class), equalTo("knownPrefix10"));
+ }
+
+ @Test
+ void pipeline_with_single_batch_of_records() {
+ final int recordsToCreate = 200;
+ final List> inputRecords = IntStream.range(0, recordsToCreate)
+ .mapToObj(i -> UUID.randomUUID().toString())
+ .map(JacksonEvent::fromMessage)
+ .map(Record::new)
+ .collect(Collectors.toList());
+
+ LOG.info("Submitting a batch of record.");
+ inMemorySourceAccessor.submit(IN_MEMORY_IDENTIFIER, inputRecords);
+
+ await().atMost(400, TimeUnit.MILLISECONDS)
+ .untilAsserted(() -> {
+ assertThat(inMemorySinkAccessor.get(IN_MEMORY_IDENTIFIER), not(empty()));
+ });
+
+ assertThat(inMemorySinkAccessor.get(IN_MEMORY_IDENTIFIER).size(), equalTo(recordsToCreate));
+
+ final List> sinkRecords = inMemorySinkAccessor.get(IN_MEMORY_IDENTIFIER);
+
+ for (int i = 0; i < sinkRecords.size(); i++) {
+ final Record inputRecord = inputRecords.get(i);
+ final Record sinkRecord = sinkRecords.get(i);
+ assertThat(sinkRecord, notNullValue());
+ final Event recordData = sinkRecord.getData();
+ assertThat(recordData, notNullValue());
+ assertThat(
+ recordData.get("message", String.class),
+ equalTo(inputRecord.getData().get("message", String.class)));
+ assertThat(recordData.get("test1", String.class),
+ equalTo("knownPrefix1" + i));
+ assertThat(recordData.get("test1_copy", String.class),
+ equalTo("knownPrefix1" + i));
+ }
+ }
+}
diff --git a/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/integration/Router_ThreeRoutesDefaultIT.java b/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/integration/Router_ThreeRoutesDefaultIT.java
new file mode 100644
index 0000000000..fbc61053a5
--- /dev/null
+++ b/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/integration/Router_ThreeRoutesDefaultIT.java
@@ -0,0 +1,130 @@
+/*
+ * Copyright OpenSearch Contributors
+ * SPDX-License-Identifier: Apache-2.0
+ */
+
+package org.opensearch.dataprepper.integration;
+
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+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.InMemorySinkAccessor;
+import org.opensearch.dataprepper.plugins.InMemorySourceAccessor;
+import org.opensearch.dataprepper.test.framework.DataPrepperTestRunner;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+import static org.awaitility.Awaitility.await;
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.hamcrest.CoreMatchers.not;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.empty;
+
+class Router_ThreeRoutesDefaultIT {
+ private static final String TESTING_KEY = "ConditionalRoutingIT";
+ private static final String ALL_SOURCE_KEY = TESTING_KEY + "_all";
+ private static final String ALPHA_SOURCE_KEY = TESTING_KEY + "_alpha";
+ private static final String BETA_SOURCE_KEY = TESTING_KEY + "_beta";
+ private static final String ALPHA_DEFAULT_SOURCE_KEY = TESTING_KEY + "_alpha_default";
+ private static final String ALPHA_BETA_GAMMA_SOURCE_KEY = TESTING_KEY + "_alpha_beta_gamma";
+ private static final String DEFAULT_SOURCE_KEY = TESTING_KEY + "_default";
+ private static final String KNOWN_CONDITIONAL_KEY = "value";
+ private static final String ALPHA_VALUE = "a";
+ private static final String BETA_VALUE = "b";
+ private static final String GAMMA_VALUE = "g";
+ private static final String DEFAULT_VALUE = "z";
+ private DataPrepperTestRunner dataPrepperTestRunner;
+ private InMemorySourceAccessor inMemorySourceAccessor;
+ private InMemorySinkAccessor inMemorySinkAccessor;
+
+ @BeforeEach
+ void setUp() {
+ dataPrepperTestRunner = DataPrepperTestRunner.builder()
+ .withPipelinesDirectoryOrFile("route/three-route-with-default-route.yaml")
+ .build();
+
+ dataPrepperTestRunner.start();
+ inMemorySourceAccessor = dataPrepperTestRunner.getInMemorySourceAccessor();
+ inMemorySinkAccessor = dataPrepperTestRunner.getInMemorySinkAccessor();
+ }
+
+ @AfterEach
+ void tearDown() {
+ dataPrepperTestRunner.stop();
+ }
+
+ @Test
+ void test_default_route() {
+ final List> alphaEvents = createEvents(ALPHA_VALUE, 10);
+ final List> betaEvents = createEvents(BETA_VALUE, 20);
+ final List> gammaEvents = createEvents(GAMMA_VALUE, 20);
+ final List> defaultEvents = createEvents(DEFAULT_VALUE, 20);
+
+ final List> allEvents = new ArrayList<>(alphaEvents);
+ allEvents.addAll(betaEvents);
+ allEvents.addAll(gammaEvents);
+ allEvents.addAll(defaultEvents);
+ Collections.shuffle(allEvents);
+
+ inMemorySourceAccessor.submit(TESTING_KEY, allEvents);
+
+ await().atMost(2, TimeUnit.SECONDS)
+ .untilAsserted(() -> {
+ assertThat(inMemorySinkAccessor.get(ALPHA_SOURCE_KEY), not(empty()));
+ assertThat(inMemorySinkAccessor.get(BETA_SOURCE_KEY), not(empty()));
+ assertThat(inMemorySinkAccessor.get(ALL_SOURCE_KEY), not(empty()));
+ assertThat(inMemorySinkAccessor.get(ALPHA_DEFAULT_SOURCE_KEY), not(empty()));
+ assertThat(inMemorySinkAccessor.get(ALPHA_BETA_GAMMA_SOURCE_KEY), not(empty()));
+ assertThat(inMemorySinkAccessor.get(DEFAULT_SOURCE_KEY), not(empty()));
+ });
+
+ final List> actualAlphaRecords = inMemorySinkAccessor.get(ALPHA_SOURCE_KEY);
+
+ assertThat(actualAlphaRecords.size(), equalTo(alphaEvents.size()));
+
+ assertThat(actualAlphaRecords, containsInAnyOrder(allEvents.stream()
+ .filter(alphaEvents::contains).toArray()));
+
+ final List> actualBetaRecords = inMemorySinkAccessor.get(BETA_SOURCE_KEY);
+
+ assertThat(actualBetaRecords.size(), equalTo(betaEvents.size()));
+
+ assertThat(actualBetaRecords, containsInAnyOrder(allEvents.stream()
+ .filter(betaEvents::contains).toArray()));
+
+ final List> actualDefaultRecords = inMemorySinkAccessor.get(DEFAULT_SOURCE_KEY);
+
+ assertThat(actualDefaultRecords.size(), equalTo(defaultEvents.size()));
+ assertThat(actualDefaultRecords, containsInAnyOrder(allEvents.stream()
+ .filter(defaultEvents::contains).toArray()));
+
+ final List> actualAlphaDefaultRecords = new ArrayList<>();
+ actualAlphaDefaultRecords.addAll(actualAlphaRecords);
+ actualAlphaDefaultRecords.addAll(actualDefaultRecords);
+ assertThat(actualAlphaDefaultRecords.size(), equalTo(defaultEvents.size()+alphaEvents.size()));
+ assertThat(actualAlphaDefaultRecords, containsInAnyOrder(allEvents.stream()
+ .filter(event -> defaultEvents.contains(event) || alphaEvents.contains(event)).toArray()));
+
+ }
+
+ private List> createEvents(final String value, final int numberToCreate) {
+ return IntStream.range(0, numberToCreate)
+ .mapToObj(i -> Map.of(KNOWN_CONDITIONAL_KEY, value, "arbitrary_field", UUID.randomUUID().toString()))
+ .map(map -> JacksonEvent.builder().withData(map).withEventType("TEST").build())
+ .map(jacksonEvent -> (Event) jacksonEvent)
+ .map(Record::new)
+ .collect(Collectors.toList());
+ }
+}
+
diff --git a/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/plugins/InMemorySink.java b/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/plugins/InMemorySink.java
index dec7aa5c1f..360367a1e4 100644
--- a/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/plugins/InMemorySink.java
+++ b/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/plugins/InMemorySink.java
@@ -40,7 +40,7 @@ public void output(final Collection> records) {
records.stream().forEach((record) -> {
EventHandle eventHandle = ((Event)record.getData()).getEventHandle();
if (acknowledgements) {
- acknowledgementSetManager.releaseEventReference(eventHandle, result);
+ eventHandle.release(result);
}
});
}
diff --git a/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/plugins/InMemorySourceAccessor.java b/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/plugins/InMemorySourceAccessor.java
index 71151be22e..3957d259a9 100644
--- a/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/plugins/InMemorySourceAccessor.java
+++ b/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/plugins/InMemorySourceAccessor.java
@@ -6,20 +6,19 @@
package org.opensearch.dataprepper.plugins;
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.model.event.EventFactory;
import org.opensearch.dataprepper.model.event.EventBuilder;
+import org.opensearch.dataprepper.model.event.EventFactory;
+import org.opensearch.dataprepper.model.record.Record;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
-import java.util.UUID;
import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantLock;
-import java.util.concurrent.atomic.AtomicBoolean;
/**
* Provides a mechanism to write records to an in_memory source. This allows the pipeline to execute
@@ -62,8 +61,8 @@ public void submit(final String testingKey, int numRecords) {
for (int i = 0; i < numRecords; i++) {
Map eventMap = Map.of("message", UUID.randomUUID().toString());
EventBuilder eventBuilder = (EventBuilder) eventFactory.eventBuilder(EventBuilder.class).withData(eventMap);
- JacksonEvent event = (JacksonEvent) eventBuilder.build();
- records.add(new Record(event));
+ Event event = eventBuilder.build();
+ records.add(new Record<>(event));
}
submit(testingKey, records);
}
@@ -79,8 +78,8 @@ public void submitWithStatus(final String testingKey, int numRecords) {
int status = (int)(Math.random() * (max - min + 1) + min);
Map eventMap = Map.of("message", UUID.randomUUID().toString(), "status", status);
EventBuilder eventBuilder = (EventBuilder) eventFactory.eventBuilder(EventBuilder.class).withData(eventMap);
- JacksonEvent event = (JacksonEvent) eventBuilder.build();
- records.add(new Record(event));
+ Event event = eventBuilder.build();
+ records.add(new Record<>(event));
}
submit(testingKey, records);
}
diff --git a/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/plugins/SimpleCopyProcessor.java b/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/plugins/SimpleCopyProcessor.java
new file mode 100644
index 0000000000..a786f09128
--- /dev/null
+++ b/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/plugins/SimpleCopyProcessor.java
@@ -0,0 +1,51 @@
+/*
+ * Copyright OpenSearch Contributors
+ * SPDX-License-Identifier: Apache-2.0
+ */
+
+package org.opensearch.dataprepper.plugins;
+
+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.processor.Processor;
+import org.opensearch.dataprepper.model.record.Record;
+
+import java.util.Collection;
+
+@DataPrepperPlugin(name = "simple_copy_test", pluginType = Processor.class, pluginConfigurationType = SimpleCopyProcessorConfig.class)
+public class SimpleCopyProcessor implements Processor, Record> {
+ private final SimpleCopyProcessorConfig simpleCopyProcessorConfig;
+ int count = 0;
+
+ @DataPrepperPluginConstructor
+ public SimpleCopyProcessor(final SimpleCopyProcessorConfig simpleCopyProcessorConfig) {
+ this.simpleCopyProcessorConfig = simpleCopyProcessorConfig;
+ }
+
+ @Override
+ public Collection> execute(final Collection> records) {
+ for (final Record record : records) {
+ final Object value = record.getData().get(simpleCopyProcessorConfig.getSource(), Object.class);
+ record.getData().put(simpleCopyProcessorConfig.getTarget(), value);
+ count++;
+ }
+
+ return records;
+ }
+
+ @Override
+ public void prepareForShutdown() {
+
+ }
+
+ @Override
+ public boolean isReadyForShutdown() {
+ return false;
+ }
+
+ @Override
+ public void shutdown() {
+
+ }
+}
diff --git a/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/plugins/SimpleCopyProcessorConfig.java b/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/plugins/SimpleCopyProcessorConfig.java
new file mode 100644
index 0000000000..ded7f6212f
--- /dev/null
+++ b/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/plugins/SimpleCopyProcessorConfig.java
@@ -0,0 +1,24 @@
+/*
+ * Copyright OpenSearch Contributors
+ * SPDX-License-Identifier: Apache-2.0
+ */
+
+package org.opensearch.dataprepper.plugins;
+
+import org.opensearch.dataprepper.model.event.EventKey;
+import org.opensearch.dataprepper.model.event.EventKeyConfiguration;
+import org.opensearch.dataprepper.model.event.EventKeyFactory;
+
+public class SimpleCopyProcessorConfig {
+ @EventKeyConfiguration(EventKeyFactory.EventAction.GET)
+ private EventKey source;
+ private EventKey target;
+
+ public EventKey getSource() {
+ return source;
+ }
+
+ public EventKey getTarget() {
+ return target;
+ }
+}
diff --git a/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/plugins/SimpleProcessor.java b/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/plugins/SimpleProcessor.java
new file mode 100644
index 0000000000..b0450d06d1
--- /dev/null
+++ b/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/plugins/SimpleProcessor.java
@@ -0,0 +1,53 @@
+/*
+ * Copyright OpenSearch Contributors
+ * SPDX-License-Identifier: Apache-2.0
+ */
+
+package org.opensearch.dataprepper.plugins;
+
+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.EventKey;
+import org.opensearch.dataprepper.model.processor.Processor;
+import org.opensearch.dataprepper.model.record.Record;
+
+import java.util.Collection;
+
+@DataPrepperPlugin(name = "simple_test", pluginType = Processor.class, pluginConfigurationType = SimpleProcessorConfig.class)
+public class SimpleProcessor implements Processor, Record> {
+ private final EventKey eventKey1;
+ private final String valuePrefix1;
+ int count = 0;
+
+ @DataPrepperPluginConstructor
+ public SimpleProcessor(final SimpleProcessorConfig simpleProcessorConfig) {
+ eventKey1 = simpleProcessorConfig.getKey1();
+ valuePrefix1 = simpleProcessorConfig.getValuePrefix1();
+ }
+
+ @Override
+ public Collection> execute(final Collection> records) {
+ for (final Record record : records) {
+ record.getData().put(eventKey1, valuePrefix1 + count);
+ count++;
+ }
+
+ return records;
+ }
+
+ @Override
+ public void prepareForShutdown() {
+
+ }
+
+ @Override
+ public boolean isReadyForShutdown() {
+ return false;
+ }
+
+ @Override
+ public void shutdown() {
+
+ }
+}
diff --git a/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/plugins/SimpleProcessorConfig.java b/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/plugins/SimpleProcessorConfig.java
new file mode 100644
index 0000000000..932d91c936
--- /dev/null
+++ b/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/plugins/SimpleProcessorConfig.java
@@ -0,0 +1,24 @@
+/*
+ * Copyright OpenSearch Contributors
+ * SPDX-License-Identifier: Apache-2.0
+ */
+
+package org.opensearch.dataprepper.plugins;
+
+import org.opensearch.dataprepper.model.event.EventKey;
+import org.opensearch.dataprepper.model.event.EventKeyConfiguration;
+import org.opensearch.dataprepper.model.event.EventKeyFactory;
+
+public class SimpleProcessorConfig {
+ @EventKeyConfiguration(EventKeyFactory.EventAction.PUT)
+ private EventKey key1;
+ private String valuePrefix1;
+
+ public EventKey getKey1() {
+ return key1;
+ }
+
+ public String getValuePrefix1() {
+ return valuePrefix1;
+ }
+}
diff --git a/data-prepper-core/src/integrationTest/resources/org/opensearch/dataprepper/pipeline/processor-pipeline.yaml b/data-prepper-core/src/integrationTest/resources/org/opensearch/dataprepper/pipeline/processor-pipeline.yaml
new file mode 100644
index 0000000000..be0e18a283
--- /dev/null
+++ b/data-prepper-core/src/integrationTest/resources/org/opensearch/dataprepper/pipeline/processor-pipeline.yaml
@@ -0,0 +1,17 @@
+processor-pipeline:
+ delay: 10
+ source:
+ in_memory:
+ testing_key: ProcessorPipelineIT
+
+ processor:
+ - simple_test:
+ key1: /test1
+ value_prefix1: knownPrefix1
+ - simple_copy_test:
+ source: /test1
+ target: /test1_copy
+
+ sink:
+ - in_memory:
+ testing_key: ProcessorPipelineIT
diff --git a/data-prepper-core/src/integrationTest/resources/org/opensearch/dataprepper/pipeline/route/three-route-with-default-route.yaml b/data-prepper-core/src/integrationTest/resources/org/opensearch/dataprepper/pipeline/route/three-route-with-default-route.yaml
new file mode 100644
index 0000000000..6d608a0d0b
--- /dev/null
+++ b/data-prepper-core/src/integrationTest/resources/org/opensearch/dataprepper/pipeline/route/three-route-with-default-route.yaml
@@ -0,0 +1,41 @@
+routing-pipeline:
+ workers: 4
+ delay: 10
+ source:
+ in_memory:
+ testing_key: ConditionalRoutingIT
+ buffer:
+ bounded_blocking:
+ # Use a small batch size to help ensure that multiple threads
+ # are picking up the different routes.
+ batch_size: 10
+ route:
+ - alpha: '/value == "a"'
+ - beta: '/value == "b"'
+ - gamma: '/value == "g"'
+ sink:
+ - in_memory:
+ testing_key: ConditionalRoutingIT_alpha
+ routes:
+ - alpha
+ - in_memory:
+ testing_key: ConditionalRoutingIT_beta
+ routes:
+ - beta
+ - in_memory:
+ testing_key: ConditionalRoutingIT_alpha_default
+ routes:
+ - alpha
+ - _default
+ - in_memory:
+ testing_key: ConditionalRoutingIT_alpha_beta_gamma
+ routes:
+ - alpha
+ - beta
+ - gamma
+ - in_memory:
+ testing_key: ConditionalRoutingIT_default
+ routes:
+ - _default
+ - in_memory:
+ testing_key: ConditionalRoutingIT_all
diff --git a/data-prepper-core/src/main/java/org/opensearch/dataprepper/acknowledgements/AcknowledgementSetMonitor.java b/data-prepper-core/src/main/java/org/opensearch/dataprepper/acknowledgements/AcknowledgementSetMonitor.java
index af9860cc9a..8c911346db 100644
--- a/data-prepper-core/src/main/java/org/opensearch/dataprepper/acknowledgements/AcknowledgementSetMonitor.java
+++ b/data-prepper-core/src/main/java/org/opensearch/dataprepper/acknowledgements/AcknowledgementSetMonitor.java
@@ -5,9 +5,6 @@
package org.opensearch.dataprepper.acknowledgements;
-import org.opensearch.dataprepper.model.event.EventHandle;
-import org.opensearch.dataprepper.model.event.DefaultEventHandle;
-import org.opensearch.dataprepper.model.event.InternalEventHandle;
import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSet;
import java.util.concurrent.locks.ReentrantLock;
@@ -33,15 +30,6 @@ class AcknowledgementSetMonitor implements Runnable {
private final AtomicInteger numInvalidReleases;
private final AtomicInteger numNullHandles;
- private DefaultAcknowledgementSet getAcknowledgementSet(final EventHandle eventHandle) {
- if (eventHandle instanceof DefaultEventHandle) {
- InternalEventHandle internalEventHandle = (InternalEventHandle)(DefaultEventHandle)eventHandle;
- return (DefaultAcknowledgementSet)internalEventHandle.getAcknowledgementSet();
- } else {
- throw new RuntimeException("Unsupported event handle");
- }
- }
-
public AcknowledgementSetMonitor() {
this.acknowledgementSets = new HashSet<>();
this.lock = new ReentrantLock(true);
@@ -67,55 +55,6 @@ public void add(final AcknowledgementSet acknowledgementSet) {
}
}
- public void acquire(final EventHandle eventHandle) {
- if (eventHandle == null) {
- numNullHandles.incrementAndGet();
- return;
- }
-
- DefaultAcknowledgementSet acknowledgementSet = getAcknowledgementSet(eventHandle);
- lock.lock();
- boolean exists = false;
- try {
- exists = acknowledgementSets.contains(acknowledgementSet);
- } finally {
- lock.unlock();
- }
- // if acknowledgementSet doesn't exist then it means that the
- // event still active even after the acknowledgement set is
- // cleaned up.
- if (exists) {
- acknowledgementSet.acquire(eventHandle);
- } else {
- LOG.warn("Trying acquire an event in an AcknowledgementSet that does not exist");
- numInvalidAcquires.incrementAndGet();
- }
- }
-
- public void release(final EventHandle eventHandle, final boolean success) {
- if (eventHandle == null) {
- numNullHandles.incrementAndGet();
- return;
- }
- DefaultAcknowledgementSet acknowledgementSet = getAcknowledgementSet(eventHandle);
- lock.lock();
- boolean exists = false;
- try {
- exists = acknowledgementSets.contains(acknowledgementSet);
- } finally {
- lock.unlock();
- }
- // if acknowledgementSet doesn't exist then it means some late
- // arrival of event handle release after the acknowledgement set
- // is cleaned up.
- if (exists) {
- boolean b = acknowledgementSet.release(eventHandle, success);
- } else {
- LOG.warn("Trying to release from an AcknowledgementSet that does not exist");
- numInvalidReleases.incrementAndGet();
- }
- }
-
/**
* for testing
* @return the size
@@ -131,6 +70,8 @@ public void run() {
if (acknowledgementSets.size() > 0) {
acknowledgementSets.removeIf((ackSet) -> ((DefaultAcknowledgementSet) ackSet).isDone());
}
+ Thread.sleep(1000);
+ } catch (InterruptedException e) {
} finally {
lock.unlock();
}
diff --git a/data-prepper-core/src/main/java/org/opensearch/dataprepper/acknowledgements/DefaultAcknowledgementSet.java b/data-prepper-core/src/main/java/org/opensearch/dataprepper/acknowledgements/DefaultAcknowledgementSet.java
index c2823203fe..fd26d10c72 100644
--- a/data-prepper-core/src/main/java/org/opensearch/dataprepper/acknowledgements/DefaultAcknowledgementSet.java
+++ b/data-prepper-core/src/main/java/org/opensearch/dataprepper/acknowledgements/DefaultAcknowledgementSet.java
@@ -82,7 +82,7 @@ public void add(Event event) {
EventHandle eventHandle = event.getEventHandle();
if (eventHandle instanceof DefaultEventHandle) {
InternalEventHandle internalEventHandle = (InternalEventHandle)(DefaultEventHandle)eventHandle;
- internalEventHandle.setAcknowledgementSet(this);
+ internalEventHandle.addAcknowledgementSet(this);
pendingAcknowledgments.put(eventHandle, new AtomicInteger(1));
totalEventsAdded.incrementAndGet();
}
diff --git a/data-prepper-core/src/main/java/org/opensearch/dataprepper/acknowledgements/DefaultAcknowledgementSetManager.java b/data-prepper-core/src/main/java/org/opensearch/dataprepper/acknowledgements/DefaultAcknowledgementSetManager.java
index 3f2e3761bd..b8f81dbfc1 100644
--- a/data-prepper-core/src/main/java/org/opensearch/dataprepper/acknowledgements/DefaultAcknowledgementSetManager.java
+++ b/data-prepper-core/src/main/java/org/opensearch/dataprepper/acknowledgements/DefaultAcknowledgementSetManager.java
@@ -7,8 +7,6 @@
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.event.EventHandle;
import org.opensearch.dataprepper.metrics.PluginMetrics;
import javax.inject.Inject;
@@ -49,18 +47,6 @@ public AcknowledgementSet create(final Consumer callback, final Duratio
return acknowledgementSet;
}
- public void acquireEventReference(final Event event) {
- acquireEventReference(event.getEventHandle());
- }
-
- public void acquireEventReference(final EventHandle eventHandle) {
- acknowledgementSetMonitor.acquire(eventHandle);
- }
-
- public void releaseEventReference(final EventHandle eventHandle, final boolean success) {
- acknowledgementSetMonitor.release(eventHandle, success);
- }
-
public void shutdown() {
acknowledgementSetMonitorThread.stop();
}
diff --git a/data-prepper-core/src/main/java/org/opensearch/dataprepper/acknowledgements/InactiveAcknowledgementSetManager.java b/data-prepper-core/src/main/java/org/opensearch/dataprepper/acknowledgements/InactiveAcknowledgementSetManager.java
index 2e112b4560..52f0e1978f 100644
--- a/data-prepper-core/src/main/java/org/opensearch/dataprepper/acknowledgements/InactiveAcknowledgementSetManager.java
+++ b/data-prepper-core/src/main/java/org/opensearch/dataprepper/acknowledgements/InactiveAcknowledgementSetManager.java
@@ -5,8 +5,6 @@
package org.opensearch.dataprepper.acknowledgements;
-import org.opensearch.dataprepper.model.event.Event;
-import org.opensearch.dataprepper.model.event.EventHandle;
import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSetManager;
import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSet;
import java.util.function.Consumer;
@@ -26,15 +24,4 @@ public AcknowledgementSet create(final Consumer callback, final Duratio
throw new UnsupportedOperationException("create operation not supported");
}
- public void acquireEventReference(final Event event) {
- throw new UnsupportedOperationException("acquire operation not supported");
- }
-
- public void acquireEventReference(final EventHandle eventHandle) {
- throw new UnsupportedOperationException("acquire operation not supported");
- }
-
- public void releaseEventReference(final EventHandle eventHandle, boolean success) {
- throw new UnsupportedOperationException("release operation not supported");
- }
}
diff --git a/data-prepper-core/src/main/java/org/opensearch/dataprepper/peerforwarder/DefaultPeerForwarderProvider.java b/data-prepper-core/src/main/java/org/opensearch/dataprepper/peerforwarder/DefaultPeerForwarderProvider.java
new file mode 100644
index 0000000000..ff638ee26f
--- /dev/null
+++ b/data-prepper-core/src/main/java/org/opensearch/dataprepper/peerforwarder/DefaultPeerForwarderProvider.java
@@ -0,0 +1,102 @@
+/*
+ * Copyright OpenSearch Contributors
+ * SPDX-License-Identifier: Apache-2.0
+ */
+
+package org.opensearch.dataprepper.peerforwarder;
+
+import org.opensearch.dataprepper.metrics.PluginMetrics;
+import org.opensearch.dataprepper.model.event.Event;
+import org.opensearch.dataprepper.model.record.Record;
+import org.opensearch.dataprepper.model.processor.Processor;
+import org.opensearch.dataprepper.peerforwarder.client.PeerForwarderClient;
+import org.opensearch.dataprepper.peerforwarder.discovery.DiscoveryMode;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+
+public class DefaultPeerForwarderProvider implements PeerForwarderProvider {
+
+ private final PeerForwarderClientFactory peerForwarderClientFactory;
+ private final PeerForwarderClient peerForwarderClient;
+ private final PeerForwarderConfiguration peerForwarderConfiguration;
+ private final PluginMetrics pluginMetrics;
+ private final Map>>> pipelinePeerForwarderReceiveBufferMap = new HashMap<>();
+ private HashRing hashRing;
+
+ DefaultPeerForwarderProvider(final PeerForwarderClientFactory peerForwarderClientFactory,
+ final PeerForwarderClient peerForwarderClient,
+ final PeerForwarderConfiguration peerForwarderConfiguration,
+ final PluginMetrics pluginMetrics) {
+ this.peerForwarderClientFactory = peerForwarderClientFactory;
+ this.peerForwarderClient = peerForwarderClient;
+ this.peerForwarderConfiguration = peerForwarderConfiguration;
+ this.pluginMetrics = pluginMetrics;
+ }
+
+ public PeerForwarder register(final String pipelineName, final Processor processor, final String pluginId, final Set identificationKeys,
+ final Integer pipelineWorkerThreads) {
+ if (pipelinePeerForwarderReceiveBufferMap.containsKey(pipelineName) &&
+ pipelinePeerForwarderReceiveBufferMap.get(pipelineName).containsKey(pluginId)) {
+ throw new RuntimeException("Data Prepper 2.0 will only support a single peer-forwarder per pipeline/plugin type");
+ }
+
+ final PeerForwarderReceiveBuffer> peerForwarderReceiveBuffer = createBufferPerPipelineProcessor(pipelineName, pluginId);
+
+ if (isPeerForwardingRequired()) {
+ if (hashRing == null) {
+ hashRing = peerForwarderClientFactory.createHashRing();
+ }
+ return new RemotePeerForwarder(
+ peerForwarderClient,
+ hashRing,
+ peerForwarderReceiveBuffer,
+ pipelineName,
+ pluginId,
+ identificationKeys,
+ pluginMetrics,
+ peerForwarderConfiguration.getBatchDelay(),
+ peerForwarderConfiguration.getFailedForwardingRequestLocalWriteTimeout(),
+ peerForwarderConfiguration.getForwardingBatchSize(),
+ peerForwarderConfiguration.getForwardingBatchQueueDepth(),
+ peerForwarderConfiguration.getForwardingBatchTimeout(),
+ pipelineWorkerThreads
+ );
+ }
+ else {
+ return new LocalPeerForwarder();
+ }
+ }
+
+ private PeerForwarderReceiveBuffer> createBufferPerPipelineProcessor(final String pipelineName, final String pluginId) {
+ final PeerForwarderReceiveBuffer> peerForwarderReceiveBuffer = new
+ PeerForwarderReceiveBuffer<>(peerForwarderConfiguration.getBufferSize(), peerForwarderConfiguration.getBatchSize(), pipelineName, pluginId);
+
+ final Map>> pluginsBufferMap =
+ pipelinePeerForwarderReceiveBufferMap.computeIfAbsent(pipelineName, k -> new HashMap<>());
+
+ pluginsBufferMap.put(pluginId, peerForwarderReceiveBuffer);
+
+ return peerForwarderReceiveBuffer;
+ }
+
+ public boolean isPeerForwardingRequired() {
+ return arePeersConfigured() && pipelinePeerForwarderReceiveBufferMap.size() > 0;
+ }
+
+ public boolean arePeersConfigured() {
+ final DiscoveryMode discoveryMode = peerForwarderConfiguration.getDiscoveryMode();
+ if (discoveryMode.equals(DiscoveryMode.LOCAL_NODE)) {
+ return false;
+ }
+ else if (discoveryMode.equals(DiscoveryMode.STATIC) && peerForwarderConfiguration.getStaticEndpoints().size() <= 1) {
+ return false;
+ }
+ return true;
+ }
+
+ public Map>>> getPipelinePeerForwarderReceiveBufferMap() {
+ return pipelinePeerForwarderReceiveBufferMap;
+ }
+}
diff --git a/data-prepper-core/src/main/java/org/opensearch/dataprepper/peerforwarder/LocalModePeerForwarderProvider.java b/data-prepper-core/src/main/java/org/opensearch/dataprepper/peerforwarder/LocalModePeerForwarderProvider.java
new file mode 100644
index 0000000000..6c2c4fe688
--- /dev/null
+++ b/data-prepper-core/src/main/java/org/opensearch/dataprepper/peerforwarder/LocalModePeerForwarderProvider.java
@@ -0,0 +1,51 @@
+/*
+ * Copyright OpenSearch Contributors
+ * SPDX-License-Identifier: Apache-2.0
+ */
+
+package org.opensearch.dataprepper.peerforwarder;
+
+import org.opensearch.dataprepper.model.event.Event;
+import org.opensearch.dataprepper.model.record.Record;
+import org.opensearch.dataprepper.model.processor.Processor;
+import org.opensearch.dataprepper.model.peerforwarder.RequiresPeerForwarding;
+
+import java.util.Map;
+import java.util.Set;
+
+public class LocalModePeerForwarderProvider implements PeerForwarderProvider {
+
+ private final PeerForwarderProvider peerForwarderProvider;
+ private boolean isRemotePeerForwarderRegistered;
+
+ public LocalModePeerForwarderProvider(final PeerForwarderProvider peerForwarderProvider) {
+ this.peerForwarderProvider = peerForwarderProvider;
+ this.isRemotePeerForwarderRegistered = false;
+ }
+
+ @Override
+ public PeerForwarder register(final String pipelineName, final Processor processor, final String pluginId, final Set identificationKeys, final Integer pipelineWorkerThreads) {
+ if (((RequiresPeerForwarding)processor).isForLocalProcessingOnly(null)) {
+ return new LocalPeerForwarder();
+ }
+ isRemotePeerForwarderRegistered = true;
+ return peerForwarderProvider.register(pipelineName, processor, pluginId, identificationKeys, pipelineWorkerThreads);
+ }
+
+ @Override
+ public boolean isPeerForwardingRequired() {
+ return isRemotePeerForwarderRegistered;
+ }
+
+ @Override
+ public Map>>> getPipelinePeerForwarderReceiveBufferMap() {
+ return (isRemotePeerForwarderRegistered) ?
+ peerForwarderProvider.getPipelinePeerForwarderReceiveBufferMap() :
+ Map.of();
+ }
+
+ @Override
+ public boolean arePeersConfigured() {
+ return isRemotePeerForwarderRegistered ? peerForwarderProvider.arePeersConfigured() : false;
+ }
+}
diff --git a/data-prepper-core/src/main/java/org/opensearch/dataprepper/peerforwarder/PeerForwarderAppConfig.java b/data-prepper-core/src/main/java/org/opensearch/dataprepper/peerforwarder/PeerForwarderAppConfig.java
index e3123b67f1..4cca81819d 100644
--- a/data-prepper-core/src/main/java/org/opensearch/dataprepper/peerforwarder/PeerForwarderAppConfig.java
+++ b/data-prepper-core/src/main/java/org/opensearch/dataprepper/peerforwarder/PeerForwarderAppConfig.java
@@ -20,6 +20,10 @@
import org.springframework.beans.factory.annotation.Qualifier;
import org.springframework.context.annotation.Bean;
import org.springframework.context.annotation.Configuration;
+import org.springframework.context.annotation.Primary;
+
+import javax.inject.Named;
+
@Configuration
class PeerForwarderAppConfig {
@@ -71,12 +75,18 @@ public PeerForwarderClient peerForwarderClient(final PeerForwarderConfiguration
peerForwarderConfiguration, peerForwarderClientFactory, peerForwarderCodec, pluginMetrics);
}
- @Bean
- public PeerForwarderProvider peerForwarderProvider(final PeerForwarderClientFactory peerForwarderClientFactory,
+ @Bean(name = "defaultPeerForwarder")
+ public DefaultPeerForwarderProvider peerForwarderProvider(final PeerForwarderClientFactory peerForwarderClientFactory,
final PeerForwarderClient peerForwarderClient,
final PeerForwarderConfiguration peerForwarderConfiguration,
@Qualifier("peerForwarderMetrics") final PluginMetrics pluginMetrics) {
- return new PeerForwarderProvider(peerForwarderClientFactory, peerForwarderClient, peerForwarderConfiguration, pluginMetrics);
+ return new DefaultPeerForwarderProvider(peerForwarderClientFactory, peerForwarderClient, peerForwarderConfiguration, pluginMetrics);
+ }
+
+ @Bean
+ @Primary
+ public PeerForwarderProvider peerForwarderProvider(@Named("defaultPeerForwarder") final PeerForwarderProvider peerForwarderProvider) {
+ return new LocalModePeerForwarderProvider(peerForwarderProvider);
}
@Bean
diff --git a/data-prepper-core/src/main/java/org/opensearch/dataprepper/peerforwarder/PeerForwarderProvider.java b/data-prepper-core/src/main/java/org/opensearch/dataprepper/peerforwarder/PeerForwarderProvider.java
index ea89c1cbdd..40b3a03deb 100644
--- a/data-prepper-core/src/main/java/org/opensearch/dataprepper/peerforwarder/PeerForwarderProvider.java
+++ b/data-prepper-core/src/main/java/org/opensearch/dataprepper/peerforwarder/PeerForwarderProvider.java
@@ -5,97 +5,49 @@
package org.opensearch.dataprepper.peerforwarder;
-import org.opensearch.dataprepper.metrics.PluginMetrics;
import org.opensearch.dataprepper.model.event.Event;
import org.opensearch.dataprepper.model.record.Record;
-import org.opensearch.dataprepper.peerforwarder.client.PeerForwarderClient;
-import org.opensearch.dataprepper.peerforwarder.discovery.DiscoveryMode;
+import org.opensearch.dataprepper.model.processor.Processor;
-import java.util.HashMap;
import java.util.Map;
import java.util.Set;
-public class PeerForwarderProvider {
-
- private final PeerForwarderClientFactory peerForwarderClientFactory;
- private final PeerForwarderClient peerForwarderClient;
- private final PeerForwarderConfiguration peerForwarderConfiguration;
- private final PluginMetrics pluginMetrics;
- private final Map>>> pipelinePeerForwarderReceiveBufferMap = new HashMap<>();
- private HashRing hashRing;
-
- PeerForwarderProvider(final PeerForwarderClientFactory peerForwarderClientFactory,
- final PeerForwarderClient peerForwarderClient,
- final PeerForwarderConfiguration peerForwarderConfiguration,
- final PluginMetrics pluginMetrics) {
- this.peerForwarderClientFactory = peerForwarderClientFactory;
- this.peerForwarderClient = peerForwarderClient;
- this.peerForwarderConfiguration = peerForwarderConfiguration;
- this.pluginMetrics = pluginMetrics;
- }
-
- public PeerForwarder register(final String pipelineName, final String pluginId, final Set identificationKeys,
- final Integer pipelineWorkerThreads) {
- if (pipelinePeerForwarderReceiveBufferMap.containsKey(pipelineName) &&
- pipelinePeerForwarderReceiveBufferMap.get(pipelineName).containsKey(pluginId)) {
- throw new RuntimeException("Data Prepper 2.0 will only support a single peer-forwarder per pipeline/plugin type");
- }
-
- final PeerForwarderReceiveBuffer> peerForwarderReceiveBuffer = createBufferPerPipelineProcessor(pipelineName, pluginId);
-
- if (isPeerForwardingRequired()) {
- if (hashRing == null) {
- hashRing = peerForwarderClientFactory.createHashRing();
- }
- return new RemotePeerForwarder(
- peerForwarderClient,
- hashRing,
- peerForwarderReceiveBuffer,
- pipelineName,
- pluginId,
- identificationKeys,
- pluginMetrics,
- peerForwarderConfiguration.getBatchDelay(),
- peerForwarderConfiguration.getFailedForwardingRequestLocalWriteTimeout(),
- peerForwarderConfiguration.getForwardingBatchSize(),
- peerForwarderConfiguration.getForwardingBatchQueueDepth(),
- peerForwarderConfiguration.getForwardingBatchTimeout(),
- pipelineWorkerThreads
- );
- }
- else {
- return new LocalPeerForwarder();
- }
- }
-
- private PeerForwarderReceiveBuffer> createBufferPerPipelineProcessor(final String pipelineName, final String pluginId) {
- final PeerForwarderReceiveBuffer> peerForwarderReceiveBuffer = new
- PeerForwarderReceiveBuffer<>(peerForwarderConfiguration.getBufferSize(), peerForwarderConfiguration.getBatchSize(), pipelineName, pluginId);
-
- final Map>> pluginsBufferMap =
- pipelinePeerForwarderReceiveBufferMap.computeIfAbsent(pipelineName, k -> new HashMap<>());
-
- pluginsBufferMap.put(pluginId, peerForwarderReceiveBuffer);
-
- return peerForwarderReceiveBuffer;
- }
-
- public boolean isPeerForwardingRequired() {
- return arePeersConfigured() && pipelinePeerForwarderReceiveBufferMap.size() > 0;
- }
-
- private boolean arePeersConfigured() {
- final DiscoveryMode discoveryMode = peerForwarderConfiguration.getDiscoveryMode();
- if (discoveryMode.equals(DiscoveryMode.LOCAL_NODE)) {
- return false;
- }
- else if (discoveryMode.equals(DiscoveryMode.STATIC) && peerForwarderConfiguration.getStaticEndpoints().size() <= 1) {
- return false;
- }
- return true;
- }
-
- public Map>>> getPipelinePeerForwarderReceiveBufferMap() {
- return pipelinePeerForwarderReceiveBufferMap;
- }
+public interface PeerForwarderProvider {
+ /**
+ * Registers a pipeline and identification keys
+ *
+ * @param pipelineName pipeline name
+ * @param processor processor
+ * @param pluginId plugin id
+ * @param identificationKeys identification keys
+ * @param pipelineWorkerThreads number of pipeline worker threads
+ * @return peer forwarder
+ * @since 2.9
+ */
+ PeerForwarder register(final String pipelineName, final Processor processor, final String pluginId, final Set identificationKeys, final Integer pipelineWorkerThreads);
+
+ /**
+ * Returns if peer forwarding required
+ *
+ * @return returns if peer forwarding required or nto
+ * @since 2.9
+ */
+ boolean isPeerForwardingRequired();
+
+ /**
+ * Returns if peers configured
+ *
+ * @return returns if peers configured
+ * @since 2.9
+ */
+ boolean arePeersConfigured();
+
+ /**
+ * Returns pipeline peer forwarder receive buffer map
+ *
+ * @return Map of buffer per pipeline per pluginId
+ * @since 2.9
+ */
+ Map>>> getPipelinePeerForwarderReceiveBufferMap();
}
+
diff --git a/data-prepper-core/src/main/java/org/opensearch/dataprepper/peerforwarder/PeerForwardingProcessorDecorator.java b/data-prepper-core/src/main/java/org/opensearch/dataprepper/peerforwarder/PeerForwardingProcessorDecorator.java
index 58a99aadae..038bdb28c5 100644
--- a/data-prepper-core/src/main/java/org/opensearch/dataprepper/peerforwarder/PeerForwardingProcessorDecorator.java
+++ b/data-prepper-core/src/main/java/org/opensearch/dataprepper/peerforwarder/PeerForwardingProcessorDecorator.java
@@ -67,7 +67,7 @@ public static List decorateProcessors(
"Peer Forwarder Plugin: %s cannot have empty identification keys." + pluginId);
}
- final PeerForwarder peerForwarder = peerForwarderProvider.register(pipelineName, pluginId, identificationKeys, pipelineWorkerThreads);
+ final PeerForwarder peerForwarder = peerForwarderProvider.register(pipelineName, firstInnerProcessor, pluginId, identificationKeys, pipelineWorkerThreads);
return processors.stream().map(processor -> new PeerForwardingProcessorDecorator(peerForwarder, processor))
.collect(Collectors.toList());
diff --git a/data-prepper-core/src/main/java/org/opensearch/dataprepper/pipeline/ProcessWorker.java b/data-prepper-core/src/main/java/org/opensearch/dataprepper/pipeline/ProcessWorker.java
index 2178fd6bcc..b5538dfe73 100644
--- a/data-prepper-core/src/main/java/org/opensearch/dataprepper/pipeline/ProcessWorker.java
+++ b/data-prepper-core/src/main/java/org/opensearch/dataprepper/pipeline/ProcessWorker.java
@@ -100,7 +100,7 @@ private void processAcknowledgements(List inputEvents, Collection void route(final Collection allRecords,
final DataFlowComponent dataFlowComponent,
final Map> recordsToRoutes,
@@ -37,7 +38,9 @@ void route(final Collection allRecords,
final Set routesForEvent = recordsToRoutes
.getOrDefault(record, Collections.emptySet());
- if (routesForEvent.stream().anyMatch(dataFlowComponentRoutes::contains)) {
+ if (routesForEvent.size() == 0 && dataFlowComponentRoutes.contains(DEFAULT_ROUTE)) {
+ recordsForComponent.add(getRecordStrategy.getRecord(record));
+ } else if (routesForEvent.stream().anyMatch(dataFlowComponentRoutes::contains)) {
recordsForComponent.add(getRecordStrategy.getRecord(record));
}
}
diff --git a/data-prepper-core/src/main/java/org/opensearch/dataprepper/pipeline/router/RouterCopyRecordStrategy.java b/data-prepper-core/src/main/java/org/opensearch/dataprepper/pipeline/router/RouterCopyRecordStrategy.java
index 1bd2944c2e..b4982c5b07 100644
--- a/data-prepper-core/src/main/java/org/opensearch/dataprepper/pipeline/router/RouterCopyRecordStrategy.java
+++ b/data-prepper-core/src/main/java/org/opensearch/dataprepper/pipeline/router/RouterCopyRecordStrategy.java
@@ -16,6 +16,7 @@
import org.opensearch.dataprepper.model.event.JacksonEvent;
import org.opensearch.dataprepper.model.event.EventFactory;
import org.opensearch.dataprepper.model.event.EventHandle;
+import org.opensearch.dataprepper.model.event.InternalEventHandle;
import org.opensearch.dataprepper.model.event.EventBuilder;
import org.opensearch.dataprepper.model.event.EventMetadata;
import org.opensearch.dataprepper.model.event.DefaultEventHandle;
@@ -65,8 +66,8 @@ private void acquireEventReference(final Record record) {
}
if (referencedRecords.contains(record) || ((routedRecords != null) && routedRecords.contains(record))) {
EventHandle eventHandle = ((JacksonEvent)record.getData()).getEventHandle();
- if (eventHandle != null && eventHandle instanceof DefaultEventHandle) {
- acknowledgementSetManager.acquireEventReference(eventHandle);
+ if (eventHandle != null && eventHandle instanceof InternalEventHandle) {
+ ((InternalEventHandle)eventHandle).acquireReference();
}
} else if (!referencedRecords.contains(record)) {
referencedRecords.add(record);
diff --git a/data-prepper-core/src/test/java/org/opensearch/dataprepper/acknowledgements/AcknowledgementSetMonitorTests.java b/data-prepper-core/src/test/java/org/opensearch/dataprepper/acknowledgements/AcknowledgementSetMonitorTests.java
index 6c85b5c4de..158841a44a 100644
--- a/data-prepper-core/src/test/java/org/opensearch/dataprepper/acknowledgements/AcknowledgementSetMonitorTests.java
+++ b/data-prepper-core/src/test/java/org/opensearch/dataprepper/acknowledgements/AcknowledgementSetMonitorTests.java
@@ -13,7 +13,6 @@
import org.junit.jupiter.api.extension.ExtendWith;
import static org.mockito.Mockito.when;
import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.doAnswer;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.MatcherAssert.assertThat;
@@ -71,57 +70,4 @@ public void testMultipleAcknowledgementSets() {
acknowledgementSetMonitor.run();
assertThat(acknowledgementSetMonitor.getSize(), equalTo(1));
}
-
- @Test
- public void testAcknowledgementSetAcquireRelease() {
- when(eventHandle1.getAcknowledgementSet()).thenReturn(acknowledgementSet1);
- try {
- doAnswer((i) -> {return null; }).when(acknowledgementSet1).acquire(eventHandle1);
- } catch (Exception e){}
- acknowledgementSetMonitor.add(acknowledgementSet1);
- acknowledgementSetMonitor.acquire(eventHandle1);
- acknowledgementSetMonitor.release(eventHandle1, true);
- Thread shutdownThread = new Thread(() -> {
- try {
- Thread.sleep(DEFAULT_WAIT_TIME_MS);
- } catch (Exception e){}
- });
- shutdownThread.start();
- acknowledgementSetMonitor.run();
- assertThat(acknowledgementSetMonitor.getSize(), equalTo(0));
- }
-
- @Test
- public void testAcknowledgementSetInvalidAcquire() {
- acknowledgementSet2 = mock(DefaultAcknowledgementSet.class);
- when(eventHandle1.getAcknowledgementSet()).thenReturn(acknowledgementSet2);
- acknowledgementSetMonitor.add(acknowledgementSet1);
- acknowledgementSetMonitor.acquire(eventHandle1);
- Thread shutdownThread = new Thread(() -> {
- try {
- Thread.sleep(DEFAULT_WAIT_TIME_MS);
- } catch (Exception e){}
- });
- shutdownThread.start();
- acknowledgementSetMonitor.run();
- assertThat(acknowledgementSetMonitor.getSize(), equalTo(0));
- assertThat(acknowledgementSetMonitor.getNumInvalidAcquires(), equalTo(1));
- }
-
- @Test
- public void testAcknowledgementSetInvalidRelease() {
- acknowledgementSet2 = mock(DefaultAcknowledgementSet.class);
- when(eventHandle1.getAcknowledgementSet()).thenReturn(acknowledgementSet2);
- acknowledgementSetMonitor.add(acknowledgementSet1);
- acknowledgementSetMonitor.release(eventHandle1, true);
- Thread shutdownThread = new Thread(() -> {
- try {
- Thread.sleep(DEFAULT_WAIT_TIME_MS);
- } catch (Exception e){}
- });
- shutdownThread.start();
- acknowledgementSetMonitor.run();
- assertThat(acknowledgementSetMonitor.getSize(), equalTo(0));
- assertThat(acknowledgementSetMonitor.getNumInvalidReleases(), equalTo(1));
- }
}
diff --git a/data-prepper-core/src/test/java/org/opensearch/dataprepper/acknowledgements/DefaultAcknowledgementSetManagerTests.java b/data-prepper-core/src/test/java/org/opensearch/dataprepper/acknowledgements/DefaultAcknowledgementSetManagerTests.java
index 1b87d6c849..a083f5ea85 100644
--- a/data-prepper-core/src/test/java/org/opensearch/dataprepper/acknowledgements/DefaultAcknowledgementSetManagerTests.java
+++ b/data-prepper-core/src/test/java/org/opensearch/dataprepper/acknowledgements/DefaultAcknowledgementSetManagerTests.java
@@ -14,6 +14,8 @@
import org.junit.jupiter.api.Test;
import org.mockito.junit.jupiter.MockitoExtension;
import org.junit.jupiter.api.extension.ExtendWith;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.ArgumentMatchers.any;
import org.mockito.Mock;
import static org.awaitility.Awaitility.await;
@@ -53,17 +55,27 @@ class DefaultAcknowledgementSetManagerTests {
void setup() {
currentRatio = 0;
callbackExecutor = Executors.newScheduledThreadPool(2);
+ acknowledgementSetManager = createObjectUnderTest();
+ AcknowledgementSet acknowledgementSet1 = acknowledgementSetManager.create((flag) -> { result = flag; }, TEST_TIMEOUT);
event1 = mock(JacksonEvent.class);
eventHandle1 = mock(DefaultEventHandle.class);
+ lenient().doAnswer(a -> {
+ Boolean result = (Boolean)a.getArgument(0);
+ acknowledgementSet1.release(eventHandle1, result);
+ return null;
+ }).when(eventHandle1).release(any(Boolean.class));
lenient().when(event1.getEventHandle()).thenReturn(eventHandle1);
pluginMetrics = mock(PluginMetrics.class);
event2 = mock(JacksonEvent.class);
eventHandle2 = mock(DefaultEventHandle.class);
+ lenient().doAnswer(a -> {
+ Boolean result = (Boolean)a.getArgument(0);
+ acknowledgementSet1.release(eventHandle2, result);
+ return null;
+ }).when(eventHandle2).release(any(Boolean.class));
lenient().when(event2.getEventHandle()).thenReturn(eventHandle2);
- acknowledgementSetManager = createObjectUnderTest();
- AcknowledgementSet acknowledgementSet1 = acknowledgementSetManager.create((flag) -> { result = flag; }, TEST_TIMEOUT);
acknowledgementSet1.add(event1);
acknowledgementSet1.add(event2);
lenient().when(eventHandle1.getAcknowledgementSet()).thenReturn(acknowledgementSet1);
@@ -77,8 +89,8 @@ DefaultAcknowledgementSetManager createObjectUnderTest() {
@Test
void testBasic() {
- acknowledgementSetManager.releaseEventReference(eventHandle2, true);
- acknowledgementSetManager.releaseEventReference(eventHandle1, true);
+ eventHandle2.release(true);
+ eventHandle1.release(true);
await().atMost(TEST_TIMEOUT.multipliedBy(5))
.untilAsserted(() -> {
assertThat(acknowledgementSetManager.getAcknowledgementSetMonitor().getSize(), equalTo(0));
@@ -88,7 +100,7 @@ void testBasic() {
@Test
void testExpirations() throws InterruptedException {
- acknowledgementSetManager.releaseEventReference(eventHandle2, true);
+ eventHandle2.release(true);
Thread.sleep(TEST_TIMEOUT.multipliedBy(5).toMillis());
assertThat(acknowledgementSetManager.getAcknowledgementSetMonitor().getSize(), equalTo(0));
await().atMost(TEST_TIMEOUT.multipliedBy(5))
@@ -99,17 +111,22 @@ void testExpirations() throws InterruptedException {
@Test
void testMultipleAcknowledgementSets() {
+ AcknowledgementSet acknowledgementSet2 = acknowledgementSetManager.create((flag) -> { result = flag; }, TEST_TIMEOUT);
event3 = mock(JacksonEvent.class);
eventHandle3 = mock(DefaultEventHandle.class);
+ doAnswer(a -> {
+ Boolean result = (Boolean)a.getArgument(0);
+ acknowledgementSet2.release(eventHandle3, result);
+ return null;
+ }).when(eventHandle3).release(any(Boolean.class));
lenient().when(event3.getEventHandle()).thenReturn(eventHandle3);
- AcknowledgementSet acknowledgementSet2 = acknowledgementSetManager.create((flag) -> { result = flag; }, TEST_TIMEOUT);
acknowledgementSet2.add(event3);
lenient().when(eventHandle3.getAcknowledgementSet()).thenReturn(acknowledgementSet2);
acknowledgementSet2.complete();
- acknowledgementSetManager.releaseEventReference(eventHandle2, true);
- acknowledgementSetManager.releaseEventReference(eventHandle3, true);
+ eventHandle2.release(true);
+ eventHandle3.release(true);
await().atMost(TEST_TIMEOUT.multipliedBy(5))
.untilAsserted(() -> {
assertThat(acknowledgementSetManager.getAcknowledgementSetMonitor().getSize(), equalTo(0));
@@ -119,22 +136,42 @@ void testMultipleAcknowledgementSets() {
@Test
void testWithProgressCheckCallbacks() {
+ AcknowledgementSet acknowledgementSet2 = acknowledgementSetManager.create((flag) -> { result = flag; }, Duration.ofMillis(10000));
eventHandle3 = mock(DefaultEventHandle.class);
+ doAnswer(a -> {
+ Boolean result = (Boolean)a.getArgument(0);
+ acknowledgementSet2.release(eventHandle3, result);
+ return null;
+ }).when(eventHandle3).release(any(Boolean.class));
lenient().when(event3.getEventHandle()).thenReturn(eventHandle3);
eventHandle4 = mock(DefaultEventHandle.class);
+ doAnswer(a -> {
+ Boolean result = (Boolean)a.getArgument(0);
+ acknowledgementSet2.release(eventHandle4, result);
+ return null;
+ }).when(eventHandle4).release(any(Boolean.class));
JacksonEvent event4 = mock(JacksonEvent.class);
lenient().when(event4.getEventHandle()).thenReturn(eventHandle4);
eventHandle5 = mock(DefaultEventHandle.class);
+ doAnswer(a -> {
+ Boolean result = (Boolean)a.getArgument(0);
+ acknowledgementSet2.release(eventHandle5, result);
+ return null;
+ }).when(eventHandle5).release(any(Boolean.class));
JacksonEvent event5 = mock(JacksonEvent.class);
lenient().when(event5.getEventHandle()).thenReturn(eventHandle5);
eventHandle6 = mock(DefaultEventHandle.class);
+ doAnswer(a -> {
+ Boolean result = (Boolean)a.getArgument(0);
+ acknowledgementSet2.release(eventHandle6, result);
+ return null;
+ }).when(eventHandle6).release(any(Boolean.class));
JacksonEvent event6 = mock(JacksonEvent.class);
lenient().when(event6.getEventHandle()).thenReturn(eventHandle6);
- AcknowledgementSet acknowledgementSet2 = acknowledgementSetManager.create((flag) -> { result = flag; }, Duration.ofMillis(10000));
acknowledgementSet2.addProgressCheck((progressCheck) -> {currentRatio = progressCheck.getRatio();}, Duration.ofSeconds(1));
acknowledgementSet2.add(event3);
acknowledgementSet2.add(event4);
@@ -145,22 +182,22 @@ void testWithProgressCheckCallbacks() {
lenient().when(eventHandle5.getAcknowledgementSet()).thenReturn(acknowledgementSet2);
lenient().when(eventHandle6.getAcknowledgementSet()).thenReturn(acknowledgementSet2);
acknowledgementSet2.complete();
- acknowledgementSetManager.releaseEventReference(eventHandle3, true);
+ eventHandle3.release(true);
await().atMost(TEST_TIMEOUT.multipliedBy(5))
.untilAsserted(() -> {
assertThat(currentRatio, equalTo(0.75));
});
- acknowledgementSetManager.releaseEventReference(eventHandle4, true);
+ eventHandle4.release(true);
await().atMost(TEST_TIMEOUT.multipliedBy(5))
.untilAsserted(() -> {
assertThat(currentRatio, equalTo(0.5));
});
- acknowledgementSetManager.releaseEventReference(eventHandle5, true);
+ eventHandle5.release(true);
await().atMost(TEST_TIMEOUT.multipliedBy(5))
.untilAsserted(() -> {
assertThat(currentRatio, equalTo(0.25));
});
- acknowledgementSetManager.releaseEventReference(eventHandle6, true);
+ eventHandle6.release(true);
await().atMost(TEST_TIMEOUT.multipliedBy(5))
.untilAsserted(() -> {
assertThat(result, equalTo(true));
@@ -170,14 +207,30 @@ void testWithProgressCheckCallbacks() {
@Test
void testWithProgressCheckCallbacks_AcksExpire() {
+ AcknowledgementSet acknowledgementSet2 = acknowledgementSetManager.create((flag) -> { result = flag; }, Duration.ofSeconds(10));
eventHandle3 = mock(DefaultEventHandle.class);
+ doAnswer(a -> {
+ Boolean result = (Boolean)a.getArgument(0);
+ acknowledgementSet2.release(eventHandle3, result);
+ return null;
+ }).when(eventHandle3).release(any(Boolean.class));
lenient().when(event3.getEventHandle()).thenReturn(eventHandle3);
eventHandle4 = mock(DefaultEventHandle.class);
+ doAnswer(a -> {
+ Boolean result = (Boolean)a.getArgument(0);
+ acknowledgementSet2.release(eventHandle4, result);
+ return null;
+ }).when(eventHandle4).release(any(Boolean.class));
JacksonEvent event4 = mock(JacksonEvent.class);
lenient().when(event4.getEventHandle()).thenReturn(eventHandle4);
eventHandle5 = mock(DefaultEventHandle.class);
+ doAnswer(a -> {
+ Boolean result = (Boolean)a.getArgument(0);
+ acknowledgementSet2.release(eventHandle5, result);
+ return null;
+ }).when(eventHandle5).release(any(Boolean.class));
JacksonEvent event5 = mock(JacksonEvent.class);
lenient().when(event5.getEventHandle()).thenReturn(eventHandle5);
@@ -185,7 +238,6 @@ void testWithProgressCheckCallbacks_AcksExpire() {
JacksonEvent event6 = mock(JacksonEvent.class);
lenient().when(event6.getEventHandle()).thenReturn(eventHandle6);
- AcknowledgementSet acknowledgementSet2 = acknowledgementSetManager.create((flag) -> { result = flag; }, Duration.ofSeconds(10));
acknowledgementSet2.addProgressCheck((progressCheck) -> {currentRatio = progressCheck.getRatio();}, Duration.ofSeconds(1));
acknowledgementSet2.add(event3);
acknowledgementSet2.add(event4);
@@ -196,17 +248,17 @@ void testWithProgressCheckCallbacks_AcksExpire() {
lenient().when(eventHandle5.getAcknowledgementSet()).thenReturn(acknowledgementSet2);
lenient().when(eventHandle6.getAcknowledgementSet()).thenReturn(acknowledgementSet2);
acknowledgementSet2.complete();
- acknowledgementSetManager.releaseEventReference(eventHandle3, true);
+ eventHandle3.release(true);
await().atMost(TEST_TIMEOUT.multipliedBy(5))
.untilAsserted(() -> {
assertThat(currentRatio, equalTo(0.75));
});
- acknowledgementSetManager.releaseEventReference(eventHandle4, true);
+ eventHandle4.release(true);
await().atMost(TEST_TIMEOUT.multipliedBy(5))
.untilAsserted(() -> {
assertThat(currentRatio, equalTo(0.5));
});
- acknowledgementSetManager.releaseEventReference(eventHandle5, true);
+ eventHandle5.release(true);
await().atMost(TEST_TIMEOUT.multipliedBy(5))
.untilAsserted(() -> {
assertThat(currentRatio, equalTo(0.25));
diff --git a/data-prepper-core/src/test/java/org/opensearch/dataprepper/acknowledgements/DefaultAcknowledgementSetTests.java b/data-prepper-core/src/test/java/org/opensearch/dataprepper/acknowledgements/DefaultAcknowledgementSetTests.java
index 28e17d77cc..a3ee665adf 100644
--- a/data-prepper-core/src/test/java/org/opensearch/dataprepper/acknowledgements/DefaultAcknowledgementSetTests.java
+++ b/data-prepper-core/src/test/java/org/opensearch/dataprepper/acknowledgements/DefaultAcknowledgementSetTests.java
@@ -91,14 +91,14 @@ void setupEvent() {
AcknowledgementSet acknowledgementSet = a.getArgument(0);
lenient().when(handle.getAcknowledgementSet()).thenReturn(acknowledgementSet);
return null;
- }).when(handle).setAcknowledgementSet(any(AcknowledgementSet.class));
+ }).when(handle).addAcknowledgementSet(any(AcknowledgementSet.class));
lenient().when(event.getEventHandle()).thenReturn(handle);
event2 = mock(JacksonEvent.class);
lenient().doAnswer(a -> {
AcknowledgementSet acknowledgementSet = a.getArgument(0);
lenient().when(handle2.getAcknowledgementSet()).thenReturn(acknowledgementSet);
return null;
- }).when(handle2).setAcknowledgementSet(any(AcknowledgementSet.class));
+ }).when(handle2).addAcknowledgementSet(any(AcknowledgementSet.class));
handle2 = mock(DefaultEventHandle.class);
lenient().when(event2.getEventHandle()).thenReturn(handle2);
}
@@ -186,7 +186,7 @@ void testDefaultAcknowledgementSetNegativeAcknowledgements() throws Exception {
AcknowledgementSet acknowledgementSet = a.getArgument(0);
lenient().when(handle.getAcknowledgementSet()).thenReturn(acknowledgementSet);
return null;
- }).when(handle).setAcknowledgementSet(any(AcknowledgementSet.class));
+ }).when(handle).addAcknowledgementSet(any(AcknowledgementSet.class));
assertThat(handle.getAcknowledgementSet(), equalTo(defaultAcknowledgementSet));
defaultAcknowledgementSet.acquire(handle);
assertThat(defaultAcknowledgementSet.release(handle, true), equalTo(false));
@@ -219,7 +219,7 @@ void testDefaultAcknowledgementSetExpirations() throws Exception {
AcknowledgementSet acknowledgementSet = a.getArgument(0);
lenient().when(handle.getAcknowledgementSet()).thenReturn(acknowledgementSet);
return null;
- }).when(handle).setAcknowledgementSet(any(AcknowledgementSet.class));
+ }).when(handle).addAcknowledgementSet(any(AcknowledgementSet.class));
assertThat(handle, not(equalTo(null)));
assertThat(handle.getAcknowledgementSet(), equalTo(defaultAcknowledgementSet));
assertThat(defaultAcknowledgementSet.release(handle, true), equalTo(true));
@@ -253,7 +253,7 @@ void testDefaultAcknowledgementSetWithProgressCheck() throws Exception {
AcknowledgementSet acknowledgementSet = a.getArgument(0);
lenient().when(handle.getAcknowledgementSet()).thenReturn(acknowledgementSet);
return null;
- }).when(handle).setAcknowledgementSet(any(AcknowledgementSet.class));
+ }).when(handle).addAcknowledgementSet(any(AcknowledgementSet.class));
assertThat(handle, not(equalTo(null)));
assertThat(handle.getAcknowledgementSet(), equalTo(defaultAcknowledgementSet));
await().atMost(Duration.ofSeconds(5))
diff --git a/data-prepper-core/src/test/java/org/opensearch/dataprepper/acknowledgements/InactiveAcknowledgementSetManagerTests.java b/data-prepper-core/src/test/java/org/opensearch/dataprepper/acknowledgements/InactiveAcknowledgementSetManagerTests.java
index eb1303d487..8a0a4d2ffd 100644
--- a/data-prepper-core/src/test/java/org/opensearch/dataprepper/acknowledgements/InactiveAcknowledgementSetManagerTests.java
+++ b/data-prepper-core/src/test/java/org/opensearch/dataprepper/acknowledgements/InactiveAcknowledgementSetManagerTests.java
@@ -7,12 +7,9 @@
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
-import static org.mockito.Mockito.mock;
import static org.hamcrest.MatcherAssert.assertThat;
import static org.hamcrest.CoreMatchers.notNullValue;
import static org.junit.jupiter.api.Assertions.assertThrows;
-import org.opensearch.dataprepper.model.event.EventHandle;
-import org.opensearch.dataprepper.model.event.Event;
import java.time.Duration;
@@ -30,25 +27,4 @@ void testCreateAPI() {
assertThrows(UnsupportedOperationException.class, () -> acknowledgementSetManager.create((a)->{}, Duration.ofMillis(10)));
}
- @Test
- void testEventAcquireAPI() {
- assertThat(acknowledgementSetManager, notNullValue());
- Event event = mock(Event.class);
- assertThrows(UnsupportedOperationException.class, () -> acknowledgementSetManager.acquireEventReference(event));
- }
-
- @Test
- void testEventHandleAcquireAPI() {
- assertThat(acknowledgementSetManager, notNullValue());
- EventHandle eventHandle = mock(EventHandle.class);
- assertThrows(UnsupportedOperationException.class, () -> acknowledgementSetManager.acquireEventReference(eventHandle));
- }
-
- @Test
- void testReleaseAPI() {
- assertThat(acknowledgementSetManager, notNullValue());
- EventHandle eventHandle = mock(EventHandle.class);
- assertThrows(UnsupportedOperationException.class, () -> acknowledgementSetManager.releaseEventReference(eventHandle, true));
- }
-
}
diff --git a/data-prepper-core/src/test/java/org/opensearch/dataprepper/peerforwarder/PeerForwarderProviderTest.java b/data-prepper-core/src/test/java/org/opensearch/dataprepper/peerforwarder/DefaultPeerForwarderProviderTest.java
similarity index 82%
rename from data-prepper-core/src/test/java/org/opensearch/dataprepper/peerforwarder/PeerForwarderProviderTest.java
rename to data-prepper-core/src/test/java/org/opensearch/dataprepper/peerforwarder/DefaultPeerForwarderProviderTest.java
index 08964d3a80..4c1c36482c 100644
--- a/data-prepper-core/src/test/java/org/opensearch/dataprepper/peerforwarder/PeerForwarderProviderTest.java
+++ b/data-prepper-core/src/test/java/org/opensearch/dataprepper/peerforwarder/DefaultPeerForwarderProviderTest.java
@@ -13,6 +13,7 @@
import org.junit.jupiter.api.extension.ExtendWith;
import org.mockito.Mock;
import org.mockito.junit.jupiter.MockitoExtension;
+import org.opensearch.dataprepper.model.processor.Processor;
import org.opensearch.dataprepper.peerforwarder.client.PeerForwarderClient;
import org.opensearch.dataprepper.peerforwarder.discovery.DiscoveryMode;
@@ -35,7 +36,7 @@
import static org.mockito.Mockito.when;
@ExtendWith(MockitoExtension.class)
-class PeerForwarderProviderTest {
+class DefaultPeerForwarderProviderTest {
private static final int PIPELINE_WORKER_THREADS = new Random().nextInt(10) + 1;
@Mock
@@ -50,6 +51,9 @@ class PeerForwarderProviderTest {
@Mock
private HashRing hashRing;
+ @Mock
+ private Processor processor;
+
@Mock
private PluginMetrics pluginMetrics;
@@ -71,13 +75,13 @@ void setUp() {
}
private PeerForwarderProvider createObjectUnderTest() {
- return new PeerForwarderProvider(peerForwarderClientFactory, peerForwarderClient, peerForwarderConfiguration, pluginMetrics);
+ return new DefaultPeerForwarderProvider(peerForwarderClientFactory, peerForwarderClient, peerForwarderConfiguration, pluginMetrics);
}
@Test
void register_creates_a_new_RemotePeerForwarder_with_cloud_map_discovery_mode() {
when(peerForwarderConfiguration.getDiscoveryMode()).thenReturn(DiscoveryMode.AWS_CLOUD_MAP);
- final PeerForwarder peerForwarder = createObjectUnderTest().register(pipelineName, pluginId, identificationKeys, PIPELINE_WORKER_THREADS);
+ final PeerForwarder peerForwarder = createObjectUnderTest().register(pipelineName, processor, pluginId, identificationKeys, PIPELINE_WORKER_THREADS);
assertThat(peerForwarder, instanceOf(RemotePeerForwarder.class));
}
@@ -86,7 +90,7 @@ void register_creates_a_new_RemotePeerForwarder_with_cloud_map_discovery_mode()
void register_creates_a_new_RemotePeerForwarder_with_static_discovery_mode_of_size_grater_than_one() {
when(peerForwarderConfiguration.getDiscoveryMode()).thenReturn(DiscoveryMode.STATIC);
when(peerForwarderConfiguration.getStaticEndpoints()).thenReturn(List.of("endpoint1", "endpoint2"));
- final PeerForwarder peerForwarder = createObjectUnderTest().register(pipelineName, pluginId, identificationKeys, PIPELINE_WORKER_THREADS);
+ final PeerForwarder peerForwarder = createObjectUnderTest().register(pipelineName, processor, pluginId, identificationKeys, PIPELINE_WORKER_THREADS);
assertThat(peerForwarder, instanceOf(RemotePeerForwarder.class));
}
@@ -95,14 +99,14 @@ void register_creates_a_new_RemotePeerForwarder_with_static_discovery_mode_of_si
void register_creates_a_new_RemotePeerForwarder_with_static_discovery_mode_of_size_one() {
when(peerForwarderConfiguration.getDiscoveryMode()).thenReturn(DiscoveryMode.STATIC);
when(peerForwarderConfiguration.getStaticEndpoints()).thenReturn(List.of("endpoint1"));
- final PeerForwarder peerForwarder = createObjectUnderTest().register(pipelineName, pluginId, identificationKeys, PIPELINE_WORKER_THREADS);
+ final PeerForwarder peerForwarder = createObjectUnderTest().register(pipelineName, processor, pluginId, identificationKeys, PIPELINE_WORKER_THREADS);
assertThat(peerForwarder, instanceOf(LocalPeerForwarder.class));
}
@Test
void register_creates_a_new_LocalPeerForwarder_with_local_discovery_mode() {
- final PeerForwarder peerForwarder = createObjectUnderTest().register(pipelineName, pluginId, identificationKeys, PIPELINE_WORKER_THREADS);
+ final PeerForwarder peerForwarder = createObjectUnderTest().register(pipelineName, processor, pluginId, identificationKeys, PIPELINE_WORKER_THREADS);
assertThat(peerForwarder, instanceOf(LocalPeerForwarder.class));
}
@@ -110,7 +114,7 @@ void register_creates_a_new_LocalPeerForwarder_with_local_discovery_mode() {
@Test
void register_creates_HashRing_if_peer_forwarding_is_required() {
when(peerForwarderConfiguration.getDiscoveryMode()).thenReturn(DiscoveryMode.AWS_CLOUD_MAP);
- createObjectUnderTest().register(pipelineName, pluginId, identificationKeys, PIPELINE_WORKER_THREADS);
+ createObjectUnderTest().register(pipelineName, processor, pluginId, identificationKeys, PIPELINE_WORKER_THREADS);
verify(peerForwarderClientFactory).createHashRing();
}
@@ -121,7 +125,7 @@ void register_called_multiple_times_creates_only_one_HashRing_if_peer_forwarding
final PeerForwarderProvider objectUnderTest = createObjectUnderTest();
for (int i = 0; i < 10; i++)
- objectUnderTest.register(pipelineName, UUID.randomUUID().toString(), identificationKeys, PIPELINE_WORKER_THREADS);
+ objectUnderTest.register(pipelineName, processor, UUID.randomUUID().toString(), identificationKeys, PIPELINE_WORKER_THREADS);
verify(peerForwarderClientFactory, times(1)).createHashRing();
}
@@ -137,17 +141,17 @@ void isAtLeastOnePeerForwarderRegistered_should_return_false_if_register_is_not_
void isAtLeastOnePeerForwarderRegistered_should_throw_when_register_is_called_with_same_pipeline_and_plugin() {
final PeerForwarderProvider objectUnderTest = createObjectUnderTest();
- objectUnderTest.register(pipelineName, pluginId, identificationKeys, PIPELINE_WORKER_THREADS);
+ objectUnderTest.register(pipelineName, processor, pluginId, identificationKeys, PIPELINE_WORKER_THREADS);
assertThrows(RuntimeException.class, () ->
- objectUnderTest.register(pipelineName, pluginId, identificationKeys, PIPELINE_WORKER_THREADS));
+ objectUnderTest.register(pipelineName, processor, pluginId, identificationKeys, PIPELINE_WORKER_THREADS));
}
@Test
void isAtLeastOnePeerForwarderRegistered_should_return_false_if_register_is_called_with_local_discovery_mode() {
final PeerForwarderProvider objectUnderTest = createObjectUnderTest();
- objectUnderTest.register(pipelineName, pluginId, identificationKeys, PIPELINE_WORKER_THREADS);
+ objectUnderTest.register(pipelineName, processor, pluginId, identificationKeys, PIPELINE_WORKER_THREADS);
assertThat(objectUnderTest.isPeerForwardingRequired(), equalTo(false));
}
@@ -157,7 +161,7 @@ void isAtLeastOnePeerForwarderRegistered_should_return_true_if_register_is_calle
when(peerForwarderConfiguration.getDiscoveryMode()).thenReturn(DiscoveryMode.AWS_CLOUD_MAP);
final PeerForwarderProvider objectUnderTest = createObjectUnderTest();
- objectUnderTest.register(pipelineName, pluginId, identificationKeys, PIPELINE_WORKER_THREADS);
+ objectUnderTest.register(pipelineName, processor, pluginId, identificationKeys, PIPELINE_WORKER_THREADS);
assertThat(objectUnderTest.isPeerForwardingRequired(), equalTo(true));
}
@@ -179,7 +183,7 @@ void getPipelinePeerForwarderReceiveBufferMap_should_return_empty_map_when_regis
void getPipelinePeerForwarderReceiveBufferMap_should_return_non_empty_map_when_register_is_called() {
final PeerForwarderProvider objectUnderTest = createObjectUnderTest();
- objectUnderTest.register(pipelineName, UUID.randomUUID().toString(), identificationKeys, PIPELINE_WORKER_THREADS);
+ objectUnderTest.register(pipelineName, processor, UUID.randomUUID().toString(), identificationKeys, PIPELINE_WORKER_THREADS);
final Map>>> pipelinePeerForwarderReceiveBufferMap = objectUnderTest
.getPipelinePeerForwarderReceiveBufferMap();
@@ -189,4 +193,4 @@ void getPipelinePeerForwarderReceiveBufferMap_should_return_non_empty_map_when_r
assertThat(pipelinePeerForwarderReceiveBufferMap.size(), equalTo(1));
assertThat(pipelinePeerForwarderReceiveBufferMap.containsKey(pipelineName), equalTo(true));
}
-}
\ No newline at end of file
+}
diff --git a/data-prepper-core/src/test/java/org/opensearch/dataprepper/peerforwarder/PeerForwarder_ClientServerIT.java b/data-prepper-core/src/test/java/org/opensearch/dataprepper/peerforwarder/PeerForwarder_ClientServerIT.java
index 2b4d875e45..f706cb97d7 100644
--- a/data-prepper-core/src/test/java/org/opensearch/dataprepper/peerforwarder/PeerForwarder_ClientServerIT.java
+++ b/data-prepper-core/src/test/java/org/opensearch/dataprepper/peerforwarder/PeerForwarder_ClientServerIT.java
@@ -22,6 +22,7 @@
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.model.processor.Processor;
import org.opensearch.dataprepper.peerforwarder.certificate.CertificateProviderFactory;
import org.opensearch.dataprepper.peerforwarder.client.PeerForwarderClient;
import org.opensearch.dataprepper.peerforwarder.codec.PeerForwarderCodecAppConfig;
@@ -133,7 +134,7 @@ private PeerForwarderProvider createPeerForwarderProvider(
final PeerForwarderClient clientForProvider = createClient(peerForwarderConfiguration);
final PeerClientPool peerClientPool = new PeerClientPool();
final PeerForwarderClientFactory clientFactoryForProvider = new PeerForwarderClientFactory(peerForwarderConfiguration, peerClientPool, certificateProviderFactory, pluginMetrics);
- return new PeerForwarderProvider(clientFactoryForProvider, clientForProvider, peerForwarderConfiguration, pluginMetrics);
+ return new DefaultPeerForwarderProvider(clientFactoryForProvider, clientForProvider, peerForwarderConfiguration, pluginMetrics);
}
private PeerForwarderClient createClient(
@@ -160,6 +161,7 @@ private Collection> getServerSideRecords(final PeerForwarderProvid
class WithSSL {
private PeerForwarderServer server;
private PeerForwarderProvider peerForwarderProvider;
+ private Processor processor;
void setUpServer(final boolean binaryCodec) {
peerForwarderConfiguration = createConfiguration(true, ForwardingAuthentication.UNAUTHENTICATED, binaryCodec);
@@ -168,7 +170,7 @@ void setUpServer(final boolean binaryCodec) {
final CertificateProviderFactory certificateProviderFactory = new CertificateProviderFactory(peerForwarderConfiguration);
peerForwarderProvider = createPeerForwarderProvider(peerForwarderConfiguration, certificateProviderFactory);
- peerForwarderProvider.register(pipelineName, pluginId, Collections.singleton(UUID.randomUUID().toString()), PIPELINE_WORKER_THREADS);
+ peerForwarderProvider.register(pipelineName, processor, pluginId, Collections.singleton(UUID.randomUUID().toString()), PIPELINE_WORKER_THREADS);
server = createServer(peerForwarderConfiguration, certificateProviderFactory, peerForwarderProvider);
server.start();
}
@@ -280,6 +282,7 @@ void send_Events_with_fingerprint_verification_to_unknown_server_should_throw(fi
class WithoutSSL {
private PeerForwarderServer server;
private PeerForwarderProvider peerForwarderProvider;
+ private Processor processor;
void setUpServer(final boolean binaryCodec) {
peerForwarderConfiguration = createConfiguration(false, ForwardingAuthentication.UNAUTHENTICATED, binaryCodec);
@@ -288,7 +291,7 @@ void setUpServer(final boolean binaryCodec) {
final CertificateProviderFactory certificateProviderFactory = new CertificateProviderFactory(peerForwarderConfiguration);
peerForwarderProvider = createPeerForwarderProvider(peerForwarderConfiguration, certificateProviderFactory);
- peerForwarderProvider.register(pipelineName, pluginId, Collections.singleton(UUID.randomUUID().toString()), PIPELINE_WORKER_THREADS);
+ peerForwarderProvider.register(pipelineName, processor, pluginId, Collections.singleton(UUID.randomUUID().toString()), PIPELINE_WORKER_THREADS);
server = createServer(peerForwarderConfiguration, certificateProviderFactory, peerForwarderProvider);
server.start();
}
@@ -339,6 +342,7 @@ void send_Events_to_server_when_expecting_SSL_should_throw(final boolean binaryC
class WithMutualTls {
private PeerForwarderServer server;
private PeerForwarderProvider peerForwarderProvider;
+ private Processor processor;
void setUpServer(final boolean binaryCodec) {
peerForwarderConfiguration = createConfiguration(true, ForwardingAuthentication.MUTUAL_TLS, binaryCodec);
@@ -347,7 +351,7 @@ void setUpServer(final boolean binaryCodec) {
final CertificateProviderFactory certificateProviderFactory = new CertificateProviderFactory(peerForwarderConfiguration);
peerForwarderProvider = createPeerForwarderProvider(peerForwarderConfiguration, certificateProviderFactory);
- peerForwarderProvider.register(pipelineName, pluginId, Collections.singleton(UUID.randomUUID().toString()), PIPELINE_WORKER_THREADS);
+ peerForwarderProvider.register(pipelineName, processor, pluginId, Collections.singleton(UUID.randomUUID().toString()), PIPELINE_WORKER_THREADS);
server = createServer(peerForwarderConfiguration, certificateProviderFactory, peerForwarderProvider);
server.start();
}
diff --git a/data-prepper-core/src/test/java/org/opensearch/dataprepper/peerforwarder/PeerForwardingProcessingDecoratorTest.java b/data-prepper-core/src/test/java/org/opensearch/dataprepper/peerforwarder/PeerForwardingProcessingDecoratorTest.java
index d0c71a52d0..7a85033842 100644
--- a/data-prepper-core/src/test/java/org/opensearch/dataprepper/peerforwarder/PeerForwardingProcessingDecoratorTest.java
+++ b/data-prepper-core/src/test/java/org/opensearch/dataprepper/peerforwarder/PeerForwardingProcessingDecoratorTest.java
@@ -17,6 +17,8 @@
import org.apache.commons.collections.CollectionUtils;
import org.mockito.Mock;
import org.mockito.junit.jupiter.MockitoExtension;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.lenient;
import org.opensearch.dataprepper.peerforwarder.exception.EmptyPeerForwarderPluginIdentificationKeysException;
import org.opensearch.dataprepper.peerforwarder.exception.UnsupportedPeerForwarderPluginException;
@@ -37,6 +39,7 @@
import static org.mockito.Mockito.times;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.verifyNoInteractions;
+import static org.mockito.Mockito.verifyNoMoreInteractions;
import static org.mockito.Mockito.when;
@ExtendWith(MockitoExtension.class)
@@ -68,13 +71,13 @@ record = mock(Record.class);
pluginId = UUID.randomUUID().toString();
}
- private List createObjectUnderTesDecoratedProcessors(final List processors) {
+ private List createObjectUnderTestDecoratedProcessors(final List processors) {
return PeerForwardingProcessorDecorator.decorateProcessors(processors, peerForwarderProvider, pipelineName, pluginId, PIPELINE_WORKER_THREADS);
}
@Test
void PeerForwardingProcessingDecorator_should_not_have_any_interactions_if_its_not_an_instance_of_RequiresPeerForwarding() {
- assertThrows(UnsupportedPeerForwarderPluginException.class, () -> createObjectUnderTesDecoratedProcessors(Collections.singletonList(processor)));
+ assertThrows(UnsupportedPeerForwarderPluginException.class, () -> createObjectUnderTestDecoratedProcessors(Collections.singletonList(processor)));
verifyNoInteractions(peerForwarderProvider);
}
@@ -83,7 +86,7 @@ void PeerForwardingProcessingDecorator_should_not_have_any_interactions_if_its_n
void PeerForwardingProcessingDecorator_execute_with_empty_identification_keys_should_throw() {
when(requiresPeerForwarding.getIdentificationKeys()).thenReturn(Collections.emptySet());
- assertThrows(EmptyPeerForwarderPluginIdentificationKeysException.class, () -> createObjectUnderTesDecoratedProcessors(Collections.singletonList((Processor) requiresPeerForwarding)));
+ assertThrows(EmptyPeerForwarderPluginIdentificationKeysException.class, () -> createObjectUnderTestDecoratedProcessors(Collections.singletonList((Processor) requiresPeerForwarding)));
}
@Test
@@ -95,12 +98,12 @@ void decorateProcessors_with_different_identification_key_should_throw() {
when(requiresPeerForwarding.getIdentificationKeys()).thenReturn(Set.of(UUID.randomUUID().toString()));
when(requiresPeerForwardingCopy.getIdentificationKeys()).thenReturn(Set.of(UUID.randomUUID().toString()));
- assertThrows(RuntimeException.class, () -> createObjectUnderTesDecoratedProcessors(List.of(((Processor) requiresPeerForwarding), (Processor) requiresPeerForwardingCopy)));
+ assertThrows(RuntimeException.class, () -> createObjectUnderTestDecoratedProcessors(List.of(((Processor) requiresPeerForwarding), (Processor) requiresPeerForwardingCopy)));
}
@Test
void decorateProcessors_with_empty_processors_should_return_empty_list_of_processors() {
- final List processors = createObjectUnderTesDecoratedProcessors(Collections.emptyList());
+ final List processors = createObjectUnderTestDecoratedProcessors(Collections.emptyList());
assertThat(processors.size(), equalTo(0));
}
@@ -115,16 +118,66 @@ class WithRegisteredPeerForwarder {
void setUp() {
identificationKeys = Set.of(TEST_IDENTIFICATION_KEY);
- when(peerForwarderProvider.register(pipelineName, pluginId, identificationKeys, PIPELINE_WORKER_THREADS)).thenReturn(peerForwarder);
- when(requiresPeerForwarding.getIdentificationKeys()).thenReturn(identificationKeys);
processor = (Processor) requiresPeerForwarding;
+ lenient().when(peerForwarderProvider.register(pipelineName, processor, pluginId, identificationKeys, PIPELINE_WORKER_THREADS)).thenReturn(peerForwarder);
+ when(requiresPeerForwarding.getIdentificationKeys()).thenReturn(identificationKeys);
}
@Test
void PeerForwardingProcessingDecorator_should_have_interaction_with_getIdentificationKeys() {
- createObjectUnderTesDecoratedProcessors(Collections.singletonList(processor));
+ createObjectUnderTestDecoratedProcessors(Collections.singletonList(processor));
+ verify(requiresPeerForwarding, times(2)).getIdentificationKeys();
+ verify(peerForwarderProvider).register(pipelineName, processor, pluginId, identificationKeys, PIPELINE_WORKER_THREADS);
+ verifyNoMoreInteractions(peerForwarderProvider);
+ }
+
+ @Test
+ void PeerForwardingProcessingDecorator_should_have_interaction_with_getIdentificationKeys_when_list_of_processors() {
+ when(requiresPeerForwarding.getIdentificationKeys()).thenReturn(identificationKeys);
+ when(requiresPeerForwardingCopy.getIdentificationKeys()).thenReturn(identificationKeys);
+
+ createObjectUnderTestDecoratedProcessors(List.of((Processor) requiresPeerForwarding, (Processor) requiresPeerForwardingCopy));
+
verify(requiresPeerForwarding, times(2)).getIdentificationKeys();
- verify(peerForwarderProvider).register(pipelineName, pluginId, identificationKeys, PIPELINE_WORKER_THREADS);
+ verify(peerForwarderProvider).register(pipelineName, processor, pluginId, identificationKeys, PIPELINE_WORKER_THREADS);
+ verifyNoMoreInteractions(peerForwarderProvider);
+ }
+
+ @Test
+ void PeerForwardingProcessingDecorator_with_localProcessingOnly() {
+ List processorList = new ArrayList<>();
+ processorList.add((Processor) requiresPeerForwarding);
+ processorList.add((Processor) requiresPeerForwardingCopy);
+
+ LocalPeerForwarder localPeerForwarder = mock(LocalPeerForwarder.class);
+ when(peerForwarderProvider.register(pipelineName, (Processor) requiresPeerForwarding, pluginId, identificationKeys, PIPELINE_WORKER_THREADS)).thenReturn(localPeerForwarder);
+ Event event = mock(Event.class);
+ when(record.getData()).thenReturn(event);
+ List> testData = Collections.singletonList(record);
+ when(requiresPeerForwarding.isApplicableEventForPeerForwarding(event)).thenReturn(false);
+ when(requiresPeerForwardingCopy.isApplicableEventForPeerForwarding(event)).thenReturn(false);
+
+ Processor processor1 = (Processor)requiresPeerForwarding;
+ Processor processor2 = (Processor)requiresPeerForwardingCopy;
+ when(processor1.execute(testData)).thenReturn(testData);
+ when(processor2.execute(testData)).thenReturn(testData);
+
+ when(requiresPeerForwarding.getIdentificationKeys()).thenReturn(identificationKeys);
+ when(requiresPeerForwardingCopy.getIdentificationKeys()).thenReturn(identificationKeys);
+
+ when(requiresPeerForwarding.isForLocalProcessingOnly(any())).thenReturn(true);
+ when(requiresPeerForwardingCopy.isForLocalProcessingOnly(any())).thenReturn(true);
+
+ final List processors = createObjectUnderTestDecoratedProcessors(processorList);
+ assertThat(processors.size(), equalTo(2));
+ verify(peerForwarderProvider, times(1)).register(pipelineName, processor, pluginId, identificationKeys, PIPELINE_WORKER_THREADS);
+ verifyNoMoreInteractions(peerForwarderProvider);
+ Collection> result = processors.get(0).execute(testData);
+ assertThat(result.size(), equalTo(testData.size()));
+ assertThat(result, equalTo(testData));
+ result = processors.get(1).execute(testData);
+ assertThat(result.size(), equalTo(testData.size()));
+ assertThat(result, equalTo(testData));
}
@Test
@@ -138,7 +191,7 @@ void PeerForwardingProcessingDecorator_execute_should_forwardRecords_with_correc
when(processor.execute(testData)).thenReturn(testData);
- final List processors = createObjectUnderTesDecoratedProcessors(Collections.singletonList(processor));
+ final List processors = createObjectUnderTestDecoratedProcessors(Collections.singletonList(processor));
assertThat(processors.size(), equalTo(1));
final Collection> records = processors.get(0).execute(testData);
@@ -164,7 +217,7 @@ void PeerForwardingProcessingDecorator_execute_should_receiveRecords() {
when(((Processor) requiresPeerForwarding).execute(anyCollection())).thenReturn(expectedRecordsToProcessLocally);
- final List processors = createObjectUnderTesDecoratedProcessors(Collections.singletonList((Processor) requiresPeerForwarding));
+ final List processors = createObjectUnderTestDecoratedProcessors(Collections.singletonList((Processor) requiresPeerForwarding));
assertThat(processors.size(), equalTo(1));
final Collection> records = processors.get(0).execute(forwardTestData);
@@ -181,7 +234,7 @@ void PeerForwardingProcessingDecorator_execute_will_call_inner_processors_execut
Event event = mock(Event.class);
when(record.getData()).thenReturn(event);
when(requiresPeerForwarding.isApplicableEventForPeerForwarding(event)).thenReturn(true);
- final List processors = createObjectUnderTesDecoratedProcessors(Collections.singletonList(processor));
+ final List processors = createObjectUnderTestDecoratedProcessors(Collections.singletonList(processor));
Collection> testData = Collections.singletonList(record);
assertThat(processors.size(), equalTo(1));
@@ -195,9 +248,9 @@ void PeerForwardingProcessingDecorator_execute_will_call_inner_processors_execut
Event event = mock(Event.class);
when(record.getData()).thenReturn(event);
when(requiresPeerForwarding.isApplicableEventForPeerForwarding(event)).thenReturn(false);
- when(requiresPeerForwarding.isForLocalProcessingOnly(event)).thenReturn(true);
+ when(requiresPeerForwarding.isForLocalProcessingOnly(any())).thenReturn(true);
- final List processors = createObjectUnderTesDecoratedProcessors(Collections.singletonList(processor));
+ final List processors = createObjectUnderTestDecoratedProcessors(Collections.singletonList(processor));
Collection> testData = Collections.singletonList(record);
assertThat(processors.size(), equalTo(1));
@@ -220,10 +273,8 @@ void PeerForwardingProcessingDecorator_inner_processor_with_is_applicable_event_
when(requiresPeerForwarding.isApplicableEventForPeerForwarding(event1)).thenReturn(false);
when(requiresPeerForwarding.isApplicableEventForPeerForwarding(event2)).thenReturn(false);
when(requiresPeerForwarding.isApplicableEventForPeerForwarding(event3)).thenReturn(false);
- when(requiresPeerForwarding.isForLocalProcessingOnly(event1)).thenReturn(true);
- when(requiresPeerForwarding.isForLocalProcessingOnly(event2)).thenReturn(true);
- when(requiresPeerForwarding.isForLocalProcessingOnly(event3)).thenReturn(true);
- final List processors = createObjectUnderTesDecoratedProcessors(Collections.singletonList(processor));
+ when(requiresPeerForwarding.isForLocalProcessingOnly(any())).thenReturn(true);
+ final List processors = createObjectUnderTestDecoratedProcessors(Collections.singletonList(processor));
when(record1.getData()).thenReturn(event1);
when(record2.getData()).thenReturn(event2);
when(record3.getData()).thenReturn(event3);
@@ -253,8 +304,8 @@ void PeerForwardingProcessingDecorator_inner_processor_with_is_applicable_event_
when(requiresPeerForwarding.isApplicableEventForPeerForwarding(event1)).thenReturn(true);
when(requiresPeerForwarding.isApplicableEventForPeerForwarding(event2)).thenReturn(false);
when(requiresPeerForwarding.isApplicableEventForPeerForwarding(event3)).thenReturn(true);
- when(requiresPeerForwarding.isForLocalProcessingOnly(event2)).thenReturn(false);
- final List processors = createObjectUnderTesDecoratedProcessors(Collections.singletonList(processor));
+ when(requiresPeerForwarding.isForLocalProcessingOnly(any())).thenReturn(false);
+ final List processors = createObjectUnderTestDecoratedProcessors(Collections.singletonList(processor));
when(record1.getData()).thenReturn(event1);
when(record2.getData()).thenReturn(event2);
when(record3.getData()).thenReturn(event3);
@@ -273,7 +324,7 @@ void PeerForwardingProcessingDecorator_inner_processor_with_is_applicable_event_
@Test
void PeerForwardingProcessingDecorator_prepareForShutdown_will_call_inner_processors_prepareForShutdown() {
- final List processors = createObjectUnderTesDecoratedProcessors(Collections.singletonList(processor));
+ final List processors = createObjectUnderTestDecoratedProcessors(Collections.singletonList(processor));
assertThat(processors.size(), equalTo(1));
processors.get(0).prepareForShutdown();
@@ -282,7 +333,7 @@ void PeerForwardingProcessingDecorator_prepareForShutdown_will_call_inner_proces
@Test
void PeerForwardingProcessingDecorator_isReadyForShutdown_will_call_inner_processors_isReadyForShutdown() {
- final List processors = createObjectUnderTesDecoratedProcessors(Collections.singletonList(processor));
+ final List processors = createObjectUnderTestDecoratedProcessors(Collections.singletonList(processor));
assertThat(processors.size(), equalTo(1));
processors.get(0).isReadyForShutdown();
@@ -291,7 +342,7 @@ void PeerForwardingProcessingDecorator_isReadyForShutdown_will_call_inner_proces
@Test
void PeerForwardingProcessingDecorator_shutdown_will_call_inner_processors_shutdown() {
- final List processors = createObjectUnderTesDecoratedProcessors(Collections.singletonList(processor));
+ final List processors = createObjectUnderTestDecoratedProcessors(Collections.singletonList(processor));
assertThat(processors.size(), equalTo(1));
processors.get(0).shutdown();
diff --git a/data-prepper-core/src/test/java/org/opensearch/dataprepper/peerforwarder/codec/JavaPeerForwarderCodecTest.java b/data-prepper-core/src/test/java/org/opensearch/dataprepper/peerforwarder/codec/JavaPeerForwarderCodecTest.java
index 70a1e737d8..bd0b26e05f 100644
--- a/data-prepper-core/src/test/java/org/opensearch/dataprepper/peerforwarder/codec/JavaPeerForwarderCodecTest.java
+++ b/data-prepper-core/src/test/java/org/opensearch/dataprepper/peerforwarder/codec/JavaPeerForwarderCodecTest.java
@@ -78,7 +78,7 @@ void testCodec_with_acknowledgementSet() throws IOException, ClassNotFoundExcept
inputEvents.getEvents().stream()
.map(Event::getEventHandle)
.map(handle -> (InternalEventHandle)handle)
- .forEach(handle -> handle.setAcknowledgementSet(mock(AcknowledgementSet.class)));
+ .forEach(handle -> handle.addAcknowledgementSet(mock(AcknowledgementSet.class)));
final byte[] bytes = createObjectUnderTest().serialize(inputEvents);
final PeerForwardingEvents outputEvents = createObjectUnderTest().deserialize(bytes);
assertThat(outputEvents.getDestinationPipelineName(), equalTo(inputEvents.getDestinationPipelineName()));
@@ -119,4 +119,4 @@ private PeerForwardingEvents generatePeerForwardingEvents(final int numEvents) {
}
return new PeerForwardingEvents(events, pluginId, pipelineName);
}
-}
\ No newline at end of file
+}
diff --git a/data-prepper-core/src/test/java/org/opensearch/dataprepper/peerforwarder/discovery/DnsPeerListProviderTest.java b/data-prepper-core/src/test/java/org/opensearch/dataprepper/peerforwarder/discovery/DnsPeerListProviderTest.java
index 1083eea9f0..3bdee15368 100644
--- a/data-prepper-core/src/test/java/org/opensearch/dataprepper/peerforwarder/discovery/DnsPeerListProviderTest.java
+++ b/data-prepper-core/src/test/java/org/opensearch/dataprepper/peerforwarder/discovery/DnsPeerListProviderTest.java
@@ -7,30 +7,33 @@
import com.linecorp.armeria.client.Endpoint;
import com.linecorp.armeria.client.endpoint.dns.DnsAddressEndpointGroup;
-import io.micrometer.core.instrument.Measurement;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.runner.RunWith;
+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.MockitoJUnitRunner;
-import org.opensearch.dataprepper.metrics.MetricNames;
-import org.opensearch.dataprepper.metrics.MetricsTestUtil;
+import org.mockito.junit.jupiter.MockitoExtension;
import org.opensearch.dataprepper.metrics.PluginMetrics;
import org.opensearch.dataprepper.peerforwarder.HashRing;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
-import java.util.StringJoiner;
import java.util.concurrent.CompletableFuture;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
+import java.util.function.ToDoubleFunction;
+
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.mockito.ArgumentMatchers.eq;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.when;
+import static org.opensearch.dataprepper.peerforwarder.discovery.PeerListProvider.PEER_ENDPOINTS;
-@RunWith(MockitoJUnitRunner.class)
+@ExtendWith(MockitoExtension.class)
public class DnsPeerListProviderTest {
private static final String ENDPOINT_1 = "10.1.1.1";
@@ -39,8 +42,6 @@ public class DnsPeerListProviderTest {
Endpoint.of(ENDPOINT_1),
Endpoint.of(ENDPOINT_2)
);
- private static final String COMPONENT_SCOPE = "testComponentScope";
- private static final String COMPONENT_ID = "testComponentId";
@Mock
private DnsAddressEndpointGroup dnsAddressEndpointGroup;
@@ -48,34 +49,33 @@ public class DnsPeerListProviderTest {
@Mock
private HashRing hashRing;
+ @Mock
private PluginMetrics pluginMetrics;
private CompletableFuture completableFuture;
private DnsPeerListProvider dnsPeerListProvider;
- @Before
+ @BeforeEach
public void setup() {
- MetricsTestUtil.initMetrics();
completableFuture = CompletableFuture.completedFuture(null);
when(dnsAddressEndpointGroup.whenReady()).thenReturn(completableFuture);
- pluginMetrics = PluginMetrics.fromNames(COMPONENT_ID, COMPONENT_SCOPE);
dnsPeerListProvider = new DnsPeerListProvider(dnsAddressEndpointGroup, pluginMetrics);
}
- @Test(expected = NullPointerException.class)
+ @Test
public void testDefaultListProviderWithNullHostname() {
- new DnsPeerListProvider(null, pluginMetrics);
+ assertThrows(NullPointerException.class, () -> new DnsPeerListProvider(null, pluginMetrics));
}
- @Test(expected = RuntimeException.class)
+ @Test
public void testConstructWithInterruptedException() throws Exception {
CompletableFuture mockFuture = mock(CompletableFuture.class);
when(mockFuture.get()).thenThrow(new InterruptedException());
when(dnsAddressEndpointGroup.whenReady()).thenReturn(mockFuture);
- new DnsPeerListProvider(dnsAddressEndpointGroup, pluginMetrics);
+ assertThrows(RuntimeException.class, () -> new DnsPeerListProvider(dnsAddressEndpointGroup, pluginMetrics));
}
@Test
@@ -90,17 +90,27 @@ public void testGetPeerList() {
}
@Test
- public void testActivePeerCounter() {
+ public void testActivePeerCounter_with_list() {
when(dnsAddressEndpointGroup.endpoints()).thenReturn(ENDPOINT_LIST);
- final List endpointsMeasures = MetricsTestUtil.getMeasurementList(new StringJoiner(MetricNames.DELIMITER).add(COMPONENT_SCOPE).add(COMPONENT_ID)
- .add(PeerListProvider.PEER_ENDPOINTS).toString());
- assertEquals(1, endpointsMeasures.size());
- final Measurement endpointsMeasure = endpointsMeasures.get(0);
- assertEquals(2.0, endpointsMeasure.getValue(), 0);
+ final ArgumentCaptor> gaugeFunctionCaptor = ArgumentCaptor.forClass(ToDoubleFunction.class);
+ verify(pluginMetrics).gauge(eq(PEER_ENDPOINTS), eq(dnsAddressEndpointGroup), gaugeFunctionCaptor.capture());
+
+ final ToDoubleFunction gaugeFunction = gaugeFunctionCaptor.getValue();
+ assertThat(gaugeFunction.applyAsDouble(dnsAddressEndpointGroup), equalTo(2.0));
+ }
+
+ @Test
+ public void testActivePeerCounter_with_single() {
when(dnsAddressEndpointGroup.endpoints()).thenReturn(Collections.singletonList(Endpoint.of(ENDPOINT_1)));
- assertEquals(1.0, endpointsMeasure.getValue(), 0);
+
+ final ArgumentCaptor> gaugeFunctionCaptor = ArgumentCaptor.forClass(ToDoubleFunction.class);
+ verify(pluginMetrics).gauge(eq(PEER_ENDPOINTS), eq(dnsAddressEndpointGroup), gaugeFunctionCaptor.capture());
+
+ final ToDoubleFunction gaugeFunction = gaugeFunctionCaptor.getValue();
+
+ assertThat(gaugeFunction.applyAsDouble(dnsAddressEndpointGroup), equalTo(1.0));
}
@Test
diff --git a/data-prepper-core/src/test/java/org/opensearch/dataprepper/peerforwarder/discovery/StaticPeerListProviderTest.java b/data-prepper-core/src/test/java/org/opensearch/dataprepper/peerforwarder/discovery/StaticPeerListProviderTest.java
index 14bc836e36..589329b108 100644
--- a/data-prepper-core/src/test/java/org/opensearch/dataprepper/peerforwarder/discovery/StaticPeerListProviderTest.java
+++ b/data-prepper-core/src/test/java/org/opensearch/dataprepper/peerforwarder/discovery/StaticPeerListProviderTest.java
@@ -5,56 +5,58 @@
package org.opensearch.dataprepper.peerforwarder.discovery;
-import io.micrometer.core.instrument.Measurement;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.runner.RunWith;
+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.MockitoJUnitRunner;
-import org.opensearch.dataprepper.metrics.MetricNames;
-import org.opensearch.dataprepper.metrics.MetricsTestUtil;
+import org.mockito.junit.jupiter.MockitoExtension;
import org.opensearch.dataprepper.metrics.PluginMetrics;
import org.opensearch.dataprepper.peerforwarder.HashRing;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
-import java.util.StringJoiner;
-
-import static org.junit.Assert.assertEquals;
+import java.util.function.ToDoubleFunction;
+
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.eq;
+import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.verifyNoInteractions;
+import static org.opensearch.dataprepper.peerforwarder.discovery.PeerListProvider.PEER_ENDPOINTS;
-@RunWith(MockitoJUnitRunner.class)
+@ExtendWith(MockitoExtension.class)
public class StaticPeerListProviderTest {
private static final String ENDPOINT_1 = "10.10.0.1";
private static final String ENDPOINT_2 = "10.10.0.2";
private static final List ENDPOINT_LIST = Arrays.asList(ENDPOINT_1, ENDPOINT_2);
- private static final String COMPONENT_SCOPE = "testComponentScope";
- private static final String COMPONENT_ID = "testComponentId";
@Mock
private HashRing hashRing;
+ @Mock
private PluginMetrics pluginMetrics;
private StaticPeerListProvider staticPeerListProvider;
- @Before
+ @BeforeEach
public void setup() {
- MetricsTestUtil.initMetrics();
- pluginMetrics = PluginMetrics.fromNames(COMPONENT_ID, COMPONENT_SCOPE);
staticPeerListProvider = new StaticPeerListProvider(ENDPOINT_LIST, pluginMetrics);
}
- @Test(expected = RuntimeException.class)
+ @Test
public void testListProviderWithEmptyList() {
- new StaticPeerListProvider(Collections.emptyList(), pluginMetrics);
+ assertThrows(RuntimeException.class, () -> new StaticPeerListProvider(Collections.emptyList(), pluginMetrics));
}
- @Test(expected = RuntimeException.class)
+ @Test
public void testListProviderWithNullList() {
- new StaticPeerListProvider(null, pluginMetrics);
+ assertThrows(RuntimeException.class, () -> new StaticPeerListProvider(null, pluginMetrics));
}
@Test
@@ -65,11 +67,12 @@ public void testListProviderWithNonEmptyList() {
@Test
public void testActivePeerCounter() {
- final List endpointsMeasures = MetricsTestUtil.getMeasurementList(
- new StringJoiner(MetricNames.DELIMITER).add(COMPONENT_SCOPE).add(COMPONENT_ID).add(PeerListProvider.PEER_ENDPOINTS).toString());
- assertEquals(1, endpointsMeasures.size());
- final Measurement endpointsMeasure = endpointsMeasures.get(0);
- assertEquals(2.0, endpointsMeasure.getValue(), 0);
+ final ArgumentCaptor>> gaugeFunctionCaptor = ArgumentCaptor.forClass(ToDoubleFunction.class);
+ verify(pluginMetrics).gauge(eq(PEER_ENDPOINTS), any(List.class), gaugeFunctionCaptor.capture());
+
+ final ToDoubleFunction> gaugeFunction = gaugeFunctionCaptor.getValue();
+
+ assertThat(gaugeFunction.applyAsDouble(ENDPOINT_LIST), equalTo(2.0));
}
@Test
diff --git a/data-prepper-core/src/test/java/org/opensearch/dataprepper/pipeline/PipelineConnectorTest.java b/data-prepper-core/src/test/java/org/opensearch/dataprepper/pipeline/PipelineConnectorTest.java
index fb54d532b7..e2af218c25 100644
--- a/data-prepper-core/src/test/java/org/opensearch/dataprepper/pipeline/PipelineConnectorTest.java
+++ b/data-prepper-core/src/test/java/org/opensearch/dataprepper/pipeline/PipelineConnectorTest.java
@@ -23,7 +23,7 @@
import org.junit.Test;
import org.junit.runner.RunWith;
import org.mockito.Mock;
-import org.mockito.runners.MockitoJUnitRunner;
+import org.mockito.junit.MockitoJUnitRunner;
import org.opensearch.dataprepper.plugins.buffer.blockingbuffer.BlockingBuffer;
diff --git a/data-prepper-core/src/test/java/org/opensearch/dataprepper/pipeline/PipelineTests.java b/data-prepper-core/src/test/java/org/opensearch/dataprepper/pipeline/PipelineTests.java
index 5c0a9a974e..c2e0ad769f 100644
--- a/data-prepper-core/src/test/java/org/opensearch/dataprepper/pipeline/PipelineTests.java
+++ b/data-prepper-core/src/test/java/org/opensearch/dataprepper/pipeline/PipelineTests.java
@@ -441,7 +441,7 @@ void publishToSinks_calls_route_with_Events_and_Sinks_verify_AcknowledgementSetM
Pipeline pipeline = createObjectUnderTest();
when(mockSource.areAcknowledgementsEnabled()).thenReturn(true);
pipeline.publishToSinks(records);
- verify(acknowledgementSetManager).acquireEventReference(any(DefaultEventHandle.class));
+ verify(eventHandle).acquireReference();
verify(router)
.route(anyCollection(), eq(dataFlowComponents), any(RouterGetRecordStrategy.class), any(BiConsumer.class));
diff --git a/data-prepper-core/src/test/java/org/opensearch/dataprepper/pipeline/ProcessWorkerTest.java b/data-prepper-core/src/test/java/org/opensearch/dataprepper/pipeline/ProcessWorkerTest.java
index 3d13c0d49f..455da07a93 100644
--- a/data-prepper-core/src/test/java/org/opensearch/dataprepper/pipeline/ProcessWorkerTest.java
+++ b/data-prepper-core/src/test/java/org/opensearch/dataprepper/pipeline/ProcessWorkerTest.java
@@ -12,6 +12,7 @@
import org.opensearch.dataprepper.model.event.DefaultEventHandle;
import org.opensearch.dataprepper.model.event.Event;
import org.opensearch.dataprepper.model.event.EventHandle;
+import org.opensearch.dataprepper.model.event.InternalEventHandle;
import org.opensearch.dataprepper.model.processor.Processor;
import org.opensearch.dataprepper.model.record.Record;
import org.opensearch.dataprepper.model.source.Source;
@@ -27,7 +28,6 @@
import java.util.concurrent.Future;
import static org.mockito.ArgumentMatchers.any;
-import static org.mockito.Mockito.doNothing;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.mockStatic;
import static org.mockito.Mockito.never;
@@ -104,7 +104,6 @@ void testProcessWorkerHappyPathWithAcknowledgments() {
final Record mockRecord = mock(Record.class);
final Event mockEvent = mock(Event.class);
final EventHandle eventHandle = mock(DefaultEventHandle.class);
- when(((DefaultEventHandle) eventHandle).getAcknowledgementSet()).thenReturn(mock(AcknowledgementSet.class));
when(mockRecord.getData()).thenReturn(mockEvent);
when(mockEvent.getEventHandle()).thenReturn(eventHandle);
@@ -174,8 +173,8 @@ void testProcessWorkerWithProcessorThrowingExceptionAndAcknowledgmentsEnabledIsH
final Record mockRecord = mock(Record.class);
final Event mockEvent = mock(Event.class);
final EventHandle eventHandle = mock(DefaultEventHandle.class);
- when(((DefaultEventHandle) eventHandle).getAcknowledgementSet()).thenReturn(mock(AcknowledgementSet.class));
- doNothing().when(eventHandle).release(true);
+ final AcknowledgementSet acknowledgementSet = mock(AcknowledgementSet.class);
+ ((InternalEventHandle)eventHandle).addAcknowledgementSet(acknowledgementSet);
when(mockRecord.getData()).thenReturn(mockEvent);
when(mockEvent.getEventHandle()).thenReturn(eventHandle);
@@ -218,8 +217,8 @@ void testProcessWorkerWithProcessorDroppingAllRecordsAndAcknowledgmentsEnabledIs
final Record mockRecord = mock(Record.class);
final Event mockEvent = mock(Event.class);
final EventHandle eventHandle = mock(DefaultEventHandle.class);
- when(((DefaultEventHandle) eventHandle).getAcknowledgementSet()).thenReturn(mock(AcknowledgementSet.class));
- doNothing().when(eventHandle).release(true);
+ final AcknowledgementSet acknowledgementSet = mock(AcknowledgementSet.class);
+ ((InternalEventHandle)eventHandle).addAcknowledgementSet(acknowledgementSet);
when(mockRecord.getData()).thenReturn(mockEvent);
when(mockEvent.getEventHandle()).thenReturn(eventHandle);
diff --git a/data-prepper-core/src/test/java/org/opensearch/dataprepper/pipeline/common/FutureHelperTest.java b/data-prepper-core/src/test/java/org/opensearch/dataprepper/pipeline/common/FutureHelperTest.java
index c572766ac2..ba8a9714de 100644
--- a/data-prepper-core/src/test/java/org/opensearch/dataprepper/pipeline/common/FutureHelperTest.java
+++ b/data-prepper-core/src/test/java/org/opensearch/dataprepper/pipeline/common/FutureHelperTest.java
@@ -9,7 +9,7 @@
import org.junit.Test;
import org.junit.runner.RunWith;
import org.mockito.Mock;
-import org.mockito.runners.MockitoJUnitRunner;
+import org.mockito.junit.MockitoJUnitRunner;
import java.util.Arrays;
import java.util.concurrent.ExecutionException;
diff --git a/data-prepper-core/src/test/java/org/opensearch/dataprepper/pipeline/router/DataFlowComponentRouterTest.java b/data-prepper-core/src/test/java/org/opensearch/dataprepper/pipeline/router/DataFlowComponentRouterTest.java
index 3802356592..1ea74afe70 100644
--- a/data-prepper-core/src/test/java/org/opensearch/dataprepper/pipeline/router/DataFlowComponentRouterTest.java
+++ b/data-prepper-core/src/test/java/org/opensearch/dataprepper/pipeline/router/DataFlowComponentRouterTest.java
@@ -158,6 +158,17 @@ void route_no_Events_when_none_have_matching_routes() {
verify(componentRecordsConsumer).accept(testComponent, Collections.emptyList());
}
+ @Test
+ void route_no_Events_when_none_have_matching_routes_with_default_route() {
+ when(dataFlowComponent.getRoutes()).thenReturn(Set.of(DataFlowComponentRouter.DEFAULT_ROUTE));
+ final Map> noMatchingRoutes = recordsIn.stream()
+ .collect(Collectors.toMap(Function.identity(), r -> Collections.emptySet()));
+
+ createObjectUnderTest().route(recordsIn, dataFlowComponent, noMatchingRoutes, getRecordStrategy, componentRecordsConsumer);
+
+ verify(componentRecordsConsumer).accept(testComponent, recordsIn);
+ }
+
@Test
void route_all_Events_when_all_have_matched_route() {
@@ -236,6 +247,33 @@ void route_no_Events_when_none_have_matching_routes() {
verify(componentRecordsConsumer).accept(testComponent, Collections.emptyList());
}
+ @Test
+ void route_no_Events_when_none_have_matching_routes_with_default_route() {
+ when(dataFlowComponent.getRoutes()).thenReturn(Set.of(DataFlowComponentRouter.DEFAULT_ROUTE));
+ final Map> noMatchingRoutes = recordsIn.stream()
+ .collect(Collectors.toMap(Function.identity(), r -> Collections.emptySet()));
+
+ createObjectUnderTest().route(recordsIn, dataFlowComponent, noMatchingRoutes, getRecordStrategy, componentRecordsConsumer);
+
+ verify(componentRecordsConsumer).accept(testComponent, recordsIn);
+ }
+
+ @Test
+ void route_matched_events_with_none_to_default_route() {
+ DataFlowComponent dataFlowComponent2 = mock(DataFlowComponent.class);
+ when(dataFlowComponent2.getRoutes()).thenReturn(Set.of(DataFlowComponentRouter.DEFAULT_ROUTE));
+ final Map> allMatchingRoutes = recordsIn.stream()
+ .collect(Collectors.toMap(Function.identity(), r -> Collections.singleton(knownRoute)));
+
+ createObjectUnderTest().route(recordsIn, dataFlowComponent2, allMatchingRoutes, getRecordStrategy, componentRecordsConsumer);
+ verify(componentRecordsConsumer).accept(null, Collections.emptyList());
+ createObjectUnderTest().route(recordsIn, dataFlowComponent, allMatchingRoutes, getRecordStrategy, componentRecordsConsumer);
+
+ verify(componentRecordsConsumer).accept(testComponent, recordsIn);
+
+ }
+
+
@Test
void route_all_Events_when_all_have_matched_route() {
diff --git a/data-prepper-core/src/test/java/org/opensearch/dataprepper/pipeline/router/RouterCopyRecordStrategyTests.java b/data-prepper-core/src/test/java/org/opensearch/dataprepper/pipeline/router/RouterCopyRecordStrategyTests.java
index 4c56113323..c971cd5b8d 100644
--- a/data-prepper-core/src/test/java/org/opensearch/dataprepper/pipeline/router/RouterCopyRecordStrategyTests.java
+++ b/data-prepper-core/src/test/java/org/opensearch/dataprepper/pipeline/router/RouterCopyRecordStrategyTests.java
@@ -83,7 +83,7 @@ void setUp() {
int v = handleRefCount.getOrDefault(handle, 0);
handleRefCount.put(handle, v+1);
return null;
- }).when(acknowledgementSetManager).acquireEventReference(any(DefaultEventHandle.class));
+ }).when(acknowledgementSet1).acquire(any(DefaultEventHandle.class));
} catch (Exception e){}
mockRecordsIn = IntStream.range(0, 10)
.mapToObj(i -> mock(Record.class))
@@ -103,7 +103,7 @@ private void attachEventHandlesToRecordsIn(List eventHandles
while (iter.hasNext()) {
Record r = (Record) iter.next();
DefaultEventHandle handle = (DefaultEventHandle)((JacksonEvent)r.getData()).getEventHandle();
- handle.setAcknowledgementSet(acknowledgementSet1);
+ handle.addAcknowledgementSet(acknowledgementSet1);
eventHandles.add(handle);
}
}
@@ -195,6 +195,7 @@ void test_one_record_with_acknowledgements() {
assertTrue(getRecordStrategy.getReferencedRecords().contains(firstRecord));
recordOut = getRecordStrategy.getRecord(firstRecord);
assertThat(recordOut, sameInstance(firstRecord));
+ firstHandle.addAcknowledgementSet(acknowledgementSet1);
assertThat(handleRefCount.get(firstHandle), equalTo(1));
recordOut = getRecordStrategy.getRecord(firstRecord);
assertThat(recordOut, sameInstance(firstRecord));
@@ -242,7 +243,7 @@ void test_one_record_with_acknowledgements_and_multi_components() {
try {
doAnswer((i) -> {
JacksonEvent e1 = (JacksonEvent) i.getArgument(0);
- ((DefaultEventHandle)e1.getEventHandle()).setAcknowledgementSet(acknowledgementSet1);
+ ((DefaultEventHandle)e1.getEventHandle()).addAcknowledgementSet(acknowledgementSet1);
return null;
}).when(acknowledgementSet1).add(any(JacksonEvent.class));
} catch (Exception e){}
@@ -280,7 +281,7 @@ void test_multiple_records_with_acknowledgements_and_multi_components() {
try {
doAnswer((i) -> {
JacksonEvent e1 = (JacksonEvent) i.getArgument(0);
- ((DefaultEventHandle)e1.getEventHandle()).setAcknowledgementSet(acknowledgementSet1);
+ ((DefaultEventHandle)e1.getEventHandle()).addAcknowledgementSet(acknowledgementSet1);
return null;
}).when(acknowledgementSet1).add(any(JacksonEvent.class));
} catch (Exception e){}
diff --git a/data-prepper-core/src/test/java/org/opensearch/dataprepper/pipeline/server/CloudWatchMeterRegistryProviderTest.java b/data-prepper-core/src/test/java/org/opensearch/dataprepper/pipeline/server/CloudWatchMeterRegistryProviderTest.java
index 53db40d1a6..9dc744981b 100644
--- a/data-prepper-core/src/test/java/org/opensearch/dataprepper/pipeline/server/CloudWatchMeterRegistryProviderTest.java
+++ b/data-prepper-core/src/test/java/org/opensearch/dataprepper/pipeline/server/CloudWatchMeterRegistryProviderTest.java
@@ -9,7 +9,7 @@
import org.junit.Test;
import org.junit.runner.RunWith;
import org.mockito.Mock;
-import org.mockito.runners.MockitoJUnitRunner;
+import org.mockito.junit.MockitoJUnitRunner;
import software.amazon.awssdk.services.cloudwatch.CloudWatchAsyncClient;
import static org.hamcrest.CoreMatchers.notNullValue;
diff --git a/data-prepper-core/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker b/data-prepper-core/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker
deleted file mode 100644
index 23c33feb6d..0000000000
--- a/data-prepper-core/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker
+++ /dev/null
@@ -1,3 +0,0 @@
-# To enable mocking of final classes with vanilla Mockito
-# https://github.com/mockito/mockito/wiki/What%27s-new-in-Mockito-2#mock-the-unmockable-opt-in-mocking-of-final-classesmethods
-mock-maker-inline
diff --git a/data-prepper-event/src/main/java/org/opensearch/dataprepper/core/event/DefaultEventKeyFactory.java b/data-prepper-event/src/main/java/org/opensearch/dataprepper/core/event/DefaultEventKeyFactory.java
new file mode 100644
index 0000000000..605b5bcb41
--- /dev/null
+++ b/data-prepper-event/src/main/java/org/opensearch/dataprepper/core/event/DefaultEventKeyFactory.java
@@ -0,0 +1,20 @@
+/*
+ * Copyright OpenSearch Contributors
+ * SPDX-License-Identifier: Apache-2.0
+ */
+
+package org.opensearch.dataprepper.core.event;
+
+import org.opensearch.dataprepper.model.event.EventKey;
+import org.opensearch.dataprepper.model.event.EventKeyFactory;
+import org.opensearch.dataprepper.model.event.InternalOnlyEventKeyBridge;
+
+import javax.inject.Named;
+
+@Named
+public class DefaultEventKeyFactory implements EventKeyFactory {
+ @Override
+ public EventKey createEventKey(final String key, final EventAction... forActions) {
+ return InternalOnlyEventKeyBridge.createEventKey(key, forActions);
+ }
+}
diff --git a/data-prepper-event/src/main/java/org/opensearch/dataprepper/model/event/InternalOnlyEventKeyBridge.java b/data-prepper-event/src/main/java/org/opensearch/dataprepper/model/event/InternalOnlyEventKeyBridge.java
new file mode 100644
index 0000000000..130b94db0e
--- /dev/null
+++ b/data-prepper-event/src/main/java/org/opensearch/dataprepper/model/event/InternalOnlyEventKeyBridge.java
@@ -0,0 +1,17 @@
+/*
+ * Copyright OpenSearch Contributors
+ * SPDX-License-Identifier: Apache-2.0
+ */
+
+package org.opensearch.dataprepper.model.event;
+
+/**
+ * Until we remove {@link JacksonEvent} from data-prepper-api,
+ * we will need this class to give us access to the package-protected
+ * {@link JacksonEventKey}.
+ */
+public class InternalOnlyEventKeyBridge {
+ public static EventKey createEventKey(final String key, final EventKeyFactory.EventAction... forAction) {
+ return new JacksonEventKey(key, forAction);
+ }
+}
diff --git a/data-prepper-event/src/test/java/org/opensearch/dataprepper/core/event/DefaultEventKeyFactoryTest.java b/data-prepper-event/src/test/java/org/opensearch/dataprepper/core/event/DefaultEventKeyFactoryTest.java
new file mode 100644
index 0000000000..8d034fcc83
--- /dev/null
+++ b/data-prepper-event/src/test/java/org/opensearch/dataprepper/core/event/DefaultEventKeyFactoryTest.java
@@ -0,0 +1,52 @@
+/*
+ * Copyright OpenSearch Contributors
+ * SPDX-License-Identifier: Apache-2.0
+ */
+
+package org.opensearch.dataprepper.core.event;
+
+import org.junit.jupiter.api.Test;
+import org.opensearch.dataprepper.model.event.EventKey;
+import org.opensearch.dataprepper.model.event.EventKeyFactory;
+
+import java.util.UUID;
+
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.hamcrest.CoreMatchers.notNullValue;
+import static org.hamcrest.MatcherAssert.assertThat;
+
+class DefaultEventKeyFactoryTest {
+
+ private DefaultEventKeyFactory createObjectUnderTest() {
+ return new DefaultEventKeyFactory();
+ }
+
+ @Test
+ void createEventKey_returns_correct_EventKey() {
+ final String keyPath = UUID.randomUUID().toString();
+ final EventKey eventKey = createObjectUnderTest().createEventKey(keyPath);
+
+ assertThat(eventKey, notNullValue());
+ assertThat(eventKey.getKey(), equalTo(keyPath));
+ }
+
+ @Test
+ void createEventKey_with_EventAction_returns_correct_EventKey() {
+ final String keyPath = UUID.randomUUID().toString();
+ final EventKey eventKey = createObjectUnderTest().createEventKey(keyPath, EventKeyFactory.EventAction.GET);
+
+ assertThat(eventKey, notNullValue());
+ assertThat(eventKey.getKey(), equalTo(keyPath));
+ }
+
+ @Test
+ void createEventKey_returns_JacksonEventKey() {
+ final String keyPath = UUID.randomUUID().toString();
+ final EventKey eventKey = createObjectUnderTest().createEventKey(keyPath);
+
+ assertThat(eventKey, notNullValue());
+ assertThat(eventKey.getClass().getSimpleName(), equalTo("JacksonEventKey"));
+
+ assertThat(eventKey.getKey(), equalTo(keyPath));
+ }
+}
\ No newline at end of file
diff --git a/data-prepper-expression/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker b/data-prepper-expression/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker
deleted file mode 100644
index 1f0955d450..0000000000
--- a/data-prepper-expression/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker
+++ /dev/null
@@ -1 +0,0 @@
-mock-maker-inline
diff --git a/data-prepper-logstash-configuration/build.gradle b/data-prepper-logstash-configuration/build.gradle
index 6e328b7adc..002ae15516 100644
--- a/data-prepper-logstash-configuration/build.gradle
+++ b/data-prepper-logstash-configuration/build.gradle
@@ -25,7 +25,6 @@ dependencies {
implementation 'com.fasterxml.jackson.core:jackson-databind'
implementation libs.commons.lang3
testImplementation testLibs.slf4j.simple
- testImplementation testLibs.mockito.inline
}
generateGrammarSource {
diff --git a/data-prepper-pipeline-parser/build.gradle b/data-prepper-pipeline-parser/build.gradle
index 09c89eb15c..a94f63fc1d 100644
--- a/data-prepper-pipeline-parser/build.gradle
+++ b/data-prepper-pipeline-parser/build.gradle
@@ -18,6 +18,7 @@ dependencies {
implementation 'org.projectlombok:lombok:1.18.22'
implementation 'com.jayway.jsonpath:json-path:2.6.0'
implementation 'javax.inject:javax.inject:1'
+ implementation 'javax.annotation:javax.annotation-api:1.3.2'
implementation(libs.spring.core) {
exclude group: 'commons-logging', module: 'commons-logging'
}
@@ -29,12 +30,7 @@ dependencies {
testImplementation testLibs.bundles.junit
testImplementation testLibs.bundles.mockito
testImplementation testLibs.hamcrest
- testImplementation 'org.powermock:powermock-module-junit4:2.0.9'
- testImplementation 'org.powermock:powermock-api-mockito2:2.0.9'
testImplementation 'org.assertj:assertj-core:3.20.2'
- testImplementation 'junit:junit:4.13.2'
- testImplementation 'org.powermock:powermock-module-junit4:2.0.9'
- testImplementation 'org.powermock:powermock-api-mockito2:2.0.9'
compileOnly 'org.projectlombok:lombok:1.18.20'
annotationProcessor 'org.projectlombok:lombok:1.18.20'
}
\ No newline at end of file
diff --git a/data-prepper-pipeline-parser/src/main/java/org/opensearch/dataprepper/core/validators/NotEmptyValidatorForEventKey.java b/data-prepper-pipeline-parser/src/main/java/org/opensearch/dataprepper/core/validators/NotEmptyValidatorForEventKey.java
new file mode 100644
index 0000000000..507d2e9637
--- /dev/null
+++ b/data-prepper-pipeline-parser/src/main/java/org/opensearch/dataprepper/core/validators/NotEmptyValidatorForEventKey.java
@@ -0,0 +1,22 @@
+/*
+ * Copyright OpenSearch Contributors
+ * SPDX-License-Identifier: Apache-2.0
+ */
+
+package org.opensearch.dataprepper.core.validators;
+
+import jakarta.validation.ConstraintValidator;
+import jakarta.validation.ConstraintValidatorContext;
+import org.opensearch.dataprepper.model.event.EventKey;
+
+import jakarta.validation.constraints.NotEmpty;
+
+public class NotEmptyValidatorForEventKey implements ConstraintValidator {
+ @Override
+ public boolean isValid(final EventKey eventKey, final ConstraintValidatorContext constraintValidatorContext) {
+ if(eventKey == null) {
+ return false;
+ }
+ return !eventKey.getKey().isEmpty();
+ }
+}
diff --git a/data-prepper-pipeline-parser/src/main/java/org/opensearch/dataprepper/pipeline/parser/EventKeyDeserializer.java b/data-prepper-pipeline-parser/src/main/java/org/opensearch/dataprepper/pipeline/parser/EventKeyDeserializer.java
new file mode 100644
index 0000000000..fbc27edc8b
--- /dev/null
+++ b/data-prepper-pipeline-parser/src/main/java/org/opensearch/dataprepper/pipeline/parser/EventKeyDeserializer.java
@@ -0,0 +1,60 @@
+/*
+ * Copyright OpenSearch Contributors
+ * SPDX-License-Identifier: Apache-2.0
+ */
+
+package org.opensearch.dataprepper.pipeline.parser;
+
+import com.fasterxml.jackson.core.JsonParser;
+import com.fasterxml.jackson.databind.BeanProperty;
+import com.fasterxml.jackson.databind.DeserializationContext;
+import com.fasterxml.jackson.databind.JsonDeserializer;
+import com.fasterxml.jackson.databind.deser.ContextualDeserializer;
+import com.fasterxml.jackson.databind.deser.std.StdDeserializer;
+import org.opensearch.dataprepper.model.event.EventKey;
+import org.opensearch.dataprepper.model.event.EventKeyConfiguration;
+import org.opensearch.dataprepper.model.event.EventKeyFactory;
+
+import java.io.IOException;
+
+public class EventKeyDeserializer extends StdDeserializer implements ContextualDeserializer {
+ private final EventKeyFactory eventKeyFactory;
+ private final EventKeyFactory.EventAction[] eventAction;
+
+ /**
+ * Constructs a new {@link EventKeyDeserializer} from an {@link EventKeyFactory}.
+ *
+ * @param eventKeyFactory The factory for creating {@link EventKey} objects.
+ */
+ public EventKeyDeserializer(final EventKeyFactory eventKeyFactory) {
+ this(eventKeyFactory, new EventKeyFactory.EventAction[] {EventKeyFactory.EventAction.ALL});
+ }
+
+ private EventKeyDeserializer(final EventKeyFactory eventKeyFactory, final EventKeyFactory.EventAction[] eventAction) {
+ super(EventKey.class);
+ this.eventKeyFactory = eventKeyFactory;
+ this.eventAction = eventAction;
+ }
+
+ @Override
+ public EventKey deserialize(final JsonParser parser, final DeserializationContext ctxt) throws IOException {
+ final String eventKeyString = parser.getValueAsString();
+
+ return eventKeyFactory.createEventKey(eventKeyString, eventAction);
+ }
+
+ @Override
+ public JsonDeserializer> createContextual(final DeserializationContext deserializationContext, final BeanProperty property) {
+ if(property == null)
+ return this;
+
+ final EventKeyConfiguration eventKeyConfiguration = property.getAnnotation(EventKeyConfiguration.class);
+
+ if(eventKeyConfiguration == null)
+ return this;
+
+ final EventKeyFactory.EventAction[] eventAction = eventKeyConfiguration.value();
+
+ return new EventKeyDeserializer(eventKeyFactory, eventAction);
+ }
+}
diff --git a/data-prepper-pipeline-parser/src/main/resources/META-INF/services/jakarta.validation.ConstraintValidator b/data-prepper-pipeline-parser/src/main/resources/META-INF/services/jakarta.validation.ConstraintValidator
new file mode 100644
index 0000000000..ab6fb40c08
--- /dev/null
+++ b/data-prepper-pipeline-parser/src/main/resources/META-INF/services/jakarta.validation.ConstraintValidator
@@ -0,0 +1,6 @@
+#
+# Copyright OpenSearch Contributors
+# SPDX-License-Identifier: Apache-2.0
+#
+
+org.opensearch.dataprepper.core.validators.NotEmptyValidatorForEventKey
\ No newline at end of file
diff --git a/data-prepper-pipeline-parser/src/test/java/org/opensearch/dataprepper/core/validators/NotEmptyValidatorForEventKeyTest.java b/data-prepper-pipeline-parser/src/test/java/org/opensearch/dataprepper/core/validators/NotEmptyValidatorForEventKeyTest.java
new file mode 100644
index 0000000000..d49ca2c161
--- /dev/null
+++ b/data-prepper-pipeline-parser/src/test/java/org/opensearch/dataprepper/core/validators/NotEmptyValidatorForEventKeyTest.java
@@ -0,0 +1,50 @@
+/*
+ * Copyright OpenSearch Contributors
+ * SPDX-License-Identifier: Apache-2.0
+ */
+
+package org.opensearch.dataprepper.core.validators;
+
+import jakarta.validation.ConstraintValidatorContext;
+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.event.EventKey;
+
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.mockito.Mockito.when;
+
+@ExtendWith(MockitoExtension.class)
+class NotEmptyValidatorForEventKeyTest {
+ @Mock
+ private EventKey eventKey;
+
+ @Mock
+ private ConstraintValidatorContext context;
+
+ private NotEmptyValidatorForEventKey createObjectUnderTest() {
+ return new NotEmptyValidatorForEventKey();
+ }
+
+ @Test
+ void isValid_returns_false_if_EventKey_is_empty() {
+ assertThat(createObjectUnderTest().isValid(null, context), equalTo(false));
+ }
+
+ @Test
+ void isValid_returns_false_if_EventKey_getKey_is_empty() {
+ when(eventKey.getKey()).thenReturn("");
+ assertThat(createObjectUnderTest().isValid(eventKey, context), equalTo(false));
+ }
+
+ @ParameterizedTest
+ @ValueSource(strings = {"/", "a", "/abcdefghijklmnopqrstuvwxyz"})
+ void isValid_returns_true_if_EventKey_getKey_is_not_empty(final String key) {
+ when(eventKey.getKey()).thenReturn(key);
+ assertThat(createObjectUnderTest().isValid(eventKey, context), equalTo(true));
+ }
+}
\ No newline at end of file
diff --git a/data-prepper-pipeline-parser/src/test/java/org/opensearch/dataprepper/pipeline/parser/EventKeyDeserializerTest.java b/data-prepper-pipeline-parser/src/test/java/org/opensearch/dataprepper/pipeline/parser/EventKeyDeserializerTest.java
new file mode 100644
index 0000000000..240c14dd37
--- /dev/null
+++ b/data-prepper-pipeline-parser/src/test/java/org/opensearch/dataprepper/pipeline/parser/EventKeyDeserializerTest.java
@@ -0,0 +1,142 @@
+/*
+ * Copyright OpenSearch Contributors
+ * SPDX-License-Identifier: Apache-2.0
+ */
+
+package org.opensearch.dataprepper.pipeline.parser;
+
+import com.fasterxml.jackson.core.JsonParser;
+import com.fasterxml.jackson.databind.BeanProperty;
+import com.fasterxml.jackson.databind.DeserializationContext;
+import com.fasterxml.jackson.databind.JsonDeserializer;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.module.SimpleModule;
+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.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.EnumSource;
+import org.mockito.Mock;
+import org.mockito.junit.jupiter.MockitoExtension;
+import org.opensearch.dataprepper.model.event.EventKey;
+import org.opensearch.dataprepper.model.event.EventKeyConfiguration;
+import org.opensearch.dataprepper.model.event.EventKeyFactory;
+
+import java.io.IOException;
+import java.util.UUID;
+
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.hamcrest.CoreMatchers.notNullValue;
+import static org.hamcrest.CoreMatchers.sameInstance;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+@ExtendWith(MockitoExtension.class)
+class EventKeyDeserializerTest {
+
+ @Mock
+ private EventKeyFactory eventKeyFactory;
+
+ @Mock
+ private DeserializationContext deserializationContext;
+ @Mock
+ private BeanProperty property;
+ @Mock(lenient = true)
+ private JsonParser parser;
+ @Mock
+ private EventKey eventKey;
+
+ private String eventKeyString;
+
+ @BeforeEach
+ void setUp() throws IOException {
+ eventKeyString = UUID.randomUUID().toString();
+
+ when(parser.getValueAsString()).thenReturn(eventKeyString);
+ }
+
+ private EventKeyDeserializer createObjectUnderTest() {
+ return new EventKeyDeserializer(eventKeyFactory);
+ }
+
+ @Test
+ void createContextual_returns_EventKeyDeserializer_that_deserializes_with_ALL_when_no_BeanProperty() throws IOException {
+ when(eventKeyFactory.createEventKey(eventKeyString, EventKeyFactory.EventAction.ALL)).thenReturn(eventKey);
+ final JsonDeserializer> contextualDeserializer = createObjectUnderTest().createContextual(deserializationContext, null);
+ assertThat(contextualDeserializer, notNullValue());
+ assertThat(contextualDeserializer.deserialize(parser, deserializationContext), equalTo(eventKey));
+ }
+
+ @Test
+ void createContextual_returns_EventKeyDeserializer_that_deserializes_with_ALL_when_no_annotation() throws IOException {
+ when(eventKeyFactory.createEventKey(eventKeyString, EventKeyFactory.EventAction.ALL)).thenReturn(eventKey);
+ final JsonDeserializer> contextualDeserializer = createObjectUnderTest().createContextual(deserializationContext, property);
+ assertThat(contextualDeserializer, notNullValue());
+ assertThat(contextualDeserializer.deserialize(parser, deserializationContext), equalTo(eventKey));
+ }
+
+ @Test
+ void createContextual_returns_same_EventKeyDeserializer_as_self_when_no_BeanProperty() {
+ final EventKeyDeserializer objectUnderTest = createObjectUnderTest();
+ final JsonDeserializer> contextualDeserializer = objectUnderTest.createContextual(deserializationContext, null);
+ assertThat(contextualDeserializer, sameInstance(objectUnderTest));
+ }
+
+ @Test
+ void createContextual_returns_same_EventKeyDeserializer_as_self_when_no_annotation() {
+ final EventKeyDeserializer objectUnderTest = createObjectUnderTest();
+ final JsonDeserializer> contextualDeserializer = objectUnderTest.createContextual(deserializationContext, property);
+ assertThat(contextualDeserializer, sameInstance(objectUnderTest));
+ }
+
+ @ParameterizedTest
+ @EnumSource(value = EventKeyFactory.EventAction.class)
+ void createContextual_returns_EventKeyDeserializer_that_deserializes_with_action_from_annotated_Event(final EventKeyFactory.EventAction eventAction) throws IOException {
+ final EventKeyConfiguration eventKeyConfiguration = mock(EventKeyConfiguration.class);
+ when(eventKeyConfiguration.value()).thenReturn(new EventKeyFactory.EventAction[] { eventAction });
+ when(property.getAnnotation(EventKeyConfiguration.class)).thenReturn(eventKeyConfiguration);
+ when(eventKeyFactory.createEventKey(eventKeyString, eventAction)).thenReturn(eventKey);
+
+ final JsonDeserializer> contextualDeserializer = createObjectUnderTest().createContextual(deserializationContext, property);
+
+ assertThat(contextualDeserializer, notNullValue());
+ assertThat(contextualDeserializer.deserialize(parser, deserializationContext), equalTo(eventKey));
+ }
+
+ @Test
+ void createContextual_returns_EventKeyDeserializer_that_deserializes_with_action_from_annotated_Event_when_multiple() throws IOException {
+ final EventKeyConfiguration eventKeyConfiguration = mock(EventKeyConfiguration.class);
+ when(eventKeyConfiguration.value()).thenReturn(new EventKeyFactory.EventAction[] { EventKeyFactory.EventAction.PUT, EventKeyFactory.EventAction.DELETE });
+ when(property.getAnnotation(EventKeyConfiguration.class)).thenReturn(eventKeyConfiguration);
+ when(eventKeyFactory.createEventKey(eventKeyString, EventKeyFactory.EventAction.PUT, EventKeyFactory.EventAction.DELETE)).thenReturn(eventKey);
+
+ final JsonDeserializer> contextualDeserializer = createObjectUnderTest().createContextual(deserializationContext, property);
+
+ assertThat(contextualDeserializer, notNullValue());
+ assertThat(contextualDeserializer.deserialize(parser, deserializationContext), equalTo(eventKey));
+ }
+
+ @Nested
+ class UsingRealObjectMapper {
+ private ObjectMapper objectMapper;
+
+ @BeforeEach
+ void setUp() {
+ objectMapper = new ObjectMapper();
+
+ final SimpleModule simpleModule = new SimpleModule();
+ simpleModule.addDeserializer(EventKey.class, createObjectUnderTest());
+ objectMapper.registerModule(simpleModule);
+ }
+
+ @Test
+ void quick() {
+ when(eventKeyFactory.createEventKey(eventKeyString, EventKeyFactory.EventAction.ALL)).thenReturn(eventKey);
+
+ assertThat(objectMapper.convertValue(eventKeyString, EventKey.class),
+ equalTo(eventKey));
+ }
+ }
+}
\ No newline at end of file
diff --git a/data-prepper-plugin-framework/build.gradle b/data-prepper-plugin-framework/build.gradle
index f77212a6b2..14f03fe15d 100644
--- a/data-prepper-plugin-framework/build.gradle
+++ b/data-prepper-plugin-framework/build.gradle
@@ -24,5 +24,4 @@ dependencies {
}
implementation libs.reflections.core
implementation 'com.fasterxml.jackson.core:jackson-databind'
- testImplementation testLibs.mockito.inline
}
\ No newline at end of file
diff --git a/data-prepper-plugin-framework/src/main/java/org/opensearch/dataprepper/plugin/ApplicationContextToTypedSuppliers.java b/data-prepper-plugin-framework/src/main/java/org/opensearch/dataprepper/plugin/ApplicationContextToTypedSuppliers.java
index f5ceebbde6..f9e1abddb7 100644
--- a/data-prepper-plugin-framework/src/main/java/org/opensearch/dataprepper/plugin/ApplicationContextToTypedSuppliers.java
+++ b/data-prepper-plugin-framework/src/main/java/org/opensearch/dataprepper/plugin/ApplicationContextToTypedSuppliers.java
@@ -8,6 +8,7 @@
import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSetManager;
import org.opensearch.dataprepper.model.breaker.CircuitBreaker;
import org.opensearch.dataprepper.model.event.EventFactory;
+import org.opensearch.dataprepper.model.event.EventKeyFactory;
import org.springframework.beans.factory.annotation.Autowired;
import javax.inject.Inject;
@@ -31,6 +32,7 @@ class ApplicationContextToTypedSuppliers {
@Inject
ApplicationContextToTypedSuppliers(
final EventFactory eventFactory,
+ final EventKeyFactory eventKeyFactory,
final AcknowledgementSetManager acknowledgementSetManager,
@Autowired(required = false) final CircuitBreaker circuitBreaker
) {
@@ -39,6 +41,7 @@ class ApplicationContextToTypedSuppliers {
typedSuppliers = Map.of(
EventFactory.class, () -> eventFactory,
+ EventKeyFactory.class, () -> eventKeyFactory,
AcknowledgementSetManager.class, () -> acknowledgementSetManager,
CircuitBreaker.class, () -> circuitBreaker
);
diff --git a/data-prepper-plugin-framework/src/main/java/org/opensearch/dataprepper/plugin/ObjectMapperConfiguration.java b/data-prepper-plugin-framework/src/main/java/org/opensearch/dataprepper/plugin/ObjectMapperConfiguration.java
index 5865d5b29a..ca2cea4ee8 100644
--- a/data-prepper-plugin-framework/src/main/java/org/opensearch/dataprepper/plugin/ObjectMapperConfiguration.java
+++ b/data-prepper-plugin-framework/src/main/java/org/opensearch/dataprepper/plugin/ObjectMapperConfiguration.java
@@ -8,9 +8,12 @@
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.PropertyNamingStrategies;
import com.fasterxml.jackson.databind.module.SimpleModule;
+import org.opensearch.dataprepper.model.event.EventKey;
+import org.opensearch.dataprepper.model.event.EventKeyFactory;
import org.opensearch.dataprepper.model.types.ByteCount;
import org.opensearch.dataprepper.pipeline.parser.ByteCountDeserializer;
import org.opensearch.dataprepper.pipeline.parser.DataPrepperDurationDeserializer;
+import org.opensearch.dataprepper.pipeline.parser.EventKeyDeserializer;
import org.springframework.context.annotation.Bean;
import javax.inject.Named;
@@ -38,10 +41,13 @@ ObjectMapper extensionPluginConfigObjectMapper() {
}
@Bean(name = "pluginConfigObjectMapper")
- ObjectMapper pluginConfigObjectMapper(final VariableExpander variableExpander) {
+ ObjectMapper pluginConfigObjectMapper(
+ final VariableExpander variableExpander,
+ final EventKeyFactory eventKeyFactory) {
final SimpleModule simpleModule = new SimpleModule();
simpleModule.addDeserializer(Duration.class, new DataPrepperDurationDeserializer());
simpleModule.addDeserializer(ByteCount.class, new ByteCountDeserializer());
+ simpleModule.addDeserializer(EventKey.class, new EventKeyDeserializer(eventKeyFactory));
TRANSLATE_VALUE_SUPPORTED_JAVA_TYPES.stream().forEach(clazz -> simpleModule.addDeserializer(
clazz, new DataPrepperScalarTypeDeserializer<>(variableExpander, clazz)));
diff --git a/data-prepper-plugin-framework/src/test/java/org/opensearch/dataprepper/plugin/ApplicationContextToTypedSuppliersTest.java b/data-prepper-plugin-framework/src/test/java/org/opensearch/dataprepper/plugin/ApplicationContextToTypedSuppliersTest.java
index 0cd008559a..a12540a46a 100644
--- a/data-prepper-plugin-framework/src/test/java/org/opensearch/dataprepper/plugin/ApplicationContextToTypedSuppliersTest.java
+++ b/data-prepper-plugin-framework/src/test/java/org/opensearch/dataprepper/plugin/ApplicationContextToTypedSuppliersTest.java
@@ -12,6 +12,7 @@
import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSetManager;
import org.opensearch.dataprepper.model.breaker.CircuitBreaker;
import org.opensearch.dataprepper.model.event.EventFactory;
+import org.opensearch.dataprepper.model.event.EventKeyFactory;
import java.util.Map;
import java.util.function.Supplier;
@@ -28,6 +29,9 @@ class ApplicationContextToTypedSuppliersTest {
@Mock
private EventFactory eventFactory;
+ @Mock
+ private EventKeyFactory eventKeyFactory;
+
@Mock
private AcknowledgementSetManager acknowledgementSetManager;
@@ -37,6 +41,7 @@ class ApplicationContextToTypedSuppliersTest {
private ApplicationContextToTypedSuppliers createObjectUnderTest() {
return new ApplicationContextToTypedSuppliers(
eventFactory,
+ eventKeyFactory,
acknowledgementSetManager,
circuitBreaker
);
@@ -58,12 +63,16 @@ void constructor_throws_with_null_AcknowledgementSetManager() {
void getArgumentsSuppliers_returns_map_with_expected_classes() {
final Map, Supplier> argumentsSuppliers = createObjectUnderTest().getArgumentsSuppliers();
- assertThat(argumentsSuppliers.size(), equalTo(3));
+ assertThat(argumentsSuppliers.size(), equalTo(4));
assertThat(argumentsSuppliers, hasKey(EventFactory.class));
assertThat(argumentsSuppliers.get(EventFactory.class), notNullValue());
assertThat(argumentsSuppliers.get(EventFactory.class).get(), equalTo(eventFactory));
+ assertThat(argumentsSuppliers, hasKey(EventKeyFactory.class));
+ assertThat(argumentsSuppliers.get(EventKeyFactory.class), notNullValue());
+ assertThat(argumentsSuppliers.get(EventKeyFactory.class).get(), equalTo(eventKeyFactory));
+
assertThat(argumentsSuppliers, hasKey(AcknowledgementSetManager.class));
assertThat(argumentsSuppliers.get(AcknowledgementSetManager.class), notNullValue());
assertThat(argumentsSuppliers.get(AcknowledgementSetManager.class).get(), equalTo(acknowledgementSetManager));
@@ -79,12 +88,16 @@ void getArgumentsSuppliers_returns_map_with_null_optional_CircuitBreaker() {
final Map, Supplier> argumentsSuppliers = createObjectUnderTest().getArgumentsSuppliers();
- assertThat(argumentsSuppliers.size(), equalTo(3));
+ assertThat(argumentsSuppliers.size(), equalTo(4));
assertThat(argumentsSuppliers, hasKey(EventFactory.class));
assertThat(argumentsSuppliers.get(EventFactory.class), notNullValue());
assertThat(argumentsSuppliers.get(EventFactory.class).get(), equalTo(eventFactory));
+ assertThat(argumentsSuppliers, hasKey(EventKeyFactory.class));
+ assertThat(argumentsSuppliers.get(EventKeyFactory.class), notNullValue());
+ assertThat(argumentsSuppliers.get(EventKeyFactory.class).get(), equalTo(eventKeyFactory));
+
assertThat(argumentsSuppliers, hasKey(AcknowledgementSetManager.class));
assertThat(argumentsSuppliers.get(AcknowledgementSetManager.class), notNullValue());
assertThat(argumentsSuppliers.get(AcknowledgementSetManager.class).get(), equalTo(acknowledgementSetManager));
diff --git a/data-prepper-plugin-framework/src/test/java/org/opensearch/dataprepper/plugin/ObjectMapperConfigurationTest.java b/data-prepper-plugin-framework/src/test/java/org/opensearch/dataprepper/plugin/ObjectMapperConfigurationTest.java
index d839566680..594d3a47c2 100644
--- a/data-prepper-plugin-framework/src/test/java/org/opensearch/dataprepper/plugin/ObjectMapperConfigurationTest.java
+++ b/data-prepper-plugin-framework/src/test/java/org/opensearch/dataprepper/plugin/ObjectMapperConfigurationTest.java
@@ -11,6 +11,8 @@
import org.junit.jupiter.api.extension.ExtendWith;
import org.mockito.Mock;
import org.mockito.junit.jupiter.MockitoExtension;
+import org.opensearch.dataprepper.model.event.EventKey;
+import org.opensearch.dataprepper.model.event.EventKeyFactory;
import java.time.Duration;
import java.util.Arrays;
@@ -20,6 +22,8 @@
import static org.hamcrest.MatcherAssert.assertThat;
import static org.hamcrest.Matchers.equalTo;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
@ExtendWith(MockitoExtension.class)
class ObjectMapperConfigurationTest {
@@ -28,10 +32,13 @@ class ObjectMapperConfigurationTest {
@Mock
private VariableExpander variableExpander;
+ @Mock
+ private EventKeyFactory eventKeyFactory;
+
@Test
void test_duration_with_pluginConfigObjectMapper() {
final String durationTestString = "10s";
- final ObjectMapper objectMapper = objectMapperConfiguration.pluginConfigObjectMapper(variableExpander);
+ final ObjectMapper objectMapper = objectMapperConfiguration.pluginConfigObjectMapper(variableExpander, eventKeyFactory);
final Duration duration = objectMapper.convertValue(durationTestString, Duration.class);
assertThat(duration, equalTo(Duration.ofSeconds(10)));
}
@@ -39,7 +46,7 @@ void test_duration_with_pluginConfigObjectMapper() {
@Test
void test_enum_with_pluginConfigObjectMapper() {
final String testString = "test";
- final ObjectMapper objectMapper = objectMapperConfiguration.pluginConfigObjectMapper(variableExpander);
+ final ObjectMapper objectMapper = objectMapperConfiguration.pluginConfigObjectMapper(variableExpander, eventKeyFactory);
final TestType duration = objectMapper.convertValue(testString, TestType.class);
assertThat(duration, equalTo(TestType.fromOptionValue(testString)));
}
@@ -60,6 +67,16 @@ void test_enum_with_extensionPluginConfigObjectMapper() {
assertThat(duration, equalTo(TestType.fromOptionValue(testString)));
}
+ @Test
+ void test_eventKey_with_pluginConfigObjectMapper() {
+ final String testKey = "test";
+ final EventKey eventKey = mock(EventKey.class);
+ when(eventKeyFactory.createEventKey(testKey, EventKeyFactory.EventAction.ALL)).thenReturn(eventKey);
+ final ObjectMapper objectMapper = objectMapperConfiguration.pluginConfigObjectMapper(variableExpander, eventKeyFactory);
+ final EventKey actualEventKey = objectMapper.convertValue(testKey, EventKey.class);
+ assertThat(actualEventKey, equalTo(eventKey));
+ }
+
private enum TestType {
TEST("test");
diff --git a/data-prepper-plugins/aggregate-processor/build.gradle b/data-prepper-plugins/aggregate-processor/build.gradle
index 744986e924..9a3eb4551a 100644
--- a/data-prepper-plugins/aggregate-processor/build.gradle
+++ b/data-prepper-plugins/aggregate-processor/build.gradle
@@ -19,7 +19,6 @@ dependencies {
implementation libs.opentelemetry.proto
implementation 'com.fasterxml.jackson.core:jackson-databind'
implementation 'io.micrometer:micrometer-core'
- testImplementation testLibs.mockito.inline
}
jacocoTestCoverageVerification {
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 8a6f9d5a6a..68cb6f6e65 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
@@ -20,6 +20,7 @@
import io.micrometer.core.instrument.Counter;
import org.opensearch.dataprepper.plugins.hasher.IdentificationKeysHasher;
+import java.math.BigDecimal;
import java.util.Collection;
import java.util.LinkedList;
import java.util.List;
@@ -147,6 +148,23 @@ public static long getTimeNanos(final Instant time) {
return currentTimeNanos;
}
+ public static Instant convertObjectToInstant(Object timeObject) {
+ if (timeObject instanceof Instant) {
+ return (Instant)timeObject;
+ } else if (timeObject instanceof String) {
+ return Instant.parse((String)timeObject);
+ } else if (timeObject instanceof Integer || timeObject instanceof Long) {
+ long value = ((Number)timeObject).longValue();
+ return (value > 1E10) ? Instant.ofEpochMilli(value) : Instant.ofEpochSecond(value);
+ } else if (timeObject instanceof Double || timeObject instanceof Float || timeObject instanceof BigDecimal) {
+ double value = ((Number)timeObject).doubleValue();
+ long seconds = (long) value;
+ long nanos = (long) ((value - seconds) * 1_000_000_000);
+ return Instant.ofEpochSecond(seconds, nanos);
+ } else {
+ throw new RuntimeException("Invalid format for time "+timeObject);
+ }
+ }
@Override
public void prepareForShutdown() {
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 fae6a19289..c8fd772336 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
@@ -18,15 +18,21 @@
import org.opensearch.dataprepper.plugins.processor.aggregate.AggregateActionInput;
import org.opensearch.dataprepper.plugins.processor.aggregate.AggregateActionOutput;
import org.opensearch.dataprepper.plugins.processor.aggregate.AggregateActionResponse;
+import org.opensearch.dataprepper.plugins.processor.aggregate.AggregateProcessor;
+import static org.opensearch.dataprepper.plugins.processor.aggregate.AggregateProcessor.getTimeNanos;
import org.opensearch.dataprepper.plugins.processor.aggregate.GroupState;
import io.opentelemetry.proto.metrics.v1.AggregationTemporality;
+import org.opensearch.dataprepper.plugins.hasher.IdentificationKeysHasher;
import java.time.Instant;
-import java.util.List;
import java.time.ZoneId;
import java.time.format.DateTimeFormatter;
-import java.util.Map;
+
import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
/**
* An AggregateAction that combines multiple Events into a single Event. This action will count the number of events with same keys and will create a combined event
@@ -36,28 +42,32 @@
@DataPrepperPlugin(name = "count", pluginType = AggregateAction.class, pluginConfigurationType = CountAggregateActionConfig.class)
public class CountAggregateAction implements AggregateAction {
private static final String DATE_FORMAT = "yyyy-MM-dd'T'HH:mm:ss.SSSXXX";
+ private static final String UNIQUE_KEYS_SETKEY = "__unique_keys";
private static final String exemplarKey = "__exemplar";
static final String EVENT_TYPE = "event";
- static final String SUM_METRIC_NAME = "count";
static final String SUM_METRIC_DESCRIPTION = "Number of events";
static final String SUM_METRIC_UNIT = "1";
static final boolean SUM_METRIC_IS_MONOTONIC = true;
public final String countKey;
public final String startTimeKey;
+ public final String endTimeKey;
public final String outputFormat;
private long startTimeNanos;
+ private final String metricName;
+ private final IdentificationKeysHasher uniqueKeysHasher;
@DataPrepperPluginConstructor
public CountAggregateAction(final CountAggregateActionConfig countAggregateActionConfig) {
this.countKey = countAggregateActionConfig.getCountKey();
this.startTimeKey = countAggregateActionConfig.getStartTimeKey();
+ this.endTimeKey = countAggregateActionConfig.getEndTimeKey();
this.outputFormat = countAggregateActionConfig.getOutputFormat();
- }
-
- private long getTimeNanos(Instant time) {
- final long NANO_MULTIPLIER = 1_000 * 1_000 * 1_000;
- long currentTimeNanos = time.getEpochSecond() * NANO_MULTIPLIER + time.getNano();
- return currentTimeNanos;
+ this.metricName = countAggregateActionConfig.getMetricName();
+ if (countAggregateActionConfig.getUniqueKeys() != null) {
+ this.uniqueKeysHasher = new IdentificationKeysHasher(countAggregateActionConfig.getUniqueKeys());
+ } else {
+ this.uniqueKeysHasher = null;
+ }
}
public Exemplar createExemplar(final Event event) {
@@ -81,15 +91,45 @@ public Exemplar createExemplar(final Event event) {
@Override
public AggregateActionResponse handleEvent(final Event event, final AggregateActionInput aggregateActionInput) {
final GroupState groupState = aggregateActionInput.getGroupState();
+ Instant eventStartTime = Instant.now();
+ Instant eventEndTime = eventStartTime;
+ Object startTime = event.get(startTimeKey, Object.class);
+ Object endTime = event.get(endTimeKey, Object.class);
+
+ if (startTime != null) {
+ eventStartTime = AggregateProcessor.convertObjectToInstant(startTime);
+ }
+ if (endTime != null) {
+ eventEndTime = AggregateProcessor.convertObjectToInstant(endTime);
+ }
if (groupState.get(countKey) == null) {
- groupState.put(startTimeKey, Instant.now());
groupState.putAll(aggregateActionInput.getIdentificationKeys());
+ if (uniqueKeysHasher != null) {
+ Set uniqueKeysMapSet = new HashSet<>();
+
+ uniqueKeysMapSet.add(uniqueKeysHasher.createIdentificationKeysMapFromEvent(event));
+ groupState.put(UNIQUE_KEYS_SETKEY, uniqueKeysMapSet);
+ }
groupState.put(countKey, 1);
groupState.put(exemplarKey, createExemplar(event));
+ groupState.put(startTimeKey, eventStartTime);
+ groupState.put(endTimeKey, eventEndTime);
} else {
Integer v = (Integer)groupState.get(countKey) + 1;
+
+ if (uniqueKeysHasher != null) {
+ Set uniqueKeysMapSet = (Set) groupState.get(UNIQUE_KEYS_SETKEY);
+ uniqueKeysMapSet.add(uniqueKeysHasher.createIdentificationKeysMapFromEvent(event));
+ v = uniqueKeysMapSet.size();
+ }
groupState.put(countKey, v);
- }
+ Instant groupStartTime = (Instant)groupState.get(startTimeKey);
+ Instant groupEndTime = (Instant)groupState.get(endTimeKey);
+ if (eventStartTime.isBefore(groupStartTime))
+ groupState.put(startTimeKey, eventStartTime);
+ if (eventEndTime.isAfter(groupEndTime))
+ groupState.put(endTimeKey, eventEndTime);
+ }
return AggregateActionResponse.nullEventResponse();
}
@@ -98,6 +138,9 @@ public AggregateActionOutput concludeGroup(final AggregateActionInput aggregateA
GroupState groupState = aggregateActionInput.getGroupState();
Event event;
Instant startTime = (Instant)groupState.get(startTimeKey);
+ Instant endTime = (Instant)groupState.get(endTimeKey);
+ groupState.remove(endTimeKey);
+ groupState.remove(UNIQUE_KEYS_SETKEY);
if (outputFormat.equals(OutputFormat.RAW.toString())) {
groupState.put(startTimeKey, startTime.atZone(ZoneId.of(ZoneId.systemDefault().toString())).format(DateTimeFormatter.ofPattern(DATE_FORMAT)));
event = JacksonEvent.builder()
@@ -110,14 +153,14 @@ public AggregateActionOutput concludeGroup(final AggregateActionInput aggregateA
groupState.remove(exemplarKey);
groupState.remove(countKey);
groupState.remove(startTimeKey);
- long currentTimeNanos = getTimeNanos(Instant.now());
+ long endTimeNanos = getTimeNanos(endTime);
long startTimeNanos = getTimeNanos(startTime);
Map attr = new HashMap();
groupState.forEach((k, v) -> attr.put((String)k, v));
JacksonSum sum = JacksonSum.builder()
- .withName(SUM_METRIC_NAME)
+ .withName(this.metricName)
.withDescription(SUM_METRIC_DESCRIPTION)
- .withTime(OTelProtoCodec.convertUnixNanosToISO8601(currentTimeNanos))
+ .withTime(OTelProtoCodec.convertUnixNanosToISO8601(endTimeNanos))
.withStartTime(OTelProtoCodec.convertUnixNanosToISO8601(startTimeNanos))
.withIsMonotonic(SUM_METRIC_IS_MONOTONIC)
.withUnit(SUM_METRIC_UNIT)
@@ -128,7 +171,7 @@ public AggregateActionOutput concludeGroup(final AggregateActionInput aggregateA
.build(false);
event = (Event)sum;
}
-
+
return new AggregateActionOutput(List.of(event));
}
}
diff --git a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/CountAggregateActionConfig.java b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/CountAggregateActionConfig.java
index cbe5ebb20b..1144aee261 100644
--- a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/CountAggregateActionConfig.java
+++ b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/CountAggregateActionConfig.java
@@ -5,28 +5,52 @@
package org.opensearch.dataprepper.plugins.processor.aggregate.actions;
-import java.util.Set;
import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
import com.fasterxml.jackson.annotation.JsonProperty;
public class CountAggregateActionConfig {
+ static final String SUM_METRIC_NAME = "count";
public static final String DEFAULT_COUNT_KEY = "aggr._count";
public static final String DEFAULT_START_TIME_KEY = "aggr._start_time";
+ public static final String DEFAULT_END_TIME_KEY = "aggr._end_time";
public static final Set validOutputFormats = new HashSet<>(Set.of(OutputFormat.OTEL_METRICS.toString(), OutputFormat.RAW.toString()));
@JsonProperty("count_key")
String countKey = DEFAULT_COUNT_KEY;
+ @JsonProperty("metric_name")
+ String metricName = SUM_METRIC_NAME;
+
+ @JsonProperty("unique_keys")
+ List uniqueKeys = null;
+
@JsonProperty("start_time_key")
String startTimeKey = DEFAULT_START_TIME_KEY;
+ @JsonProperty("end_time_key")
+ String endTimeKey = DEFAULT_END_TIME_KEY;
+
@JsonProperty("output_format")
String outputFormat = OutputFormat.OTEL_METRICS.toString();
+ public String getMetricName() {
+ return metricName;
+ }
+
+ public List getUniqueKeys() {
+ return uniqueKeys;
+ }
+
public String getCountKey() {
return countKey;
}
+ public String getEndTimeKey() {
+ return endTimeKey;
+ }
+
public String getStartTimeKey() {
return startTimeKey;
}
@@ -37,4 +61,4 @@ public String getOutputFormat() {
}
return outputFormat;
}
-}
+}
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 5e93f305bc..bdb9a3fad6 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
@@ -17,6 +17,7 @@
import org.opensearch.dataprepper.model.event.JacksonEvent;
import org.opensearch.dataprepper.plugins.processor.aggregate.AggregateAction;
import org.opensearch.dataprepper.plugins.processor.aggregate.AggregateActionInput;
+import org.opensearch.dataprepper.plugins.processor.aggregate.AggregateProcessor;
import org.opensearch.dataprepper.plugins.processor.aggregate.AggregateActionOutput;
import org.opensearch.dataprepper.plugins.processor.aggregate.AggregateActionResponse;
import org.opensearch.dataprepper.plugins.processor.aggregate.GroupState;
@@ -35,15 +36,14 @@
import java.util.ArrayList;
/**
- * An AggregateAction that combines multiple Events into a single Event. This action will create a combined event with histogram buckets of the values
- * of specified list of keys from the groupState on concludeGroup.
+ * An AggregateAction that combines multiple Events into a single Event. This action will create a combined event with histogram buckets of the values
+ * of specified list of keys from the groupState on concludeGroup.
* @since 2.1
*/
@DataPrepperPlugin(name = "histogram", pluginType = AggregateAction.class, pluginConfigurationType = HistogramAggregateActionConfig.class)
public class HistogramAggregateAction implements AggregateAction {
private static final String DATE_FORMAT = "yyyy-MM-dd'T'HH:mm:ss.SSSXXX";
private static final String EVENT_TYPE = "event";
- public static final String HISTOGRAM_METRIC_NAME = "histogram";
private final String countKey;
private final String bucketCountsKey;
private final String bucketsKey;
@@ -61,6 +61,7 @@ public class HistogramAggregateAction implements AggregateAction {
private Event maxEvent;
private double minValue;
private double maxValue;
+ private final String metricName;
private long startTimeNanos;
private double[] buckets;
@@ -71,6 +72,7 @@ public HistogramAggregateAction(final HistogramAggregateActionConfig histogramAg
List bucketList = histogramAggregateActionConfig.getBuckets();
this.buckets = new double[bucketList.size()+2];
int bucketIdx = 0;
+ this.metricName = histogramAggregateActionConfig.getMetricName();
this.buckets[bucketIdx++] = -Float.MAX_VALUE;
for (int i = 0; i < bucketList.size(); i++) {
this.buckets[bucketIdx++] = convertToDouble(bucketList.get(i));
@@ -137,16 +139,29 @@ public AggregateActionResponse handleEvent(final Event event, final AggregateAct
return AggregateActionResponse.nullEventResponse();
}
double doubleValue = convertToDouble(value);
-
+
int idx = Arrays.binarySearch(this.buckets, doubleValue);
if (idx < 0) {
idx = -idx-2;
}
+ Instant eventTime = Instant.now();
+ Instant eventStartTime = eventTime;
+ Instant eventEndTime = eventTime;
+ Object startTime = event.get(startTimeKey, Object.class);
+ Object endTime = event.get(endTimeKey, Object.class);
+ if (startTime != null) {
+ eventStartTime = AggregateProcessor.convertObjectToInstant(startTime);
+ }
+ if (endTime != null) {
+ eventEndTime = AggregateProcessor.convertObjectToInstant(endTime);
+ }
if (groupState.get(bucketCountsKey) == null) {
+ groupState.put(startTimeKey, eventStartTime);
+ groupState.put(endTimeKey, eventEndTime);
Long[] bucketCountsList = new Long[buckets.length-1];
Arrays.fill(bucketCountsList, (long)0);
bucketCountsList[idx]++;
- groupState.put(startTimeKey, Instant.now());
+ groupState.put(startTimeKey, eventTime);
groupState.putAll(aggregateActionInput.getIdentificationKeys());
groupState.put(sumKey, doubleValue);
groupState.put(countKey, 1);
@@ -180,9 +195,13 @@ public AggregateActionResponse handleEvent(final Event event, final AggregateAct
maxValue = doubleValue;
}
}
- }
- // Keep over-writing endTime to get the last time a record of this group received
- groupState.put(endTimeKey, Instant.now());
+ Instant groupStartTime = (Instant)groupState.get(startTimeKey);
+ Instant groupEndTime = (Instant)groupState.get(endTimeKey);
+ if (eventStartTime.isBefore(groupStartTime))
+ groupState.put(startTimeKey, eventStartTime);
+ if (eventEndTime.isAfter(groupEndTime))
+ groupState.put(endTimeKey, eventEndTime);
+ }
return AggregateActionResponse.nullEventResponse();
}
@@ -194,7 +213,7 @@ public AggregateActionOutput concludeGroup(final AggregateActionInput aggregateA
Instant endTime = (Instant)groupState.get(endTimeKey);
long startTimeNanos = getTimeNanos(startTime);
long endTimeNanos = getTimeNanos(endTime);
- String histogramKey = HISTOGRAM_METRIC_NAME + "_key";
+ String histogramKey = this.metricName + "_key";
List exemplarList = new ArrayList<>();
exemplarList.add(createExemplar("min", minEvent, minValue));
exemplarList.add(createExemplar("max", maxEvent, maxValue));
@@ -227,7 +246,7 @@ public AggregateActionOutput concludeGroup(final AggregateActionInput aggregateA
Integer count = (Integer)groupState.get(countKey);
String description = String.format("Histogram of %s in the events", key);
JacksonHistogram histogram = JacksonHistogram.builder()
- .withName(HISTOGRAM_METRIC_NAME)
+ .withName(this.metricName)
.withDescription(description)
.withTime(OTelProtoCodec.convertUnixNanosToISO8601(endTimeNanos))
.withStartTime(OTelProtoCodec.convertUnixNanosToISO8601(startTimeNanos))
@@ -247,7 +266,7 @@ public AggregateActionOutput concludeGroup(final AggregateActionInput aggregateA
.build(false);
event = (Event)histogram;
}
-
+
return new AggregateActionOutput(List.of(event));
}
}
diff --git a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/HistogramAggregateActionConfig.java b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/HistogramAggregateActionConfig.java
index a173671836..7c998c123d 100644
--- a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/HistogramAggregateActionConfig.java
+++ b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/HistogramAggregateActionConfig.java
@@ -12,6 +12,7 @@
import jakarta.validation.constraints.NotNull;
public class HistogramAggregateActionConfig {
+ public static final String HISTOGRAM_METRIC_NAME = "histogram";
public static final String DEFAULT_GENERATED_KEY_PREFIX = "aggr._";
public static final String SUM_KEY = "sum";
public static final String COUNT_KEY = "count";
@@ -32,6 +33,9 @@ public class HistogramAggregateActionConfig {
@NotNull
String units;
+ @JsonProperty("metric_name")
+ String metricName = HISTOGRAM_METRIC_NAME;
+
@JsonProperty("generated_key_prefix")
String generatedKeyPrefix = DEFAULT_GENERATED_KEY_PREFIX;
@@ -45,6 +49,10 @@ public class HistogramAggregateActionConfig {
@JsonProperty("record_minmax")
boolean recordMinMax = false;
+ public String getMetricName() {
+ return metricName;
+ }
+
public boolean getRecordMinMax() {
return recordMinMax;
}
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 ea7b6eb416..fc416b0e45 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
@@ -533,7 +533,7 @@ void aggregateWithHistogramAggregateAction() throws InterruptedException, NoSuch
countDownLatch.countDown();
});
}
- Thread.sleep(GROUP_DURATION_FOR_ONLY_SINGLE_CONCLUDE * 1000);
+ Thread.sleep(GROUP_DURATION_FOR_ONLY_SINGLE_CONCLUDE * 1500);
boolean allThreadsFinished = countDownLatch.await(5L, TimeUnit.SECONDS);
assertThat(allThreadsFinished, equalTo(true));
diff --git a/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateProcessorStaticFunctionsTest.java b/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateProcessorStaticFunctionsTest.java
new file mode 100644
index 0000000000..8c9892ab29
--- /dev/null
+++ b/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateProcessorStaticFunctionsTest.java
@@ -0,0 +1,40 @@
+/*
+ * Copyright OpenSearch Contributors
+ * SPDX-License-Identifier: Apache-2.0
+ */
+
+package org.opensearch.dataprepper.plugins.processor.aggregate;
+
+import org.junit.jupiter.api.Test;
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+import java.time.Instant;
+import java.time.Duration;
+
+public class AggregateProcessorStaticFunctionsTest {
+ @Test
+ public void testConvertObjectToInstant() {
+ Instant now = Instant.now();
+ assertThat(AggregateProcessor.convertObjectToInstant(now), equalTo(now));
+ String nowStr = now.toString();
+ long nowSeconds = now.getEpochSecond();
+ long nowMillis = now.toEpochMilli();
+ int nowNanos = now.getNano();
+ double nowDouble = nowSeconds+(double)nowNanos/1000_000_000;
+ assertThat(AggregateProcessor.convertObjectToInstant(nowStr), equalTo(now));
+ assertThat(AggregateProcessor.convertObjectToInstant(nowSeconds), equalTo(Instant.ofEpochSecond(nowSeconds)));
+ assertThat(AggregateProcessor.convertObjectToInstant(nowMillis), equalTo(Instant.ofEpochMilli(nowMillis)));
+ Duration tolerance = Duration.ofNanos(1000);
+ assertTrue((Duration.between(AggregateProcessor.convertObjectToInstant(nowDouble), Instant.ofEpochSecond(nowSeconds, nowNanos))).abs().compareTo(tolerance) <= 0);
+ }
+
+ @Test
+ public void testGetTimeNanos() {
+ Instant now = Instant.now();
+ assertThat(AggregateProcessor.getTimeNanos(now) / 1000_000_000, equalTo(now.getEpochSecond()));
+ assertThat(AggregateProcessor.getTimeNanos(now) % 1000_000_000, equalTo((long)now.getNano()));
+ }
+}
+
diff --git a/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/CountAggregateActionConfigTests.java b/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/CountAggregateActionConfigTests.java
index f022ac9148..1975918e37 100644
--- a/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/CountAggregateActionConfigTests.java
+++ b/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/CountAggregateActionConfigTests.java
@@ -13,6 +13,8 @@
import static org.opensearch.dataprepper.plugins.processor.aggregate.actions.CountAggregateActionConfig.DEFAULT_COUNT_KEY;
import static org.opensearch.dataprepper.plugins.processor.aggregate.actions.CountAggregateActionConfig.DEFAULT_START_TIME_KEY;
+import java.util.ArrayList;
+import java.util.List;
import java.util.UUID;
import static org.hamcrest.CoreMatchers.equalTo;
@@ -38,6 +40,8 @@ void testDefault() {
assertThat(countAggregateActionConfig.getCountKey(), equalTo(DEFAULT_COUNT_KEY));
assertThat(countAggregateActionConfig.getStartTimeKey(), equalTo(DEFAULT_START_TIME_KEY));
assertThat(countAggregateActionConfig.getOutputFormat(), equalTo(OutputFormat.OTEL_METRICS.toString()));
+ assertThat(countAggregateActionConfig.getMetricName(), equalTo(CountAggregateActionConfig.SUM_METRIC_NAME));
+ assertThat(countAggregateActionConfig.getUniqueKeys(), equalTo(null));
}
@Test
@@ -51,6 +55,14 @@ void testValidConfig() throws NoSuchFieldException, IllegalAccessException {
final String testOutputFormat = OutputFormat.OTEL_METRICS.toString();
setField(CountAggregateActionConfig.class, countAggregateActionConfig, "outputFormat", testOutputFormat);
assertThat(countAggregateActionConfig.getOutputFormat(), equalTo(OutputFormat.OTEL_METRICS.toString()));
+ final String testName = UUID.randomUUID().toString();
+ setField(CountAggregateActionConfig.class, countAggregateActionConfig, "metricName", testName);
+ assertThat(countAggregateActionConfig.getMetricName(), equalTo(testName));
+ final List uniqueKeys = new ArrayList<>();
+ uniqueKeys.add(UUID.randomUUID().toString());
+ uniqueKeys.add(UUID.randomUUID().toString());
+ setField(CountAggregateActionConfig.class, countAggregateActionConfig, "uniqueKeys", uniqueKeys);
+ assertThat(countAggregateActionConfig.getUniqueKeys(), equalTo(uniqueKeys));
}
@Test
diff --git a/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/CountAggregateActionTest.java b/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/CountAggregateActionTest.java
index 66936fa7f8..af81ca001f 100644
--- a/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/CountAggregateActionTest.java
+++ b/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/CountAggregateActionTest.java
@@ -25,8 +25,12 @@
import java.util.HashMap;
import java.util.Map;
import java.util.List;
+import java.util.Random;
import java.util.UUID;
+import java.time.Instant;
+import static org.mockito.Mockito.when;
+import static org.mockito.Mockito.mock;
import static org.hamcrest.CoreMatchers.equalTo;
import static org.hamcrest.CoreMatchers.not;
import static org.hamcrest.MatcherAssert.assertThat;
@@ -46,6 +50,7 @@ private AggregateAction createObjectUnderTest(CountAggregateActionConfig config)
@ParameterizedTest
@ValueSource(ints = {1, 2, 10, 100})
void testCountAggregate(int testCount) throws NoSuchFieldException, IllegalAccessException {
+ final String testName = UUID.randomUUID().toString();
CountAggregateActionConfig countAggregateActionConfig = new CountAggregateActionConfig();
setField(CountAggregateActionConfig.class, countAggregateActionConfig, "outputFormat", OutputFormat.RAW.toString());
countAggregateAction = createObjectUnderTest(countAggregateActionConfig);
@@ -75,8 +80,10 @@ void testCountAggregate(int testCount) throws NoSuchFieldException, IllegalAcces
@ParameterizedTest
@ValueSource(ints = {1, 2, 10, 100})
- void testCountAggregateOTelFormat(int testCount) {
+ void testCountAggregateOTelFormat(int testCount) throws NoSuchFieldException, IllegalAccessException {
CountAggregateActionConfig countAggregateActionConfig = new CountAggregateActionConfig();
+ final String testName = UUID.randomUUID().toString();
+ setField(CountAggregateActionConfig.class, countAggregateActionConfig, "metricName", testName);
countAggregateAction = createObjectUnderTest(countAggregateActionConfig);
final String key1 = "key-"+UUID.randomUUID().toString();
final String value1 = UUID.randomUUID().toString();
@@ -115,6 +122,7 @@ void testCountAggregateOTelFormat(int testCount) {
expectedEventMap.put("isMonotonic", true);
expectedEventMap.put("aggregationTemporality", "AGGREGATION_TEMPORALITY_DELTA");
expectedEventMap.put("unit", "1");
+ expectedEventMap.put("name", testName);
expectedEventMap.forEach((k, v) -> assertThat(result.get(0).toMap(), hasEntry(k,v)));
assertThat(result.get(0).toMap().get("attributes"), equalTo(eventMap));
JacksonMetric metric = (JacksonMetric) result.get(0);
@@ -139,4 +147,144 @@ void testCountAggregateOTelFormat(int testCount) {
assertThat(attributes.get(key2), equalTo(value2));
assertTrue(attributes.containsKey(dataKey2));
}
+
+ @ParameterizedTest
+ @ValueSource(ints = {1, 2, 10, 100})
+ void testCountAggregateOTelFormatWithStartAndEndTimesInTheEvent(int testCount) {
+ CountAggregateActionConfig mockConfig = mock(CountAggregateActionConfig.class);
+ when(mockConfig.getCountKey()).thenReturn(CountAggregateActionConfig.DEFAULT_COUNT_KEY);
+ when(mockConfig.getUniqueKeys()).thenReturn(null);
+ final String testName = UUID.randomUUID().toString();
+ when(mockConfig.getMetricName()).thenReturn(testName);
+ String startTimeKey = UUID.randomUUID().toString();
+ String endTimeKey = UUID.randomUUID().toString();
+ when(mockConfig.getStartTimeKey()).thenReturn(startTimeKey);
+ when(mockConfig.getEndTimeKey()).thenReturn(endTimeKey);
+ when(mockConfig.getOutputFormat()).thenReturn(OutputFormat.OTEL_METRICS.toString());
+ countAggregateAction = createObjectUnderTest(mockConfig);
+ final String key1 = "key-"+UUID.randomUUID().toString();
+ final String value1 = UUID.randomUUID().toString();
+ final String dataKey1 = "datakey-"+UUID.randomUUID().toString();
+ final String key2 = "key-"+UUID.randomUUID().toString();
+ final String value2 = UUID.randomUUID().toString();
+ final String dataKey2 = "datakey-"+UUID.randomUUID().toString();
+ final Instant testTime = Instant.ofEpochSecond(Instant.now().getEpochSecond());
+ Map eventMap = Collections.singletonMap(key1, value1);
+ Event testEvent = JacksonEvent.builder()
+ .withEventType("event")
+ .withData(eventMap)
+ .build();
+ Map eventMap2 = Collections.singletonMap(key2, value2);
+ JacksonEvent testEvent2 = JacksonEvent.builder()
+ .withEventType("event")
+ .withData(eventMap2)
+ .build();
+ AggregateActionInput aggregateActionInput = new AggregateActionTestUtils.TestAggregateActionInput(eventMap);
+ AggregateActionInput aggregateActionInput2 = new AggregateActionTestUtils.TestAggregateActionInput(eventMap2);
+ Random random = new Random();
+ for (int i = 0; i < testCount; i++) {
+ testEvent.put(dataKey1, UUID.randomUUID().toString());
+ Instant sTime = (i == 0) ? testTime : testTime.plusSeconds(random.nextInt(5));
+ Instant eTime = (i == testCount-1) ? testTime.plusSeconds(100) : testTime.plusSeconds (50+random.nextInt(45));
+ testEvent.put(startTimeKey, sTime);
+ testEvent.put(endTimeKey, eTime);
+ testEvent2.put(dataKey2, UUID.randomUUID().toString());
+ testEvent2.put(startTimeKey, sTime.toString());
+ testEvent2.put(endTimeKey, eTime.toString());
+ AggregateActionResponse aggregateActionResponse = countAggregateAction.handleEvent(testEvent, aggregateActionInput);
+ assertThat(aggregateActionResponse.getEvent(), equalTo(null));
+ aggregateActionResponse = countAggregateAction.handleEvent(testEvent2, aggregateActionInput2);
+ assertThat(aggregateActionResponse.getEvent(), equalTo(null));
+ }
+
+ AggregateActionOutput actionOutput = countAggregateAction.concludeGroup(aggregateActionInput);
+ final List result = actionOutput.getEvents();
+ assertThat(result.size(), equalTo(1));
+ Map expectedEventMap = new HashMap<>();
+ expectedEventMap.put("value", (double)testCount);
+ expectedEventMap.put("name", testName);
+ expectedEventMap.put("description", "Number of events");
+ expectedEventMap.put("isMonotonic", true);
+ expectedEventMap.put("aggregationTemporality", "AGGREGATION_TEMPORALITY_DELTA");
+ expectedEventMap.put("unit", "1");
+ expectedEventMap.forEach((k, v) -> assertThat(result.get(0).toMap(), hasEntry(k,v)));
+ assertThat(result.get(0).toMap().get("attributes"), equalTo(eventMap));
+ JacksonMetric metric = (JacksonMetric) result.get(0);
+ assertThat(metric.toJsonString().indexOf("attributes"), not(-1));
+ assertThat(result.get(0).get("startTime", String.class), equalTo(testTime.toString()));
+ assertThat(result.get(0).get("time", String.class), equalTo(testTime.plusSeconds(100).toString()));
+
+ assertThat(result.get(0).toMap(), hasKey("startTime"));
+ assertThat(result.get(0).toMap(), hasKey("time"));
+ List> exemplars = (List >)result.get(0).toMap().get("exemplars");
+ assertThat(exemplars.size(), equalTo(1));
+ Map exemplar = exemplars.get(0);
+ Map attributes = (Map)exemplar.get("attributes");
+ assertThat(attributes.get(key1), equalTo(value1));
+ assertTrue(attributes.containsKey(dataKey1));
+
+ actionOutput = countAggregateAction.concludeGroup(aggregateActionInput2);
+ final List result2 = actionOutput.getEvents();
+ assertThat(result2.size(), equalTo(1));
+
+ exemplars = (List >)result2.get(0).toMap().get("exemplars");
+ assertThat(exemplars.size(), equalTo(1));
+ exemplar = exemplars.get(0);
+ attributes = (Map)exemplar.get("attributes");
+ assertThat(attributes.get(key2), equalTo(value2));
+ assertTrue(attributes.containsKey(dataKey2));
+ }
+
+ @ParameterizedTest
+ @ValueSource(ints = {1, 2, 3, 10, 20})
+ void testCountAggregateOTelFormatUniqueKeys(int testCount) throws NoSuchFieldException, IllegalAccessException {
+ CountAggregateActionConfig countAggregateActionConfig = new CountAggregateActionConfig();
+ final String testName = UUID.randomUUID().toString();
+ setField(CountAggregateActionConfig.class, countAggregateActionConfig, "metricName", testName);
+ final String key1 = "key-"+UUID.randomUUID().toString();
+ final String value1 = UUID.randomUUID().toString();
+ final String dataKey1 = "datakey-"+UUID.randomUUID().toString();
+ setField(CountAggregateActionConfig.class, countAggregateActionConfig, "uniqueKeys", List.of(dataKey1));
+ countAggregateAction = createObjectUnderTest(countAggregateActionConfig);
+ Map eventMap = Collections.singletonMap(key1, value1);
+ Event testEvent = JacksonEvent.builder()
+ .withEventType("event")
+ .withData(eventMap)
+ .build();
+ AggregateActionInput aggregateActionInput = new AggregateActionTestUtils.TestAggregateActionInput(eventMap);
+ final String dataKey1_1 = UUID.randomUUID().toString();
+ final String dataKey1_2 = UUID.randomUUID().toString();
+ final String dataKey1_3 = UUID.randomUUID().toString();
+ final String[] dataKeysList = {dataKey1_1, dataKey1_2, dataKey1_3};
+ for (int i = 0; i < testCount; i++) {
+ testEvent.put(dataKey1, dataKeysList[i % 3]);
+ AggregateActionResponse aggregateActionResponse = countAggregateAction.handleEvent(testEvent, aggregateActionInput);
+ assertThat(aggregateActionResponse.getEvent(), equalTo(null));
+ }
+
+ AggregateActionOutput actionOutput = countAggregateAction.concludeGroup(aggregateActionInput);
+ final List result = actionOutput.getEvents();
+ assertThat(result.size(), equalTo(1));
+ Map expectedEventMap = new HashMap<>();
+ double expectedCount = (testCount >= 3) ? 3 : testCount;
+ expectedEventMap.put("value", expectedCount);
+ expectedEventMap.put("description", "Number of events");
+ expectedEventMap.put("isMonotonic", true);
+ expectedEventMap.put("aggregationTemporality", "AGGREGATION_TEMPORALITY_DELTA");
+ expectedEventMap.put("unit", "1");
+ expectedEventMap.put("name", testName);
+ expectedEventMap.forEach((k, v) -> assertThat(result.get(0).toMap(), hasEntry(k,v)));
+ assertThat(result.get(0).toMap().get("attributes"), equalTo(eventMap));
+ JacksonMetric metric = (JacksonMetric) result.get(0);
+ assertThat(metric.toJsonString().indexOf("attributes"), not(-1));
+ assertThat(result.get(0).toMap(), hasKey("startTime"));
+ assertThat(result.get(0).toMap(), hasKey("time"));
+ List> exemplars = (List >)result.get(0).toMap().get("exemplars");
+ assertThat(exemplars.size(), equalTo(1));
+ Map exemplar = exemplars.get(0);
+ Map attributes = (Map)exemplar.get("attributes");
+ assertThat(attributes.get(key1), equalTo(value1));
+ assertTrue(attributes.containsKey(dataKey1));
+
+ }
}
diff --git a/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/HistogramAggregateActionConfigTests.java b/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/HistogramAggregateActionConfigTests.java
index f3e1e19d25..60ba8dc202 100644
--- a/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/HistogramAggregateActionConfigTests.java
+++ b/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/HistogramAggregateActionConfigTests.java
@@ -11,6 +11,8 @@
import org.mockito.junit.jupiter.MockitoExtension;
import org.apache.commons.lang3.RandomStringUtils;
+import java.util.UUID;
+
import static org.opensearch.dataprepper.plugins.processor.aggregate.actions.HistogramAggregateActionConfig.DEFAULT_GENERATED_KEY_PREFIX;
import java.util.concurrent.ThreadLocalRandom;
@@ -41,6 +43,7 @@ void testDefault() {
assertThat(histogramAggregateActionConfig.getGeneratedKeyPrefix(), equalTo(DEFAULT_GENERATED_KEY_PREFIX));
assertThat(histogramAggregateActionConfig.getRecordMinMax(), equalTo(false));
assertThat(histogramAggregateActionConfig.getOutputFormat(), equalTo(OutputFormat.OTEL_METRICS.toString()));
+ assertThat(histogramAggregateActionConfig.getMetricName(), equalTo(HistogramAggregateActionConfig.HISTOGRAM_METRIC_NAME));
}
@Test
@@ -106,6 +109,9 @@ void testValidConfig() throws NoSuchFieldException, IllegalAccessException {
longBuckets.add(longValue2);
setField(HistogramAggregateActionConfig.class, histogramAggregateActionConfig, "buckets", longBuckets);
assertThat(histogramAggregateActionConfig.getBuckets(), containsInAnyOrder(longBuckets.toArray()));
+ final String testName = UUID.randomUUID().toString();
+ setField(HistogramAggregateActionConfig.class, histogramAggregateActionConfig, "metricName", testName);
+ assertThat(histogramAggregateActionConfig.getMetricName(), equalTo(testName));
}
@Test
diff --git a/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/HistogramAggregateActionTests.java b/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/HistogramAggregateActionTests.java
index b2b498306b..155acee918 100644
--- a/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/HistogramAggregateActionTests.java
+++ b/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/HistogramAggregateActionTests.java
@@ -23,13 +23,18 @@
import org.apache.commons.lang3.RandomStringUtils;
import java.util.Arrays;
+import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
import java.util.List;
-import java.util.ArrayList;
+import java.util.Random;
+import java.util.UUID;
+import java.time.Instant;
import java.util.concurrent.ThreadLocalRandom;
+import static org.mockito.Mockito.when;
+import static org.mockito.Mockito.mock;
import static org.hamcrest.CoreMatchers.equalTo;
import static org.hamcrest.CoreMatchers.not;
import static org.hamcrest.MatcherAssert.assertThat;
@@ -193,7 +198,7 @@ void testHistogramAggregateOTelFormat(int testCount) throws NoSuchFieldException
final String expectedStartTimeKey = histogramAggregateActionConfig.getStartTimeKey();
Map expectedEventMap = new HashMap<>(Collections.singletonMap("count", (long)testCount));
expectedEventMap.put("unit", testUnits);
- expectedEventMap.put("name", HistogramAggregateAction.HISTOGRAM_METRIC_NAME);
+ expectedEventMap.put("name", HistogramAggregateActionConfig.HISTOGRAM_METRIC_NAME);
expectedEventMap.put("sum", expectedSum);
expectedEventMap.put("min", expectedMin);
expectedEventMap.put("max", expectedMax);
@@ -207,7 +212,7 @@ void testHistogramAggregateOTelFormat(int testCount) throws NoSuchFieldException
for (int i = 0; i < expectedBucketCounts.length; i++) {
assertThat(expectedBucketCounts[i], equalTo(bucketCountsFromResult.get(i)));
}
- assertThat(((Map)result.get(0).toMap().get("attributes")), hasEntry(HistogramAggregateAction.HISTOGRAM_METRIC_NAME+"_key", testKey));
+ assertThat(((Map)result.get(0).toMap().get("attributes")), hasEntry(HistogramAggregateActionConfig.HISTOGRAM_METRIC_NAME+"_key", testKey));
List exemplars = (List )result.get(0).toMap().get("exemplars");
assertThat(exemplars.size(), equalTo(2));
assertThat(((Map)result.get(0).toMap().get("attributes")), hasEntry(dataKey, dataValue));
@@ -235,4 +240,134 @@ void testHistogramAggregateOTelFormat(int testCount) throws NoSuchFieldException
}
}
}
+
+ @ParameterizedTest
+ @ValueSource(ints = {10, 20, 50, 100})
+ void testHistogramAggregateOTelFormatWithStartAndEndTimesInTheEvent(int testCount) throws NoSuchFieldException, IllegalAccessException {
+ HistogramAggregateActionConfig mockConfig = mock(HistogramAggregateActionConfig.class);
+ String startTimeKey = UUID.randomUUID().toString();
+ String endTimeKey = UUID.randomUUID().toString();
+ final String testKeyPrefix = RandomStringUtils.randomAlphabetic(5)+"_";
+ when(mockConfig.getStartTimeKey()).thenReturn(startTimeKey);
+ when(mockConfig.getEndTimeKey()).thenReturn(endTimeKey);
+ final String testName = UUID.randomUUID().toString();
+ when(mockConfig.getMetricName()).thenReturn(testName);
+ when(mockConfig.getOutputFormat()).thenReturn(OutputFormat.OTEL_METRICS.toString());
+ String keyPrefix = UUID.randomUUID().toString();
+ final String testUnits = "ms";
+ when(mockConfig.getUnits()).thenReturn(testUnits);
+ when(mockConfig.getRecordMinMax()).thenReturn(true);
+ final double TEST_VALUE_RANGE_MIN = 0.0;
+ final double TEST_VALUE_RANGE_MAX = 6.0;
+ final double TEST_VALUE_RANGE_STEP = 2.0;
+ final double bucket1 = TEST_VALUE_RANGE_MIN;
+ final double bucket2 = bucket1 + TEST_VALUE_RANGE_STEP;
+ final double bucket3 = bucket2 + TEST_VALUE_RANGE_STEP;
+ List buckets = new ArrayList();
+ buckets.add(bucket1);
+ buckets.add(bucket2);
+ buckets.add(bucket3);
+ when(mockConfig.getBuckets()).thenReturn(buckets);
+ final String testKey = RandomStringUtils.randomAlphabetic(10);
+ when(mockConfig.getKey()).thenReturn(testKey);
+ final String testPrefix = RandomStringUtils.randomAlphabetic(7);
+ when(mockConfig.getSumKey()).thenReturn(testPrefix+"sum");
+ when(mockConfig.getMinKey()).thenReturn(testPrefix+"min");
+ when(mockConfig.getMaxKey()).thenReturn(testPrefix+"max");
+ when(mockConfig.getCountKey()).thenReturn(testPrefix+"count");
+ when(mockConfig.getBucketsKey()).thenReturn(testPrefix+"buckets");
+ when(mockConfig.getBucketCountsKey()).thenReturn(testPrefix+"bucketcounts");
+ when(mockConfig.getDurationKey()).thenReturn(testPrefix+"duration");
+ histogramAggregateAction = createObjectUnderTest(mockConfig);
+ final String dataKey = RandomStringUtils.randomAlphabetic(10);
+ final String dataValue = RandomStringUtils.randomAlphabetic(15);
+ final AggregateActionInput aggregateActionInput = new AggregateActionTestUtils.TestAggregateActionInput(Map.of(dataKey, dataValue));
+ Long[] expectedBucketCounts = new Long[buckets.size()+1];
+ double expectedSum = 0.0;
+ double expectedMin = TEST_VALUE_RANGE_MAX+TEST_VALUE_RANGE_STEP+1.0;
+ double expectedMax = TEST_VALUE_RANGE_MIN-TEST_VALUE_RANGE_STEP-1.0;
+ Arrays.fill(expectedBucketCounts, (long)0);
+ Random random = new Random();
+ final Instant testTime = Instant.ofEpochSecond(Instant.now().getEpochSecond());
+ for (int i = 0; i < testCount; i++) {
+ final double value = ThreadLocalRandom.current().nextDouble(TEST_VALUE_RANGE_MIN-TEST_VALUE_RANGE_STEP, TEST_VALUE_RANGE_MAX+TEST_VALUE_RANGE_STEP);
+ if (value < bucket1) {
+ expectedBucketCounts[0]++;
+ } else if (value < bucket2) {
+ expectedBucketCounts[1]++;
+ } else if (value < bucket3) {
+ expectedBucketCounts[2]++;
+ } else {
+ expectedBucketCounts[3]++;
+ }
+ expectedSum += value;
+ if (value < expectedMin) {
+ expectedMin = value;
+ }
+ if (value > expectedMax) {
+ expectedMax = value;
+ }
+ Instant sTime = (i == 0) ? testTime : testTime.plusSeconds(random.nextInt(5));
+ Instant eTime = (i == testCount-1) ? testTime.plusSeconds(100) : testTime.plusSeconds (50+random.nextInt(45));
+ Map eventMap = Collections.synchronizedMap(Map.of(testKey, value, startTimeKey, sTime, endTimeKey, eTime));
+ Event testEvent = JacksonEvent.builder()
+ .withEventType("event")
+ .withData(eventMap)
+ .build();
+ final AggregateActionResponse aggregateActionResponse = histogramAggregateAction.handleEvent(testEvent, aggregateActionInput);
+ assertThat(aggregateActionResponse.getEvent(), equalTo(null));
+ }
+
+ final AggregateActionOutput actionOutput = histogramAggregateAction.concludeGroup(aggregateActionInput);
+ final List result = actionOutput.getEvents();
+ assertThat(result.size(), equalTo(1));
+ final String expectedCountKey = mockConfig.getCountKey();
+ final String expectedStartTimeKey = mockConfig.getStartTimeKey();
+ Map expectedEventMap = new HashMap<>(Collections.singletonMap("count", (long)testCount));
+ expectedEventMap.put("unit", testUnits);
+ expectedEventMap.put("name", testName);
+ expectedEventMap.put("sum", expectedSum);
+ expectedEventMap.put("min", expectedMin);
+ expectedEventMap.put("max", expectedMax);
+ expectedEventMap.put("bucketCounts", expectedBucketCounts.length);
+ expectedEventMap.put("explicitBoundsCount", expectedBucketCounts.length-1);
+
+ expectedEventMap.forEach((k, v) -> assertThat(result.get(0).toMap(), hasEntry(k, v)));
+ assertThat(result.get(0).toMap(), hasKey("startTime"));
+ assertThat(result.get(0).toMap(), hasKey("time"));
+ final List bucketCountsFromResult = (ArrayList)result.get(0).toMap().get("bucketCountsList");
+ for (int i = 0; i < expectedBucketCounts.length; i++) {
+ assertThat(expectedBucketCounts[i], equalTo(bucketCountsFromResult.get(i)));
+ }
+ assertThat(((Map)result.get(0).toMap().get("attributes")), hasEntry(testName+"_key", testKey));
+ List exemplars = (List )result.get(0).toMap().get("exemplars");
+ assertThat(exemplars.size(), equalTo(2));
+ assertThat(((Map)result.get(0).toMap().get("attributes")), hasEntry(dataKey, dataValue));
+ final String expectedDurationKey = mockConfig.getDurationKey();
+ assertThat(((Map)result.get(0).toMap().get("attributes")), hasKey(expectedDurationKey));
+ JacksonMetric metric = (JacksonMetric) result.get(0);
+ assertThat(metric.toJsonString().indexOf("attributes"), not(-1));
+ final List explicitBoundsFromResult = (ArrayList)result.get(0).toMap().get("explicitBounds");
+ double bucketVal = TEST_VALUE_RANGE_MIN;
+ for (int i = 0; i < explicitBoundsFromResult.size(); i++) {
+ assertThat(explicitBoundsFromResult.get(i), equalTo(bucketVal));
+ bucketVal += TEST_VALUE_RANGE_STEP;
+ }
+ final List> bucketsFromResult = (ArrayList>)result.get(0).toMap().get("buckets");
+ double expectedBucketMin = -Float.MAX_VALUE;
+ double expectedBucketMax = TEST_VALUE_RANGE_MIN;
+ for (int i = 0; i < bucketsFromResult.size(); i++) {
+ assertThat(bucketsFromResult.get(i), hasEntry("min", expectedBucketMin));
+ assertThat(bucketsFromResult.get(i), hasEntry("max", expectedBucketMax));
+ assertThat(bucketsFromResult.get(i), hasEntry("count", expectedBucketCounts[i]));
+ expectedBucketMin = expectedBucketMax;
+ expectedBucketMax += TEST_VALUE_RANGE_STEP;
+ if (i == bucketsFromResult.size()-2) {
+ expectedBucketMax = Float.MAX_VALUE;
+ }
+ }
+
+ assertThat(result.get(0).get("startTime", String.class), equalTo(testTime.toString()));
+ assertThat(result.get(0).get("time", String.class), equalTo(testTime.plusSeconds(100).toString()));
+ }
}
diff --git a/data-prepper-plugins/armeria-common/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker b/data-prepper-plugins/armeria-common/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker
deleted file mode 100644
index ca6ee9cea8..0000000000
--- a/data-prepper-plugins/armeria-common/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker
+++ /dev/null
@@ -1 +0,0 @@
-mock-maker-inline
\ No newline at end of file
diff --git a/data-prepper-plugins/avro-codecs/build.gradle b/data-prepper-plugins/avro-codecs/build.gradle
index e6c5ea5e54..2bce28bbe0 100644
--- a/data-prepper-plugins/avro-codecs/build.gradle
+++ b/data-prepper-plugins/avro-codecs/build.gradle
@@ -6,7 +6,7 @@
dependencies {
implementation project(path: ':data-prepper-api')
implementation libs.avro.core
- implementation 'org.apache.parquet:parquet-common:1.14.0'
+ implementation libs.parquet.common
implementation 'software.amazon.awssdk:s3'
implementation 'software.amazon.awssdk:apache-client'
testImplementation 'org.json:json:20240205'
diff --git a/data-prepper-plugins/avro-codecs/src/test/java/org/opensearch/dataprepper/avro/AvroAutoSchemaGeneratorTest.java b/data-prepper-plugins/avro-codecs/src/test/java/org/opensearch/dataprepper/avro/AvroAutoSchemaGeneratorTest.java
index 622eb56a1b..1b66b62c37 100644
--- a/data-prepper-plugins/avro-codecs/src/test/java/org/opensearch/dataprepper/avro/AvroAutoSchemaGeneratorTest.java
+++ b/data-prepper-plugins/avro-codecs/src/test/java/org/opensearch/dataprepper/avro/AvroAutoSchemaGeneratorTest.java
@@ -17,7 +17,7 @@
import java.util.Collections;
import java.util.List;
import java.util.Map;
-import java.util.Random;
+import java.util.Timer;
import java.util.UUID;
import java.util.stream.Stream;
@@ -218,7 +218,7 @@ static class SomeUnknownTypesArgumentsProvider implements ArgumentsProvider {
@Override
public Stream extends Arguments> provideArguments(ExtensionContext context) {
return Stream.of(
- arguments(Random.class),
+ arguments(Timer.class),
arguments(InputStream.class),
arguments(File.class)
);
diff --git a/data-prepper-plugins/aws-plugin/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker b/data-prepper-plugins/aws-plugin/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker
deleted file mode 100644
index 23c33feb6d..0000000000
--- a/data-prepper-plugins/aws-plugin/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker
+++ /dev/null
@@ -1,3 +0,0 @@
-# To enable mocking of final classes with vanilla Mockito
-# https://github.com/mockito/mockito/wiki/What%27s-new-in-Mockito-2#mock-the-unmockable-opt-in-mocking-of-final-classesmethods
-mock-maker-inline
diff --git a/data-prepper-plugins/blocking-buffer/src/test/java/org/opensearch/dataprepper/plugins/buffer/blockingbuffer/BlockingBufferTests.java b/data-prepper-plugins/blocking-buffer/src/test/java/org/opensearch/dataprepper/plugins/buffer/blockingbuffer/BlockingBufferTests.java
index 194c810ec4..f3f28db174 100644
--- a/data-prepper-plugins/blocking-buffer/src/test/java/org/opensearch/dataprepper/plugins/buffer/blockingbuffer/BlockingBufferTests.java
+++ b/data-prepper-plugins/blocking-buffer/src/test/java/org/opensearch/dataprepper/plugins/buffer/blockingbuffer/BlockingBufferTests.java
@@ -328,7 +328,7 @@ public Stream extends Arguments> provideArguments(final ExtensionContext conte
return Stream.of(
Arguments.of(0, randomInt + 1, 0.0),
Arguments.of(1, 100, 1.0),
- Arguments.of(randomInt, randomInt, 100.0),
+ Arguments.of(randomInt + 1, randomInt + 1, 100.0),
Arguments.of(randomInt, randomInt + 250, ((double) randomInt / (randomInt + 250)) * 100),
Arguments.of(6, 9, 66.66666666666666),
Arguments.of(531, 1000, 53.1),
diff --git a/data-prepper-plugins/cloudwatch-logs/build.gradle b/data-prepper-plugins/cloudwatch-logs/build.gradle
index dc374997f0..3bbb24f443 100644
--- a/data-prepper-plugins/cloudwatch-logs/build.gradle
+++ b/data-prepper-plugins/cloudwatch-logs/build.gradle
@@ -16,7 +16,6 @@ dependencies {
implementation 'org.projectlombok:lombok:1.18.26'
implementation 'org.hibernate.validator:hibernate-validator:8.0.0.Final'
testImplementation project(path: ':data-prepper-test-common')
- testImplementation testLibs.mockito.inline
compileOnly 'org.projectlombok:lombok:1.18.24'
annotationProcessor 'org.projectlombok:lombok:1.18.24'
}
diff --git a/data-prepper-plugins/cloudwatch-metrics-source/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker b/data-prepper-plugins/cloudwatch-metrics-source/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker
deleted file mode 100644
index 23c33feb6d..0000000000
--- a/data-prepper-plugins/cloudwatch-metrics-source/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker
+++ /dev/null
@@ -1,3 +0,0 @@
-# To enable mocking of final classes with vanilla Mockito
-# https://github.com/mockito/mockito/wiki/What%27s-new-in-Mockito-2#mock-the-unmockable-opt-in-mocking-of-final-classesmethods
-mock-maker-inline
diff --git a/data-prepper-plugins/common/build.gradle b/data-prepper-plugins/common/build.gradle
index 947d1234d4..cdfdeab9ef 100644
--- a/data-prepper-plugins/common/build.gradle
+++ b/data-prepper-plugins/common/build.gradle
@@ -19,12 +19,11 @@ dependencies {
implementation libs.bouncycastle.bcpkix
implementation libs.reflections.core
implementation 'io.micrometer:micrometer-core'
- implementation 'org.apache.parquet:parquet-common:1.14.0'
+ implementation libs.parquet.common
implementation 'org.xerial.snappy:snappy-java:1.1.10.5'
testImplementation project(':data-prepper-plugins:blocking-buffer')
testImplementation project(':data-prepper-test-event')
testImplementation libs.commons.io
- testImplementation testLibs.mockito.inline
}
jacocoTestCoverageVerification {
diff --git a/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/processor/StringProcessor.java b/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/processor/StringProcessor.java
index aa2930e634..3cf2953e06 100644
--- a/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/processor/StringProcessor.java
+++ b/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/processor/StringProcessor.java
@@ -5,6 +5,7 @@
package org.opensearch.dataprepper.plugins.processor;
+import com.fasterxml.jackson.annotation.JsonPropertyDescription;
import org.opensearch.dataprepper.model.annotations.DataPrepperPlugin;
import org.opensearch.dataprepper.model.annotations.DataPrepperPluginConstructor;
import org.opensearch.dataprepper.model.configuration.PluginSetting;
@@ -40,6 +41,7 @@ public class StringProcessor implements Processor, Record>
private final boolean upperCase;
public static class Configuration {
+ @JsonPropertyDescription("Whether to convert to uppercase (`true`) or lowercase (`false`).")
private boolean upperCase = true;
public boolean getUpperCase() {
diff --git a/data-prepper-plugins/csv-processor/build.gradle b/data-prepper-plugins/csv-processor/build.gradle
index 56c02daf83..cda0694a66 100644
--- a/data-prepper-plugins/csv-processor/build.gradle
+++ b/data-prepper-plugins/csv-processor/build.gradle
@@ -12,7 +12,7 @@ dependencies {
implementation project(':data-prepper-api')
implementation 'com.fasterxml.jackson.dataformat:jackson-dataformat-csv'
implementation 'io.micrometer:micrometer-core'
- implementation 'org.apache.parquet:parquet-common:1.14.0'
+ implementation libs.parquet.common
implementation 'software.amazon.awssdk:s3'
implementation 'software.amazon.awssdk:apache-client'
testImplementation project(':data-prepper-plugins:log-generator-source')
diff --git a/data-prepper-plugins/csv-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/csv/CsvProcessorConfig.java b/data-prepper-plugins/csv-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/csv/CsvProcessorConfig.java
index ec5d685b7e..8c770b597a 100644
--- a/data-prepper-plugins/csv-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/csv/CsvProcessorConfig.java
+++ b/data-prepper-plugins/csv-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/csv/CsvProcessorConfig.java
@@ -6,6 +6,7 @@
package org.opensearch.dataprepper.plugins.processor.csv;
import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonPropertyDescription;
import jakarta.validation.constraints.AssertTrue;
import java.util.List;
@@ -20,24 +21,45 @@ public class CsvProcessorConfig {
static final Boolean DEFAULT_DELETE_HEADERS = true;
@JsonProperty("source")
+ @JsonPropertyDescription("The field in the event that will be parsed. Default value is `message`.")
private String source = DEFAULT_SOURCE;
@JsonProperty("delimiter")
+ @JsonPropertyDescription("The character separating each column. Default value is `,`.")
private String delimiter = DEFAULT_DELIMITER;
@JsonProperty("delete_header")
+ @JsonPropertyDescription("If specified, the event header (`column_names_source_key`) is deleted after the event " +
+ "is parsed. If there is no event header, no action is taken. Default value is true.")
private Boolean deleteHeader = DEFAULT_DELETE_HEADERS;
@JsonProperty("quote_character")
+ @JsonPropertyDescription("The character used as a text qualifier for a single column of data. " +
+ "Default value is `\"`.")
private String quoteCharacter = DEFAULT_QUOTE_CHARACTER;
@JsonProperty("column_names_source_key")
+ @JsonPropertyDescription("The field in the event that specifies the CSV column names, which will be " +
+ "automatically detected. If there need to be extra column names, the column names are automatically " +
+ "generated according to their index. If `column_names` is also defined, the header in " +
+ "`column_names_source_key` can also be used to generate the event fields. " +
+ "If too few columns are specified in this field, the remaining column names are automatically generated. " +
+ "If too many column names are specified in this field, the CSV processor omits the extra column names.")
private String columnNamesSourceKey;
@JsonProperty("column_names")
+ @JsonPropertyDescription("User-specified names for the CSV columns. " +
+ "Default value is `[column1, column2, ..., columnN]` if there are no columns of data in the CSV " +
+ "record and `column_names_source_key` is not defined. If `column_names_source_key` is defined, " +
+ "the header in `column_names_source_key` generates the event fields. If too few columns are specified " +
+ "in this field, the remaining column names are automatically generated. " +
+ "If too many column names are specified in this field, the CSV processor omits the extra column names.")
private List columnNames;
@JsonProperty("csv_when")
+ @JsonPropertyDescription("Allows you to specify a [conditional expression](https://opensearch.org/docs/latest/data-prepper/pipelines/expression-syntax/), " +
+ "such as `/some-key == \"test\"`, that will be evaluated to determine whether " +
+ "the processor should be applied to the event.")
private String csvWhen;
/**
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 a74b2e9d38..aed3a38674 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
@@ -7,6 +7,7 @@
import com.fasterxml.jackson.annotation.JsonIgnore;
import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonPropertyDescription;
import jakarta.validation.constraints.AssertTrue;
import java.time.ZoneId;
@@ -24,8 +25,16 @@ public class DateProcessorConfig {
public static class DateMatch {
@JsonProperty("key")
+ @JsonPropertyDescription("Represents the event key against which to match patterns. " +
+ "Required if `match` is configured. ")
private String key;
@JsonProperty("patterns")
+ @JsonPropertyDescription("A list of possible patterns that the timestamp value of the key can have. The patterns " +
+ "are based on a sequence of letters and symbols. The `patterns` support all the patterns listed in the " +
+ "Java [DatetimeFormatter](https://docs.oracle.com/javase/8/docs/api/java/time/format/DateTimeFormatter.html) reference. " +
+ "The timestamp value also supports `epoch_second`, `epoch_milli`, and `epoch_nano` values, " +
+ "which represent the timestamp as the number of seconds, milliseconds, and nanoseconds since the epoch. " +
+ "Epoch values always use the UTC time zone.")
private List patterns;
public DateMatch() {
@@ -82,30 +91,57 @@ public static boolean isValidPattern(final String pattern) {
}
@JsonProperty("from_time_received")
+ @JsonPropertyDescription("When `true`, the timestamp from the event metadata, " +
+ "which is the time at which the source receives the event, is added to the event data. " +
+ "This option cannot be defined at the same time as `match`. Default is `false`.")
private Boolean fromTimeReceived = DEFAULT_FROM_TIME_RECEIVED;
@JsonProperty("to_origination_metadata")
+ @JsonPropertyDescription("When `true`, the matched time is also added to the event's metadata as an instance of " +
+ "`Instant`. Default is `false`.")
private Boolean toOriginationMetadata = DEFAULT_TO_ORIGINATION_METADATA;
@JsonProperty("match")
+ @JsonPropertyDescription("The date match configuration. " +
+ "This option cannot be defined at the same time as `from_time_received`. There is no default value.")
private List match;
@JsonProperty("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;
@JsonProperty("output_format")
+ @JsonPropertyDescription("Determines the format of the timestamp added to an event. " +
+ "Default is `yyyy-MM-dd'T'HH:mm:ss.SSSXXX`.")
private String outputFormat = DEFAULT_OUTPUT_FORMAT;
@JsonProperty("source_timezone")
+ @JsonPropertyDescription("The time zone used to parse dates, including when the zone or offset cannot be extracted " +
+ "from the value. If the zone or offset are part of the value, then the time zone is ignored. " +
+ "A list of all the available time zones is contained in the **TZ database name** column of " +
+ "[the list of database time zones](https://en.wikipedia.org/wiki/List_of_tz_database_time_zones#List).")
private String sourceTimezone = DEFAULT_SOURCE_TIMEZONE;
@JsonProperty("destination_timezone")
+ @JsonPropertyDescription("The time zone used for storing the timestamp in the `destination` field. " +
+ "A list of all the available time zones is contained in the **TZ database name** column of " +
+ "[the list of database time zones](https://en.wikipedia.org/wiki/List_of_tz_database_time_zones#List).")
private String destinationTimezone = DEFAULT_DESTINATION_TIMEZONE;
@JsonProperty("locale")
+ @JsonPropertyDescription("The location used for parsing dates. Commonly used for parsing month names (`MMM`). " +
+ "The value can contain language, country, or variant fields in IETF BCP 47, such as `en-US`, " +
+ "or a string representation of the " +
+ "[locale](https://docs.oracle.com/javase/8/docs/api/java/util/Locale.html) object, such as `en_US`. " +
+ "A full list of locale fields, including language, country, and variant, can be found in " +
+ "[the language subtag registry](https://www.iana.org/assignments/language-subtag-registry/language-subtag-registry). " +
+ "Default is `Locale.ROOT`.")
private String locale;
@JsonProperty("date_when")
+ @JsonPropertyDescription("Specifies under what condition the `date` processor should perform matching. " +
+ "Default is no condition.")
private String dateWhen;
@JsonIgnore
diff --git a/data-prepper-plugins/decompress-processor/build.gradle b/data-prepper-plugins/decompress-processor/build.gradle
index 9d67cffc3b..1068830a59 100644
--- a/data-prepper-plugins/decompress-processor/build.gradle
+++ b/data-prepper-plugins/decompress-processor/build.gradle
@@ -9,5 +9,4 @@ dependencies {
implementation project(':data-prepper-plugins:common')
implementation 'com.fasterxml.jackson.core:jackson-databind'
implementation 'io.micrometer:micrometer-core'
- testImplementation testLibs.mockito.inline
}
\ No newline at end of file
diff --git a/data-prepper-plugins/dynamodb-source-coordination-store/build.gradle b/data-prepper-plugins/dynamodb-source-coordination-store/build.gradle
index 4b9fb2a8f4..1912c2ae9b 100644
--- a/data-prepper-plugins/dynamodb-source-coordination-store/build.gradle
+++ b/data-prepper-plugins/dynamodb-source-coordination-store/build.gradle
@@ -10,7 +10,6 @@ dependencies {
implementation 'software.amazon.awssdk:dynamodb'
implementation 'software.amazon.awssdk:dynamodb-enhanced'
implementation 'software.amazon.awssdk:sts'
- testImplementation testLibs.mockito.inline
}
test {
diff --git a/data-prepper-plugins/dynamodb-source/build.gradle b/data-prepper-plugins/dynamodb-source/build.gradle
index 8fdc037470..3b3046434a 100644
--- a/data-prepper-plugins/dynamodb-source/build.gradle
+++ b/data-prepper-plugins/dynamodb-source/build.gradle
@@ -25,6 +25,5 @@ dependencies {
implementation project(path: ':data-prepper-plugins:buffer-common')
- testImplementation testLibs.mockito.inline
testImplementation 'com.fasterxml.jackson.dataformat:jackson-dataformat-yaml'
}
\ No newline at end of file
diff --git a/data-prepper-plugins/event-json-codecs/build.gradle b/data-prepper-plugins/event-json-codecs/build.gradle
index aad563d19d..2278bf6033 100644
--- a/data-prepper-plugins/event-json-codecs/build.gradle
+++ b/data-prepper-plugins/event-json-codecs/build.gradle
@@ -15,7 +15,7 @@ dependencies {
implementation 'com.fasterxml.jackson.dataformat:jackson-dataformat-xml'
implementation 'com.fasterxml.jackson.datatype:jackson-datatype-jsr310:2.17.0'
testImplementation 'com.fasterxml.jackson.datatype:jackson-datatype-jsr310:2.17.0'
- implementation 'org.apache.parquet:parquet-common:1.14.0'
+ implementation libs.parquet.common
testImplementation project(':data-prepper-test-common')
}
diff --git a/data-prepper-plugins/event-json-codecs/src/test/java/org/opensearch/dataprepper/plugins/codec/event_json/EventJsonInputCodecTest.java b/data-prepper-plugins/event-json-codecs/src/test/java/org/opensearch/dataprepper/plugins/codec/event_json/EventJsonInputCodecTest.java
index f85d1c6605..a4b0377963 100644
--- a/data-prepper-plugins/event-json-codecs/src/test/java/org/opensearch/dataprepper/plugins/codec/event_json/EventJsonInputCodecTest.java
+++ b/data-prepper-plugins/event-json-codecs/src/test/java/org/opensearch/dataprepper/plugins/codec/event_json/EventJsonInputCodecTest.java
@@ -11,9 +11,12 @@
import org.junit.jupiter.api.Test;
import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.ValueSource;
+
import static org.mockito.Mockito.when;
import static org.mockito.Mockito.mock;
+
import org.mockito.Mock;
+
import static org.hamcrest.MatcherAssert.assertThat;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.CoreMatchers.not;
@@ -28,6 +31,7 @@
import java.io.ByteArrayInputStream;
import java.time.Instant;
+import java.time.temporal.ChronoUnit;
import java.util.List;
import java.util.LinkedList;
import java.util.Map;
@@ -56,7 +60,7 @@ public EventJsonInputCodec createInputCodec() {
@ParameterizedTest
@ValueSource(strings = {"", "{}"})
public void emptyTest(String input) throws Exception {
- input = "{\""+EventJsonDefines.VERSION+"\":\""+DataPrepperVersion.getCurrentVersion().toString()+"\", \""+EventJsonDefines.EVENTS+"\":["+input+"]}";
+ input = "{\"" + EventJsonDefines.VERSION + "\":\"" + DataPrepperVersion.getCurrentVersion().toString() + "\", \"" + EventJsonDefines.EVENTS + "\":[" + input + "]}";
ByteArrayInputStream inputStream = new ByteArrayInputStream(input.getBytes());
inputCodec = createInputCodec();
Consumer> consumer = mock(Consumer.class);
@@ -70,15 +74,15 @@ public void inCompatibleVersionTest() throws Exception {
final String key = UUID.randomUUID().toString();
final String value = UUID.randomUUID().toString();
Map data = Map.of(key, value);
- Instant startTime = Instant.now();
+ Instant startTime = Instant.now().truncatedTo(ChronoUnit.MICROS);
Event event = createEvent(data, startTime);
Map dataMap = event.toMap();
Map metadataMap = objectMapper.convertValue(event.getMetadata(), Map.class);
- String input = "{\""+EventJsonDefines.VERSION+"\":\"3.0\", \""+EventJsonDefines.EVENTS+"\":[";
+ String input = "{\"" + EventJsonDefines.VERSION + "\":\"3.0\", \"" + EventJsonDefines.EVENTS + "\":[";
String comma = "";
for (int i = 0; i < 2; i++) {
- input += comma+"{\"data\":"+objectMapper.writeValueAsString(dataMap)+","+"\"metadata\":"+objectMapper.writeValueAsString(metadataMap)+"}";
+ input += comma + "{\"data\":" + objectMapper.writeValueAsString(dataMap) + "," + "\"metadata\":" + objectMapper.writeValueAsString(metadataMap) + "}";
comma = ",";
}
input += "]}";
@@ -95,15 +99,15 @@ public void basicTest() throws Exception {
final String key = UUID.randomUUID().toString();
final String value = UUID.randomUUID().toString();
Map data = Map.of(key, value);
- Instant startTime = Instant.now();
+ Instant startTime = Instant.now().truncatedTo(ChronoUnit.MICROS);
Event event = createEvent(data, startTime);
Map dataMap = event.toMap();
Map metadataMap = objectMapper.convertValue(event.getMetadata(), Map.class);
- String input = "{\""+EventJsonDefines.VERSION+"\":\""+DataPrepperVersion.getCurrentVersion().toString()+"\", \""+EventJsonDefines.EVENTS+"\":[";
+ String input = "{\"" + EventJsonDefines.VERSION + "\":\"" + DataPrepperVersion.getCurrentVersion().toString() + "\", \"" + EventJsonDefines.EVENTS + "\":[";
String comma = "";
for (int i = 0; i < 2; i++) {
- input += comma+"{\"data\":"+objectMapper.writeValueAsString(dataMap)+","+"\"metadata\":"+objectMapper.writeValueAsString(metadataMap)+"}";
+ input += comma + "{\"data\":" + objectMapper.writeValueAsString(dataMap) + "," + "\"metadata\":" + objectMapper.writeValueAsString(metadataMap) + "}";
comma = ",";
}
input += "]}";
@@ -111,8 +115,8 @@ public void basicTest() throws Exception {
List> records = new LinkedList<>();
inputCodec.parse(inputStream, records::add);
assertThat(records.size(), equalTo(2));
- for(Record record : records) {
- Event e = (Event)record.getData();
+ for (Record record : records) {
+ Event e = (Event) record.getData();
assertThat(e.get(key, String.class), equalTo(value));
assertThat(e.getMetadata().getTimeReceived(), equalTo(startTime));
assertThat(e.getMetadata().getTags().size(), equalTo(0));
@@ -126,15 +130,15 @@ public void test_with_timeReceivedOverridden() throws Exception {
final String key = UUID.randomUUID().toString();
final String value = UUID.randomUUID().toString();
Map