-
Notifications
You must be signed in to change notification settings - Fork 214
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Add Internal and external latency to OpenSearch and S3 sinks. #3583
Changes from 12 commits
16cfa3f
72eda21
a31d9bb
fa8e91e
4c2af3b
123109a
c2bbbfb
07a0896
b22a682
e014114
2d32e77
7709f0e
39c6160
e794577
ed07c9d
b7a574c
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,36 @@ | ||
/* | ||
* Copyright OpenSearch Contributors | ||
* SPDX-License-Identifier: Apache-2.0 | ||
*/ | ||
|
||
package org.opensearch.dataprepper.plugins.sink; | ||
|
||
import org.opensearch.dataprepper.metrics.PluginMetrics; | ||
import io.micrometer.core.instrument.DistributionSummary; | ||
import org.opensearch.dataprepper.model.event.EventHandle; | ||
|
||
import java.time.Duration; | ||
import java.time.Instant; | ||
|
||
public class LatencyMetrics { | ||
public static final String INTERNAL_LATENCY = "internalLatency"; | ||
public static final String EXTERNAL_LATENCY = "externalLatency"; | ||
private final DistributionSummary internalLatencySummary; | ||
private final DistributionSummary externalLatencySummary; | ||
|
||
public LatencyMetrics(PluginMetrics pluginMetrics) { | ||
internalLatencySummary = pluginMetrics.summary(INTERNAL_LATENCY); | ||
externalLatencySummary = pluginMetrics.summary(EXTERNAL_LATENCY); | ||
} | ||
public void update(final EventHandle eventHandle) { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I tend to think that this should take the
|
||
if (eventHandle == null) { | ||
return; | ||
} | ||
Instant now = Instant.now(); | ||
internalLatencySummary.record(Duration.between(eventHandle.getInternalOriginationTime(), now).toMillis()); | ||
if (eventHandle.getExternalOriginationTime() == null) { | ||
return; | ||
} | ||
externalLatencySummary.record(Duration.between(eventHandle.getExternalOriginationTime(), now).toMillis()); | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,71 @@ | ||
/* | ||
* Copyright OpenSearch Contributors | ||
* SPDX-License-Identifier: Apache-2.0 | ||
*/ | ||
|
||
package org.opensearch.dataprepper.plugins.sink; | ||
|
||
import org.opensearch.dataprepper.metrics.PluginMetrics; | ||
import org.opensearch.dataprepper.model.event.EventHandle; | ||
import io.micrometer.core.instrument.simple.SimpleMeterRegistry; | ||
import io.micrometer.core.instrument.DistributionSummary; | ||
|
||
import org.junit.jupiter.api.BeforeEach; | ||
import org.junit.jupiter.api.Test; | ||
|
||
import static org.mockito.Mockito.mock; | ||
import static org.mockito.Mockito.when; | ||
import static org.hamcrest.MatcherAssert.assertThat; | ||
import static org.hamcrest.Matchers.equalTo; | ||
import static org.hamcrest.Matchers.greaterThanOrEqualTo; | ||
|
||
import java.time.Instant; | ||
|
||
class LatencyMetricsTest { | ||
|
||
private PluginMetrics pluginMetrics; | ||
private EventHandle eventHandle; | ||
private LatencyMetrics latencyMetrics; | ||
private DistributionSummary internalLatencySummary; | ||
private DistributionSummary externalLatencySummary; | ||
|
||
public LatencyMetrics createObjectUnderTest() { | ||
return new LatencyMetrics(pluginMetrics); | ||
} | ||
|
||
@BeforeEach | ||
void setup() { | ||
pluginMetrics = mock(PluginMetrics.class); | ||
SimpleMeterRegistry registry = new SimpleMeterRegistry(); | ||
internalLatencySummary = DistributionSummary | ||
.builder("internalLatency") | ||
.baseUnit("milliseconds") | ||
.register(registry); | ||
externalLatencySummary = DistributionSummary | ||
.builder("externalLatency") | ||
.baseUnit("milliseconds") | ||
.register(registry); | ||
when(pluginMetrics.summary(LatencyMetrics.INTERNAL_LATENCY)).thenReturn(internalLatencySummary); | ||
when(pluginMetrics.summary(LatencyMetrics.EXTERNAL_LATENCY)).thenReturn(externalLatencySummary); | ||
eventHandle = mock(EventHandle.class); | ||
when(eventHandle.getInternalOriginationTime()).thenReturn(Instant.now()); | ||
latencyMetrics = createObjectUnderTest(); | ||
} | ||
|
||
@Test | ||
public void testInternalOriginationTime() { | ||
latencyMetrics.update(eventHandle); | ||
assertThat(internalLatencySummary.count(), equalTo(1L)); | ||
} | ||
|
||
@Test | ||
public void testExternalOriginationTime() { | ||
when(eventHandle.getExternalOriginationTime()).thenReturn(Instant.now().minusMillis(10)); | ||
latencyMetrics.update(eventHandle); | ||
assertThat(internalLatencySummary.count(), equalTo(1L)); | ||
assertThat(externalLatencySummary.count(), equalTo(1L)); | ||
assertThat(externalLatencySummary.max(), greaterThanOrEqualTo(10.0)); | ||
} | ||
} | ||
|
||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -27,6 +27,7 @@ | |
import java.util.List; | ||
import java.util.Objects; | ||
import java.util.stream.Collectors; | ||
import org.apache.commons.lang3.tuple.Pair; | ||
|
||
@DataPrepperPlugin(name = "date", pluginType = Processor.class, pluginConfigurationType = DateProcessorConfig.class) | ||
public class DateProcessor extends AbstractProcessor<Record<Event>, Record<Event>> { | ||
|
@@ -71,7 +72,16 @@ public Collection<Record<Event>> doExecute(Collection<Record<Event>> records) { | |
zonedDateTime = getDateTimeFromTimeReceived(record); | ||
|
||
else if (keyToParse != null && !keyToParse.isEmpty()) { | ||
zonedDateTime = getDateTimeFromMatch(record); | ||
Pair<String, Instant> result = getDateTimeFromMatch(record); | ||
if (result != null) { | ||
zonedDateTime = result.getLeft(); | ||
Instant timeStamp = result.getRight(); | ||
if (dateProcessorConfig.getToOriginationMetadata()) { | ||
Event event = (Event)record.getData(); | ||
event.getMetadata().setExternalOriginationTime(timeStamp); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Can we have There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I did not do that because I am not sure if we want to tie them together. What if we don't want external latency metric but set metadata only? This way, we could introduce another flag in date processor in future if such requirement arises. |
||
event.getEventHandle().setExternalOriginationTime(timeStamp); | ||
} | ||
} | ||
populateDateProcessorMetrics(zonedDateTime); | ||
} | ||
|
||
|
@@ -119,7 +129,7 @@ private String getDateTimeFromTimeReceived(final Record<Event> record) { | |
return timeReceived.atZone(dateProcessorConfig.getDestinationZoneId()).format(getOutputFormatter()); | ||
} | ||
|
||
private String getDateTimeFromMatch(final Record<Event> record) { | ||
private Pair<String, Instant> getDateTimeFromMatch(final Record<Event> record) { | ||
final String sourceTimestamp = getSourceTimestamp(record); | ||
if (sourceTimestamp == null) | ||
return null; | ||
|
@@ -136,12 +146,12 @@ private String getSourceTimestamp(final Record<Event> record) { | |
} | ||
} | ||
|
||
private String getFormattedDateTimeString(final String sourceTimestamp) { | ||
private Pair<String, Instant> getFormattedDateTimeString(final String sourceTimestamp) { | ||
for (DateTimeFormatter formatter : dateTimeFormatters) { | ||
try { | ||
return ZonedDateTime.parse(sourceTimestamp, formatter).format(getOutputFormatter().withZone(dateProcessorConfig.getDestinationZoneId())); | ||
ZonedDateTime tmp = ZonedDateTime.parse(sourceTimestamp, formatter); | ||
return Pair.of(tmp.format(getOutputFormatter().withZone(dateProcessorConfig.getDestinationZoneId())), tmp.toInstant()); | ||
} catch (Exception ignored) { | ||
|
||
} | ||
} | ||
|
||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -45,25 +45,22 @@ public class BulkOperationWrapper { | |
private final EventHandle eventHandle; | ||
private final BulkOperation bulkOperation; | ||
private final SerializedJson jsonNode; | ||
private final OpenSearchSink sink; | ||
|
||
public BulkOperationWrapper(final BulkOperation bulkOperation) { | ||
this.bulkOperation = bulkOperation; | ||
this.eventHandle = null; | ||
this.jsonNode = null; | ||
public BulkOperationWrapper(final OpenSearchSink sink, final BulkOperation bulkOperation) { | ||
this(sink, bulkOperation, null, null); | ||
} | ||
|
||
public BulkOperationWrapper(final BulkOperation bulkOperation, final EventHandle eventHandle, final SerializedJson jsonNode) { | ||
public BulkOperationWrapper(final OpenSearchSink sink, final BulkOperation bulkOperation, final EventHandle eventHandle, final SerializedJson jsonNode) { | ||
checkNotNull(bulkOperation); | ||
this.sink = sink; | ||
this.bulkOperation = bulkOperation; | ||
this.eventHandle = eventHandle; | ||
this.jsonNode = jsonNode; | ||
} | ||
|
||
public BulkOperationWrapper(final BulkOperation bulkOperation, final EventHandle eventHandle) { | ||
checkNotNull(bulkOperation); | ||
this.bulkOperation = bulkOperation; | ||
this.eventHandle = eventHandle; | ||
this.jsonNode = null; | ||
public BulkOperationWrapper(final OpenSearchSink sink, final BulkOperation bulkOperation, final EventHandle eventHandle) { | ||
this(sink, bulkOperation, eventHandle, null); | ||
} | ||
|
||
public BulkOperation getBulkOperation() { | ||
|
@@ -76,6 +73,7 @@ public EventHandle getEventHandle() { | |
|
||
public void releaseEventHandle(boolean result) { | ||
if (eventHandle != null) { | ||
sink.updateLatencyMetrics(eventHandle); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Ultimately we want this called because
Then, in
The current design results in too many classes having to be aware of these interactions. |
||
eventHandle.release(result); | ||
} | ||
} | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -30,6 +30,7 @@ | |
import org.opensearch.dataprepper.model.configuration.PluginModel; | ||
import org.opensearch.dataprepper.model.configuration.PluginSetting; | ||
import org.opensearch.dataprepper.model.event.Event; | ||
import org.opensearch.dataprepper.model.event.EventHandle; | ||
import org.opensearch.dataprepper.model.event.exceptions.EventKeyNotFoundException; | ||
import org.opensearch.dataprepper.model.failures.DlqObject; | ||
import org.opensearch.dataprepper.model.opensearch.OpenSearchBulkActions; | ||
|
@@ -59,6 +60,7 @@ | |
import org.opensearch.dataprepper.plugins.sink.opensearch.index.IndexTemplateAPIWrapperFactory; | ||
import org.opensearch.dataprepper.plugins.sink.opensearch.index.IndexType; | ||
import org.opensearch.dataprepper.plugins.sink.opensearch.index.TemplateStrategy; | ||
import org.opensearch.dataprepper.plugins.sink.LatencyMetrics; | ||
import org.slf4j.Logger; | ||
import org.slf4j.LoggerFactory; | ||
import software.amazon.awssdk.services.opensearchserverless.OpenSearchServerlessClient; | ||
|
@@ -91,6 +93,7 @@ public class OpenSearchSink extends AbstractSink<Record<Event>> { | |
private static final Logger LOG = LoggerFactory.getLogger(OpenSearchSink.class); | ||
private static final int INITIALIZE_RETRY_WAIT_TIME_MS = 5000; | ||
private final AwsCredentialsSupplier awsCredentialsSupplier; | ||
private final LatencyMetrics latencyMetrics; | ||
|
||
private DlqWriter dlqWriter; | ||
private BufferedWriter dlqFileWriter; | ||
|
@@ -141,6 +144,7 @@ public OpenSearchSink(final PluginSetting pluginSetting, | |
this.awsCredentialsSupplier = awsCredentialsSupplier; | ||
this.sinkContext = sinkContext != null ? sinkContext : new SinkContext(null, Collections.emptyList(), Collections.emptyList(), Collections.emptyList()); | ||
this.expressionEvaluator = expressionEvaluator; | ||
this.latencyMetrics = new LatencyMetrics(pluginMetrics); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Is there really no way to give the There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. How do we do that? Metrics are sink specific, right? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Yes but we could "fake" that the metrics are owned by the sink by adding the name of the sink here (since you can name the metrics anything and that it can still follow our metric naming pattern There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I do not think that's the clean way because an event handle can only have one name for metrics. How can it have multiple names? I think this is way cleaner. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Every sink will need to provide this latency and we want to make it as easy as possible for sinks to use this feature. I think that data-prepper-core could hold a One thing we would need to be sure to do here is ensure that sinks always call |
||
bulkRequestTimer = pluginMetrics.timer(BULKREQUEST_LATENCY); | ||
bulkRequestErrorsCounter = pluginMetrics.counter(BULKREQUEST_ERRORS); | ||
invalidActionErrorsCounter = pluginMetrics.counter(INVALID_ACTION_ERRORS); | ||
|
@@ -316,6 +320,10 @@ private BulkOperation getBulkOperationForAction(final String action, final Seria | |
return bulkOperation; | ||
} | ||
|
||
public void updateLatencyMetrics(final EventHandle eventHandle) { | ||
latencyMetrics.update(eventHandle); | ||
} | ||
|
||
@Override | ||
public void doOutput(final Collection<Record<Event>> records) { | ||
final long threadId = Thread.currentThread().getId(); | ||
|
@@ -376,7 +384,7 @@ public void doOutput(final Collection<Record<Event>> records) { | |
} | ||
BulkOperation bulkOperation = getBulkOperationForAction(eventAction, document, indexName, event.getJsonNode()); | ||
|
||
BulkOperationWrapper bulkOperationWrapper = new BulkOperationWrapper(bulkOperation, event.getEventHandle(), serializedJsonNode); | ||
BulkOperationWrapper bulkOperationWrapper = new BulkOperationWrapper(this, bulkOperation, event.getEventHandle(), serializedJsonNode); | ||
final long estimatedBytesBeforeAdd = bulkRequest.estimateSizeInBytesWithDocument(bulkOperationWrapper); | ||
if (bulkSize >= 0 && estimatedBytesBeforeAdd >= bulkSize && bulkRequest.getOperationsCount() > 0) { | ||
flushBatch(bulkRequest); | ||
|
@@ -449,6 +457,7 @@ private void logFailureForDlqObjects(final List<DlqObject> dlqObjects, final Thr | |
try { | ||
dlqFileWriter.write(String.format("{\"Document\": [%s], \"failure\": %s}\n", | ||
BulkOperationWriter.dlqObjectToString(dlqObject), message)); | ||
updateLatencyMetrics(dlqObject.getEventHandle()); | ||
dlqObject.releaseEventHandle(true); | ||
} catch (final IOException e) { | ||
LOG.error("Failed to write a document to the DLQ", e); | ||
|
@@ -459,6 +468,7 @@ private void logFailureForDlqObjects(final List<DlqObject> dlqObjects, final Thr | |
try { | ||
dlqWriter.write(dlqObjects, pluginSetting.getPipelineName(), pluginSetting.getName()); | ||
dlqObjects.forEach((dlqObject) -> { | ||
updateLatencyMetrics(dlqObject.getEventHandle()); | ||
dlqObject.releaseEventHandle(true); | ||
}); | ||
} catch (final IOException e) { | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
If we are able to move the tracking of this into
data-prepper-core
, please move this class intodata-prepper-core
. There should be no need to expose it in that situation.