mirror of
https://github.com/honeymoose/OpenSearch.git
synced 2025-02-25 14:26:27 +00:00
INGEST: Simplify IngestService (#33008)
* INGEST: Simplify IngestService * Follow up to #32617 * Flatten redundant inner classes of `IngestService`
This commit is contained in:
parent
b595b1a20c
commit
200078734c
@ -71,26 +71,31 @@ public class IngestService implements ClusterStateApplier {
|
||||
public static final String NOOP_PIPELINE_NAME = "_none";
|
||||
|
||||
private final ClusterService clusterService;
|
||||
private final PipelineStore pipelineStore;
|
||||
private final PipelineExecutionService pipelineExecutionService;
|
||||
private final Map<String, Processor.Factory> processorFactories;
|
||||
// Ideally this should be in IngestMetadata class, but we don't have the processor factories around there.
|
||||
// We know of all the processor factories when a node with all its plugin have been initialized. Also some
|
||||
// processor factories rely on other node services. Custom metadata is statically registered when classes
|
||||
// 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();
|
||||
|
||||
public IngestService(ClusterService clusterService, ThreadPool threadPool,
|
||||
Environment env, ScriptService scriptService, AnalysisRegistry analysisRegistry,
|
||||
List<IngestPlugin> ingestPlugins) {
|
||||
this.clusterService = clusterService;
|
||||
this.pipelineStore = new PipelineStore(
|
||||
processorFactories(
|
||||
ingestPlugins,
|
||||
new Processor.Parameters(
|
||||
env, scriptService, analysisRegistry,
|
||||
threadPool.getThreadContext(), threadPool::relativeTimeInMillis,
|
||||
(delay, command) -> threadPool.schedule(
|
||||
TimeValue.timeValueMillis(delay), ThreadPool.Names.GENERIC, command
|
||||
)
|
||||
this.processorFactories = processorFactories(
|
||||
ingestPlugins,
|
||||
new Processor.Parameters(
|
||||
env, scriptService, analysisRegistry,
|
||||
threadPool.getThreadContext(), threadPool::relativeTimeInMillis,
|
||||
(delay, command) -> threadPool.schedule(
|
||||
TimeValue.timeValueMillis(delay), ThreadPool.Names.GENERIC, command
|
||||
)
|
||||
)
|
||||
);
|
||||
this.pipelineExecutionService = new PipelineExecutionService(pipelineStore, threadPool);
|
||||
this.threadPool = threadPool;
|
||||
}
|
||||
|
||||
private static Map<String, Processor.Factory> processorFactories(List<IngestPlugin> ingestPlugins,
|
||||
@ -114,8 +119,7 @@ public class IngestService implements ClusterStateApplier {
|
||||
/**
|
||||
* Deletes the pipeline specified by id in the request.
|
||||
*/
|
||||
public void delete(DeletePipelineRequest request,
|
||||
ActionListener<AcknowledgedResponse> listener) {
|
||||
public void delete(DeletePipelineRequest request, ActionListener<AcknowledgedResponse> listener) {
|
||||
clusterService.submitStateUpdateTask("delete-pipeline-" + request.getId(),
|
||||
new AckedClusterStateUpdateTask<AcknowledgedResponse>(request, listener) {
|
||||
|
||||
@ -198,32 +202,23 @@ public class IngestService implements ClusterStateApplier {
|
||||
return result;
|
||||
}
|
||||
|
||||
public void executeBulkRequest(Iterable<DocWriteRequest<?>> actionRequests, BiConsumer<IndexRequest, Exception> itemFailureHandler,
|
||||
Consumer<Exception> completionHandler) {
|
||||
pipelineExecutionService.executeBulkRequest(actionRequests, itemFailureHandler, completionHandler);
|
||||
}
|
||||
|
||||
public IngestStats stats() {
|
||||
return pipelineExecutionService.stats();
|
||||
}
|
||||
|
||||
/**
|
||||
* Stores the specified pipeline definition in the request.
|
||||
*/
|
||||
public void putPipeline(Map<DiscoveryNode, IngestInfo> ingestInfos, PutPipelineRequest request,
|
||||
ActionListener<AcknowledgedResponse> listener) throws Exception {
|
||||
pipelineStore.put(clusterService, ingestInfos, request, listener);
|
||||
put(clusterService, ingestInfos, request, listener);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the pipeline by the specified id
|
||||
*/
|
||||
public Pipeline getPipeline(String id) {
|
||||
return pipelineStore.get(id);
|
||||
return pipelines.get(id);
|
||||
}
|
||||
|
||||
public Map<String, Processor.Factory> getProcessorFactories() {
|
||||
return pipelineStore.getProcessorFactories();
|
||||
return processorFactories;
|
||||
}
|
||||
|
||||
public IngestInfo info() {
|
||||
@ -236,99 +231,64 @@ public class IngestService implements ClusterStateApplier {
|
||||
}
|
||||
|
||||
Map<String, Pipeline> pipelines() {
|
||||
return pipelineStore.pipelines;
|
||||
}
|
||||
|
||||
void validatePipeline(Map<DiscoveryNode, IngestInfo> ingestInfos, PutPipelineRequest request) throws Exception {
|
||||
pipelineStore.validatePipeline(ingestInfos, request);
|
||||
}
|
||||
|
||||
void updatePipelineStats(IngestMetadata ingestMetadata) {
|
||||
pipelineExecutionService.updatePipelineStats(ingestMetadata);
|
||||
return pipelines;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void applyClusterState(final ClusterChangedEvent event) {
|
||||
ClusterState state = event.state();
|
||||
pipelineStore.innerUpdatePipelines(event.previousState(), state);
|
||||
innerUpdatePipelines(event.previousState(), state);
|
||||
IngestMetadata ingestMetadata = state.getMetaData().custom(IngestMetadata.TYPE);
|
||||
if (ingestMetadata != null) {
|
||||
pipelineExecutionService.updatePipelineStats(ingestMetadata);
|
||||
updatePipelineStats(ingestMetadata);
|
||||
}
|
||||
}
|
||||
|
||||
public static final class PipelineStore {
|
||||
|
||||
private final Map<String, Processor.Factory> processorFactories;
|
||||
|
||||
// Ideally this should be in IngestMetadata class, but we don't have the processor factories around there.
|
||||
// We know of all the processor factories when a node with all its plugin have been initialized. Also some
|
||||
// processor factories rely on other node services. Custom metadata is statically registered when classes
|
||||
// are loaded, so in the cluster state we just save the pipeline config and here we keep the actual pipelines around.
|
||||
volatile Map<String, Pipeline> pipelines = new HashMap<>();
|
||||
|
||||
private PipelineStore(Map<String, Processor.Factory> processorFactories) {
|
||||
this.processorFactories = processorFactories;
|
||||
}
|
||||
|
||||
void innerUpdatePipelines(ClusterState previousState, ClusterState state) {
|
||||
if (state.blocks().hasGlobalBlock(GatewayService.STATE_NOT_RECOVERED_BLOCK)) {
|
||||
return;
|
||||
private static Pipeline substitutePipeline(String id, ElasticsearchParseException e) {
|
||||
String tag = e.getHeaderKeys().contains("processor_tag") ? e.getHeader("processor_tag").get(0) : null;
|
||||
String type = e.getHeaderKeys().contains("processor_type") ? e.getHeader("processor_type").get(0) : "unknown";
|
||||
String errorMessage = "pipeline with id [" + id + "] could not be loaded, caused by [" + e.getDetailedMessage() + "]";
|
||||
Processor failureProcessor = new AbstractProcessor(tag) {
|
||||
@Override
|
||||
public void execute(IngestDocument ingestDocument) {
|
||||
throw new IllegalStateException(errorMessage);
|
||||
}
|
||||
|
||||
IngestMetadata ingestMetadata = state.getMetaData().custom(IngestMetadata.TYPE);
|
||||
IngestMetadata previousIngestMetadata = previousState.getMetaData().custom(IngestMetadata.TYPE);
|
||||
if (Objects.equals(ingestMetadata, previousIngestMetadata)) {
|
||||
return;
|
||||
@Override
|
||||
public String getType() {
|
||||
return type;
|
||||
}
|
||||
};
|
||||
String description = "this is a place holder pipeline, because pipeline with id [" + id + "] could not be loaded";
|
||||
return new Pipeline(id, description, null, new CompoundProcessor(failureProcessor));
|
||||
}
|
||||
|
||||
Map<String, Pipeline> pipelines = new HashMap<>();
|
||||
List<ElasticsearchParseException> exceptions = new ArrayList<>();
|
||||
for (PipelineConfiguration pipeline : ingestMetadata.getPipelines().values()) {
|
||||
try {
|
||||
pipelines.put(pipeline.getId(), Pipeline.create(pipeline.getId(), pipeline.getConfigAsMap(), processorFactories));
|
||||
} catch (ElasticsearchParseException e) {
|
||||
pipelines.put(pipeline.getId(), substitutePipeline(pipeline.getId(), e));
|
||||
exceptions.add(e);
|
||||
} catch (Exception e) {
|
||||
ElasticsearchParseException parseException = new ElasticsearchParseException(
|
||||
"Error updating pipeline with id [" + pipeline.getId() + "]", e);
|
||||
pipelines.put(pipeline.getId(), substitutePipeline(pipeline.getId(), parseException));
|
||||
exceptions.add(parseException);
|
||||
}
|
||||
}
|
||||
this.pipelines = Collections.unmodifiableMap(pipelines);
|
||||
ExceptionsHelper.rethrowAndSuppress(exceptions);
|
||||
static ClusterState innerPut(PutPipelineRequest request, ClusterState currentState) {
|
||||
IngestMetadata currentIngestMetadata = currentState.metaData().custom(IngestMetadata.TYPE);
|
||||
Map<String, PipelineConfiguration> pipelines;
|
||||
if (currentIngestMetadata != null) {
|
||||
pipelines = new HashMap<>(currentIngestMetadata.getPipelines());
|
||||
} else {
|
||||
pipelines = new HashMap<>();
|
||||
}
|
||||
|
||||
private static Pipeline substitutePipeline(String id, ElasticsearchParseException e) {
|
||||
String tag = e.getHeaderKeys().contains("processor_tag") ? e.getHeader("processor_tag").get(0) : null;
|
||||
String type = e.getHeaderKeys().contains("processor_type") ? e.getHeader("processor_type").get(0) : "unknown";
|
||||
String errorMessage = "pipeline with id [" + id + "] could not be loaded, caused by [" + e.getDetailedMessage() + "]";
|
||||
Processor failureProcessor = new AbstractProcessor(tag) {
|
||||
@Override
|
||||
public void execute(IngestDocument ingestDocument) {
|
||||
throw new IllegalStateException(errorMessage);
|
||||
}
|
||||
pipelines.put(request.getId(), new PipelineConfiguration(request.getId(), request.getSource(), request.getXContentType()));
|
||||
ClusterState.Builder newState = ClusterState.builder(currentState);
|
||||
newState.metaData(MetaData.builder(currentState.getMetaData())
|
||||
.putCustom(IngestMetadata.TYPE, new IngestMetadata(pipelines))
|
||||
.build());
|
||||
return newState.build();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getType() {
|
||||
return type;
|
||||
}
|
||||
};
|
||||
String description = "this is a place holder pipeline, because pipeline with id [" + id + "] could not be loaded";
|
||||
return new Pipeline(id, description, null, new CompoundProcessor(failureProcessor));
|
||||
}
|
||||
|
||||
/**
|
||||
* Stores the specified pipeline definition in the request.
|
||||
*/
|
||||
public void put(ClusterService clusterService, Map<DiscoveryNode, IngestInfo> ingestInfos, PutPipelineRequest request,
|
||||
ActionListener<AcknowledgedResponse> listener) throws Exception {
|
||||
// validates the pipeline and processor configuration before submitting a cluster update task:
|
||||
validatePipeline(ingestInfos, request);
|
||||
clusterService.submitStateUpdateTask("put-pipeline-" + request.getId(),
|
||||
new AckedClusterStateUpdateTask<AcknowledgedResponse>(request, listener) {
|
||||
/**
|
||||
* Stores the specified pipeline definition in the request.
|
||||
*/
|
||||
public void put(ClusterService clusterService, Map<DiscoveryNode, IngestInfo> ingestInfos, PutPipelineRequest request,
|
||||
ActionListener<AcknowledgedResponse> listener) throws Exception {
|
||||
// validates the pipeline and processor configuration before submitting a cluster update task:
|
||||
validatePipeline(ingestInfos, request);
|
||||
clusterService.submitStateUpdateTask("put-pipeline-" + request.getId(),
|
||||
new AckedClusterStateUpdateTask<AcknowledgedResponse>(request, listener) {
|
||||
|
||||
@Override
|
||||
protected AcknowledgedResponse newResponse(boolean acknowledged) {
|
||||
@ -340,222 +300,202 @@ public class IngestService implements ClusterStateApplier {
|
||||
return innerPut(request, currentState);
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
void validatePipeline(Map<DiscoveryNode, IngestInfo> ingestInfos, PutPipelineRequest request) throws Exception {
|
||||
if (ingestInfos.isEmpty()) {
|
||||
throw new IllegalStateException("Ingest info is empty");
|
||||
}
|
||||
|
||||
void validatePipeline(Map<DiscoveryNode, IngestInfo> ingestInfos, PutPipelineRequest request) throws Exception {
|
||||
if (ingestInfos.isEmpty()) {
|
||||
throw new IllegalStateException("Ingest info is empty");
|
||||
}
|
||||
|
||||
Map<String, Object> pipelineConfig = XContentHelper.convertToMap(request.getSource(), false, request.getXContentType()).v2();
|
||||
Pipeline pipeline = Pipeline.create(request.getId(), pipelineConfig, processorFactories);
|
||||
List<Exception> exceptions = new ArrayList<>();
|
||||
for (Processor processor : pipeline.flattenAllProcessors()) {
|
||||
for (Map.Entry<DiscoveryNode, IngestInfo> entry : ingestInfos.entrySet()) {
|
||||
if (entry.getValue().containsProcessor(processor.getType()) == false) {
|
||||
String message = "Processor type [" + processor.getType() + "] is not installed on node [" + entry.getKey() + "]";
|
||||
exceptions.add(
|
||||
ConfigurationUtils.newConfigurationException(processor.getType(), processor.getTag(), null, message)
|
||||
);
|
||||
}
|
||||
Map<String, Object> pipelineConfig = XContentHelper.convertToMap(request.getSource(), false, request.getXContentType()).v2();
|
||||
Pipeline pipeline = Pipeline.create(request.getId(), pipelineConfig, processorFactories);
|
||||
List<Exception> exceptions = new ArrayList<>();
|
||||
for (Processor processor : pipeline.flattenAllProcessors()) {
|
||||
for (Map.Entry<DiscoveryNode, IngestInfo> entry : ingestInfos.entrySet()) {
|
||||
if (entry.getValue().containsProcessor(processor.getType()) == false) {
|
||||
String message = "Processor type [" + processor.getType() + "] is not installed on node [" + entry.getKey() + "]";
|
||||
exceptions.add(
|
||||
ConfigurationUtils.newConfigurationException(processor.getType(), processor.getTag(), null, message)
|
||||
);
|
||||
}
|
||||
}
|
||||
ExceptionsHelper.rethrowAndSuppress(exceptions);
|
||||
}
|
||||
ExceptionsHelper.rethrowAndSuppress(exceptions);
|
||||
}
|
||||
|
||||
static ClusterState innerPut(PutPipelineRequest request, ClusterState currentState) {
|
||||
IngestMetadata currentIngestMetadata = currentState.metaData().custom(IngestMetadata.TYPE);
|
||||
Map<String, PipelineConfiguration> pipelines;
|
||||
if (currentIngestMetadata != null) {
|
||||
pipelines = new HashMap<>(currentIngestMetadata.getPipelines());
|
||||
} else {
|
||||
pipelines = new HashMap<>();
|
||||
public void executeBulkRequest(Iterable<DocWriteRequest<?>> actionRequests,
|
||||
BiConsumer<IndexRequest, Exception> itemFailureHandler, Consumer<Exception> completionHandler) {
|
||||
threadPool.executor(ThreadPool.Names.WRITE).execute(new AbstractRunnable() {
|
||||
|
||||
@Override
|
||||
public void onFailure(Exception e) {
|
||||
completionHandler.accept(e);
|
||||
}
|
||||
|
||||
pipelines.put(request.getId(), new PipelineConfiguration(request.getId(), request.getSource(), request.getXContentType()));
|
||||
ClusterState.Builder newState = ClusterState.builder(currentState);
|
||||
newState.metaData(MetaData.builder(currentState.getMetaData())
|
||||
.putCustom(IngestMetadata.TYPE, new IngestMetadata(pipelines))
|
||||
.build());
|
||||
return newState.build();
|
||||
@Override
|
||||
protected void doRun() {
|
||||
for (DocWriteRequest<?> actionRequest : actionRequests) {
|
||||
IndexRequest indexRequest = null;
|
||||
if (actionRequest instanceof IndexRequest) {
|
||||
indexRequest = (IndexRequest) actionRequest;
|
||||
} else if (actionRequest instanceof UpdateRequest) {
|
||||
UpdateRequest updateRequest = (UpdateRequest) actionRequest;
|
||||
indexRequest = updateRequest.docAsUpsert() ? updateRequest.doc() : updateRequest.upsertRequest();
|
||||
}
|
||||
if (indexRequest == null) {
|
||||
continue;
|
||||
}
|
||||
String pipelineId = indexRequest.getPipeline();
|
||||
if (NOOP_PIPELINE_NAME.equals(pipelineId) == false) {
|
||||
try {
|
||||
Pipeline pipeline = pipelines.get(pipelineId);
|
||||
if (pipeline == null) {
|
||||
throw new IllegalArgumentException("pipeline with id [" + pipelineId + "] does not exist");
|
||||
}
|
||||
innerExecute(indexRequest, pipeline);
|
||||
//this shouldn't be needed here but we do it for consistency with index api
|
||||
// which requires it to prevent double execution
|
||||
indexRequest.setPipeline(NOOP_PIPELINE_NAME);
|
||||
} catch (Exception e) {
|
||||
itemFailureHandler.accept(indexRequest, e);
|
||||
}
|
||||
}
|
||||
}
|
||||
completionHandler.accept(null);
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
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());
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the pipeline by the specified id
|
||||
*/
|
||||
public Pipeline get(String id) {
|
||||
return pipelines.get(id);
|
||||
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;
|
||||
}
|
||||
}
|
||||
|
||||
public Map<String, Processor.Factory> getProcessorFactories() {
|
||||
return processorFactories;
|
||||
if (changed) {
|
||||
statsHolderPerPipeline = Collections.unmodifiableMap(newStatsPerPipeline);
|
||||
}
|
||||
}
|
||||
|
||||
private static final class PipelineExecutionService {
|
||||
|
||||
private final PipelineStore store;
|
||||
private final ThreadPool threadPool;
|
||||
|
||||
private final StatsHolder totalStats = new StatsHolder();
|
||||
private volatile Map<String, StatsHolder> statsHolderPerPipeline = Collections.emptyMap();
|
||||
|
||||
PipelineExecutionService(PipelineStore store, ThreadPool threadPool) {
|
||||
this.store = store;
|
||||
this.threadPool = threadPool;
|
||||
private void innerExecute(IndexRequest indexRequest, Pipeline pipeline) throws Exception {
|
||||
if (pipeline.getProcessors().isEmpty()) {
|
||||
return;
|
||||
}
|
||||
|
||||
void executeBulkRequest(Iterable<DocWriteRequest<?>> actionRequests,
|
||||
BiConsumer<IndexRequest, Exception> itemFailureHandler,
|
||||
Consumer<Exception> completionHandler) {
|
||||
threadPool.executor(ThreadPool.Names.WRITE).execute(new AbstractRunnable() {
|
||||
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);
|
||||
String index = indexRequest.index();
|
||||
String type = indexRequest.type();
|
||||
String id = indexRequest.id();
|
||||
String routing = indexRequest.routing();
|
||||
Long version = indexRequest.version();
|
||||
VersionType versionType = indexRequest.versionType();
|
||||
Map<String, Object> sourceAsMap = indexRequest.sourceAsMap();
|
||||
IngestDocument ingestDocument = new IngestDocument(index, type, id, routing, version, versionType, sourceAsMap);
|
||||
pipeline.execute(ingestDocument);
|
||||
|
||||
@Override
|
||||
public void onFailure(Exception e) {
|
||||
completionHandler.accept(e);
|
||||
}
|
||||
Map<IngestDocument.MetaData, Object> metadataMap = ingestDocument.extractMetadata();
|
||||
//it's fine to set all metadata fields all the time, as ingest document holds their starting values
|
||||
//before ingestion, which might also get modified during ingestion.
|
||||
indexRequest.index((String) metadataMap.get(IngestDocument.MetaData.INDEX));
|
||||
indexRequest.type((String) metadataMap.get(IngestDocument.MetaData.TYPE));
|
||||
indexRequest.id((String) metadataMap.get(IngestDocument.MetaData.ID));
|
||||
indexRequest.routing((String) metadataMap.get(IngestDocument.MetaData.ROUTING));
|
||||
indexRequest.version(((Number) metadataMap.get(IngestDocument.MetaData.VERSION)).longValue());
|
||||
if (metadataMap.get(IngestDocument.MetaData.VERSION_TYPE) != null) {
|
||||
indexRequest.versionType(VersionType.fromString((String) metadataMap.get(IngestDocument.MetaData.VERSION_TYPE)));
|
||||
}
|
||||
indexRequest.source(ingestDocument.getSourceAndMetadata());
|
||||
} catch (Exception e) {
|
||||
totalStats.ingestFailed();
|
||||
pipelineStats.ifPresent(StatsHolder::ingestFailed);
|
||||
throw e;
|
||||
} finally {
|
||||
long ingestTimeInMillis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTimeInNanos);
|
||||
totalStats.postIngest(ingestTimeInMillis);
|
||||
pipelineStats.ifPresent(statsHolder -> statsHolder.postIngest(ingestTimeInMillis));
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void doRun() {
|
||||
for (DocWriteRequest<?> actionRequest : actionRequests) {
|
||||
IndexRequest indexRequest = null;
|
||||
if (actionRequest instanceof IndexRequest) {
|
||||
indexRequest = (IndexRequest) actionRequest;
|
||||
} else if (actionRequest instanceof UpdateRequest) {
|
||||
UpdateRequest updateRequest = (UpdateRequest) actionRequest;
|
||||
indexRequest = updateRequest.docAsUpsert() ? updateRequest.doc() : updateRequest.upsertRequest();
|
||||
}
|
||||
if (indexRequest == null) {
|
||||
continue;
|
||||
}
|
||||
String pipeline = indexRequest.getPipeline();
|
||||
if (NOOP_PIPELINE_NAME.equals(pipeline) == false) {
|
||||
try {
|
||||
innerExecute(indexRequest, getPipeline(indexRequest.getPipeline()));
|
||||
//this shouldn't be needed here but we do it for consistency with index api
|
||||
// which requires it to prevent double execution
|
||||
indexRequest.setPipeline(NOOP_PIPELINE_NAME);
|
||||
} catch (Exception e) {
|
||||
itemFailureHandler.accept(indexRequest, e);
|
||||
}
|
||||
}
|
||||
}
|
||||
completionHandler.accept(null);
|
||||
}
|
||||
});
|
||||
private void innerUpdatePipelines(ClusterState previousState, ClusterState state) {
|
||||
if (state.blocks().hasGlobalBlock(GatewayService.STATE_NOT_RECOVERED_BLOCK)) {
|
||||
return;
|
||||
}
|
||||
|
||||
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());
|
||||
}
|
||||
|
||||
return new IngestStats(totalStats.createStats(), statsPerPipeline);
|
||||
IngestMetadata ingestMetadata = state.getMetaData().custom(IngestMetadata.TYPE);
|
||||
IngestMetadata previousIngestMetadata = previousState.getMetaData().custom(IngestMetadata.TYPE);
|
||||
if (Objects.equals(ingestMetadata, previousIngestMetadata)) {
|
||||
return;
|
||||
}
|
||||
|
||||
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);
|
||||
}
|
||||
}
|
||||
|
||||
private void innerExecute(IndexRequest indexRequest, Pipeline pipeline) throws Exception {
|
||||
if (pipeline.getProcessors().isEmpty()) {
|
||||
return;
|
||||
}
|
||||
|
||||
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()));
|
||||
Map<String, Pipeline> pipelines = new HashMap<>();
|
||||
List<ElasticsearchParseException> exceptions = new ArrayList<>();
|
||||
for (PipelineConfiguration pipeline : ingestMetadata.getPipelines().values()) {
|
||||
try {
|
||||
totalStats.preIngest();
|
||||
pipelineStats.ifPresent(StatsHolder::preIngest);
|
||||
String index = indexRequest.index();
|
||||
String type = indexRequest.type();
|
||||
String id = indexRequest.id();
|
||||
String routing = indexRequest.routing();
|
||||
Long version = indexRequest.version();
|
||||
VersionType versionType = indexRequest.versionType();
|
||||
Map<String, Object> sourceAsMap = indexRequest.sourceAsMap();
|
||||
IngestDocument ingestDocument = new IngestDocument(index, type, id, routing, version, versionType, sourceAsMap);
|
||||
pipeline.execute(ingestDocument);
|
||||
|
||||
Map<IngestDocument.MetaData, Object> metadataMap = ingestDocument.extractMetadata();
|
||||
//it's fine to set all metadata fields all the time, as ingest document holds their starting values
|
||||
//before ingestion, which might also get modified during ingestion.
|
||||
indexRequest.index((String) metadataMap.get(IngestDocument.MetaData.INDEX));
|
||||
indexRequest.type((String) metadataMap.get(IngestDocument.MetaData.TYPE));
|
||||
indexRequest.id((String) metadataMap.get(IngestDocument.MetaData.ID));
|
||||
indexRequest.routing((String) metadataMap.get(IngestDocument.MetaData.ROUTING));
|
||||
indexRequest.version(((Number) metadataMap.get(IngestDocument.MetaData.VERSION)).longValue());
|
||||
if (metadataMap.get(IngestDocument.MetaData.VERSION_TYPE) != null) {
|
||||
indexRequest.versionType(VersionType.fromString((String) metadataMap.get(IngestDocument.MetaData.VERSION_TYPE)));
|
||||
}
|
||||
indexRequest.source(ingestDocument.getSourceAndMetadata());
|
||||
pipelines.put(pipeline.getId(), Pipeline.create(pipeline.getId(), pipeline.getConfigAsMap(), processorFactories));
|
||||
} catch (ElasticsearchParseException e) {
|
||||
pipelines.put(pipeline.getId(), substitutePipeline(pipeline.getId(), e));
|
||||
exceptions.add(e);
|
||||
} catch (Exception e) {
|
||||
totalStats.ingestFailed();
|
||||
pipelineStats.ifPresent(StatsHolder::ingestFailed);
|
||||
throw e;
|
||||
} finally {
|
||||
long ingestTimeInMillis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTimeInNanos);
|
||||
totalStats.postIngest(ingestTimeInMillis);
|
||||
pipelineStats.ifPresent(statsHolder -> statsHolder.postIngest(ingestTimeInMillis));
|
||||
ElasticsearchParseException parseException = new ElasticsearchParseException(
|
||||
"Error updating pipeline with id [" + pipeline.getId() + "]", e);
|
||||
pipelines.put(pipeline.getId(), substitutePipeline(pipeline.getId(), parseException));
|
||||
exceptions.add(parseException);
|
||||
}
|
||||
}
|
||||
this.pipelines = Collections.unmodifiableMap(pipelines);
|
||||
ExceptionsHelper.rethrowAndSuppress(exceptions);
|
||||
}
|
||||
|
||||
private Pipeline getPipeline(String pipelineId) {
|
||||
Pipeline pipeline = store.get(pipelineId);
|
||||
if (pipeline == null) {
|
||||
throw new IllegalArgumentException("pipeline with id [" + pipelineId + "] does not exist");
|
||||
}
|
||||
return pipeline;
|
||||
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();
|
||||
}
|
||||
|
||||
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());
|
||||
}
|
||||
|
||||
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());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -206,7 +206,7 @@ public class IngestServiceTests extends ESTestCase {
|
||||
PutPipelineRequest putRequest = new PutPipelineRequest(id,
|
||||
new BytesArray("{\"processors\": [{\"set\" : {\"field\": \"_field\", \"value\": \"_value\"}}]}"), XContentType.JSON);
|
||||
ClusterState previousClusterState = clusterState;
|
||||
clusterState = IngestService.PipelineStore.innerPut(putRequest, clusterState);
|
||||
clusterState = IngestService.innerPut(putRequest, clusterState);
|
||||
ingestService.applyClusterState(new ClusterChangedEvent("", clusterState, previousClusterState));
|
||||
pipeline = ingestService.getPipeline(id);
|
||||
assertThat(pipeline, notNullValue());
|
||||
@ -233,7 +233,7 @@ public class IngestServiceTests extends ESTestCase {
|
||||
// add a new pipeline:
|
||||
PutPipelineRequest putRequest = new PutPipelineRequest(id, new BytesArray("{\"processors\": []}"), XContentType.JSON);
|
||||
ClusterState previousClusterState = clusterState;
|
||||
clusterState = IngestService.PipelineStore.innerPut(putRequest, clusterState);
|
||||
clusterState = IngestService.innerPut(putRequest, clusterState);
|
||||
ingestService.applyClusterState(new ClusterChangedEvent("", clusterState, previousClusterState));
|
||||
pipeline = ingestService.getPipeline(id);
|
||||
assertThat(pipeline, notNullValue());
|
||||
@ -245,7 +245,7 @@ public class IngestServiceTests extends ESTestCase {
|
||||
putRequest =
|
||||
new PutPipelineRequest(id, new BytesArray("{\"processors\": [], \"description\": \"_description\"}"), XContentType.JSON);
|
||||
previousClusterState = clusterState;
|
||||
clusterState = IngestService.PipelineStore.innerPut(putRequest, clusterState);
|
||||
clusterState = IngestService.innerPut(putRequest, clusterState);
|
||||
ingestService.applyClusterState(new ClusterChangedEvent("", clusterState, previousClusterState));
|
||||
pipeline = ingestService.getPipeline(id);
|
||||
assertThat(pipeline, notNullValue());
|
||||
@ -264,7 +264,7 @@ public class IngestServiceTests extends ESTestCase {
|
||||
PutPipelineRequest putRequest =
|
||||
new PutPipelineRequest(id, new BytesArray("{\"description\": \"empty processors\"}"), XContentType.JSON);
|
||||
ClusterState previousClusterState = clusterState;
|
||||
clusterState = IngestService.PipelineStore.innerPut(putRequest, clusterState);
|
||||
clusterState = IngestService.innerPut(putRequest, clusterState);
|
||||
try {
|
||||
ingestService.applyClusterState(new ClusterChangedEvent("", clusterState, previousClusterState));
|
||||
fail("should fail");
|
||||
@ -439,7 +439,7 @@ public class IngestServiceTests extends ESTestCase {
|
||||
PutPipelineRequest putRequest = new PutPipelineRequest(id,
|
||||
new BytesArray("{\"processors\": [{\"mock\" : {}}]}"), XContentType.JSON);
|
||||
ClusterState previousClusterState = clusterState;
|
||||
clusterState = IngestService.PipelineStore.innerPut(putRequest, clusterState);
|
||||
clusterState = IngestService.innerPut(putRequest, clusterState);
|
||||
ingestService.applyClusterState(new ClusterChangedEvent("", clusterState, previousClusterState));
|
||||
final SetOnce<Boolean> failure = new SetOnce<>();
|
||||
final IndexRequest indexRequest = new IndexRequest("_index", "_type", "_id").source(emptyMap()).setPipeline(id);
|
||||
@ -467,7 +467,7 @@ public class IngestServiceTests extends ESTestCase {
|
||||
new BytesArray("{\"processors\": [{\"mock\" : {}}]}"), XContentType.JSON);
|
||||
ClusterState clusterState = ClusterState.builder(new ClusterName("_name")).build(); // Start empty
|
||||
ClusterState previousClusterState = clusterState;
|
||||
clusterState = IngestService.PipelineStore.innerPut(putRequest, clusterState);
|
||||
clusterState = IngestService.innerPut(putRequest, clusterState);
|
||||
ingestService.applyClusterState(new ClusterChangedEvent("", clusterState, previousClusterState));
|
||||
|
||||
BulkRequest bulkRequest = new BulkRequest();
|
||||
@ -507,7 +507,7 @@ public class IngestServiceTests extends ESTestCase {
|
||||
new BytesArray("{\"processors\": [{\"mock\" : {}}]}"), XContentType.JSON);
|
||||
ClusterState clusterState = ClusterState.builder(new ClusterName("_name")).build(); // Start empty
|
||||
ClusterState previousClusterState = clusterState;
|
||||
clusterState = IngestService.PipelineStore.innerPut(putRequest, clusterState);
|
||||
clusterState = IngestService.innerPut(putRequest, clusterState);
|
||||
ingestService.applyClusterState(new ClusterChangedEvent("", clusterState, previousClusterState));
|
||||
final IndexRequest indexRequest = new IndexRequest("_index", "_type", "_id").source(emptyMap()).setPipeline("_id");
|
||||
@SuppressWarnings("unchecked")
|
||||
@ -525,7 +525,7 @@ public class IngestServiceTests extends ESTestCase {
|
||||
new PutPipelineRequest("_id", new BytesArray("{\"processors\": [], \"description\": \"_description\"}"), XContentType.JSON);
|
||||
ClusterState clusterState = ClusterState.builder(new ClusterName("_name")).build(); // Start empty
|
||||
ClusterState previousClusterState = clusterState;
|
||||
clusterState = IngestService.PipelineStore.innerPut(putRequest, clusterState);
|
||||
clusterState = IngestService.innerPut(putRequest, clusterState);
|
||||
ingestService.applyClusterState(new ClusterChangedEvent("", clusterState, previousClusterState));
|
||||
final IndexRequest indexRequest = new IndexRequest("_index", "_type", "_id").source(emptyMap()).setPipeline("_id");
|
||||
@SuppressWarnings("unchecked")
|
||||
@ -545,7 +545,7 @@ public class IngestServiceTests extends ESTestCase {
|
||||
new BytesArray("{\"processors\": [{\"mock\" : {}}]}"), XContentType.JSON);
|
||||
ClusterState clusterState = ClusterState.builder(new ClusterName("_name")).build(); // Start empty
|
||||
ClusterState previousClusterState = clusterState;
|
||||
clusterState = IngestService.PipelineStore.innerPut(putRequest, clusterState);
|
||||
clusterState = IngestService.innerPut(putRequest, clusterState);
|
||||
ingestService.applyClusterState(new ClusterChangedEvent("", clusterState, previousClusterState));
|
||||
final long newVersion = randomLong();
|
||||
final String versionType = randomFrom("internal", "external", "external_gt", "external_gte");
|
||||
@ -587,7 +587,7 @@ public class IngestServiceTests extends ESTestCase {
|
||||
new BytesArray("{\"processors\": [{\"mock\" : {}}]}"), XContentType.JSON);
|
||||
ClusterState clusterState = ClusterState.builder(new ClusterName("_name")).build(); // Start empty
|
||||
ClusterState previousClusterState = clusterState;
|
||||
clusterState = IngestService.PipelineStore.innerPut(putRequest, clusterState);
|
||||
clusterState = IngestService.innerPut(putRequest, clusterState);
|
||||
ingestService.applyClusterState(new ClusterChangedEvent("", clusterState, previousClusterState));
|
||||
final IndexRequest indexRequest = new IndexRequest("_index", "_type", "_id").source(emptyMap()).setPipeline("_id");
|
||||
doThrow(new RuntimeException())
|
||||
@ -616,7 +616,7 @@ public class IngestServiceTests extends ESTestCase {
|
||||
new BytesArray("{\"processors\": [{\"mock\" : {}}]}"), XContentType.JSON);
|
||||
ClusterState clusterState = ClusterState.builder(new ClusterName("_name")).build(); // Start empty
|
||||
ClusterState previousClusterState = clusterState;
|
||||
clusterState = IngestService.PipelineStore.innerPut(putRequest, clusterState);
|
||||
clusterState = IngestService.innerPut(putRequest, clusterState);
|
||||
ingestService.applyClusterState(new ClusterChangedEvent("", clusterState, previousClusterState));
|
||||
final IndexRequest indexRequest = new IndexRequest("_index", "_type", "_id").source(emptyMap()).setPipeline("_id");
|
||||
doThrow(new RuntimeException()).when(processor).execute(eqIndexTypeId(emptyMap()));
|
||||
@ -645,7 +645,7 @@ public class IngestServiceTests extends ESTestCase {
|
||||
new BytesArray("{\"processors\": [{\"mock\" : {}}]}"), XContentType.JSON);
|
||||
ClusterState clusterState = ClusterState.builder(new ClusterName("_name")).build(); // Start empty
|
||||
ClusterState previousClusterState = clusterState;
|
||||
clusterState = IngestService.PipelineStore.innerPut(putRequest, clusterState);
|
||||
clusterState = IngestService.innerPut(putRequest, clusterState);
|
||||
ingestService.applyClusterState(new ClusterChangedEvent("", clusterState, previousClusterState));
|
||||
final IndexRequest indexRequest = new IndexRequest("_index", "_type", "_id").source(emptyMap()).setPipeline("_id");
|
||||
doThrow(new RuntimeException())
|
||||
@ -700,7 +700,7 @@ public class IngestServiceTests extends ESTestCase {
|
||||
new BytesArray("{\"processors\": [{\"mock\" : {}}]}"), XContentType.JSON);
|
||||
ClusterState clusterState = ClusterState.builder(new ClusterName("_name")).build(); // Start empty
|
||||
ClusterState previousClusterState = clusterState;
|
||||
clusterState = IngestService.PipelineStore.innerPut(putRequest, clusterState);
|
||||
clusterState = IngestService.innerPut(putRequest, clusterState);
|
||||
ingestService.applyClusterState(new ClusterChangedEvent("", clusterState, previousClusterState));
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
@ -734,7 +734,7 @@ public class IngestServiceTests extends ESTestCase {
|
||||
new PutPipelineRequest("_id", new BytesArray("{\"processors\": [], \"description\": \"_description\"}"), XContentType.JSON);
|
||||
ClusterState clusterState = ClusterState.builder(new ClusterName("_name")).build();
|
||||
ClusterState previousClusterState = clusterState;
|
||||
clusterState = IngestService.PipelineStore.innerPut(putRequest, clusterState);
|
||||
clusterState = IngestService.innerPut(putRequest, clusterState);
|
||||
ingestService.applyClusterState(new ClusterChangedEvent("", clusterState, previousClusterState));
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
@ -762,12 +762,12 @@ public class IngestServiceTests extends ESTestCase {
|
||||
new BytesArray("{\"processors\": [{\"mock\" : {}}]}"), XContentType.JSON);
|
||||
ClusterState clusterState = ClusterState.builder(new ClusterName("_name")).build(); // Start empty
|
||||
ClusterState previousClusterState = clusterState;
|
||||
clusterState = IngestService.PipelineStore.innerPut(putRequest, clusterState);
|
||||
clusterState = IngestService.innerPut(putRequest, clusterState);
|
||||
ingestService.applyClusterState(new ClusterChangedEvent("", clusterState, previousClusterState));
|
||||
putRequest = new PutPipelineRequest("_id2",
|
||||
new BytesArray("{\"processors\": [{\"mock\" : {}}]}"), XContentType.JSON);
|
||||
previousClusterState = clusterState;
|
||||
clusterState = IngestService.PipelineStore.innerPut(putRequest, clusterState);
|
||||
clusterState = IngestService.innerPut(putRequest, clusterState);
|
||||
ingestService.applyClusterState(new ClusterChangedEvent("", clusterState, previousClusterState));
|
||||
final Map<String, PipelineConfiguration> configurationMap = new HashMap<>();
|
||||
configurationMap.put("_id1", new PipelineConfiguration("_id1", new BytesArray("{}"), XContentType.JSON));
|
||||
|
Loading…
x
Reference in New Issue
Block a user