Skip to content

Commit

Permalink
ingest: correctly measure chained pipeline stats (#33912)
Browse files Browse the repository at this point in the history
Prior to this change when a pipeline processor called another
pipeline, only the stats for the first processor were recorded.
The stats for the subsequent pipelines were ignored. This change
properly accounts for pipelines irregardless if they are the first
or subsequently called pipelines.

This change moves the state of the stats from the IngestService
to the pipeline itself. Cluster updates are safe since the pipelines
map is atomically swapped, and if a cluster update happens
while iterating over stats (now read directly from the pipeline)
a slightly stale view of stats may be shown.
  • Loading branch information
jakelandis committed Oct 22, 2018
1 parent 2d24192 commit 8e72a68
Show file tree
Hide file tree
Showing 5 changed files with 253 additions and 110 deletions.
95 changes: 95 additions & 0 deletions server/src/main/java/org/elasticsearch/ingest/IngestMetric.java
Original file line number Diff line number Diff line change
@@ -0,0 +1,95 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/

package org.elasticsearch.ingest;

import org.elasticsearch.common.metrics.CounterMetric;
import org.elasticsearch.common.metrics.MeanMetric;

/**
* <p>Metrics to measure ingest actions.
* <p>This counts measure documents and timings for a given scope.
* The scope is determined by the calling code. For example you can use this class to count all documents across all pipeline,
* or you can use this class to count documents for a given pipeline or a specific processor.
* This class does not make assumptions about it's given scope.
*/
class IngestMetric {

/**
* The time it takes to complete the measured item.
*/
private final MeanMetric ingestTime = new MeanMetric();
/**
* The current count of things being measure. Should most likely ever be 0 or 1.
* Useful when aggregating multiple metrics to see how many things are in flight.
*/
private final CounterMetric ingestCurrent = new CounterMetric();
/**
* The ever increasing count of things being measured
*/
private final CounterMetric ingestCount = new CounterMetric();
/**
* The only increasing count of failures
*/
private final CounterMetric ingestFailed = new CounterMetric();

/**
* Call this prior to the ingest action.
*/
void preIngest() {
ingestCurrent.inc();
}

/**
* Call this after the performing the ingest action, even if the action failed.
* @param ingestTimeInMillis The time it took to perform the action.
*/
void postIngest(long ingestTimeInMillis) {
ingestCurrent.dec();
ingestTime.inc(ingestTimeInMillis);
ingestCount.inc();
}

/**
* Call this if the ingest action failed.
*/
void ingestFailed() {
ingestFailed.inc();
}

/**
* <p>Add two sets of metrics together.
* <p><strong>Note -</strong> this method does <strong>not</strong> add the current count values.
* The current count value is ephemeral and requires a increase/decrease operation pairs to keep the value correct.
*
* @param metrics The metric to add.
*/
void add(IngestMetric metrics) {
ingestCount.inc(metrics.ingestCount.count());
ingestTime.inc(metrics.ingestTime.sum());
ingestFailed.inc(metrics.ingestFailed.count());
}

/**
* Creates a serializable representation for these metrics.
*/
IngestStats.Stats createStats() {
return new IngestStats.Stats(ingestCount.count(), ingestTime.sum(), ingestCurrent.count(), ingestFailed.count());
}
}
87 changes: 19 additions & 68 deletions server/src/main/java/org/elasticsearch/ingest/IngestService.java
Original file line number Diff line number Diff line change
Expand Up @@ -23,16 +23,16 @@
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Map;

import java.util.Objects;
import java.util.Optional;
import java.util.Set;
import java.util.concurrent.TimeUnit;
import java.util.function.BiConsumer;
import java.util.function.Consumer;
import java.util.stream.Collectors;

import org.elasticsearch.ElasticsearchParseException;
import org.elasticsearch.ExceptionsHelper;
import org.elasticsearch.ResourceNotFoundException;
Expand All @@ -50,8 +50,6 @@
import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.metrics.CounterMetric;
import org.elasticsearch.common.metrics.MeanMetric;
import org.elasticsearch.common.regex.Regex;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.concurrent.AbstractRunnable;
Expand Down Expand Up @@ -80,8 +78,7 @@ public class IngestService implements ClusterStateApplier {
// are loaded, so in the cluster state we just save the pipeline config and here we keep the actual pipelines around.
private volatile Map<String, Pipeline> pipelines = new HashMap<>();
private final ThreadPool threadPool;
private final StatsHolder totalStats = new StatsHolder();
private volatile Map<String, StatsHolder> statsHolderPerPipeline = Collections.emptyMap();
private final IngestMetric totalMetrics = new IngestMetric();

public IngestService(ClusterService clusterService, ThreadPool threadPool,
Environment env, ScriptService scriptService, AnalysisRegistry analysisRegistry,
Expand Down Expand Up @@ -258,10 +255,16 @@ Map<String, Pipeline> pipelines() {
@Override
public void applyClusterState(final ClusterChangedEvent event) {
ClusterState state = event.state();
Map<String, Pipeline> originalPipelines = pipelines;
innerUpdatePipelines(event.previousState(), state);
IngestMetadata ingestMetadata = state.getMetaData().custom(IngestMetadata.TYPE);
if (ingestMetadata != null) {
updatePipelineStats(ingestMetadata);
//pipelines changed, so add the old metrics to the new metrics
if (originalPipelines != pipelines) {
pipelines.forEach((id, pipeline) -> {
Pipeline originalPipeline = originalPipelines.get(id);
if (originalPipeline != null) {
pipeline.getMetrics().add(originalPipeline.getMetrics());
}
});
}
}

Expand Down Expand Up @@ -326,6 +329,7 @@ void validatePipeline(Map<DiscoveryNode, IngestInfo> ingestInfos, PutPipelineReq
public void executeBulkRequest(Iterable<DocWriteRequest<?>> actionRequests,
BiConsumer<IndexRequest, Exception> itemFailureHandler, Consumer<Exception> completionHandler,
Consumer<IndexRequest> itemDroppedHandler) {

threadPool.executor(ThreadPool.Names.WRITE).execute(new AbstractRunnable() {

@Override
Expand Down Expand Up @@ -368,37 +372,11 @@ protected void doRun() {
}

public IngestStats stats() {
Map<String, StatsHolder> statsHolderPerPipeline = this.statsHolderPerPipeline;

Map<String, IngestStats.Stats> statsPerPipeline = new HashMap<>(statsHolderPerPipeline.size());
for (Map.Entry<String, StatsHolder> entry : statsHolderPerPipeline.entrySet()) {
statsPerPipeline.put(entry.getKey(), entry.getValue().createStats());
}
Map<String, IngestStats.Stats> statsPerPipeline =
pipelines.entrySet().stream().collect(Collectors.toMap(Map.Entry::getKey, v -> v.getValue().getMetrics().createStats()));

return new IngestStats(totalStats.createStats(), statsPerPipeline);
}

void updatePipelineStats(IngestMetadata ingestMetadata) {
boolean changed = false;
Map<String, StatsHolder> newStatsPerPipeline = new HashMap<>(statsHolderPerPipeline);
Iterator<String> iterator = newStatsPerPipeline.keySet().iterator();
while (iterator.hasNext()) {
String pipeline = iterator.next();
if (ingestMetadata.getPipelines().containsKey(pipeline) == false) {
iterator.remove();
changed = true;
}
}
for (String pipeline : ingestMetadata.getPipelines().keySet()) {
if (newStatsPerPipeline.containsKey(pipeline) == false) {
newStatsPerPipeline.put(pipeline, new StatsHolder());
changed = true;
}
}

if (changed) {
statsHolderPerPipeline = Collections.unmodifiableMap(newStatsPerPipeline);
}
return new IngestStats(totalMetrics.createStats(), statsPerPipeline);
}

private void innerExecute(IndexRequest indexRequest, Pipeline pipeline, Consumer<IndexRequest> itemDroppedHandler) throws Exception {
Expand All @@ -409,10 +387,8 @@ private void innerExecute(IndexRequest indexRequest, Pipeline pipeline, Consumer
long startTimeInNanos = System.nanoTime();
// the pipeline specific stat holder may not exist and that is fine:
// (e.g. the pipeline may have been removed while we're ingesting a document
Optional<StatsHolder> pipelineStats = Optional.ofNullable(statsHolderPerPipeline.get(pipeline.getId()));
try {
totalStats.preIngest();
pipelineStats.ifPresent(StatsHolder::preIngest);
totalMetrics.preIngest();
String index = indexRequest.index();
String type = indexRequest.type();
String id = indexRequest.id();
Expand All @@ -438,13 +414,11 @@ private void innerExecute(IndexRequest indexRequest, Pipeline pipeline, Consumer
indexRequest.source(ingestDocument.getSourceAndMetadata());
}
} catch (Exception e) {
totalStats.ingestFailed();
pipelineStats.ifPresent(StatsHolder::ingestFailed);
totalMetrics.ingestFailed();
throw e;
} finally {
long ingestTimeInMillis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTimeInNanos);
totalStats.postIngest(ingestTimeInMillis);
pipelineStats.ifPresent(statsHolder -> statsHolder.postIngest(ingestTimeInMillis));
totalMetrics.postIngest(ingestTimeInMillis);
}
}

Expand Down Expand Up @@ -481,27 +455,4 @@ private void innerUpdatePipelines(ClusterState previousState, ClusterState state
ExceptionsHelper.rethrowAndSuppress(exceptions);
}

private static class StatsHolder {

private final MeanMetric ingestMetric = new MeanMetric();
private final CounterMetric ingestCurrent = new CounterMetric();
private final CounterMetric ingestFailed = new CounterMetric();

void preIngest() {
ingestCurrent.inc();
}

void postIngest(long ingestTimeInMillis) {
ingestCurrent.dec();
ingestMetric.inc(ingestTimeInMillis);
}

void ingestFailed() {
ingestFailed.inc();
}

IngestStats.Stats createStats() {
return new IngestStats.Stats(ingestMetric.count(), ingestMetric.sum(), ingestCurrent.count(), ingestFailed.count());
}
}
}
30 changes: 29 additions & 1 deletion server/src/main/java/org/elasticsearch/ingest/Pipeline.java
Original file line number Diff line number Diff line change
Expand Up @@ -22,10 +22,12 @@
import org.elasticsearch.ElasticsearchParseException;
import org.elasticsearch.common.Nullable;

import java.time.Clock;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.Map;

import org.elasticsearch.script.ScriptService;

/**
Expand All @@ -44,12 +46,21 @@ public final class Pipeline {
@Nullable
private final Integer version;
private final CompoundProcessor compoundProcessor;
private final IngestMetric metrics;
private final Clock clock;

public Pipeline(String id, @Nullable String description, @Nullable Integer version, CompoundProcessor compoundProcessor) {
this(id, description, version, compoundProcessor, Clock.systemUTC());
}

//package private for testing
Pipeline(String id, @Nullable String description, @Nullable Integer version, CompoundProcessor compoundProcessor, Clock clock) {
this.id = id;
this.description = description;
this.compoundProcessor = compoundProcessor;
this.version = version;
this.metrics = new IngestMetric();
this.clock = clock;
}

public static Pipeline create(String id, Map<String, Object> config,
Expand Down Expand Up @@ -78,7 +89,17 @@ public static Pipeline create(String id, Map<String, Object> config,
* Modifies the data of a document to be indexed based on the processor this pipeline holds
*/
public IngestDocument execute(IngestDocument ingestDocument) throws Exception {
return compoundProcessor.execute(ingestDocument);
long startTimeInMillis = clock.millis();
try {
metrics.preIngest();
return compoundProcessor.execute(ingestDocument);
} catch (Exception e) {
metrics.ingestFailed();
throw e;
} finally {
long ingestTimeInMillis = clock.millis() - startTimeInMillis;
metrics.postIngest(ingestTimeInMillis);
}
}

/**
Expand Down Expand Up @@ -135,4 +156,11 @@ public List<Processor> getOnFailureProcessors() {
public List<Processor> flattenAllProcessors() {
return compoundProcessor.flattenProcessors();
}

/**
* The metrics associated with this pipeline.
*/
public IngestMetric getMetrics() {
return metrics;
}
}
Loading

0 comments on commit 8e72a68

Please sign in to comment.