mirror of
https://github.com/honeymoose/OpenSearch.git
synced 2025-03-25 09:28:27 +00:00
Merge pull request #16049 from elastic/feature/ingest
Merge feature/ingest branch into master branch. This adds the ingest feature to ES that allows to preprocess document before indexing on an ingest node. By default a node is an ingest node. Documents are preprocessed via a pipeline. A pipeline consists out of one or more processors Each processor makes one or more modifications to a document processed. There are many types of processors available out-of-the-box that are designed to make a specific change to a document being processed. In a cluster many pipeline can be configured via dedicated pipeline APIs. An new option on the bulk and index APIs allows to control what pipeline is picked for preprocessing. If no pipeline is specified then the ingest feature is skipped and no preprocessing takes place.
This commit is contained in:
commit
df0be87b18
@ -149,6 +149,16 @@ import org.elasticsearch.action.indexedscripts.get.GetIndexedScriptAction;
|
||||
import org.elasticsearch.action.indexedscripts.get.TransportGetIndexedScriptAction;
|
||||
import org.elasticsearch.action.indexedscripts.put.PutIndexedScriptAction;
|
||||
import org.elasticsearch.action.indexedscripts.put.TransportPutIndexedScriptAction;
|
||||
import org.elasticsearch.action.ingest.IngestActionFilter;
|
||||
import org.elasticsearch.action.ingest.IngestProxyActionFilter;
|
||||
import org.elasticsearch.action.ingest.DeletePipelineAction;
|
||||
import org.elasticsearch.action.ingest.DeletePipelineTransportAction;
|
||||
import org.elasticsearch.action.ingest.GetPipelineAction;
|
||||
import org.elasticsearch.action.ingest.GetPipelineTransportAction;
|
||||
import org.elasticsearch.action.ingest.PutPipelineAction;
|
||||
import org.elasticsearch.action.ingest.PutPipelineTransportAction;
|
||||
import org.elasticsearch.action.ingest.SimulatePipelineAction;
|
||||
import org.elasticsearch.action.ingest.SimulatePipelineTransportAction;
|
||||
import org.elasticsearch.action.percolate.MultiPercolateAction;
|
||||
import org.elasticsearch.action.percolate.PercolateAction;
|
||||
import org.elasticsearch.action.percolate.TransportMultiPercolateAction;
|
||||
@ -186,6 +196,8 @@ import org.elasticsearch.action.update.UpdateAction;
|
||||
import org.elasticsearch.common.inject.AbstractModule;
|
||||
import org.elasticsearch.common.inject.multibindings.MapBinder;
|
||||
import org.elasticsearch.common.inject.multibindings.Multibinder;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.node.NodeModule;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
@ -210,13 +222,13 @@ public class ActionModule extends AbstractModule {
|
||||
this.transportAction = transportAction;
|
||||
this.supportTransportActions = supportTransportActions;
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
||||
private final boolean ingestEnabled;
|
||||
private final boolean proxy;
|
||||
|
||||
public ActionModule(boolean proxy) {
|
||||
public ActionModule(boolean ingestEnabled, boolean proxy) {
|
||||
this.ingestEnabled = ingestEnabled;
|
||||
this.proxy = proxy;
|
||||
}
|
||||
|
||||
@ -240,6 +252,13 @@ public class ActionModule extends AbstractModule {
|
||||
|
||||
@Override
|
||||
protected void configure() {
|
||||
if (proxy == false) {
|
||||
if (ingestEnabled) {
|
||||
registerFilter(IngestActionFilter.class);
|
||||
} else {
|
||||
registerFilter(IngestProxyActionFilter.class);
|
||||
}
|
||||
}
|
||||
|
||||
Multibinder<ActionFilter> actionFilterMultibinder = Multibinder.newSetBinder(binder(), ActionFilter.class);
|
||||
for (Class<? extends ActionFilter> actionFilter : actionFilters) {
|
||||
@ -340,6 +359,11 @@ public class ActionModule extends AbstractModule {
|
||||
|
||||
registerAction(FieldStatsAction.INSTANCE, TransportFieldStatsTransportAction.class);
|
||||
|
||||
registerAction(PutPipelineAction.INSTANCE, PutPipelineTransportAction.class);
|
||||
registerAction(GetPipelineAction.INSTANCE, GetPipelineTransportAction.class);
|
||||
registerAction(DeletePipelineAction.INSTANCE, DeletePipelineTransportAction.class);
|
||||
registerAction(SimulatePipelineAction.INSTANCE, SimulatePipelineTransportAction.class);
|
||||
|
||||
// register Name -> GenericAction Map that can be injected to instances.
|
||||
MapBinder<String, GenericAction> actionsBinder
|
||||
= MapBinder.newMapBinder(binder(), String.class, GenericAction.class);
|
||||
|
@ -289,11 +289,11 @@ public class BulkProcessor implements Closeable {
|
||||
}
|
||||
|
||||
public BulkProcessor add(BytesReference data, @Nullable String defaultIndex, @Nullable String defaultType) throws Exception {
|
||||
return add(data, defaultIndex, defaultType, null);
|
||||
return add(data, defaultIndex, defaultType, null, null);
|
||||
}
|
||||
|
||||
public synchronized BulkProcessor add(BytesReference data, @Nullable String defaultIndex, @Nullable String defaultType, @Nullable Object payload) throws Exception {
|
||||
bulkRequest.add(data, defaultIndex, defaultType, null, null, payload, true);
|
||||
public synchronized BulkProcessor add(BytesReference data, @Nullable String defaultIndex, @Nullable String defaultType, @Nullable String defaultPipeline, @Nullable Object payload) throws Exception {
|
||||
bulkRequest.add(data, defaultIndex, defaultType, null, null, defaultPipeline, payload, true);
|
||||
executeIfNeeded();
|
||||
return this;
|
||||
}
|
||||
|
@ -28,6 +28,7 @@ import org.elasticsearch.action.delete.DeleteRequest;
|
||||
import org.elasticsearch.action.index.IndexRequest;
|
||||
import org.elasticsearch.action.update.UpdateRequest;
|
||||
import org.elasticsearch.common.Nullable;
|
||||
import org.elasticsearch.common.Strings;
|
||||
import org.elasticsearch.common.bytes.BytesArray;
|
||||
import org.elasticsearch.common.bytes.BytesReference;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
@ -253,17 +254,17 @@ public class BulkRequest extends ActionRequest<BulkRequest> implements Composite
|
||||
* Adds a framed data in binary format
|
||||
*/
|
||||
public BulkRequest add(BytesReference data, @Nullable String defaultIndex, @Nullable String defaultType) throws Exception {
|
||||
return add(data, defaultIndex, defaultType, null, null, null, true);
|
||||
return add(data, defaultIndex, defaultType, null, null, null, null, true);
|
||||
}
|
||||
|
||||
/**
|
||||
* Adds a framed data in binary format
|
||||
*/
|
||||
public BulkRequest add(BytesReference data, @Nullable String defaultIndex, @Nullable String defaultType, boolean allowExplicitIndex) throws Exception {
|
||||
return add(data, defaultIndex, defaultType, null, null, null, allowExplicitIndex);
|
||||
return add(data, defaultIndex, defaultType, null, null, null, null, allowExplicitIndex);
|
||||
}
|
||||
|
||||
public BulkRequest add(BytesReference data, @Nullable String defaultIndex, @Nullable String defaultType, @Nullable String defaultRouting, @Nullable String[] defaultFields, @Nullable Object payload, boolean allowExplicitIndex) throws Exception {
|
||||
public BulkRequest add(BytesReference data, @Nullable String defaultIndex, @Nullable String defaultType, @Nullable String defaultRouting, @Nullable String[] defaultFields, @Nullable String defaultPipeline, @Nullable Object payload, boolean allowExplicitIndex) throws Exception {
|
||||
XContent xContent = XContentFactory.xContent(data);
|
||||
int line = 0;
|
||||
int from = 0;
|
||||
@ -304,6 +305,7 @@ public class BulkRequest extends ActionRequest<BulkRequest> implements Composite
|
||||
long version = Versions.MATCH_ANY;
|
||||
VersionType versionType = VersionType.INTERNAL;
|
||||
int retryOnConflict = 0;
|
||||
String pipeline = defaultPipeline;
|
||||
|
||||
// at this stage, next token can either be END_OBJECT (and use default index and type, with auto generated id)
|
||||
// or START_OBJECT which will have another set of parameters
|
||||
@ -344,6 +346,8 @@ public class BulkRequest extends ActionRequest<BulkRequest> implements Composite
|
||||
versionType = VersionType.fromString(parser.text());
|
||||
} else if ("_retry_on_conflict".equals(currentFieldName) || "_retryOnConflict".equals(currentFieldName)) {
|
||||
retryOnConflict = parser.intValue();
|
||||
} else if ("pipeline".equals(currentFieldName)) {
|
||||
pipeline = parser.text();
|
||||
} else if ("fields".equals(currentFieldName)) {
|
||||
throw new IllegalArgumentException("Action/metadata line [" + line + "] contains a simple value for parameter [fields] while a list is expected");
|
||||
} else {
|
||||
@ -380,15 +384,15 @@ public class BulkRequest extends ActionRequest<BulkRequest> implements Composite
|
||||
if ("index".equals(action)) {
|
||||
if (opType == null) {
|
||||
internalAdd(new IndexRequest(index, type, id).routing(routing).parent(parent).timestamp(timestamp).ttl(ttl).version(version).versionType(versionType)
|
||||
.source(data.slice(from, nextMarker - from)), payload);
|
||||
.setPipeline(pipeline).source(data.slice(from, nextMarker - from)), payload);
|
||||
} else {
|
||||
internalAdd(new IndexRequest(index, type, id).routing(routing).parent(parent).timestamp(timestamp).ttl(ttl).version(version).versionType(versionType)
|
||||
.create("create".equals(opType))
|
||||
.create("create".equals(opType)).setPipeline(pipeline)
|
||||
.source(data.slice(from, nextMarker - from)), payload);
|
||||
}
|
||||
} else if ("create".equals(action)) {
|
||||
internalAdd(new IndexRequest(index, type, id).routing(routing).parent(parent).timestamp(timestamp).ttl(ttl).version(version).versionType(versionType)
|
||||
.create(true)
|
||||
.create(true).setPipeline(pipeline)
|
||||
.source(data.slice(from, nextMarker - from)), payload);
|
||||
} else if ("update".equals(action)) {
|
||||
UpdateRequest updateRequest = new UpdateRequest(index, type, id).routing(routing).parent(parent).retryOnConflict(retryOnConflict)
|
||||
@ -479,6 +483,22 @@ public class BulkRequest extends ActionRequest<BulkRequest> implements Composite
|
||||
return -1;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return Whether this bulk request contains index request with an ingest pipeline enabled.
|
||||
*/
|
||||
public boolean hasIndexRequestsWithPipelines() {
|
||||
for (ActionRequest actionRequest : requests) {
|
||||
if (actionRequest instanceof IndexRequest) {
|
||||
IndexRequest indexRequest = (IndexRequest) actionRequest;
|
||||
if (Strings.hasText(indexRequest.getPipeline())) {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ActionRequestValidationException validate() {
|
||||
ActionRequestValidationException validationException = null;
|
||||
|
@ -155,6 +155,8 @@ public class IndexRequest extends ReplicationRequest<IndexRequest> implements Do
|
||||
|
||||
private XContentType contentType = Requests.INDEX_CONTENT_TYPE;
|
||||
|
||||
private String pipeline;
|
||||
|
||||
public IndexRequest() {
|
||||
}
|
||||
|
||||
@ -363,6 +365,21 @@ public class IndexRequest extends ReplicationRequest<IndexRequest> implements Do
|
||||
return this.ttl;
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the ingest pipeline to be executed before indexing the document
|
||||
*/
|
||||
public IndexRequest setPipeline(String pipeline) {
|
||||
this.pipeline = pipeline;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the ingest pipeline to be executed before indexing the document
|
||||
*/
|
||||
public String getPipeline() {
|
||||
return this.pipeline;
|
||||
}
|
||||
|
||||
/**
|
||||
* The source of the document to index, recopied to a new array if it is unsage.
|
||||
*/
|
||||
@ -658,6 +675,7 @@ public class IndexRequest extends ReplicationRequest<IndexRequest> implements Do
|
||||
refresh = in.readBoolean();
|
||||
version = in.readLong();
|
||||
versionType = VersionType.fromValue(in.readByte());
|
||||
pipeline = in.readOptionalString();
|
||||
}
|
||||
|
||||
@Override
|
||||
@ -679,6 +697,7 @@ public class IndexRequest extends ReplicationRequest<IndexRequest> implements Do
|
||||
out.writeBoolean(refresh);
|
||||
out.writeLong(version);
|
||||
out.writeByte(versionType.getValue());
|
||||
out.writeOptionalString(pipeline);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -278,4 +278,12 @@ public class IndexRequestBuilder extends ReplicationRequestBuilder<IndexRequest,
|
||||
request.ttl(ttl);
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the ingest pipeline to be executed before indexing the document
|
||||
*/
|
||||
public IndexRequestBuilder setPipeline(String pipeline) {
|
||||
request.setPipeline(pipeline);
|
||||
return this;
|
||||
}
|
||||
}
|
||||
|
@ -0,0 +1,43 @@
|
||||
/*
|
||||
* 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.action.ingest;
|
||||
|
||||
import org.elasticsearch.action.Action;
|
||||
import org.elasticsearch.client.ElasticsearchClient;
|
||||
|
||||
public class DeletePipelineAction extends Action<DeletePipelineRequest, WritePipelineResponse, DeletePipelineRequestBuilder> {
|
||||
|
||||
public static final DeletePipelineAction INSTANCE = new DeletePipelineAction();
|
||||
public static final String NAME = "cluster:admin/ingest/pipeline/delete";
|
||||
|
||||
public DeletePipelineAction() {
|
||||
super(NAME);
|
||||
}
|
||||
|
||||
@Override
|
||||
public DeletePipelineRequestBuilder newRequestBuilder(ElasticsearchClient client) {
|
||||
return new DeletePipelineRequestBuilder(client, this);
|
||||
}
|
||||
|
||||
@Override
|
||||
public WritePipelineResponse newResponse() {
|
||||
return new WritePipelineResponse();
|
||||
}
|
||||
}
|
@ -0,0 +1,70 @@
|
||||
/*
|
||||
* 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.action.ingest;
|
||||
|
||||
import org.elasticsearch.action.ActionRequestValidationException;
|
||||
import org.elasticsearch.action.support.master.AcknowledgedRequest;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Objects;
|
||||
|
||||
import static org.elasticsearch.action.ValidateActions.addValidationError;
|
||||
|
||||
public class DeletePipelineRequest extends AcknowledgedRequest<DeletePipelineRequest> {
|
||||
|
||||
private String id;
|
||||
|
||||
public DeletePipelineRequest(String id) {
|
||||
if (id == null) {
|
||||
throw new IllegalArgumentException("id is missing");
|
||||
}
|
||||
this.id = id;
|
||||
}
|
||||
|
||||
DeletePipelineRequest() {
|
||||
}
|
||||
|
||||
public void setId(String id) {
|
||||
this.id = Objects.requireNonNull(id);
|
||||
}
|
||||
|
||||
public String getId() {
|
||||
return id;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ActionRequestValidationException validate() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void readFrom(StreamInput in) throws IOException {
|
||||
super.readFrom(in);
|
||||
id = in.readString();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeTo(StreamOutput out) throws IOException {
|
||||
super.writeTo(out);
|
||||
out.writeString(id);
|
||||
}
|
||||
}
|
@ -0,0 +1,35 @@
|
||||
/*
|
||||
* 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.action.ingest;
|
||||
|
||||
import org.elasticsearch.action.ActionRequestBuilder;
|
||||
import org.elasticsearch.client.ElasticsearchClient;
|
||||
|
||||
public class DeletePipelineRequestBuilder extends ActionRequestBuilder<DeletePipelineRequest, WritePipelineResponse, DeletePipelineRequestBuilder> {
|
||||
|
||||
public DeletePipelineRequestBuilder(ElasticsearchClient client, DeletePipelineAction action) {
|
||||
super(client, action, new DeletePipelineRequest());
|
||||
}
|
||||
|
||||
public DeletePipelineRequestBuilder(ElasticsearchClient client, DeletePipelineAction action, String id) {
|
||||
super(client, action, new DeletePipelineRequest(id));
|
||||
}
|
||||
|
||||
}
|
@ -0,0 +1,71 @@
|
||||
/*
|
||||
* 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.action.ingest;
|
||||
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.action.support.ActionFilters;
|
||||
import org.elasticsearch.action.support.master.TransportMasterNodeAction;
|
||||
import org.elasticsearch.cluster.ClusterService;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
import org.elasticsearch.cluster.block.ClusterBlockException;
|
||||
import org.elasticsearch.cluster.block.ClusterBlockLevel;
|
||||
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
|
||||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.ingest.PipelineStore;
|
||||
import org.elasticsearch.node.service.NodeService;
|
||||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
import org.elasticsearch.transport.TransportService;
|
||||
|
||||
public class DeletePipelineTransportAction extends TransportMasterNodeAction<DeletePipelineRequest, WritePipelineResponse> {
|
||||
|
||||
private final PipelineStore pipelineStore;
|
||||
private final ClusterService clusterService;
|
||||
|
||||
@Inject
|
||||
public DeletePipelineTransportAction(Settings settings, ThreadPool threadPool, ClusterService clusterService,
|
||||
TransportService transportService, ActionFilters actionFilters,
|
||||
IndexNameExpressionResolver indexNameExpressionResolver, NodeService nodeService) {
|
||||
super(settings, DeletePipelineAction.NAME, transportService, clusterService, threadPool, actionFilters, indexNameExpressionResolver, DeletePipelineRequest::new);
|
||||
this.clusterService = clusterService;
|
||||
this.pipelineStore = nodeService.getIngestService().getPipelineStore();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected String executor() {
|
||||
return ThreadPool.Names.SAME;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected WritePipelineResponse newResponse() {
|
||||
return new WritePipelineResponse();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void masterOperation(DeletePipelineRequest request, ClusterState state, ActionListener<WritePipelineResponse> listener) throws Exception {
|
||||
pipelineStore.delete(clusterService, request, listener);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected ClusterBlockException checkBlock(DeletePipelineRequest request, ClusterState state) {
|
||||
return state.blocks().globalBlockedException(ClusterBlockLevel.METADATA_WRITE);
|
||||
}
|
||||
|
||||
}
|
@ -0,0 +1,43 @@
|
||||
/*
|
||||
* 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.action.ingest;
|
||||
|
||||
import org.elasticsearch.action.Action;
|
||||
import org.elasticsearch.client.ElasticsearchClient;
|
||||
|
||||
public class GetPipelineAction extends Action<GetPipelineRequest, GetPipelineResponse, GetPipelineRequestBuilder> {
|
||||
|
||||
public static final GetPipelineAction INSTANCE = new GetPipelineAction();
|
||||
public static final String NAME = "cluster:admin/ingest/pipeline/get";
|
||||
|
||||
public GetPipelineAction() {
|
||||
super(NAME);
|
||||
}
|
||||
|
||||
@Override
|
||||
public GetPipelineRequestBuilder newRequestBuilder(ElasticsearchClient client) {
|
||||
return new GetPipelineRequestBuilder(client, this);
|
||||
}
|
||||
|
||||
@Override
|
||||
public GetPipelineResponse newResponse() {
|
||||
return new GetPipelineResponse();
|
||||
}
|
||||
}
|
@ -0,0 +1,66 @@
|
||||
/*
|
||||
* 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.action.ingest;
|
||||
|
||||
import org.elasticsearch.action.ActionRequestValidationException;
|
||||
import org.elasticsearch.action.support.master.MasterNodeReadRequest;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Objects;
|
||||
|
||||
import static org.elasticsearch.action.ValidateActions.addValidationError;
|
||||
|
||||
public class GetPipelineRequest extends MasterNodeReadRequest<GetPipelineRequest> {
|
||||
|
||||
private String[] ids;
|
||||
|
||||
public GetPipelineRequest(String... ids) {
|
||||
if (ids == null || ids.length == 0) {
|
||||
throw new IllegalArgumentException("No ids specified");
|
||||
}
|
||||
this.ids = ids;
|
||||
}
|
||||
|
||||
GetPipelineRequest() {
|
||||
}
|
||||
|
||||
public String[] getIds() {
|
||||
return ids;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ActionRequestValidationException validate() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void readFrom(StreamInput in) throws IOException {
|
||||
super.readFrom(in);
|
||||
ids = in.readStringArray();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeTo(StreamOutput out) throws IOException {
|
||||
super.writeTo(out);
|
||||
out.writeStringArray(ids);
|
||||
}
|
||||
}
|
@ -0,0 +1,35 @@
|
||||
/*
|
||||
* 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.action.ingest;
|
||||
|
||||
import org.elasticsearch.action.support.master.MasterNodeReadOperationRequestBuilder;
|
||||
import org.elasticsearch.client.ElasticsearchClient;
|
||||
|
||||
public class GetPipelineRequestBuilder extends MasterNodeReadOperationRequestBuilder<GetPipelineRequest, GetPipelineResponse, GetPipelineRequestBuilder> {
|
||||
|
||||
public GetPipelineRequestBuilder(ElasticsearchClient client, GetPipelineAction action) {
|
||||
super(client, action, new GetPipelineRequest());
|
||||
}
|
||||
|
||||
public GetPipelineRequestBuilder(ElasticsearchClient client, GetPipelineAction action, String[] ids) {
|
||||
super(client, action, new GetPipelineRequest(ids));
|
||||
}
|
||||
|
||||
}
|
@ -0,0 +1,86 @@
|
||||
/*
|
||||
* 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.action.ingest;
|
||||
|
||||
import org.elasticsearch.action.ActionResponse;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.xcontent.StatusToXContent;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.ingest.PipelineConfiguration;
|
||||
import org.elasticsearch.rest.RestStatus;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
public class GetPipelineResponse extends ActionResponse implements StatusToXContent {
|
||||
|
||||
private List<PipelineConfiguration> pipelines;
|
||||
|
||||
public GetPipelineResponse() {
|
||||
}
|
||||
|
||||
public GetPipelineResponse(List<PipelineConfiguration> pipelines) {
|
||||
this.pipelines = pipelines;
|
||||
}
|
||||
|
||||
public List<PipelineConfiguration> pipelines() {
|
||||
return pipelines;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void readFrom(StreamInput in) throws IOException {
|
||||
super.readFrom(in);
|
||||
int size = in.readVInt();
|
||||
pipelines = new ArrayList<>(size);
|
||||
for (int i = 0; i < size; i++) {
|
||||
pipelines.add(PipelineConfiguration.readPipelineConfiguration(in));
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeTo(StreamOutput out) throws IOException {
|
||||
super.writeTo(out);
|
||||
out.writeVInt(pipelines.size());
|
||||
for (PipelineConfiguration pipeline : pipelines) {
|
||||
pipeline.writeTo(out);
|
||||
}
|
||||
}
|
||||
|
||||
public boolean isFound() {
|
||||
return !pipelines.isEmpty();
|
||||
}
|
||||
|
||||
@Override
|
||||
public RestStatus status() {
|
||||
return isFound() ? RestStatus.OK : RestStatus.NOT_FOUND;
|
||||
}
|
||||
|
||||
@Override
|
||||
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
|
||||
builder.startArray("pipelines");
|
||||
for (PipelineConfiguration pipeline : pipelines) {
|
||||
pipeline.toXContent(builder, params);
|
||||
}
|
||||
builder.endArray();
|
||||
return builder;
|
||||
}
|
||||
}
|
@ -0,0 +1,69 @@
|
||||
/*
|
||||
* 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.action.ingest;
|
||||
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.action.support.ActionFilters;
|
||||
import org.elasticsearch.action.support.master.TransportMasterNodeReadAction;
|
||||
import org.elasticsearch.cluster.ClusterService;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
import org.elasticsearch.cluster.block.ClusterBlockException;
|
||||
import org.elasticsearch.cluster.block.ClusterBlockLevel;
|
||||
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
|
||||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.ingest.PipelineStore;
|
||||
import org.elasticsearch.node.service.NodeService;
|
||||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
import org.elasticsearch.transport.TransportService;
|
||||
|
||||
public class GetPipelineTransportAction extends TransportMasterNodeReadAction<GetPipelineRequest, GetPipelineResponse> {
|
||||
|
||||
private final PipelineStore pipelineStore;
|
||||
|
||||
@Inject
|
||||
public GetPipelineTransportAction(Settings settings, ThreadPool threadPool, ClusterService clusterService,
|
||||
TransportService transportService, ActionFilters actionFilters,
|
||||
IndexNameExpressionResolver indexNameExpressionResolver, NodeService nodeService) {
|
||||
super(settings, GetPipelineAction.NAME, transportService, clusterService, threadPool, actionFilters, indexNameExpressionResolver, GetPipelineRequest::new);
|
||||
this.pipelineStore = nodeService.getIngestService().getPipelineStore();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected String executor() {
|
||||
return ThreadPool.Names.SAME;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected GetPipelineResponse newResponse() {
|
||||
return new GetPipelineResponse();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void masterOperation(GetPipelineRequest request, ClusterState state, ActionListener<GetPipelineResponse> listener) throws Exception {
|
||||
listener.onResponse(new GetPipelineResponse(pipelineStore.getPipelines(state, request.getIds())));
|
||||
}
|
||||
|
||||
@Override
|
||||
protected ClusterBlockException checkBlock(GetPipelineRequest request, ClusterState state) {
|
||||
return state.blocks().globalBlockedException(ClusterBlockLevel.METADATA_READ);
|
||||
}
|
||||
|
||||
}
|
@ -0,0 +1,225 @@
|
||||
/*
|
||||
* 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.action.ingest;
|
||||
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.action.ActionRequest;
|
||||
import org.elasticsearch.action.ActionResponse;
|
||||
import org.elasticsearch.action.bulk.BulkAction;
|
||||
import org.elasticsearch.action.bulk.BulkItemResponse;
|
||||
import org.elasticsearch.action.bulk.BulkRequest;
|
||||
import org.elasticsearch.action.bulk.BulkResponse;
|
||||
import org.elasticsearch.action.index.IndexAction;
|
||||
import org.elasticsearch.action.index.IndexRequest;
|
||||
import org.elasticsearch.action.support.ActionFilter;
|
||||
import org.elasticsearch.action.support.ActionFilterChain;
|
||||
import org.elasticsearch.common.Strings;
|
||||
import org.elasticsearch.common.component.AbstractComponent;
|
||||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.ingest.PipelineExecutionService;
|
||||
import org.elasticsearch.node.service.NodeService;
|
||||
import org.elasticsearch.tasks.Task;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashSet;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
|
||||
public final class IngestActionFilter extends AbstractComponent implements ActionFilter {
|
||||
|
||||
private final PipelineExecutionService executionService;
|
||||
|
||||
@Inject
|
||||
public IngestActionFilter(Settings settings, NodeService nodeService) {
|
||||
super(settings);
|
||||
this.executionService = nodeService.getIngestService().getPipelineExecutionService();
|
||||
}
|
||||
|
||||
@Override
|
||||
public <Request extends ActionRequest<Request>, Response extends ActionResponse> void apply(Task task, String action, Request request, ActionListener<Response> listener, ActionFilterChain<Request, Response> chain) {
|
||||
switch (action) {
|
||||
case IndexAction.NAME:
|
||||
IndexRequest indexRequest = (IndexRequest) request;
|
||||
if (Strings.hasText(indexRequest.getPipeline())) {
|
||||
processIndexRequest(task, action, listener, chain, (IndexRequest) request);
|
||||
} else {
|
||||
chain.proceed(task, action, request, listener);
|
||||
}
|
||||
break;
|
||||
case BulkAction.NAME:
|
||||
BulkRequest bulkRequest = (BulkRequest) request;
|
||||
if (bulkRequest.hasIndexRequestsWithPipelines()) {
|
||||
@SuppressWarnings("unchecked")
|
||||
ActionListener<BulkResponse> actionListener = (ActionListener<BulkResponse>) listener;
|
||||
processBulkIndexRequest(task, bulkRequest, action, chain, actionListener);
|
||||
} else {
|
||||
chain.proceed(task, action, request, listener);
|
||||
}
|
||||
break;
|
||||
default:
|
||||
chain.proceed(task, action, request, listener);
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public <Response extends ActionResponse> void apply(String action, Response response, ActionListener<Response> listener, ActionFilterChain<?, Response> chain) {
|
||||
chain.proceed(action, response, listener);
|
||||
}
|
||||
|
||||
void processIndexRequest(Task task, String action, ActionListener listener, ActionFilterChain chain, IndexRequest indexRequest) {
|
||||
|
||||
executionService.execute(indexRequest, t -> {
|
||||
logger.error("failed to execute pipeline [{}]", t, indexRequest.getPipeline());
|
||||
listener.onFailure(t);
|
||||
}, success -> {
|
||||
// TransportIndexAction uses IndexRequest and same action name on the node that receives the request and the node that
|
||||
// processes the primary action. This could lead to a pipeline being executed twice for the same
|
||||
// index request, hence we set the pipeline to null once its execution completed.
|
||||
indexRequest.setPipeline(null);
|
||||
chain.proceed(task, action, indexRequest, listener);
|
||||
});
|
||||
}
|
||||
|
||||
void processBulkIndexRequest(Task task, BulkRequest original, String action, ActionFilterChain chain, ActionListener<BulkResponse> listener) {
|
||||
BulkRequestModifier bulkRequestModifier = new BulkRequestModifier(original);
|
||||
executionService.execute(() -> bulkRequestModifier, (indexRequest, throwable) -> {
|
||||
logger.debug("failed to execute pipeline [{}] for document [{}/{}/{}]", indexRequest.getPipeline(), indexRequest.index(), indexRequest.type(), indexRequest.id(), throwable);
|
||||
bulkRequestModifier.markCurrentItemAsFailed(throwable);
|
||||
}, (throwable) -> {
|
||||
if (throwable != null) {
|
||||
logger.error("failed to execute pipeline for a bulk request", throwable);
|
||||
listener.onFailure(throwable);
|
||||
} else {
|
||||
BulkRequest bulkRequest = bulkRequestModifier.getBulkRequest();
|
||||
ActionListener<BulkResponse> actionListener = bulkRequestModifier.wrapActionListenerIfNeeded(listener);
|
||||
if (bulkRequest.requests().isEmpty()) {
|
||||
// at this stage, the transport bulk action can't deal with a bulk request with no requests,
|
||||
// so we stop and send an empty response back to the client.
|
||||
// (this will happen if pre-processing all items in the bulk failed)
|
||||
actionListener.onResponse(new BulkResponse(new BulkItemResponse[0], 0));
|
||||
} else {
|
||||
chain.proceed(task, action, bulkRequest, actionListener);
|
||||
}
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
public int order() {
|
||||
return Integer.MAX_VALUE;
|
||||
}
|
||||
|
||||
final static class BulkRequestModifier implements Iterator<ActionRequest<?>> {
|
||||
|
||||
final BulkRequest bulkRequest;
|
||||
final Set<Integer> failedSlots;
|
||||
final List<BulkItemResponse> itemResponses;
|
||||
|
||||
int currentSlot = -1;
|
||||
int[] originalSlots;
|
||||
|
||||
BulkRequestModifier(BulkRequest bulkRequest) {
|
||||
this.bulkRequest = bulkRequest;
|
||||
this.failedSlots = new HashSet<>();
|
||||
this.itemResponses = new ArrayList<>(bulkRequest.requests().size());
|
||||
}
|
||||
|
||||
@Override
|
||||
public ActionRequest next() {
|
||||
return bulkRequest.requests().get(++currentSlot);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasNext() {
|
||||
return (currentSlot + 1) < bulkRequest.requests().size();
|
||||
}
|
||||
|
||||
BulkRequest getBulkRequest() {
|
||||
if (itemResponses.isEmpty()) {
|
||||
return bulkRequest;
|
||||
} else {
|
||||
BulkRequest modifiedBulkRequest = new BulkRequest(bulkRequest);
|
||||
modifiedBulkRequest.refresh(bulkRequest.refresh());
|
||||
modifiedBulkRequest.consistencyLevel(bulkRequest.consistencyLevel());
|
||||
modifiedBulkRequest.timeout(bulkRequest.timeout());
|
||||
|
||||
int slot = 0;
|
||||
originalSlots = new int[bulkRequest.requests().size() - failedSlots.size()];
|
||||
for (int i = 0; i < bulkRequest.requests().size(); i++) {
|
||||
ActionRequest request = bulkRequest.requests().get(i);
|
||||
if (failedSlots.contains(i) == false) {
|
||||
modifiedBulkRequest.add(request);
|
||||
originalSlots[slot++] = i;
|
||||
}
|
||||
}
|
||||
return modifiedBulkRequest;
|
||||
}
|
||||
}
|
||||
|
||||
ActionListener<BulkResponse> wrapActionListenerIfNeeded(ActionListener<BulkResponse> actionListener) {
|
||||
if (itemResponses.isEmpty()) {
|
||||
return actionListener;
|
||||
} else {
|
||||
return new IngestBulkResponseListener(originalSlots, itemResponses, actionListener);
|
||||
}
|
||||
}
|
||||
|
||||
void markCurrentItemAsFailed(Throwable e) {
|
||||
IndexRequest indexRequest = (IndexRequest) bulkRequest.requests().get(currentSlot);
|
||||
// We hit a error during preprocessing a request, so we:
|
||||
// 1) Remember the request item slot from the bulk, so that we're done processing all requests we know what failed
|
||||
// 2) Add a bulk item failure for this request
|
||||
// 3) Continue with the next request in the bulk.
|
||||
failedSlots.add(currentSlot);
|
||||
BulkItemResponse.Failure failure = new BulkItemResponse.Failure(indexRequest.index(), indexRequest.type(), indexRequest.id(), e);
|
||||
itemResponses.add(new BulkItemResponse(currentSlot, indexRequest.opType().lowercase(), failure));
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
private final static class IngestBulkResponseListener implements ActionListener<BulkResponse> {
|
||||
|
||||
private final int[] originalSlots;
|
||||
private final List<BulkItemResponse> itemResponses;
|
||||
private final ActionListener<BulkResponse> actionListener;
|
||||
|
||||
IngestBulkResponseListener(int[] originalSlots, List<BulkItemResponse> itemResponses, ActionListener<BulkResponse> actionListener) {
|
||||
this.itemResponses = itemResponses;
|
||||
this.actionListener = actionListener;
|
||||
this.originalSlots = originalSlots;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onResponse(BulkResponse bulkItemResponses) {
|
||||
for (int i = 0; i < bulkItemResponses.getItems().length; i++) {
|
||||
itemResponses.add(originalSlots[i], bulkItemResponses.getItems()[i]);
|
||||
}
|
||||
actionListener.onResponse(new BulkResponse(itemResponses.toArray(new BulkItemResponse[itemResponses.size()]), bulkItemResponses.getTookInMillis()));
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onFailure(Throwable e) {
|
||||
actionListener.onFailure(e);
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,125 @@
|
||||
/*
|
||||
* 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.action.ingest;
|
||||
|
||||
import org.elasticsearch.action.Action;
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.action.ActionListenerResponseHandler;
|
||||
import org.elasticsearch.action.ActionRequest;
|
||||
import org.elasticsearch.action.ActionResponse;
|
||||
import org.elasticsearch.action.bulk.BulkAction;
|
||||
import org.elasticsearch.action.bulk.BulkRequest;
|
||||
import org.elasticsearch.action.index.IndexAction;
|
||||
import org.elasticsearch.action.index.IndexRequest;
|
||||
import org.elasticsearch.action.support.ActionFilter;
|
||||
import org.elasticsearch.action.support.ActionFilterChain;
|
||||
import org.elasticsearch.cluster.ClusterService;
|
||||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||
import org.elasticsearch.cluster.node.DiscoveryNodes;
|
||||
import org.elasticsearch.common.Randomness;
|
||||
import org.elasticsearch.common.Strings;
|
||||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.tasks.Task;
|
||||
import org.elasticsearch.transport.TransportResponse;
|
||||
import org.elasticsearch.transport.TransportService;
|
||||
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
public final class IngestProxyActionFilter implements ActionFilter {
|
||||
|
||||
private final ClusterService clusterService;
|
||||
private final TransportService transportService;
|
||||
private final AtomicInteger randomNodeGenerator = new AtomicInteger(Randomness.get().nextInt());
|
||||
|
||||
@Inject
|
||||
public IngestProxyActionFilter(ClusterService clusterService, TransportService transportService) {
|
||||
this.clusterService = clusterService;
|
||||
this.transportService = transportService;
|
||||
}
|
||||
|
||||
@Override
|
||||
public <Request extends ActionRequest<Request>, Response extends ActionResponse> void apply(Task task, String action, Request request, ActionListener<Response> listener, ActionFilterChain<Request, Response> chain) {
|
||||
Action ingestAction;
|
||||
switch (action) {
|
||||
case IndexAction.NAME:
|
||||
ingestAction = IndexAction.INSTANCE;
|
||||
IndexRequest indexRequest = (IndexRequest) request;
|
||||
if (Strings.hasText(indexRequest.getPipeline())) {
|
||||
forwardIngestRequest(ingestAction, request, listener);
|
||||
} else {
|
||||
chain.proceed(task, action, request, listener);
|
||||
}
|
||||
break;
|
||||
case BulkAction.NAME:
|
||||
ingestAction = BulkAction.INSTANCE;
|
||||
BulkRequest bulkRequest = (BulkRequest) request;
|
||||
if (bulkRequest.hasIndexRequestsWithPipelines()) {
|
||||
forwardIngestRequest(ingestAction, request, listener);
|
||||
} else {
|
||||
chain.proceed(task, action, request, listener);
|
||||
}
|
||||
break;
|
||||
default:
|
||||
chain.proceed(task, action, request, listener);
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
private void forwardIngestRequest(Action<?, ?, ?> action, ActionRequest request, ActionListener<?> listener) {
|
||||
transportService.sendRequest(randomIngestNode(), action.name(), request, new ActionListenerResponseHandler(listener) {
|
||||
@Override
|
||||
public TransportResponse newInstance() {
|
||||
return action.newResponse();
|
||||
}
|
||||
|
||||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
public <Response extends ActionResponse> void apply(String action, Response response, ActionListener<Response> listener, ActionFilterChain<?, Response> chain) {
|
||||
chain.proceed(action, response, listener);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int order() {
|
||||
return Integer.MAX_VALUE;
|
||||
}
|
||||
|
||||
private DiscoveryNode randomIngestNode() {
|
||||
assert clusterService.localNode().isIngestNode() == false;
|
||||
DiscoveryNodes nodes = clusterService.state().getNodes();
|
||||
DiscoveryNode[] ingestNodes = nodes.getIngestNodes().values().toArray(DiscoveryNode.class);
|
||||
if (ingestNodes.length == 0) {
|
||||
throw new IllegalStateException("There are no ingest nodes in this cluster, unable to forward request to an ingest node.");
|
||||
}
|
||||
|
||||
int index = getNodeNumber();
|
||||
return ingestNodes[(index) % ingestNodes.length];
|
||||
}
|
||||
|
||||
private int getNodeNumber() {
|
||||
int index = randomNodeGenerator.incrementAndGet();
|
||||
if (index < 0) {
|
||||
index = 0;
|
||||
randomNodeGenerator.set(0);
|
||||
}
|
||||
return index;
|
||||
}
|
||||
}
|
@ -0,0 +1,44 @@
|
||||
/*
|
||||
* 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.action.ingest;
|
||||
|
||||
import org.elasticsearch.action.Action;
|
||||
import org.elasticsearch.action.index.IndexResponse;
|
||||
import org.elasticsearch.client.ElasticsearchClient;
|
||||
|
||||
public class PutPipelineAction extends Action<PutPipelineRequest, WritePipelineResponse, PutPipelineRequestBuilder> {
|
||||
|
||||
public static final PutPipelineAction INSTANCE = new PutPipelineAction();
|
||||
public static final String NAME = "cluster:admin/ingest/pipeline/put";
|
||||
|
||||
public PutPipelineAction() {
|
||||
super(NAME);
|
||||
}
|
||||
|
||||
@Override
|
||||
public PutPipelineRequestBuilder newRequestBuilder(ElasticsearchClient client) {
|
||||
return new PutPipelineRequestBuilder(client, this);
|
||||
}
|
||||
|
||||
@Override
|
||||
public WritePipelineResponse newResponse() {
|
||||
return new WritePipelineResponse();
|
||||
}
|
||||
}
|
@ -0,0 +1,79 @@
|
||||
/*
|
||||
* 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.action.ingest;
|
||||
|
||||
import org.elasticsearch.action.ActionRequestValidationException;
|
||||
import org.elasticsearch.action.support.master.AcknowledgedRequest;
|
||||
import org.elasticsearch.common.bytes.BytesReference;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Objects;
|
||||
|
||||
import static org.elasticsearch.action.ValidateActions.addValidationError;
|
||||
|
||||
public class PutPipelineRequest extends AcknowledgedRequest<PutPipelineRequest> {
|
||||
|
||||
private String id;
|
||||
private BytesReference source;
|
||||
|
||||
public PutPipelineRequest(String id, BytesReference source) {
|
||||
if (id == null) {
|
||||
throw new IllegalArgumentException("id is missing");
|
||||
}
|
||||
if (source == null) {
|
||||
throw new IllegalArgumentException("source is missing");
|
||||
}
|
||||
|
||||
this.id = id;
|
||||
this.source = source;
|
||||
}
|
||||
|
||||
PutPipelineRequest() {
|
||||
}
|
||||
|
||||
@Override
|
||||
public ActionRequestValidationException validate() {
|
||||
return null;
|
||||
}
|
||||
|
||||
public String getId() {
|
||||
return id;
|
||||
}
|
||||
|
||||
public BytesReference getSource() {
|
||||
return source;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void readFrom(StreamInput in) throws IOException {
|
||||
super.readFrom(in);
|
||||
id = in.readString();
|
||||
source = in.readBytesReference();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeTo(StreamOutput out) throws IOException {
|
||||
super.writeTo(out);
|
||||
out.writeString(id);
|
||||
out.writeBytesReference(source);
|
||||
}
|
||||
}
|
@ -0,0 +1,36 @@
|
||||
/*
|
||||
* 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.action.ingest;
|
||||
|
||||
import org.elasticsearch.action.ActionRequestBuilder;
|
||||
import org.elasticsearch.client.ElasticsearchClient;
|
||||
import org.elasticsearch.common.bytes.BytesReference;
|
||||
|
||||
public class PutPipelineRequestBuilder extends ActionRequestBuilder<PutPipelineRequest, WritePipelineResponse, PutPipelineRequestBuilder> {
|
||||
|
||||
public PutPipelineRequestBuilder(ElasticsearchClient client, PutPipelineAction action) {
|
||||
super(client, action, new PutPipelineRequest());
|
||||
}
|
||||
|
||||
public PutPipelineRequestBuilder(ElasticsearchClient client, PutPipelineAction action, String id, BytesReference source) {
|
||||
super(client, action, new PutPipelineRequest(id, source));
|
||||
}
|
||||
|
||||
}
|
@ -0,0 +1,71 @@
|
||||
/*
|
||||
* 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.action.ingest;
|
||||
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.action.support.ActionFilters;
|
||||
import org.elasticsearch.action.support.master.TransportMasterNodeAction;
|
||||
import org.elasticsearch.cluster.ClusterService;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
import org.elasticsearch.cluster.block.ClusterBlockException;
|
||||
import org.elasticsearch.cluster.block.ClusterBlockLevel;
|
||||
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
|
||||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.ingest.PipelineStore;
|
||||
import org.elasticsearch.node.service.NodeService;
|
||||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
import org.elasticsearch.transport.TransportService;
|
||||
|
||||
public class PutPipelineTransportAction extends TransportMasterNodeAction<PutPipelineRequest, WritePipelineResponse> {
|
||||
|
||||
private final PipelineStore pipelineStore;
|
||||
private final ClusterService clusterService;
|
||||
|
||||
@Inject
|
||||
public PutPipelineTransportAction(Settings settings, ThreadPool threadPool, ClusterService clusterService,
|
||||
TransportService transportService, ActionFilters actionFilters,
|
||||
IndexNameExpressionResolver indexNameExpressionResolver, NodeService nodeService) {
|
||||
super(settings, PutPipelineAction.NAME, transportService, clusterService, threadPool, actionFilters, indexNameExpressionResolver, PutPipelineRequest::new);
|
||||
this.clusterService = clusterService;
|
||||
this.pipelineStore = nodeService.getIngestService().getPipelineStore();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected String executor() {
|
||||
return ThreadPool.Names.SAME;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected WritePipelineResponse newResponse() {
|
||||
return new WritePipelineResponse();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void masterOperation(PutPipelineRequest request, ClusterState state, ActionListener<WritePipelineResponse> listener) throws Exception {
|
||||
pipelineStore.put(clusterService, request, listener);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected ClusterBlockException checkBlock(PutPipelineRequest request, ClusterState state) {
|
||||
return state.blocks().globalBlockedException(ClusterBlockLevel.METADATA_WRITE);
|
||||
}
|
||||
|
||||
}
|
@ -0,0 +1,98 @@
|
||||
/*
|
||||
* 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.action.ingest;
|
||||
|
||||
import org.elasticsearch.ElasticsearchException;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.ingest.core.IngestDocument;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
|
||||
/**
|
||||
* Holds the end result of what a pipeline did to sample document provided via the simulate api.
|
||||
*/
|
||||
public final class SimulateDocumentBaseResult implements SimulateDocumentResult<SimulateDocumentBaseResult> {
|
||||
|
||||
private static final SimulateDocumentBaseResult PROTOTYPE = new SimulateDocumentBaseResult(new WriteableIngestDocument(new IngestDocument(Collections.emptyMap(), Collections.emptyMap())));
|
||||
|
||||
private WriteableIngestDocument ingestDocument;
|
||||
private Exception failure;
|
||||
|
||||
public SimulateDocumentBaseResult(IngestDocument ingestDocument) {
|
||||
this.ingestDocument = new WriteableIngestDocument(ingestDocument);
|
||||
}
|
||||
|
||||
private SimulateDocumentBaseResult(WriteableIngestDocument ingestDocument) {
|
||||
this.ingestDocument = ingestDocument;
|
||||
}
|
||||
|
||||
public SimulateDocumentBaseResult(Exception failure) {
|
||||
this.failure = failure;
|
||||
}
|
||||
|
||||
public IngestDocument getIngestDocument() {
|
||||
if (ingestDocument == null) {
|
||||
return null;
|
||||
}
|
||||
return ingestDocument.getIngestDocument();
|
||||
}
|
||||
|
||||
public Exception getFailure() {
|
||||
return failure;
|
||||
}
|
||||
|
||||
public static SimulateDocumentBaseResult readSimulateDocumentSimpleResult(StreamInput in) throws IOException {
|
||||
return PROTOTYPE.readFrom(in);
|
||||
}
|
||||
|
||||
@Override
|
||||
public SimulateDocumentBaseResult readFrom(StreamInput in) throws IOException {
|
||||
if (in.readBoolean()) {
|
||||
Exception exception = in.readThrowable();
|
||||
return new SimulateDocumentBaseResult(exception);
|
||||
}
|
||||
return new SimulateDocumentBaseResult(new WriteableIngestDocument(in));
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeTo(StreamOutput out) throws IOException {
|
||||
if (failure == null) {
|
||||
out.writeBoolean(false);
|
||||
ingestDocument.writeTo(out);
|
||||
} else {
|
||||
out.writeBoolean(true);
|
||||
out.writeThrowable(failure);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
|
||||
builder.startObject();
|
||||
if (failure == null) {
|
||||
ingestDocument.toXContent(builder, params);
|
||||
} else {
|
||||
ElasticsearchException.renderThrowable(builder, params, failure);
|
||||
}
|
||||
builder.endObject();
|
||||
return builder;
|
||||
}
|
||||
}
|
@ -0,0 +1,26 @@
|
||||
/*
|
||||
* 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.action.ingest;
|
||||
|
||||
import org.elasticsearch.common.io.stream.Writeable;
|
||||
import org.elasticsearch.common.xcontent.ToXContent;
|
||||
|
||||
public interface SimulateDocumentResult<T extends SimulateDocumentResult> extends Writeable<T>, ToXContent {
|
||||
|
||||
}
|
@ -0,0 +1,81 @@
|
||||
/*
|
||||
* 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.action.ingest;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* Holds the result of what a pipeline did to a sample document via the simulate api, but instead of {@link SimulateDocumentBaseResult}
|
||||
* this result class holds the intermediate result each processor did to the sample document.
|
||||
*/
|
||||
public final class SimulateDocumentVerboseResult implements SimulateDocumentResult<SimulateDocumentVerboseResult> {
|
||||
|
||||
private static final SimulateDocumentVerboseResult PROTOTYPE = new SimulateDocumentVerboseResult(Collections.emptyList());
|
||||
|
||||
private final List<SimulateProcessorResult> processorResults;
|
||||
|
||||
public SimulateDocumentVerboseResult(List<SimulateProcessorResult> processorResults) {
|
||||
this.processorResults = processorResults;
|
||||
}
|
||||
|
||||
public List<SimulateProcessorResult> getProcessorResults() {
|
||||
return processorResults;
|
||||
}
|
||||
|
||||
public static SimulateDocumentVerboseResult readSimulateDocumentVerboseResultFrom(StreamInput in) throws IOException {
|
||||
return PROTOTYPE.readFrom(in);
|
||||
}
|
||||
|
||||
@Override
|
||||
public SimulateDocumentVerboseResult readFrom(StreamInput in) throws IOException {
|
||||
int size = in.readVInt();
|
||||
List<SimulateProcessorResult> processorResults = new ArrayList<>();
|
||||
for (int i = 0; i < size; i++) {
|
||||
processorResults.add(new SimulateProcessorResult(in));
|
||||
}
|
||||
return new SimulateDocumentVerboseResult(processorResults);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeTo(StreamOutput out) throws IOException {
|
||||
out.writeVInt(processorResults.size());
|
||||
for (SimulateProcessorResult result : processorResults) {
|
||||
result.writeTo(out);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
|
||||
builder.startObject();
|
||||
builder.startArray("processor_results");
|
||||
for (SimulateProcessorResult processorResult : processorResults) {
|
||||
processorResult.toXContent(builder, params);
|
||||
}
|
||||
builder.endArray();
|
||||
builder.endObject();
|
||||
return builder;
|
||||
}
|
||||
}
|
@ -0,0 +1,99 @@
|
||||
/*
|
||||
* 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.action.ingest;
|
||||
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.action.ActionRunnable;
|
||||
import org.elasticsearch.ingest.core.IngestDocument;
|
||||
import org.elasticsearch.ingest.core.Pipeline;
|
||||
import org.elasticsearch.ingest.core.Processor;
|
||||
import org.elasticsearch.ingest.core.CompoundProcessor;
|
||||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
class SimulateExecutionService {
|
||||
|
||||
private static final String THREAD_POOL_NAME = ThreadPool.Names.MANAGEMENT;
|
||||
|
||||
private final ThreadPool threadPool;
|
||||
|
||||
SimulateExecutionService(ThreadPool threadPool) {
|
||||
this.threadPool = threadPool;
|
||||
}
|
||||
|
||||
void executeVerboseDocument(Processor processor, IngestDocument ingestDocument, List<SimulateProcessorResult> processorResultList) throws Exception {
|
||||
if (processor instanceof CompoundProcessor) {
|
||||
CompoundProcessor cp = (CompoundProcessor) processor;
|
||||
try {
|
||||
for (Processor p : cp.getProcessors()) {
|
||||
executeVerboseDocument(p, ingestDocument, processorResultList);
|
||||
}
|
||||
} catch (Exception e) {
|
||||
for (Processor p : cp.getOnFailureProcessors()) {
|
||||
executeVerboseDocument(p, ingestDocument, processorResultList);
|
||||
}
|
||||
}
|
||||
} else {
|
||||
try {
|
||||
processor.execute(ingestDocument);
|
||||
processorResultList.add(new SimulateProcessorResult(processor.getTag(), new IngestDocument(ingestDocument)));
|
||||
} catch (Exception e) {
|
||||
processorResultList.add(new SimulateProcessorResult(processor.getTag(), e));
|
||||
throw e;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
SimulateDocumentResult executeDocument(Pipeline pipeline, IngestDocument ingestDocument, boolean verbose) {
|
||||
if (verbose) {
|
||||
List<SimulateProcessorResult> processorResultList = new ArrayList<>();
|
||||
IngestDocument currentIngestDocument = new IngestDocument(ingestDocument);
|
||||
CompoundProcessor pipelineProcessor = new CompoundProcessor(pipeline.getProcessors(), pipeline.getOnFailureProcessors());
|
||||
try {
|
||||
executeVerboseDocument(pipelineProcessor, currentIngestDocument, processorResultList);
|
||||
} catch (Exception e) {
|
||||
return new SimulateDocumentBaseResult(e);
|
||||
}
|
||||
return new SimulateDocumentVerboseResult(processorResultList);
|
||||
} else {
|
||||
try {
|
||||
pipeline.execute(ingestDocument);
|
||||
return new SimulateDocumentBaseResult(ingestDocument);
|
||||
} catch (Exception e) {
|
||||
return new SimulateDocumentBaseResult(e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public void execute(SimulatePipelineRequest.Parsed request, ActionListener<SimulatePipelineResponse> listener) {
|
||||
threadPool.executor(THREAD_POOL_NAME).execute(new ActionRunnable<SimulatePipelineResponse>(listener) {
|
||||
@Override
|
||||
protected void doRun() throws Exception {
|
||||
List<SimulateDocumentResult> responses = new ArrayList<>();
|
||||
for (IngestDocument ingestDocument : request.getDocuments()) {
|
||||
responses.add(executeDocument(request.getPipeline(), ingestDocument, request.isVerbose()));
|
||||
}
|
||||
listener.onResponse(new SimulatePipelineResponse(request.getPipeline().getId(), request.isVerbose(), responses));
|
||||
}
|
||||
});
|
||||
}
|
||||
}
|
@ -0,0 +1,43 @@
|
||||
/*
|
||||
* 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.action.ingest;
|
||||
|
||||
import org.elasticsearch.action.Action;
|
||||
import org.elasticsearch.client.ElasticsearchClient;
|
||||
|
||||
public class SimulatePipelineAction extends Action<SimulatePipelineRequest, SimulatePipelineResponse, SimulatePipelineRequestBuilder> {
|
||||
|
||||
public static final SimulatePipelineAction INSTANCE = new SimulatePipelineAction();
|
||||
public static final String NAME = "cluster:admin/ingest/pipeline/simulate";
|
||||
|
||||
public SimulatePipelineAction() {
|
||||
super(NAME);
|
||||
}
|
||||
|
||||
@Override
|
||||
public SimulatePipelineRequestBuilder newRequestBuilder(ElasticsearchClient client) {
|
||||
return new SimulatePipelineRequestBuilder(client, this);
|
||||
}
|
||||
|
||||
@Override
|
||||
public SimulatePipelineResponse newResponse() {
|
||||
return new SimulatePipelineResponse();
|
||||
}
|
||||
}
|
@ -0,0 +1,165 @@
|
||||
/*
|
||||
* 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.action.ingest;
|
||||
|
||||
import org.elasticsearch.action.ActionRequest;
|
||||
import org.elasticsearch.action.ActionRequestValidationException;
|
||||
import org.elasticsearch.common.bytes.BytesReference;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.ingest.core.ConfigurationUtils;
|
||||
import org.elasticsearch.ingest.core.IngestDocument;
|
||||
import org.elasticsearch.ingest.core.Pipeline;
|
||||
import org.elasticsearch.ingest.PipelineStore;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
|
||||
import static org.elasticsearch.action.ValidateActions.addValidationError;
|
||||
import static org.elasticsearch.ingest.core.IngestDocument.MetaData;
|
||||
|
||||
public class SimulatePipelineRequest extends ActionRequest<SimulatePipelineRequest> {
|
||||
|
||||
private String id;
|
||||
private boolean verbose;
|
||||
private BytesReference source;
|
||||
|
||||
public SimulatePipelineRequest(BytesReference source) {
|
||||
if (source == null) {
|
||||
throw new IllegalArgumentException("source is missing");
|
||||
}
|
||||
this.source = source;
|
||||
}
|
||||
|
||||
SimulatePipelineRequest() {
|
||||
}
|
||||
|
||||
@Override
|
||||
public ActionRequestValidationException validate() {
|
||||
return null;
|
||||
}
|
||||
|
||||
public String getId() {
|
||||
return id;
|
||||
}
|
||||
|
||||
public void setId(String id) {
|
||||
this.id = id;
|
||||
}
|
||||
|
||||
public boolean isVerbose() {
|
||||
return verbose;
|
||||
}
|
||||
|
||||
public void setVerbose(boolean verbose) {
|
||||
this.verbose = verbose;
|
||||
}
|
||||
|
||||
public BytesReference getSource() {
|
||||
return source;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void readFrom(StreamInput in) throws IOException {
|
||||
super.readFrom(in);
|
||||
id = in.readString();
|
||||
verbose = in.readBoolean();
|
||||
source = in.readBytesReference();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeTo(StreamOutput out) throws IOException {
|
||||
super.writeTo(out);
|
||||
out.writeString(id);
|
||||
out.writeBoolean(verbose);
|
||||
out.writeBytesReference(source);
|
||||
}
|
||||
|
||||
public static final class Fields {
|
||||
static final String PIPELINE = "pipeline";
|
||||
static final String DOCS = "docs";
|
||||
static final String SOURCE = "_source";
|
||||
}
|
||||
|
||||
static class Parsed {
|
||||
private final List<IngestDocument> documents;
|
||||
private final Pipeline pipeline;
|
||||
private final boolean verbose;
|
||||
|
||||
Parsed(Pipeline pipeline, List<IngestDocument> documents, boolean verbose) {
|
||||
this.pipeline = pipeline;
|
||||
this.documents = Collections.unmodifiableList(documents);
|
||||
this.verbose = verbose;
|
||||
}
|
||||
|
||||
public Pipeline getPipeline() {
|
||||
return pipeline;
|
||||
}
|
||||
|
||||
public List<IngestDocument> getDocuments() {
|
||||
return documents;
|
||||
}
|
||||
|
||||
public boolean isVerbose() {
|
||||
return verbose;
|
||||
}
|
||||
}
|
||||
|
||||
private static final Pipeline.Factory PIPELINE_FACTORY = new Pipeline.Factory();
|
||||
static final String SIMULATED_PIPELINE_ID = "_simulate_pipeline";
|
||||
|
||||
static Parsed parseWithPipelineId(String pipelineId, Map<String, Object> config, boolean verbose, PipelineStore pipelineStore) {
|
||||
if (pipelineId == null) {
|
||||
throw new IllegalArgumentException("param [pipeline] is null");
|
||||
}
|
||||
Pipeline pipeline = pipelineStore.get(pipelineId);
|
||||
List<IngestDocument> ingestDocumentList = parseDocs(config);
|
||||
return new Parsed(pipeline, ingestDocumentList, verbose);
|
||||
}
|
||||
|
||||
static Parsed parse(Map<String, Object> config, boolean verbose, PipelineStore pipelineStore) throws Exception {
|
||||
Map<String, Object> pipelineConfig = ConfigurationUtils.readMap(config, Fields.PIPELINE);
|
||||
Pipeline pipeline = PIPELINE_FACTORY.create(SIMULATED_PIPELINE_ID, pipelineConfig, pipelineStore.getProcessorFactoryRegistry());
|
||||
List<IngestDocument> ingestDocumentList = parseDocs(config);
|
||||
return new Parsed(pipeline, ingestDocumentList, verbose);
|
||||
}
|
||||
|
||||
private static List<IngestDocument> parseDocs(Map<String, Object> config) {
|
||||
List<Map<String, Object>> docs = ConfigurationUtils.readList(config, Fields.DOCS);
|
||||
List<IngestDocument> ingestDocumentList = new ArrayList<>();
|
||||
for (Map<String, Object> dataMap : docs) {
|
||||
Map<String, Object> document = ConfigurationUtils.readMap(dataMap, Fields.SOURCE);
|
||||
IngestDocument ingestDocument = new IngestDocument(ConfigurationUtils.readStringProperty(dataMap, MetaData.INDEX.getFieldName(), "_index"),
|
||||
ConfigurationUtils.readStringProperty(dataMap, MetaData.TYPE.getFieldName(), "_type"),
|
||||
ConfigurationUtils.readStringProperty(dataMap, MetaData.ID.getFieldName(), "_id"),
|
||||
ConfigurationUtils.readOptionalStringProperty(dataMap, MetaData.ROUTING.getFieldName()),
|
||||
ConfigurationUtils.readOptionalStringProperty(dataMap, MetaData.PARENT.getFieldName()),
|
||||
ConfigurationUtils.readOptionalStringProperty(dataMap, MetaData.TIMESTAMP.getFieldName()),
|
||||
ConfigurationUtils.readOptionalStringProperty(dataMap, MetaData.TTL.getFieldName()),
|
||||
document);
|
||||
ingestDocumentList.add(ingestDocument);
|
||||
}
|
||||
return ingestDocumentList;
|
||||
}
|
||||
}
|
@ -0,0 +1,46 @@
|
||||
/*
|
||||
* 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.action.ingest;
|
||||
|
||||
import org.elasticsearch.action.ActionRequestBuilder;
|
||||
import org.elasticsearch.client.ElasticsearchClient;
|
||||
import org.elasticsearch.common.bytes.BytesReference;
|
||||
|
||||
public class SimulatePipelineRequestBuilder extends ActionRequestBuilder<SimulatePipelineRequest, SimulatePipelineResponse, SimulatePipelineRequestBuilder> {
|
||||
|
||||
public SimulatePipelineRequestBuilder(ElasticsearchClient client, SimulatePipelineAction action) {
|
||||
super(client, action, new SimulatePipelineRequest());
|
||||
}
|
||||
|
||||
public SimulatePipelineRequestBuilder(ElasticsearchClient client, SimulatePipelineAction action, BytesReference source) {
|
||||
super(client, action, new SimulatePipelineRequest(source));
|
||||
}
|
||||
|
||||
public SimulatePipelineRequestBuilder setId(String id) {
|
||||
request.setId(id);
|
||||
return this;
|
||||
}
|
||||
|
||||
public SimulatePipelineRequestBuilder setVerbose(boolean verbose) {
|
||||
request.setVerbose(verbose);
|
||||
return this;
|
||||
}
|
||||
|
||||
}
|
@ -0,0 +1,103 @@
|
||||
/*
|
||||
* 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.action.ingest;
|
||||
|
||||
import org.elasticsearch.action.ActionResponse;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.xcontent.ToXContent;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilderString;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
||||
public class SimulatePipelineResponse extends ActionResponse implements ToXContent {
|
||||
private String pipelineId;
|
||||
private boolean verbose;
|
||||
private List<SimulateDocumentResult> results;
|
||||
|
||||
public SimulatePipelineResponse() {
|
||||
|
||||
}
|
||||
|
||||
public SimulatePipelineResponse(String pipelineId, boolean verbose, List<SimulateDocumentResult> responses) {
|
||||
this.pipelineId = pipelineId;
|
||||
this.verbose = verbose;
|
||||
this.results = Collections.unmodifiableList(responses);
|
||||
}
|
||||
|
||||
public String getPipelineId() {
|
||||
return pipelineId;
|
||||
}
|
||||
|
||||
public List<SimulateDocumentResult> getResults() {
|
||||
return results;
|
||||
}
|
||||
|
||||
public boolean isVerbose() {
|
||||
return verbose;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeTo(StreamOutput out) throws IOException {
|
||||
super.writeTo(out);
|
||||
out.writeString(pipelineId);
|
||||
out.writeBoolean(verbose);
|
||||
out.writeVInt(results.size());
|
||||
for (SimulateDocumentResult response : results) {
|
||||
response.writeTo(out);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void readFrom(StreamInput in) throws IOException {
|
||||
super.readFrom(in);
|
||||
this.pipelineId = in.readString();
|
||||
boolean verbose = in.readBoolean();
|
||||
int responsesLength = in.readVInt();
|
||||
results = new ArrayList<>();
|
||||
for (int i = 0; i < responsesLength; i++) {
|
||||
SimulateDocumentResult<?> simulateDocumentResult;
|
||||
if (verbose) {
|
||||
simulateDocumentResult = SimulateDocumentVerboseResult.readSimulateDocumentVerboseResultFrom(in);
|
||||
} else {
|
||||
simulateDocumentResult = SimulateDocumentBaseResult.readSimulateDocumentSimpleResult(in);
|
||||
}
|
||||
results.add(simulateDocumentResult);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
|
||||
builder.startArray(Fields.DOCUMENTS);
|
||||
for (SimulateDocumentResult response : results) {
|
||||
response.toXContent(builder, params);
|
||||
}
|
||||
builder.endArray();
|
||||
return builder;
|
||||
}
|
||||
|
||||
static final class Fields {
|
||||
static final XContentBuilderString DOCUMENTS = new XContentBuilderString("docs");
|
||||
}
|
||||
}
|
@ -0,0 +1,66 @@
|
||||
/*
|
||||
* 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.action.ingest;
|
||||
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.action.support.ActionFilters;
|
||||
import org.elasticsearch.action.support.HandledTransportAction;
|
||||
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
|
||||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.xcontent.XContentHelper;
|
||||
import org.elasticsearch.ingest.PipelineStore;
|
||||
import org.elasticsearch.node.service.NodeService;
|
||||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
import org.elasticsearch.transport.TransportService;
|
||||
|
||||
import java.util.Map;
|
||||
|
||||
public class SimulatePipelineTransportAction extends HandledTransportAction<SimulatePipelineRequest, SimulatePipelineResponse> {
|
||||
|
||||
private final PipelineStore pipelineStore;
|
||||
private final SimulateExecutionService executionService;
|
||||
|
||||
@Inject
|
||||
public SimulatePipelineTransportAction(Settings settings, ThreadPool threadPool, TransportService transportService, ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver, NodeService nodeService) {
|
||||
super(settings, SimulatePipelineAction.NAME, threadPool, transportService, actionFilters, indexNameExpressionResolver, SimulatePipelineRequest::new);
|
||||
this.pipelineStore = nodeService.getIngestService().getPipelineStore();
|
||||
this.executionService = new SimulateExecutionService(threadPool);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void doExecute(SimulatePipelineRequest request, ActionListener<SimulatePipelineResponse> listener) {
|
||||
final Map<String, Object> source = XContentHelper.convertToMap(request.getSource(), false).v2();
|
||||
|
||||
final SimulatePipelineRequest.Parsed simulateRequest;
|
||||
try {
|
||||
if (request.getId() != null) {
|
||||
simulateRequest = SimulatePipelineRequest.parseWithPipelineId(request.getId(), source, request.isVerbose(), pipelineStore);
|
||||
} else {
|
||||
simulateRequest = SimulatePipelineRequest.parse(source, request.isVerbose(), pipelineStore);
|
||||
}
|
||||
} catch (Exception e) {
|
||||
listener.onFailure(e);
|
||||
return;
|
||||
}
|
||||
|
||||
executionService.execute(simulateRequest, listener);
|
||||
}
|
||||
}
|
@ -0,0 +1,106 @@
|
||||
/*
|
||||
* 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.action.ingest;
|
||||
|
||||
import org.elasticsearch.ElasticsearchException;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.io.stream.Writeable;
|
||||
import org.elasticsearch.common.xcontent.ToXContent;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.ingest.core.AbstractProcessorFactory;
|
||||
import org.elasticsearch.ingest.core.IngestDocument;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
public class SimulateProcessorResult implements Writeable<SimulateProcessorResult>, ToXContent {
|
||||
private final String processorTag;
|
||||
private final WriteableIngestDocument ingestDocument;
|
||||
private final Exception failure;
|
||||
|
||||
public SimulateProcessorResult(StreamInput in) throws IOException {
|
||||
this.processorTag = in.readString();
|
||||
if (in.readBoolean()) {
|
||||
this.failure = in.readThrowable();
|
||||
this.ingestDocument = null;
|
||||
} else {
|
||||
this.ingestDocument = new WriteableIngestDocument(in);
|
||||
this.failure = null;
|
||||
}
|
||||
}
|
||||
|
||||
public SimulateProcessorResult(String processorTag, IngestDocument ingestDocument) {
|
||||
this.processorTag = processorTag;
|
||||
this.ingestDocument = new WriteableIngestDocument(ingestDocument);
|
||||
this.failure = null;
|
||||
}
|
||||
|
||||
public SimulateProcessorResult(String processorTag, Exception failure) {
|
||||
this.processorTag = processorTag;
|
||||
this.failure = failure;
|
||||
this.ingestDocument = null;
|
||||
}
|
||||
|
||||
public IngestDocument getIngestDocument() {
|
||||
if (ingestDocument == null) {
|
||||
return null;
|
||||
}
|
||||
return ingestDocument.getIngestDocument();
|
||||
}
|
||||
|
||||
public String getProcessorTag() {
|
||||
return processorTag;
|
||||
}
|
||||
|
||||
public Exception getFailure() {
|
||||
return failure;
|
||||
}
|
||||
|
||||
@Override
|
||||
public SimulateProcessorResult readFrom(StreamInput in) throws IOException {
|
||||
return new SimulateProcessorResult(in);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeTo(StreamOutput out) throws IOException {
|
||||
out.writeString(processorTag);
|
||||
if (failure == null) {
|
||||
out.writeBoolean(false);
|
||||
ingestDocument.writeTo(out);
|
||||
} else {
|
||||
out.writeBoolean(true);
|
||||
out.writeThrowable(failure);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
|
||||
builder.startObject();
|
||||
if (processorTag != null) {
|
||||
builder.field(AbstractProcessorFactory.TAG_KEY, processorTag);
|
||||
}
|
||||
if (failure == null) {
|
||||
ingestDocument.toXContent(builder, params);
|
||||
} else {
|
||||
ElasticsearchException.renderThrowable(builder, params, failure);
|
||||
}
|
||||
builder.endObject();
|
||||
return builder;
|
||||
}
|
||||
}
|
@ -0,0 +1,48 @@
|
||||
/*
|
||||
* 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.action.ingest;
|
||||
|
||||
import org.elasticsearch.action.support.master.AcknowledgedResponse;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
public class WritePipelineResponse extends AcknowledgedResponse {
|
||||
|
||||
WritePipelineResponse() {
|
||||
}
|
||||
|
||||
public WritePipelineResponse(boolean acknowledge) {
|
||||
super(acknowledge);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void readFrom(StreamInput in) throws IOException {
|
||||
super.readFrom(in);
|
||||
readAcknowledged(in);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeTo(StreamOutput out) throws IOException {
|
||||
super.writeTo(out);
|
||||
writeAcknowledged(out);
|
||||
}
|
||||
}
|
@ -0,0 +1,105 @@
|
||||
/*
|
||||
* 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.action.ingest;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.io.stream.Writeable;
|
||||
import org.elasticsearch.common.xcontent.ToXContent;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilderString;
|
||||
import org.elasticsearch.ingest.core.IngestDocument;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
|
||||
final class WriteableIngestDocument implements Writeable<WriteableIngestDocument>, ToXContent {
|
||||
|
||||
private final IngestDocument ingestDocument;
|
||||
|
||||
WriteableIngestDocument(IngestDocument ingestDocument) {
|
||||
assert ingestDocument != null;
|
||||
this.ingestDocument = ingestDocument;
|
||||
}
|
||||
|
||||
WriteableIngestDocument(StreamInput in) throws IOException {
|
||||
Map<String, Object> sourceAndMetadata = in.readMap();
|
||||
@SuppressWarnings("unchecked")
|
||||
Map<String, String> ingestMetadata = (Map<String, String>) in.readGenericValue();
|
||||
this.ingestDocument = new IngestDocument(sourceAndMetadata, ingestMetadata);
|
||||
}
|
||||
|
||||
IngestDocument getIngestDocument() {
|
||||
return ingestDocument;
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public WriteableIngestDocument readFrom(StreamInput in) throws IOException {
|
||||
return new WriteableIngestDocument(in);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeTo(StreamOutput out) throws IOException {
|
||||
out.writeMap(ingestDocument.getSourceAndMetadata());
|
||||
out.writeGenericValue(ingestDocument.getIngestMetadata());
|
||||
}
|
||||
|
||||
@Override
|
||||
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
|
||||
builder.startObject("doc");
|
||||
Map<IngestDocument.MetaData, String> metadataMap = ingestDocument.extractMetadata();
|
||||
for (Map.Entry<IngestDocument.MetaData, String> metadata : metadataMap.entrySet()) {
|
||||
builder.field(metadata.getKey().getFieldName(), metadata.getValue());
|
||||
}
|
||||
builder.field("_source", ingestDocument.getSourceAndMetadata());
|
||||
builder.startObject("_ingest");
|
||||
for (Map.Entry<String, String> ingestMetadata : ingestDocument.getIngestMetadata().entrySet()) {
|
||||
builder.field(ingestMetadata.getKey(), ingestMetadata.getValue());
|
||||
}
|
||||
builder.endObject();
|
||||
builder.endObject();
|
||||
return builder;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (o == null || getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
WriteableIngestDocument that = (WriteableIngestDocument) o;
|
||||
return Objects.equals(ingestDocument, that.ingestDocument);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return Objects.hash(ingestDocument);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return ingestDocument.toString();
|
||||
}
|
||||
}
|
@ -51,6 +51,17 @@ import org.elasticsearch.action.indexedscripts.get.GetIndexedScriptResponse;
|
||||
import org.elasticsearch.action.indexedscripts.put.PutIndexedScriptRequest;
|
||||
import org.elasticsearch.action.indexedscripts.put.PutIndexedScriptRequestBuilder;
|
||||
import org.elasticsearch.action.indexedscripts.put.PutIndexedScriptResponse;
|
||||
import org.elasticsearch.action.ingest.DeletePipelineRequest;
|
||||
import org.elasticsearch.action.ingest.DeletePipelineRequestBuilder;
|
||||
import org.elasticsearch.action.ingest.GetPipelineRequest;
|
||||
import org.elasticsearch.action.ingest.GetPipelineRequestBuilder;
|
||||
import org.elasticsearch.action.ingest.GetPipelineResponse;
|
||||
import org.elasticsearch.action.ingest.PutPipelineRequest;
|
||||
import org.elasticsearch.action.ingest.PutPipelineRequestBuilder;
|
||||
import org.elasticsearch.action.ingest.SimulatePipelineRequest;
|
||||
import org.elasticsearch.action.ingest.SimulatePipelineRequestBuilder;
|
||||
import org.elasticsearch.action.ingest.SimulatePipelineResponse;
|
||||
import org.elasticsearch.action.ingest.WritePipelineResponse;
|
||||
import org.elasticsearch.action.percolate.MultiPercolateRequest;
|
||||
import org.elasticsearch.action.percolate.MultiPercolateRequestBuilder;
|
||||
import org.elasticsearch.action.percolate.MultiPercolateResponse;
|
||||
@ -82,6 +93,7 @@ import org.elasticsearch.action.update.UpdateRequestBuilder;
|
||||
import org.elasticsearch.action.update.UpdateResponse;
|
||||
import org.elasticsearch.client.support.Headers;
|
||||
import org.elasticsearch.common.Nullable;
|
||||
import org.elasticsearch.common.bytes.BytesReference;
|
||||
import org.elasticsearch.common.lease.Releasable;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
|
||||
@ -592,6 +604,66 @@ public interface Client extends ElasticsearchClient, Releasable {
|
||||
|
||||
void fieldStats(FieldStatsRequest request, ActionListener<FieldStatsResponse> listener);
|
||||
|
||||
/**
|
||||
* Stores an ingest pipeline
|
||||
*/
|
||||
void putPipeline(PutPipelineRequest request, ActionListener<WritePipelineResponse> listener);
|
||||
|
||||
/**
|
||||
* Stores an ingest pipeline
|
||||
*/
|
||||
ActionFuture<WritePipelineResponse> putPipeline(PutPipelineRequest request);
|
||||
|
||||
/**
|
||||
* Stores an ingest pipeline
|
||||
*/
|
||||
PutPipelineRequestBuilder preparePutPipeline(String id, BytesReference source);
|
||||
|
||||
/**
|
||||
* Deletes a stored ingest pipeline
|
||||
*/
|
||||
void deletePipeline(DeletePipelineRequest request, ActionListener<WritePipelineResponse> listener);
|
||||
|
||||
/**
|
||||
* Deletes a stored ingest pipeline
|
||||
*/
|
||||
ActionFuture<WritePipelineResponse> deletePipeline(DeletePipelineRequest request);
|
||||
|
||||
/**
|
||||
* Deletes a stored ingest pipeline
|
||||
*/
|
||||
DeletePipelineRequestBuilder prepareDeletePipeline();
|
||||
|
||||
/**
|
||||
* Returns a stored ingest pipeline
|
||||
*/
|
||||
void getPipeline(GetPipelineRequest request, ActionListener<GetPipelineResponse> listener);
|
||||
|
||||
/**
|
||||
* Returns a stored ingest pipeline
|
||||
*/
|
||||
ActionFuture<GetPipelineResponse> getPipeline(GetPipelineRequest request);
|
||||
|
||||
/**
|
||||
* Returns a stored ingest pipeline
|
||||
*/
|
||||
GetPipelineRequestBuilder prepareGetPipeline(String... ids);
|
||||
|
||||
/**
|
||||
* Simulates an ingest pipeline
|
||||
*/
|
||||
void simulatePipeline(SimulatePipelineRequest request, ActionListener<SimulatePipelineResponse> listener);
|
||||
|
||||
/**
|
||||
* Simulates an ingest pipeline
|
||||
*/
|
||||
ActionFuture<SimulatePipelineResponse> simulatePipeline(SimulatePipelineRequest request);
|
||||
|
||||
/**
|
||||
* Simulates an ingest pipeline
|
||||
*/
|
||||
SimulatePipelineRequestBuilder prepareSimulatePipeline(BytesReference source);
|
||||
|
||||
/**
|
||||
* Returns this clients settings
|
||||
*/
|
||||
|
@ -272,6 +272,21 @@ import org.elasticsearch.action.indexedscripts.put.PutIndexedScriptAction;
|
||||
import org.elasticsearch.action.indexedscripts.put.PutIndexedScriptRequest;
|
||||
import org.elasticsearch.action.indexedscripts.put.PutIndexedScriptRequestBuilder;
|
||||
import org.elasticsearch.action.indexedscripts.put.PutIndexedScriptResponse;
|
||||
import org.elasticsearch.action.ingest.DeletePipelineAction;
|
||||
import org.elasticsearch.action.ingest.DeletePipelineRequest;
|
||||
import org.elasticsearch.action.ingest.DeletePipelineRequestBuilder;
|
||||
import org.elasticsearch.action.ingest.GetPipelineAction;
|
||||
import org.elasticsearch.action.ingest.GetPipelineRequest;
|
||||
import org.elasticsearch.action.ingest.GetPipelineRequestBuilder;
|
||||
import org.elasticsearch.action.ingest.GetPipelineResponse;
|
||||
import org.elasticsearch.action.ingest.PutPipelineAction;
|
||||
import org.elasticsearch.action.ingest.PutPipelineRequest;
|
||||
import org.elasticsearch.action.ingest.PutPipelineRequestBuilder;
|
||||
import org.elasticsearch.action.ingest.SimulatePipelineAction;
|
||||
import org.elasticsearch.action.ingest.SimulatePipelineRequest;
|
||||
import org.elasticsearch.action.ingest.SimulatePipelineRequestBuilder;
|
||||
import org.elasticsearch.action.ingest.SimulatePipelineResponse;
|
||||
import org.elasticsearch.action.ingest.WritePipelineResponse;
|
||||
import org.elasticsearch.action.percolate.MultiPercolateAction;
|
||||
import org.elasticsearch.action.percolate.MultiPercolateRequest;
|
||||
import org.elasticsearch.action.percolate.MultiPercolateRequestBuilder;
|
||||
@ -319,6 +334,7 @@ import org.elasticsearch.client.ClusterAdminClient;
|
||||
import org.elasticsearch.client.ElasticsearchClient;
|
||||
import org.elasticsearch.client.IndicesAdminClient;
|
||||
import org.elasticsearch.common.Nullable;
|
||||
import org.elasticsearch.common.bytes.BytesReference;
|
||||
import org.elasticsearch.common.component.AbstractComponent;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
@ -794,6 +810,66 @@ public abstract class AbstractClient extends AbstractComponent implements Client
|
||||
return new FieldStatsRequestBuilder(this, FieldStatsAction.INSTANCE);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void putPipeline(PutPipelineRequest request, ActionListener<WritePipelineResponse> listener) {
|
||||
execute(PutPipelineAction.INSTANCE, request, listener);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ActionFuture<WritePipelineResponse> putPipeline(PutPipelineRequest request) {
|
||||
return execute(PutPipelineAction.INSTANCE, request);
|
||||
}
|
||||
|
||||
@Override
|
||||
public PutPipelineRequestBuilder preparePutPipeline(String id, BytesReference source) {
|
||||
return new PutPipelineRequestBuilder(this, PutPipelineAction.INSTANCE, id, source);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void deletePipeline(DeletePipelineRequest request, ActionListener<WritePipelineResponse> listener) {
|
||||
execute(DeletePipelineAction.INSTANCE, request, listener);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ActionFuture<WritePipelineResponse> deletePipeline(DeletePipelineRequest request) {
|
||||
return execute(DeletePipelineAction.INSTANCE, request);
|
||||
}
|
||||
|
||||
@Override
|
||||
public DeletePipelineRequestBuilder prepareDeletePipeline() {
|
||||
return new DeletePipelineRequestBuilder(this, DeletePipelineAction.INSTANCE);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void getPipeline(GetPipelineRequest request, ActionListener<GetPipelineResponse> listener) {
|
||||
execute(GetPipelineAction.INSTANCE, request, listener);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ActionFuture<GetPipelineResponse> getPipeline(GetPipelineRequest request) {
|
||||
return execute(GetPipelineAction.INSTANCE, request);
|
||||
}
|
||||
|
||||
@Override
|
||||
public GetPipelineRequestBuilder prepareGetPipeline(String... ids) {
|
||||
return new GetPipelineRequestBuilder(this, GetPipelineAction.INSTANCE, ids);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void simulatePipeline(SimulatePipelineRequest request, ActionListener<SimulatePipelineResponse> listener) {
|
||||
execute(SimulatePipelineAction.INSTANCE, request, listener);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ActionFuture<SimulatePipelineResponse> simulatePipeline(SimulatePipelineRequest request) {
|
||||
return execute(SimulatePipelineAction.INSTANCE, request);
|
||||
}
|
||||
|
||||
@Override
|
||||
public SimulatePipelineRequestBuilder prepareSimulatePipeline(BytesReference source) {
|
||||
return new SimulatePipelineRequestBuilder(this, SimulatePipelineAction.INSTANCE, source);
|
||||
}
|
||||
|
||||
static class Admin implements AdminClient {
|
||||
|
||||
private final ClusterAdmin clusterAdmin;
|
||||
|
@ -19,10 +19,6 @@
|
||||
|
||||
package org.elasticsearch.client.transport;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import org.elasticsearch.Version;
|
||||
import org.elasticsearch.action.Action;
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
@ -59,6 +55,10 @@ import org.elasticsearch.threadpool.ThreadPoolModule;
|
||||
import org.elasticsearch.transport.TransportService;
|
||||
import org.elasticsearch.transport.netty.NettyTransport;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import static org.elasticsearch.common.settings.Settings.settingsBuilder;
|
||||
|
||||
/**
|
||||
@ -116,7 +116,7 @@ public class TransportClient extends AbstractClient {
|
||||
.put("node.client", true)
|
||||
.put(CLIENT_TYPE_SETTING, CLIENT_TYPE);
|
||||
return new PluginsService(settingsBuilder.build(), null, null, pluginClasses);
|
||||
};
|
||||
}
|
||||
|
||||
/**
|
||||
* Builds a new instance of the transport client.
|
||||
@ -150,7 +150,7 @@ public class TransportClient extends AbstractClient {
|
||||
// noop
|
||||
}
|
||||
});
|
||||
modules.add(new ActionModule(true));
|
||||
modules.add(new ActionModule(false, true));
|
||||
modules.add(new CircuitBreakerModule(settings));
|
||||
|
||||
pluginsService.processModules(modules);
|
||||
|
@ -54,6 +54,7 @@ import org.elasticsearch.index.IndexNotFoundException;
|
||||
import org.elasticsearch.index.store.IndexStoreConfig;
|
||||
import org.elasticsearch.indices.recovery.RecoverySettings;
|
||||
import org.elasticsearch.indices.ttl.IndicesTTLService;
|
||||
import org.elasticsearch.ingest.IngestMetadata;
|
||||
import org.elasticsearch.rest.RestStatus;
|
||||
|
||||
import java.io.IOException;
|
||||
@ -111,6 +112,7 @@ public class MetaData implements Iterable<IndexMetaData>, Diffable<MetaData>, Fr
|
||||
static {
|
||||
// register non plugin custom metadata
|
||||
registerPrototype(RepositoriesMetaData.TYPE, RepositoriesMetaData.PROTO);
|
||||
registerPrototype(IngestMetadata.TYPE, IngestMetadata.PROTO);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -32,6 +32,7 @@ import org.elasticsearch.common.transport.TransportAddress;
|
||||
import org.elasticsearch.common.transport.TransportAddressSerializers;
|
||||
import org.elasticsearch.common.xcontent.ToXContent;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.node.Node;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
@ -87,6 +88,10 @@ public class DiscoveryNode implements Streamable, ToXContent {
|
||||
return Booleans.isExplicitTrue(data);
|
||||
}
|
||||
|
||||
public static boolean ingestNode(Settings settings) {
|
||||
return Node.NODE_INGEST_SETTING.get(settings);
|
||||
}
|
||||
|
||||
public static final List<DiscoveryNode> EMPTY_LIST = Collections.emptyList();
|
||||
|
||||
private String nodeName = "";
|
||||
@ -316,6 +321,14 @@ public class DiscoveryNode implements Streamable, ToXContent {
|
||||
return masterNode();
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a boolean that tells whether this an ingest node or not
|
||||
*/
|
||||
public boolean isIngestNode() {
|
||||
String ingest = attributes.get("ingest");
|
||||
return ingest == null ? true : Booleans.parseBooleanExact(ingest);
|
||||
}
|
||||
|
||||
public Version version() {
|
||||
return this.version;
|
||||
}
|
||||
|
@ -52,16 +52,20 @@ public class DiscoveryNodes extends AbstractDiffable<DiscoveryNodes> implements
|
||||
private final ImmutableOpenMap<String, DiscoveryNode> nodes;
|
||||
private final ImmutableOpenMap<String, DiscoveryNode> dataNodes;
|
||||
private final ImmutableOpenMap<String, DiscoveryNode> masterNodes;
|
||||
private final ImmutableOpenMap<String, DiscoveryNode> ingestNodes;
|
||||
|
||||
private final String masterNodeId;
|
||||
private final String localNodeId;
|
||||
private final Version minNodeVersion;
|
||||
private final Version minNonClientNodeVersion;
|
||||
|
||||
private DiscoveryNodes(ImmutableOpenMap<String, DiscoveryNode> nodes, ImmutableOpenMap<String, DiscoveryNode> dataNodes, ImmutableOpenMap<String, DiscoveryNode> masterNodes, String masterNodeId, String localNodeId, Version minNodeVersion, Version minNonClientNodeVersion) {
|
||||
private DiscoveryNodes(ImmutableOpenMap<String, DiscoveryNode> nodes, ImmutableOpenMap<String, DiscoveryNode> dataNodes,
|
||||
ImmutableOpenMap<String, DiscoveryNode> masterNodes, ImmutableOpenMap<String, DiscoveryNode> ingestNodes,
|
||||
String masterNodeId, String localNodeId, Version minNodeVersion, Version minNonClientNodeVersion) {
|
||||
this.nodes = nodes;
|
||||
this.dataNodes = dataNodes;
|
||||
this.masterNodes = masterNodes;
|
||||
this.ingestNodes = ingestNodes;
|
||||
this.masterNodeId = masterNodeId;
|
||||
this.localNodeId = localNodeId;
|
||||
this.minNodeVersion = minNodeVersion;
|
||||
@ -164,6 +168,13 @@ public class DiscoveryNodes extends AbstractDiffable<DiscoveryNodes> implements
|
||||
return masterNodes();
|
||||
}
|
||||
|
||||
/**
|
||||
* @return All the ingest nodes arranged by their ids
|
||||
*/
|
||||
public ImmutableOpenMap<String, DiscoveryNode> getIngestNodes() {
|
||||
return ingestNodes;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get a {@link Map} of the discovered master and data nodes arranged by their ids
|
||||
*
|
||||
@ -654,6 +665,7 @@ public class DiscoveryNodes extends AbstractDiffable<DiscoveryNodes> implements
|
||||
public DiscoveryNodes build() {
|
||||
ImmutableOpenMap.Builder<String, DiscoveryNode> dataNodesBuilder = ImmutableOpenMap.builder();
|
||||
ImmutableOpenMap.Builder<String, DiscoveryNode> masterNodesBuilder = ImmutableOpenMap.builder();
|
||||
ImmutableOpenMap.Builder<String, DiscoveryNode> ingestNodesBuilder = ImmutableOpenMap.builder();
|
||||
Version minNodeVersion = Version.CURRENT;
|
||||
Version minNonClientNodeVersion = Version.CURRENT;
|
||||
for (ObjectObjectCursor<String, DiscoveryNode> nodeEntry : nodes) {
|
||||
@ -665,10 +677,16 @@ public class DiscoveryNodes extends AbstractDiffable<DiscoveryNodes> implements
|
||||
masterNodesBuilder.put(nodeEntry.key, nodeEntry.value);
|
||||
minNonClientNodeVersion = Version.smallest(minNonClientNodeVersion, nodeEntry.value.version());
|
||||
}
|
||||
if (nodeEntry.value.isIngestNode()) {
|
||||
ingestNodesBuilder.put(nodeEntry.key, nodeEntry.value);
|
||||
}
|
||||
minNodeVersion = Version.smallest(minNodeVersion, nodeEntry.value.version());
|
||||
}
|
||||
|
||||
return new DiscoveryNodes(nodes.build(), dataNodesBuilder.build(), masterNodesBuilder.build(), masterNodeId, localNodeId, minNodeVersion, minNonClientNodeVersion);
|
||||
return new DiscoveryNodes(
|
||||
nodes.build(), dataNodesBuilder.build(), masterNodesBuilder.build(), ingestNodesBuilder.build(),
|
||||
masterNodeId, localNodeId, minNodeVersion, minNonClientNodeVersion
|
||||
);
|
||||
}
|
||||
|
||||
public static DiscoveryNodes readFrom(StreamInput in, @Nullable DiscoveryNode localNode) throws IOException {
|
||||
|
@ -118,6 +118,10 @@ import org.elasticsearch.rest.action.get.RestGetSourceAction;
|
||||
import org.elasticsearch.rest.action.get.RestHeadAction;
|
||||
import org.elasticsearch.rest.action.get.RestMultiGetAction;
|
||||
import org.elasticsearch.rest.action.index.RestIndexAction;
|
||||
import org.elasticsearch.rest.action.ingest.RestDeletePipelineAction;
|
||||
import org.elasticsearch.rest.action.ingest.RestGetPipelineAction;
|
||||
import org.elasticsearch.rest.action.ingest.RestPutPipelineAction;
|
||||
import org.elasticsearch.rest.action.ingest.RestSimulatePipelineAction;
|
||||
import org.elasticsearch.rest.action.main.RestMainAction;
|
||||
import org.elasticsearch.rest.action.percolate.RestMultiPercolateAction;
|
||||
import org.elasticsearch.rest.action.percolate.RestPercolateAction;
|
||||
@ -258,7 +262,13 @@ public class NetworkModule extends AbstractModule {
|
||||
RestCatAction.class,
|
||||
|
||||
// Tasks API
|
||||
RestListTasksAction.class
|
||||
RestListTasksAction.class,
|
||||
|
||||
// Ingest API
|
||||
RestPutPipelineAction.class,
|
||||
RestGetPipelineAction.class,
|
||||
RestDeletePipelineAction.class,
|
||||
RestSimulatePipelineAction.class
|
||||
);
|
||||
|
||||
private static final List<Class<? extends AbstractCatAction>> builtinCatHandlers = Arrays.asList(
|
||||
|
@ -175,7 +175,7 @@ public final class ClusterSettings extends AbstractScopedSettings {
|
||||
NettyHttpServerTransport.SETTING_CORS_ENABLED,
|
||||
NettyHttpServerTransport.SETTING_CORS_MAX_AGE,
|
||||
NettyHttpServerTransport.SETTING_HTTP_DETAILED_ERRORS_ENABLED,
|
||||
NettyHttpServerTransport.SETTING_PIPELINING,
|
||||
NettyHttpServerTransport.SETTING_PIPELINING,
|
||||
HierarchyCircuitBreakerService.TOTAL_CIRCUIT_BREAKER_LIMIT_SETTING,
|
||||
HierarchyCircuitBreakerService.FIELDDATA_CIRCUIT_BREAKER_LIMIT_SETTING,
|
||||
HierarchyCircuitBreakerService.FIELDDATA_CIRCUIT_BREAKER_OVERHEAD_SETTING,
|
||||
@ -252,5 +252,6 @@ public final class ClusterSettings extends AbstractScopedSettings {
|
||||
URLRepository.ALLOWED_URLS_SETTING,
|
||||
URLRepository.REPOSITORIES_LIST_DIRECTORIES_SETTING,
|
||||
URLRepository.REPOSITORIES_URL_SETTING,
|
||||
URLRepository.SUPPORTED_PROTOCOLS_SETTING)));
|
||||
URLRepository.SUPPORTED_PROTOCOLS_SETTING,
|
||||
Node.NODE_INGEST_SETTING)));
|
||||
}
|
||||
|
@ -223,7 +223,7 @@ public final class ObjectParser<Value, Context> implements BiFunction<XContentPa
|
||||
list.add(supplier.get()); // single value
|
||||
} else {
|
||||
while (parser.nextToken() != XContentParser.Token.END_ARRAY) {
|
||||
if (parser.currentToken().isValue()) {
|
||||
if (parser.currentToken().isValue() || parser.currentToken() == XContentParser.Token.START_OBJECT) {
|
||||
list.add(supplier.get());
|
||||
} else {
|
||||
throw new IllegalStateException("expected value but got [" + parser.currentToken() + "]");
|
||||
@ -237,6 +237,11 @@ public final class ObjectParser<Value, Context> implements BiFunction<XContentPa
|
||||
declareField((p, v, c) -> consumer.accept(v, objectParser.apply(p, c)), field, ValueType.OBJECT);
|
||||
}
|
||||
|
||||
public <T> void declareObjectArray(BiConsumer<Value, List<T>> consumer, BiFunction<XContentParser, Context, T> objectParser, ParseField field) {
|
||||
declareField((p, v, c) -> consumer.accept(v, parseArray(p, () -> objectParser.apply(p, c))), field, ValueType.OBJECT_ARRAY);
|
||||
}
|
||||
|
||||
|
||||
public <T> void declareObjectOrDefault(BiConsumer<Value, T> consumer, BiFunction<XContentParser, Context, T> objectParser, Supplier<T> defaultValue, ParseField field) {
|
||||
declareField((p, v, c) -> {
|
||||
if (p.currentToken() == XContentParser.Token.VALUE_BOOLEAN) {
|
||||
@ -333,6 +338,7 @@ public final class ObjectParser<Value, Context> implements BiFunction<XContentPa
|
||||
INT_ARRAY(EnumSet.of(XContentParser.Token.START_ARRAY, XContentParser.Token.VALUE_NUMBER, XContentParser.Token.VALUE_STRING)),
|
||||
BOOLEAN_ARRAY(EnumSet.of(XContentParser.Token.START_ARRAY, XContentParser.Token.VALUE_BOOLEAN)),
|
||||
OBJECT(EnumSet.of(XContentParser.Token.START_OBJECT)),
|
||||
OBJECT_ARRAY(EnumSet.of(XContentParser.Token.START_OBJECT, XContentParser.Token.START_ARRAY)),
|
||||
OBJECT_OR_BOOLEAN(EnumSet.of(XContentParser.Token.START_OBJECT, XContentParser.Token.VALUE_BOOLEAN)),
|
||||
VALUE(EnumSet.of(XContentParser.Token.VALUE_BOOLEAN, XContentParser.Token.VALUE_NULL ,XContentParser.Token.VALUE_EMBEDDED_OBJECT,XContentParser.Token.VALUE_NUMBER,XContentParser.Token.VALUE_STRING));
|
||||
|
||||
|
121
core/src/main/java/org/elasticsearch/ingest/IngestMetadata.java
Normal file
121
core/src/main/java/org/elasticsearch/ingest/IngestMetadata.java
Normal file
@ -0,0 +1,121 @@
|
||||
/*
|
||||
* 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.cluster.AbstractDiffable;
|
||||
import org.elasticsearch.cluster.metadata.MetaData;
|
||||
import org.elasticsearch.common.ParseField;
|
||||
import org.elasticsearch.common.ParsingException;
|
||||
import org.elasticsearch.common.collect.HppcMaps;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.xcontent.ObjectParser;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.common.xcontent.XContentParser;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.EnumSet;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* Holds the ingest pipelines that are available in the cluster
|
||||
*/
|
||||
public final class IngestMetadata extends AbstractDiffable<MetaData.Custom> implements MetaData.Custom {
|
||||
|
||||
public final static String TYPE = "ingest";
|
||||
public final static IngestMetadata PROTO = new IngestMetadata();
|
||||
private static final ParseField PIPELINES_FIELD = new ParseField("pipeline");
|
||||
private static final ObjectParser<List<PipelineConfiguration>, Void> INGEST_METADATA_PARSER = new ObjectParser<>("ingest_metadata", ArrayList::new);
|
||||
|
||||
static {
|
||||
INGEST_METADATA_PARSER.declareObjectArray(List::addAll , PipelineConfiguration.getParser(), PIPELINES_FIELD);
|
||||
}
|
||||
|
||||
|
||||
// We can't use Pipeline class directly in cluster state, because we don't have the processor factories around when
|
||||
// IngestMetadata is registered as custom metadata.
|
||||
private final Map<String, PipelineConfiguration> pipelines;
|
||||
|
||||
private IngestMetadata() {
|
||||
this.pipelines = Collections.emptyMap();
|
||||
}
|
||||
|
||||
public IngestMetadata(Map<String, PipelineConfiguration> pipelines) {
|
||||
this.pipelines = Collections.unmodifiableMap(pipelines);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String type() {
|
||||
return TYPE;
|
||||
}
|
||||
|
||||
public Map<String, PipelineConfiguration> getPipelines() {
|
||||
return pipelines;
|
||||
}
|
||||
|
||||
@Override
|
||||
public MetaData.Custom readFrom(StreamInput in) throws IOException {
|
||||
int size = in.readVInt();
|
||||
Map<String, PipelineConfiguration> pipelines = new HashMap<>(size);
|
||||
for (int i = 0; i < size; i++) {
|
||||
PipelineConfiguration pipeline = PipelineConfiguration.readPipelineConfiguration(in);
|
||||
pipelines.put(pipeline.getId(), pipeline);
|
||||
}
|
||||
return new IngestMetadata(pipelines);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeTo(StreamOutput out) throws IOException {
|
||||
out.writeVInt(pipelines.size());
|
||||
for (PipelineConfiguration pipeline : pipelines.values()) {
|
||||
pipeline.writeTo(out);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public MetaData.Custom fromXContent(XContentParser parser) throws IOException {
|
||||
Map<String, PipelineConfiguration> pipelines = new HashMap<>();
|
||||
List<PipelineConfiguration> configs = INGEST_METADATA_PARSER.parse(parser);
|
||||
for (PipelineConfiguration pipeline : configs) {
|
||||
pipelines.put(pipeline.getId(), pipeline);
|
||||
}
|
||||
return new IngestMetadata(pipelines);
|
||||
}
|
||||
|
||||
@Override
|
||||
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
|
||||
builder.startArray(PIPELINES_FIELD.getPreferredName());
|
||||
for (PipelineConfiguration pipeline : pipelines.values()) {
|
||||
pipeline.toXContent(builder, params);
|
||||
}
|
||||
builder.endArray();
|
||||
return builder;
|
||||
}
|
||||
|
||||
@Override
|
||||
public EnumSet<MetaData.XContentContext> context() {
|
||||
return MetaData.API_AND_GATEWAY;
|
||||
}
|
||||
|
||||
}
|
@ -0,0 +1,63 @@
|
||||
/*
|
||||
* 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.cluster.ClusterService;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.script.ScriptService;
|
||||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
* Instantiates and wires all the services that the ingest plugin will be needing.
|
||||
* Also the bootstrapper is in charge of starting and stopping the ingest plugin based on the cluster state.
|
||||
*/
|
||||
public class IngestService implements Closeable {
|
||||
|
||||
private final PipelineStore pipelineStore;
|
||||
private final PipelineExecutionService pipelineExecutionService;
|
||||
private final ProcessorsRegistry processorsRegistry;
|
||||
|
||||
public IngestService(Settings settings, ThreadPool threadPool, ProcessorsRegistry processorsRegistry) {
|
||||
this.processorsRegistry = processorsRegistry;
|
||||
this.pipelineStore = new PipelineStore(settings);
|
||||
this.pipelineExecutionService = new PipelineExecutionService(pipelineStore, threadPool);
|
||||
}
|
||||
|
||||
public PipelineStore getPipelineStore() {
|
||||
return pipelineStore;
|
||||
}
|
||||
|
||||
public PipelineExecutionService getPipelineExecutionService() {
|
||||
return pipelineExecutionService;
|
||||
}
|
||||
|
||||
public void setScriptService(ScriptService scriptService) {
|
||||
pipelineStore.buildProcessorFactoryRegistry(processorsRegistry, scriptService);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
pipelineStore.close();
|
||||
}
|
||||
|
||||
}
|
@ -0,0 +1,92 @@
|
||||
/*
|
||||
* 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.bytes.BytesReference;
|
||||
import org.elasticsearch.ingest.core.TemplateService;
|
||||
import org.elasticsearch.script.CompiledScript;
|
||||
import org.elasticsearch.script.ExecutableScript;
|
||||
import org.elasticsearch.script.Script;
|
||||
import org.elasticsearch.script.ScriptContext;
|
||||
import org.elasticsearch.script.ScriptService;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.Map;
|
||||
|
||||
public class InternalTemplateService implements TemplateService {
|
||||
|
||||
private final ScriptService scriptService;
|
||||
|
||||
InternalTemplateService(ScriptService scriptService) {
|
||||
this.scriptService = scriptService;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Template compile(String template) {
|
||||
int mustacheStart = template.indexOf("{{");
|
||||
int mustacheEnd = template.indexOf("}}");
|
||||
if (mustacheStart != -1 && mustacheEnd != -1 && mustacheStart < mustacheEnd) {
|
||||
Script script = new Script(template, ScriptService.ScriptType.INLINE, "mustache", Collections.emptyMap());
|
||||
CompiledScript compiledScript = scriptService.compile(
|
||||
script,
|
||||
ScriptContext.Standard.INGEST,
|
||||
null /* we can supply null here, because ingest doesn't use indexed scripts */,
|
||||
Collections.emptyMap()
|
||||
);
|
||||
return new Template() {
|
||||
@Override
|
||||
public String execute(Map<String, Object> model) {
|
||||
ExecutableScript executableScript = scriptService.executable(compiledScript, model);
|
||||
Object result = executableScript.run();
|
||||
if (result instanceof BytesReference) {
|
||||
return ((BytesReference) result).toUtf8();
|
||||
}
|
||||
return String.valueOf(result);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getKey() {
|
||||
return template;
|
||||
}
|
||||
};
|
||||
} else {
|
||||
return new StringTemplate(template);
|
||||
}
|
||||
}
|
||||
|
||||
class StringTemplate implements Template {
|
||||
|
||||
private final String value;
|
||||
|
||||
public StringTemplate(String value) {
|
||||
this.value = value;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String execute(Map<String, Object> model) {
|
||||
return value;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getKey() {
|
||||
return value;
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,119 @@
|
||||
/*
|
||||
* 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.apache.lucene.util.BytesRef;
|
||||
import org.elasticsearch.Build;
|
||||
import org.elasticsearch.common.ParseField;
|
||||
import org.elasticsearch.common.bytes.BytesReference;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.io.stream.Writeable;
|
||||
import org.elasticsearch.common.xcontent.ObjectParser;
|
||||
import org.elasticsearch.common.xcontent.ToXContent;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.common.xcontent.XContentHelper;
|
||||
import org.elasticsearch.common.xcontent.XContentParser;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
import java.util.function.BiFunction;
|
||||
|
||||
/**
|
||||
* Encapsulates a pipeline's id and configuration as a blob
|
||||
*/
|
||||
public final class PipelineConfiguration implements Writeable<PipelineConfiguration>, ToXContent {
|
||||
|
||||
private final static PipelineConfiguration PROTOTYPE = new PipelineConfiguration(null, null);
|
||||
|
||||
public static PipelineConfiguration readPipelineConfiguration(StreamInput in) throws IOException {
|
||||
return PROTOTYPE.readFrom(in);
|
||||
}
|
||||
private final static ObjectParser<Builder, Void> PARSER = new ObjectParser<>("pipeline_config", Builder::new);
|
||||
static {
|
||||
PARSER.declareString(Builder::setId, new ParseField("id"));
|
||||
PARSER.declareField((parser, builder, aVoid) -> {
|
||||
XContentBuilder contentBuilder = XContentBuilder.builder(parser.contentType().xContent());
|
||||
XContentHelper.copyCurrentStructure(contentBuilder.generator(), parser);
|
||||
builder.setConfig(contentBuilder.bytes());
|
||||
}, new ParseField("config"), ObjectParser.ValueType.OBJECT);
|
||||
}
|
||||
|
||||
public static BiFunction<XContentParser, Void,PipelineConfiguration> getParser() {
|
||||
return (p, c) -> PARSER.apply(p ,c).build();
|
||||
}
|
||||
private static class Builder {
|
||||
|
||||
private String id;
|
||||
private BytesReference config;
|
||||
|
||||
void setId(String id) {
|
||||
this.id = id;
|
||||
}
|
||||
|
||||
void setConfig(BytesReference config) {
|
||||
this.config = config;
|
||||
}
|
||||
|
||||
PipelineConfiguration build() {
|
||||
return new PipelineConfiguration(id, config);
|
||||
}
|
||||
}
|
||||
|
||||
private final String id;
|
||||
// Store config as bytes reference, because the config is only used when the pipeline store reads the cluster state
|
||||
// and the way the map of maps config is read requires a deep copy (it removes instead of gets entries to check for unused options)
|
||||
// also the get pipeline api just directly returns this to the caller
|
||||
private final BytesReference config;
|
||||
|
||||
public PipelineConfiguration(String id, BytesReference config) {
|
||||
this.id = id;
|
||||
this.config = config;
|
||||
}
|
||||
|
||||
public String getId() {
|
||||
return id;
|
||||
}
|
||||
|
||||
public Map<String, Object> getConfigAsMap() {
|
||||
return XContentHelper.convertToMap(config, true).v2();
|
||||
}
|
||||
|
||||
@Override
|
||||
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
|
||||
builder.startObject();
|
||||
builder.field("id", id);
|
||||
builder.field("config", getConfigAsMap());
|
||||
builder.endObject();
|
||||
return builder;
|
||||
}
|
||||
|
||||
@Override
|
||||
public PipelineConfiguration readFrom(StreamInput in) throws IOException {
|
||||
return new PipelineConfiguration(in.readString(), in.readBytesReference());
|
||||
}
|
||||
|
||||
public void writeTo(StreamOutput out) throws IOException {
|
||||
out.writeString(id);
|
||||
out.writeBytesReference(config);
|
||||
}
|
||||
|
||||
}
|
@ -0,0 +1,124 @@
|
||||
/*
|
||||
* 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.action.ActionRequest;
|
||||
import org.elasticsearch.action.index.IndexRequest;
|
||||
import org.elasticsearch.common.Strings;
|
||||
import org.elasticsearch.common.util.concurrent.AbstractRunnable;
|
||||
import org.elasticsearch.ingest.core.IngestDocument;
|
||||
import org.elasticsearch.ingest.core.Pipeline;
|
||||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
|
||||
import java.util.Map;
|
||||
import java.util.function.BiConsumer;
|
||||
import java.util.function.Consumer;
|
||||
|
||||
public class PipelineExecutionService {
|
||||
|
||||
private final PipelineStore store;
|
||||
private final ThreadPool threadPool;
|
||||
|
||||
public PipelineExecutionService(PipelineStore store, ThreadPool threadPool) {
|
||||
this.store = store;
|
||||
this.threadPool = threadPool;
|
||||
}
|
||||
|
||||
public void execute(IndexRequest request, Consumer<Throwable> failureHandler, Consumer<Boolean> completionHandler) {
|
||||
Pipeline pipeline = getPipeline(request.getPipeline());
|
||||
threadPool.executor(ThreadPool.Names.INDEX).execute(new AbstractRunnable() {
|
||||
|
||||
@Override
|
||||
public void onFailure(Throwable t) {
|
||||
failureHandler.accept(t);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void doRun() throws Exception {
|
||||
innerExecute(request, pipeline);
|
||||
completionHandler.accept(true);
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
public void execute(Iterable<ActionRequest<?>> actionRequests,
|
||||
BiConsumer<IndexRequest, Throwable> itemFailureHandler,
|
||||
Consumer<Throwable> completionHandler) {
|
||||
threadPool.executor(ThreadPool.Names.INDEX).execute(new AbstractRunnable() {
|
||||
|
||||
@Override
|
||||
public void onFailure(Throwable t) {
|
||||
completionHandler.accept(t);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void doRun() throws Exception {
|
||||
for (ActionRequest actionRequest : actionRequests) {
|
||||
if ((actionRequest instanceof IndexRequest)) {
|
||||
IndexRequest indexRequest = (IndexRequest) actionRequest;
|
||||
if (Strings.hasText(indexRequest.getPipeline())) {
|
||||
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(null);
|
||||
} catch (Throwable e) {
|
||||
itemFailureHandler.accept(indexRequest, e);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
completionHandler.accept(null);
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
private void innerExecute(IndexRequest indexRequest, Pipeline pipeline) throws Exception {
|
||||
String index = indexRequest.index();
|
||||
String type = indexRequest.type();
|
||||
String id = indexRequest.id();
|
||||
String routing = indexRequest.routing();
|
||||
String parent = indexRequest.parent();
|
||||
String timestamp = indexRequest.timestamp();
|
||||
String ttl = indexRequest.ttl() == null ? null : indexRequest.ttl().toString();
|
||||
Map<String, Object> sourceAsMap = indexRequest.sourceAsMap();
|
||||
IngestDocument ingestDocument = new IngestDocument(index, type, id, routing, parent, timestamp, ttl, sourceAsMap);
|
||||
pipeline.execute(ingestDocument);
|
||||
|
||||
Map<IngestDocument.MetaData, String> 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(metadataMap.get(IngestDocument.MetaData.INDEX));
|
||||
indexRequest.type(metadataMap.get(IngestDocument.MetaData.TYPE));
|
||||
indexRequest.id(metadataMap.get(IngestDocument.MetaData.ID));
|
||||
indexRequest.routing(metadataMap.get(IngestDocument.MetaData.ROUTING));
|
||||
indexRequest.parent(metadataMap.get(IngestDocument.MetaData.PARENT));
|
||||
indexRequest.timestamp(metadataMap.get(IngestDocument.MetaData.TIMESTAMP));
|
||||
indexRequest.ttl(metadataMap.get(IngestDocument.MetaData.TTL));
|
||||
indexRequest.source(ingestDocument.getSourceAndMetadata());
|
||||
}
|
||||
|
||||
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;
|
||||
}
|
||||
}
|
242
core/src/main/java/org/elasticsearch/ingest/PipelineStore.java
Normal file
242
core/src/main/java/org/elasticsearch/ingest/PipelineStore.java
Normal file
@ -0,0 +1,242 @@
|
||||
/*
|
||||
* 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.apache.lucene.util.IOUtils;
|
||||
import org.elasticsearch.ResourceNotFoundException;
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.action.ingest.DeletePipelineRequest;
|
||||
import org.elasticsearch.action.ingest.PutPipelineRequest;
|
||||
import org.elasticsearch.action.ingest.WritePipelineResponse;
|
||||
import org.elasticsearch.cluster.AckedClusterStateUpdateTask;
|
||||
import org.elasticsearch.cluster.ClusterChangedEvent;
|
||||
import org.elasticsearch.cluster.ClusterService;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
import org.elasticsearch.cluster.ClusterStateListener;
|
||||
import org.elasticsearch.cluster.metadata.MetaData;
|
||||
import org.elasticsearch.common.component.AbstractComponent;
|
||||
import org.elasticsearch.common.regex.Regex;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.xcontent.XContentHelper;
|
||||
import org.elasticsearch.ingest.core.Pipeline;
|
||||
import org.elasticsearch.ingest.core.Processor;
|
||||
import org.elasticsearch.ingest.core.TemplateService;
|
||||
import org.elasticsearch.script.ScriptService;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.function.Function;
|
||||
|
||||
public class PipelineStore extends AbstractComponent implements Closeable, ClusterStateListener {
|
||||
|
||||
private final Pipeline.Factory factory = new Pipeline.Factory();
|
||||
private Map<String, Processor.Factory> processorFactoryRegistry;
|
||||
|
||||
// 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<>();
|
||||
|
||||
public PipelineStore(Settings settings) {
|
||||
super(settings);
|
||||
}
|
||||
|
||||
public void buildProcessorFactoryRegistry(ProcessorsRegistry processorsRegistry, ScriptService scriptService) {
|
||||
Map<String, Processor.Factory> processorFactories = new HashMap<>();
|
||||
TemplateService templateService = new InternalTemplateService(scriptService);
|
||||
for (Map.Entry<String, Function<TemplateService, Processor.Factory<?>>> entry : processorsRegistry.entrySet()) {
|
||||
Processor.Factory processorFactory = entry.getValue().apply(templateService);
|
||||
processorFactories.put(entry.getKey(), processorFactory);
|
||||
}
|
||||
this.processorFactoryRegistry = Collections.unmodifiableMap(processorFactories);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
// TODO: When org.elasticsearch.node.Node can close Closable instances we should try to remove this code,
|
||||
// since any wired closable should be able to close itself
|
||||
List<Closeable> closeables = new ArrayList<>();
|
||||
for (Processor.Factory factory : processorFactoryRegistry.values()) {
|
||||
if (factory instanceof Closeable) {
|
||||
closeables.add((Closeable) factory);
|
||||
}
|
||||
}
|
||||
IOUtils.close(closeables);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void clusterChanged(ClusterChangedEvent event) {
|
||||
innerUpdatePipelines(event.state());
|
||||
}
|
||||
|
||||
void innerUpdatePipelines(ClusterState state) {
|
||||
IngestMetadata ingestMetadata = state.getMetaData().custom(IngestMetadata.TYPE);
|
||||
if (ingestMetadata == null) {
|
||||
return;
|
||||
}
|
||||
|
||||
Map<String, Pipeline> pipelines = new HashMap<>();
|
||||
for (PipelineConfiguration pipeline : ingestMetadata.getPipelines().values()) {
|
||||
try {
|
||||
pipelines.put(pipeline.getId(), constructPipeline(pipeline.getId(), pipeline.getConfigAsMap()));
|
||||
} catch (Exception e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
this.pipelines = Collections.unmodifiableMap(pipelines);
|
||||
}
|
||||
|
||||
/**
|
||||
* Deletes the pipeline specified by id in the request.
|
||||
*/
|
||||
public void delete(ClusterService clusterService, DeletePipelineRequest request, ActionListener<WritePipelineResponse> listener) {
|
||||
clusterService.submitStateUpdateTask("delete-pipeline-" + request.getId(), new AckedClusterStateUpdateTask<WritePipelineResponse>(request, listener) {
|
||||
|
||||
@Override
|
||||
protected WritePipelineResponse newResponse(boolean acknowledged) {
|
||||
return new WritePipelineResponse(acknowledged);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ClusterState execute(ClusterState currentState) throws Exception {
|
||||
return innerDelete(request, currentState);
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
ClusterState innerDelete(DeletePipelineRequest request, ClusterState currentState) {
|
||||
IngestMetadata currentIngestMetadata = currentState.metaData().custom(IngestMetadata.TYPE);
|
||||
if (currentIngestMetadata == null) {
|
||||
return currentState;
|
||||
}
|
||||
Map<String, PipelineConfiguration> pipelines = currentIngestMetadata.getPipelines();
|
||||
if (pipelines.containsKey(request.getId()) == false) {
|
||||
throw new ResourceNotFoundException("pipeline [{}] is missing", request.getId());
|
||||
} else {
|
||||
pipelines = new HashMap<>(pipelines);
|
||||
pipelines.remove(request.getId());
|
||||
ClusterState.Builder newState = ClusterState.builder(currentState);
|
||||
newState.metaData(MetaData.builder(currentState.getMetaData())
|
||||
.putCustom(IngestMetadata.TYPE, new IngestMetadata(pipelines))
|
||||
.build());
|
||||
return newState.build();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Stores the specified pipeline definition in the request.
|
||||
*
|
||||
* @throws IllegalArgumentException If the pipeline holds incorrect configuration
|
||||
*/
|
||||
public void put(ClusterService clusterService, PutPipelineRequest request, ActionListener<WritePipelineResponse> listener) throws IllegalArgumentException {
|
||||
try {
|
||||
// validates the pipeline and processor configuration before submitting a cluster update task:
|
||||
Map<String, Object> pipelineConfig = XContentHelper.convertToMap(request.getSource(), false).v2();
|
||||
constructPipeline(request.getId(), pipelineConfig);
|
||||
} catch (Exception e) {
|
||||
throw new IllegalArgumentException("Invalid pipeline configuration", e);
|
||||
}
|
||||
clusterService.submitStateUpdateTask("put-pipeline-" + request.getId(), new AckedClusterStateUpdateTask<WritePipelineResponse>(request, listener) {
|
||||
|
||||
@Override
|
||||
protected WritePipelineResponse newResponse(boolean acknowledged) {
|
||||
return new WritePipelineResponse(acknowledged);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ClusterState execute(ClusterState currentState) throws Exception {
|
||||
return innerPut(request, currentState);
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
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<>();
|
||||
}
|
||||
|
||||
pipelines.put(request.getId(), new PipelineConfiguration(request.getId(), request.getSource()));
|
||||
ClusterState.Builder newState = ClusterState.builder(currentState);
|
||||
newState.metaData(MetaData.builder(currentState.getMetaData())
|
||||
.putCustom(IngestMetadata.TYPE, new IngestMetadata(pipelines))
|
||||
.build());
|
||||
return newState.build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the pipeline by the specified id
|
||||
*/
|
||||
public Pipeline get(String id) {
|
||||
return pipelines.get(id);
|
||||
}
|
||||
|
||||
public Map<String, Processor.Factory> getProcessorFactoryRegistry() {
|
||||
return processorFactoryRegistry;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return pipeline configuration specified by id. If multiple ids or wildcards are specified multiple pipelines
|
||||
* may be returned
|
||||
*/
|
||||
// Returning PipelineConfiguration instead of Pipeline, because Pipeline and Processor interface don't
|
||||
// know how to serialize themselves.
|
||||
public List<PipelineConfiguration> getPipelines(ClusterState clusterState, String... ids) {
|
||||
IngestMetadata ingestMetadata = clusterState.getMetaData().custom(IngestMetadata.TYPE);
|
||||
return innerGetPipelines(ingestMetadata, ids);
|
||||
}
|
||||
|
||||
List<PipelineConfiguration> innerGetPipelines(IngestMetadata ingestMetadata, String... ids) {
|
||||
if (ingestMetadata == null) {
|
||||
return Collections.emptyList();
|
||||
}
|
||||
|
||||
List<PipelineConfiguration> result = new ArrayList<>(ids.length);
|
||||
for (String id : ids) {
|
||||
if (Regex.isSimpleMatchPattern(id)) {
|
||||
for (Map.Entry<String, PipelineConfiguration> entry : ingestMetadata.getPipelines().entrySet()) {
|
||||
if (Regex.simpleMatch(id, entry.getKey())) {
|
||||
result.add(entry.getValue());
|
||||
}
|
||||
}
|
||||
} else {
|
||||
PipelineConfiguration pipeline = ingestMetadata.getPipelines().get(id);
|
||||
if (pipeline != null) {
|
||||
result.add(pipeline);
|
||||
}
|
||||
}
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
private Pipeline constructPipeline(String id, Map<String, Object> config) throws Exception {
|
||||
return factory.create(id, config, processorFactoryRegistry);
|
||||
}
|
||||
|
||||
}
|
@ -0,0 +1,47 @@
|
||||
/*
|
||||
* 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.ingest.core.Processor;
|
||||
import org.elasticsearch.ingest.core.TemplateService;
|
||||
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.function.Function;
|
||||
|
||||
public class ProcessorsRegistry {
|
||||
|
||||
private final Map<String, Function<TemplateService, Processor.Factory<?>>> processorFactoryProviders = new HashMap<>();
|
||||
|
||||
/**
|
||||
* Adds a processor factory under a specific name.
|
||||
*/
|
||||
public void registerProcessor(String name, Function<TemplateService, Processor.Factory<?>> processorFactoryProvider) {
|
||||
Function<TemplateService, Processor.Factory<?>> provider = processorFactoryProviders.putIfAbsent(name, processorFactoryProvider);
|
||||
if (provider != null) {
|
||||
throw new IllegalArgumentException("Processor factory already registered for name [" + name + "]");
|
||||
}
|
||||
}
|
||||
|
||||
public Set<Map.Entry<String, Function<TemplateService, Processor.Factory<?>>>> entrySet() {
|
||||
return processorFactoryProviders.entrySet();
|
||||
}
|
||||
}
|
@ -0,0 +1,38 @@
|
||||
/*
|
||||
* 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.core;
|
||||
|
||||
/**
|
||||
* An Abstract Processor that holds a processorTag field to be used
|
||||
* by other processors.
|
||||
*/
|
||||
public abstract class AbstractProcessor implements Processor {
|
||||
protected final String tag;
|
||||
|
||||
protected AbstractProcessor(String tag) {
|
||||
this.tag = tag;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getTag() {
|
||||
return tag;
|
||||
}
|
||||
}
|
@ -0,0 +1,39 @@
|
||||
/*
|
||||
* 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.core;
|
||||
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* A processor implementation may modify the data belonging to a document.
|
||||
* Whether changes are made and what exactly is modified is up to the implementation.
|
||||
*/
|
||||
public abstract class AbstractProcessorFactory<P extends Processor> implements Processor.Factory<P> {
|
||||
public static final String TAG_KEY = "tag";
|
||||
|
||||
@Override
|
||||
public P create(Map<String, Object> config) throws Exception {
|
||||
String tag = ConfigurationUtils.readOptionalStringProperty(config, TAG_KEY);
|
||||
return doCreate(tag, config);
|
||||
}
|
||||
|
||||
protected abstract P doCreate(String tag, Map<String, Object> config) throws Exception;
|
||||
}
|
@ -0,0 +1,98 @@
|
||||
/*
|
||||
* 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.
|
||||
*/
|
||||
//TODO(simonw): can all these classes go into org.elasticsearch.ingest?
|
||||
|
||||
package org.elasticsearch.ingest.core;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
* A Processor that executes a list of other "processors". It executes a separate list of
|
||||
* "onFailureProcessors" when any of the processors throw an {@link Exception}.
|
||||
*/
|
||||
public class CompoundProcessor implements Processor {
|
||||
static final String ON_FAILURE_MESSAGE_FIELD = "on_failure_message";
|
||||
static final String ON_FAILURE_PROCESSOR_FIELD = "on_failure_processor";
|
||||
|
||||
private final List<Processor> processors;
|
||||
private final List<Processor> onFailureProcessors;
|
||||
|
||||
public CompoundProcessor(Processor... processor) {
|
||||
this(Arrays.asList(processor), Collections.emptyList());
|
||||
}
|
||||
|
||||
public CompoundProcessor(List<Processor> processors, List<Processor> onFailureProcessors) {
|
||||
super();
|
||||
this.processors = processors;
|
||||
this.onFailureProcessors = onFailureProcessors;
|
||||
}
|
||||
|
||||
public List<Processor> getOnFailureProcessors() {
|
||||
return onFailureProcessors;
|
||||
}
|
||||
|
||||
public List<Processor> getProcessors() {
|
||||
return processors;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getType() {
|
||||
return "compound";
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getTag() {
|
||||
return "compound-processor-" + Objects.hash(processors, onFailureProcessors);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void execute(IngestDocument ingestDocument) throws Exception {
|
||||
for (Processor processor : processors) {
|
||||
try {
|
||||
processor.execute(ingestDocument);
|
||||
} catch (Exception e) {
|
||||
if (onFailureProcessors.isEmpty()) {
|
||||
throw e;
|
||||
} else {
|
||||
executeOnFailure(ingestDocument, e, processor.getType());
|
||||
}
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
void executeOnFailure(IngestDocument ingestDocument, Exception cause, String failedProcessorType) throws Exception {
|
||||
Map<String, String> ingestMetadata = ingestDocument.getIngestMetadata();
|
||||
try {
|
||||
ingestMetadata.put(ON_FAILURE_MESSAGE_FIELD, cause.getMessage());
|
||||
ingestMetadata.put(ON_FAILURE_PROCESSOR_FIELD, failedProcessorType);
|
||||
for (Processor processor : onFailureProcessors) {
|
||||
processor.execute(ingestDocument);
|
||||
}
|
||||
} finally {
|
||||
ingestMetadata.remove(ON_FAILURE_MESSAGE_FIELD);
|
||||
ingestMetadata.remove(ON_FAILURE_PROCESSOR_FIELD);
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,163 @@
|
||||
/*
|
||||
* 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.core;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
public final class ConfigurationUtils {
|
||||
|
||||
private ConfigurationUtils() {
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns and removes the specified optional property from the specified configuration map.
|
||||
*
|
||||
* If the property value isn't of type string a {@link IllegalArgumentException} is thrown.
|
||||
*/
|
||||
public static String readOptionalStringProperty(Map<String, Object> configuration, String propertyName) {
|
||||
Object value = configuration.remove(propertyName);
|
||||
return readString(propertyName, value);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns and removes the specified property from the specified configuration map.
|
||||
*
|
||||
* If the property value isn't of type string an {@link IllegalArgumentException} is thrown.
|
||||
* If the property is missing an {@link IllegalArgumentException} is thrown
|
||||
*/
|
||||
public static String readStringProperty(Map<String, Object> configuration, String propertyName) {
|
||||
return readStringProperty(configuration, propertyName, null);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns and removes the specified property from the specified configuration map.
|
||||
*
|
||||
* If the property value isn't of type string a {@link IllegalArgumentException} is thrown.
|
||||
* If the property is missing and no default value has been specified a {@link IllegalArgumentException} is thrown
|
||||
*/
|
||||
public static String readStringProperty(Map<String, Object> configuration, String propertyName, String defaultValue) {
|
||||
Object value = configuration.remove(propertyName);
|
||||
if (value == null && defaultValue != null) {
|
||||
return defaultValue;
|
||||
} else if (value == null) {
|
||||
throw new IllegalArgumentException("required property [" + propertyName + "] is missing");
|
||||
}
|
||||
return readString(propertyName, value);
|
||||
}
|
||||
|
||||
private static String readString(String propertyName, Object value) {
|
||||
if (value == null) {
|
||||
return null;
|
||||
}
|
||||
if (value instanceof String) {
|
||||
return (String) value;
|
||||
}
|
||||
throw new IllegalArgumentException("property [" + propertyName + "] isn't a string, but of type [" + value.getClass().getName() + "]");
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns and removes the specified property of type list from the specified configuration map.
|
||||
*
|
||||
* If the property value isn't of type list an {@link IllegalArgumentException} is thrown.
|
||||
*/
|
||||
public static <T> List<T> readOptionalList(Map<String, Object> configuration, String propertyName) {
|
||||
Object value = configuration.remove(propertyName);
|
||||
if (value == null) {
|
||||
return null;
|
||||
}
|
||||
return readList(propertyName, value);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns and removes the specified property of type list from the specified configuration map.
|
||||
*
|
||||
* If the property value isn't of type list an {@link IllegalArgumentException} is thrown.
|
||||
* If the property is missing an {@link IllegalArgumentException} is thrown
|
||||
*/
|
||||
public static <T> List<T> readList(Map<String, Object> configuration, String propertyName) {
|
||||
Object value = configuration.remove(propertyName);
|
||||
if (value == null) {
|
||||
throw new IllegalArgumentException("required property [" + propertyName + "] is missing");
|
||||
}
|
||||
|
||||
return readList(propertyName, value);
|
||||
}
|
||||
|
||||
private static <T> List<T> readList(String propertyName, Object value) {
|
||||
if (value instanceof List) {
|
||||
@SuppressWarnings("unchecked")
|
||||
List<T> stringList = (List<T>) value;
|
||||
return stringList;
|
||||
} else {
|
||||
throw new IllegalArgumentException("property [" + propertyName + "] isn't a list, but of type [" + value.getClass().getName() + "]");
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns and removes the specified property of type map from the specified configuration map.
|
||||
*
|
||||
* If the property value isn't of type map an {@link IllegalArgumentException} is thrown.
|
||||
* If the property is missing an {@link IllegalArgumentException} is thrown
|
||||
*/
|
||||
public static <T> Map<String, T> readMap(Map<String, Object> configuration, String propertyName) {
|
||||
Object value = configuration.remove(propertyName);
|
||||
if (value == null) {
|
||||
throw new IllegalArgumentException("required property [" + propertyName + "] is missing");
|
||||
}
|
||||
|
||||
return readMap(propertyName, value);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns and removes the specified property of type map from the specified configuration map.
|
||||
*
|
||||
* If the property value isn't of type map an {@link IllegalArgumentException} is thrown.
|
||||
*/
|
||||
public static <T> Map<String, T> readOptionalMap(Map<String, Object> configuration, String propertyName) {
|
||||
Object value = configuration.remove(propertyName);
|
||||
if (value == null) {
|
||||
return null;
|
||||
}
|
||||
|
||||
return readMap(propertyName, value);
|
||||
}
|
||||
|
||||
private static <T> Map<String, T> readMap(String propertyName, Object value) {
|
||||
if (value instanceof Map) {
|
||||
@SuppressWarnings("unchecked")
|
||||
Map<String, T> map = (Map<String, T>) value;
|
||||
return map;
|
||||
} else {
|
||||
throw new IllegalArgumentException("property [" + propertyName + "] isn't a map, but of type [" + value.getClass().getName() + "]");
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns and removes the specified property as an {@link Object} from the specified configuration map.
|
||||
*/
|
||||
public static Object readObject(Map<String, Object> configuration, String propertyName) {
|
||||
Object value = configuration.remove(propertyName);
|
||||
if (value == null) {
|
||||
throw new IllegalArgumentException("required property [" + propertyName + "] is missing");
|
||||
}
|
||||
return value;
|
||||
}
|
||||
}
|
@ -0,0 +1,544 @@
|
||||
/*
|
||||
* 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.core;
|
||||
|
||||
import org.elasticsearch.common.Strings;
|
||||
import org.elasticsearch.index.mapper.internal.IdFieldMapper;
|
||||
import org.elasticsearch.index.mapper.internal.IndexFieldMapper;
|
||||
import org.elasticsearch.index.mapper.internal.ParentFieldMapper;
|
||||
import org.elasticsearch.index.mapper.internal.RoutingFieldMapper;
|
||||
import org.elasticsearch.index.mapper.internal.SourceFieldMapper;
|
||||
import org.elasticsearch.index.mapper.internal.TTLFieldMapper;
|
||||
import org.elasticsearch.index.mapper.internal.TimestampFieldMapper;
|
||||
import org.elasticsearch.index.mapper.internal.TypeFieldMapper;
|
||||
|
||||
import java.text.DateFormat;
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Date;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Locale;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
import java.util.TimeZone;
|
||||
|
||||
/**
|
||||
* Represents a single document being captured before indexing and holds the source and metadata (like id, type and index).
|
||||
*/
|
||||
public final class IngestDocument {
|
||||
|
||||
public final static String INGEST_KEY = "_ingest";
|
||||
|
||||
static final String TIMESTAMP = "timestamp";
|
||||
|
||||
private final Map<String, Object> sourceAndMetadata;
|
||||
private final Map<String, String> ingestMetadata;
|
||||
|
||||
public IngestDocument(String index, String type, String id, String routing, String parent, String timestamp, String ttl, Map<String, Object> source) {
|
||||
this.sourceAndMetadata = new HashMap<>();
|
||||
this.sourceAndMetadata.putAll(source);
|
||||
this.sourceAndMetadata.put(MetaData.INDEX.getFieldName(), index);
|
||||
this.sourceAndMetadata.put(MetaData.TYPE.getFieldName(), type);
|
||||
this.sourceAndMetadata.put(MetaData.ID.getFieldName(), id);
|
||||
if (routing != null) {
|
||||
this.sourceAndMetadata.put(MetaData.ROUTING.getFieldName(), routing);
|
||||
}
|
||||
if (parent != null) {
|
||||
this.sourceAndMetadata.put(MetaData.PARENT.getFieldName(), parent);
|
||||
}
|
||||
if (timestamp != null) {
|
||||
this.sourceAndMetadata.put(MetaData.TIMESTAMP.getFieldName(), timestamp);
|
||||
}
|
||||
if (ttl != null) {
|
||||
this.sourceAndMetadata.put(MetaData.TTL.getFieldName(), ttl);
|
||||
}
|
||||
|
||||
this.ingestMetadata = new HashMap<>();
|
||||
DateFormat df = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSSZZ", Locale.ROOT);
|
||||
df.setTimeZone(TimeZone.getTimeZone("UTC"));
|
||||
this.ingestMetadata.put(TIMESTAMP, df.format(new Date()));
|
||||
}
|
||||
|
||||
/**
|
||||
* Copy constructor that creates a new {@link IngestDocument} which has exactly the same properties as the one provided as argument
|
||||
*/
|
||||
public IngestDocument(IngestDocument other) {
|
||||
this(new HashMap<>(other.sourceAndMetadata), new HashMap<>(other.ingestMetadata));
|
||||
}
|
||||
|
||||
/**
|
||||
* Constructor needed for testing that allows to create a new {@link IngestDocument} given the provided elasticsearch metadata,
|
||||
* source and ingest metadata. This is needed because the ingest metadata will be initialized with the current timestamp at
|
||||
* init time, which makes equality comparisons impossible in tests.
|
||||
*/
|
||||
public IngestDocument(Map<String, Object> sourceAndMetadata, Map<String, String> ingestMetadata) {
|
||||
this.sourceAndMetadata = sourceAndMetadata;
|
||||
this.ingestMetadata = ingestMetadata;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the value contained in the document for the provided path
|
||||
* @param path The path within the document in dot-notation
|
||||
* @param clazz The expected class of the field value
|
||||
* @return the value for the provided path if existing, null otherwise
|
||||
* @throws IllegalArgumentException if the path is null, empty, invalid, if the field doesn't exist
|
||||
* or if the field that is found at the provided path is not of the expected type.
|
||||
*/
|
||||
public <T> T getFieldValue(String path, Class<T> clazz) {
|
||||
FieldPath fieldPath = new FieldPath(path);
|
||||
Object context = fieldPath.initialContext;
|
||||
for (String pathElement : fieldPath.pathElements) {
|
||||
context = resolve(pathElement, path, context);
|
||||
}
|
||||
return cast(path, context, clazz);
|
||||
}
|
||||
|
||||
/**
|
||||
* Checks whether the document contains a value for the provided path
|
||||
* @param path The path within the document in dot-notation
|
||||
* @return true if the document contains a value for the field, false otherwise
|
||||
* @throws IllegalArgumentException if the path is null, empty or invalid.
|
||||
*/
|
||||
public boolean hasField(String path) {
|
||||
FieldPath fieldPath = new FieldPath(path);
|
||||
Object context = fieldPath.initialContext;
|
||||
for (int i = 0; i < fieldPath.pathElements.length - 1; i++) {
|
||||
String pathElement = fieldPath.pathElements[i];
|
||||
if (context == null) {
|
||||
return false;
|
||||
}
|
||||
if (context instanceof Map) {
|
||||
@SuppressWarnings("unchecked")
|
||||
Map<String, Object> map = (Map<String, Object>) context;
|
||||
context = map.get(pathElement);
|
||||
} else if (context instanceof List) {
|
||||
@SuppressWarnings("unchecked")
|
||||
List<Object> list = (List<Object>) context;
|
||||
try {
|
||||
int index = Integer.parseInt(pathElement);
|
||||
if (index < 0 || index >= list.size()) {
|
||||
return false;
|
||||
}
|
||||
context = list.get(index);
|
||||
} catch (NumberFormatException e) {
|
||||
return false;
|
||||
}
|
||||
|
||||
} else {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
String leafKey = fieldPath.pathElements[fieldPath.pathElements.length - 1];
|
||||
if (context instanceof Map) {
|
||||
@SuppressWarnings("unchecked")
|
||||
Map<String, Object> map = (Map<String, Object>) context;
|
||||
return map.containsKey(leafKey);
|
||||
}
|
||||
if (context instanceof List) {
|
||||
@SuppressWarnings("unchecked")
|
||||
List<Object> list = (List<Object>) context;
|
||||
try {
|
||||
int index = Integer.parseInt(leafKey);
|
||||
return index >= 0 && index < list.size();
|
||||
} catch (NumberFormatException e) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
/**
|
||||
* Removes the field identified by the provided path.
|
||||
* @param fieldPathTemplate Resolves to the path with dot-notation within the document
|
||||
* @throws IllegalArgumentException if the path is null, empty, invalid or if the field doesn't exist.
|
||||
*/
|
||||
public void removeField(TemplateService.Template fieldPathTemplate) {
|
||||
removeField(renderTemplate(fieldPathTemplate));
|
||||
}
|
||||
|
||||
/**
|
||||
* Removes the field identified by the provided path.
|
||||
* @param path the path of the field to be removed
|
||||
* @throws IllegalArgumentException if the path is null, empty, invalid or if the field doesn't exist.
|
||||
*/
|
||||
public void removeField(String path) {
|
||||
FieldPath fieldPath = new FieldPath(path);
|
||||
Object context = fieldPath.initialContext;
|
||||
for (int i = 0; i < fieldPath.pathElements.length - 1; i++) {
|
||||
context = resolve(fieldPath.pathElements[i], path, context);
|
||||
}
|
||||
|
||||
String leafKey = fieldPath.pathElements[fieldPath.pathElements.length - 1];
|
||||
if (context instanceof Map) {
|
||||
@SuppressWarnings("unchecked")
|
||||
Map<String, Object> map = (Map<String, Object>) context;
|
||||
if (map.containsKey(leafKey)) {
|
||||
map.remove(leafKey);
|
||||
return;
|
||||
}
|
||||
throw new IllegalArgumentException("field [" + leafKey + "] not present as part of path [" + path + "]");
|
||||
}
|
||||
if (context instanceof List) {
|
||||
@SuppressWarnings("unchecked")
|
||||
List<Object> list = (List<Object>) context;
|
||||
int index;
|
||||
try {
|
||||
index = Integer.parseInt(leafKey);
|
||||
} catch (NumberFormatException e) {
|
||||
throw new IllegalArgumentException("[" + leafKey + "] is not an integer, cannot be used as an index as part of path [" + path + "]", e);
|
||||
}
|
||||
if (index < 0 || index >= list.size()) {
|
||||
throw new IllegalArgumentException("[" + index + "] is out of bounds for array with length [" + list.size() + "] as part of path [" + path + "]");
|
||||
}
|
||||
list.remove(index);
|
||||
return;
|
||||
}
|
||||
|
||||
if (context == null) {
|
||||
throw new IllegalArgumentException("cannot remove [" + leafKey + "] from null as part of path [" + path + "]");
|
||||
}
|
||||
throw new IllegalArgumentException("cannot remove [" + leafKey + "] from object of type [" + context.getClass().getName() + "] as part of path [" + path + "]");
|
||||
}
|
||||
|
||||
private static Object resolve(String pathElement, String fullPath, Object context) {
|
||||
if (context == null) {
|
||||
throw new IllegalArgumentException("cannot resolve [" + pathElement + "] from null as part of path [" + fullPath + "]");
|
||||
}
|
||||
if (context instanceof Map) {
|
||||
@SuppressWarnings("unchecked")
|
||||
Map<String, Object> map = (Map<String, Object>) context;
|
||||
if (map.containsKey(pathElement)) {
|
||||
return map.get(pathElement);
|
||||
}
|
||||
throw new IllegalArgumentException("field [" + pathElement + "] not present as part of path [" + fullPath + "]");
|
||||
}
|
||||
if (context instanceof List) {
|
||||
@SuppressWarnings("unchecked")
|
||||
List<Object> list = (List<Object>) context;
|
||||
int index;
|
||||
try {
|
||||
index = Integer.parseInt(pathElement);
|
||||
} catch (NumberFormatException e) {
|
||||
throw new IllegalArgumentException("[" + pathElement + "] is not an integer, cannot be used as an index as part of path [" + fullPath + "]", e);
|
||||
}
|
||||
if (index < 0 || index >= list.size()) {
|
||||
throw new IllegalArgumentException("[" + index + "] is out of bounds for array with length [" + list.size() + "] as part of path [" + fullPath + "]");
|
||||
}
|
||||
return list.get(index);
|
||||
}
|
||||
throw new IllegalArgumentException("cannot resolve [" + pathElement + "] from object of type [" + context.getClass().getName() + "] as part of path [" + fullPath + "]");
|
||||
}
|
||||
|
||||
/**
|
||||
* Appends the provided value to the provided path in the document.
|
||||
* Any non existing path element will be created.
|
||||
* If the path identifies a list, the value will be appended to the existing list.
|
||||
* If the path identifies a scalar, the scalar will be converted to a list and
|
||||
* the provided value will be added to the newly created list.
|
||||
* Supports multiple values too provided in forms of list, in that case all the values will be appeneded to the
|
||||
* existing (or newly created) list.
|
||||
* @param path The path within the document in dot-notation
|
||||
* @param value The value or values to append to the existing ones
|
||||
* @throws IllegalArgumentException if the path is null, empty or invalid.
|
||||
*/
|
||||
public void appendFieldValue(String path, Object value) {
|
||||
setFieldValue(path, value, true);
|
||||
}
|
||||
|
||||
/**
|
||||
* Appends the provided value to the provided path in the document.
|
||||
* Any non existing path element will be created.
|
||||
* If the path identifies a list, the value will be appended to the existing list.
|
||||
* If the path identifies a scalar, the scalar will be converted to a list and
|
||||
* the provided value will be added to the newly created list.
|
||||
* Supports multiple values too provided in forms of list, in that case all the values will be appeneded to the
|
||||
* existing (or newly created) list.
|
||||
* @param fieldPathTemplate Resolves to the path with dot-notation within the document
|
||||
* @param valueSource The value source that will produce the value or values to append to the existing ones
|
||||
* @throws IllegalArgumentException if the path is null, empty or invalid.
|
||||
*/
|
||||
public void appendFieldValue(TemplateService.Template fieldPathTemplate, ValueSource valueSource) {
|
||||
Map<String, Object> model = createTemplateModel();
|
||||
appendFieldValue(fieldPathTemplate.execute(model), valueSource.copyAndResolve(model));
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the provided value to the provided path in the document.
|
||||
* Any non existing path element will be created.
|
||||
* If the last item in the path is a list, the value will replace the existing list as a whole.
|
||||
* Use {@link #appendFieldValue(String, Object)} to append values to lists instead.
|
||||
* @param path The path within the document in dot-notation
|
||||
* @param value The value to put in for the path key
|
||||
* @throws IllegalArgumentException if the path is null, empty, invalid or if the value cannot be set to the
|
||||
* item identified by the provided path.
|
||||
*/
|
||||
public void setFieldValue(String path, Object value) {
|
||||
setFieldValue(path, value, false);
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the provided value to the provided path in the document.
|
||||
* Any non existing path element will be created. If the last element is a list,
|
||||
* the value will replace the existing list.
|
||||
* @param fieldPathTemplate Resolves to the path with dot-notation within the document
|
||||
* @param valueSource The value source that will produce the value to put in for the path key
|
||||
* @throws IllegalArgumentException if the path is null, empty, invalid or if the value cannot be set to the
|
||||
* item identified by the provided path.
|
||||
*/
|
||||
public void setFieldValue(TemplateService.Template fieldPathTemplate, ValueSource valueSource) {
|
||||
Map<String, Object> model = createTemplateModel();
|
||||
setFieldValue(fieldPathTemplate.execute(model), valueSource.copyAndResolve(model), false);
|
||||
}
|
||||
|
||||
private void setFieldValue(String path, Object value, boolean append) {
|
||||
FieldPath fieldPath = new FieldPath(path);
|
||||
Object context = fieldPath.initialContext;
|
||||
for (int i = 0; i < fieldPath.pathElements.length - 1; i++) {
|
||||
String pathElement = fieldPath.pathElements[i];
|
||||
if (context == null) {
|
||||
throw new IllegalArgumentException("cannot resolve [" + pathElement + "] from null as part of path [" + path + "]");
|
||||
}
|
||||
if (context instanceof Map) {
|
||||
@SuppressWarnings("unchecked")
|
||||
Map<String, Object> map = (Map<String, Object>) context;
|
||||
if (map.containsKey(pathElement)) {
|
||||
context = map.get(pathElement);
|
||||
} else {
|
||||
HashMap<Object, Object> newMap = new HashMap<>();
|
||||
map.put(pathElement, newMap);
|
||||
context = newMap;
|
||||
}
|
||||
} else if (context instanceof List) {
|
||||
@SuppressWarnings("unchecked")
|
||||
List<Object> list = (List<Object>) context;
|
||||
int index;
|
||||
try {
|
||||
index = Integer.parseInt(pathElement);
|
||||
} catch (NumberFormatException e) {
|
||||
throw new IllegalArgumentException("[" + pathElement + "] is not an integer, cannot be used as an index as part of path [" + path + "]", e);
|
||||
}
|
||||
if (index < 0 || index >= list.size()) {
|
||||
throw new IllegalArgumentException("[" + index + "] is out of bounds for array with length [" + list.size() + "] as part of path [" + path + "]");
|
||||
}
|
||||
context = list.get(index);
|
||||
} else {
|
||||
throw new IllegalArgumentException("cannot resolve [" + pathElement + "] from object of type [" + context.getClass().getName() + "] as part of path [" + path + "]");
|
||||
}
|
||||
}
|
||||
|
||||
String leafKey = fieldPath.pathElements[fieldPath.pathElements.length - 1];
|
||||
if (context == null) {
|
||||
throw new IllegalArgumentException("cannot set [" + leafKey + "] with null parent as part of path [" + path + "]");
|
||||
}
|
||||
if (context instanceof Map) {
|
||||
@SuppressWarnings("unchecked")
|
||||
Map<String, Object> map = (Map<String, Object>) context;
|
||||
if (append) {
|
||||
if (map.containsKey(leafKey)) {
|
||||
Object object = map.get(leafKey);
|
||||
List<Object> list = appendValues(object, value);
|
||||
if (list != object) {
|
||||
map.put(leafKey, list);
|
||||
}
|
||||
} else {
|
||||
List<Object> list = new ArrayList<>();
|
||||
appendValues(list, value);
|
||||
map.put(leafKey, list);
|
||||
}
|
||||
return;
|
||||
}
|
||||
map.put(leafKey, value);
|
||||
} else if (context instanceof List) {
|
||||
@SuppressWarnings("unchecked")
|
||||
List<Object> list = (List<Object>) context;
|
||||
int index;
|
||||
try {
|
||||
index = Integer.parseInt(leafKey);
|
||||
} catch (NumberFormatException e) {
|
||||
throw new IllegalArgumentException("[" + leafKey + "] is not an integer, cannot be used as an index as part of path [" + path + "]", e);
|
||||
}
|
||||
if (index < 0 || index >= list.size()) {
|
||||
throw new IllegalArgumentException("[" + index + "] is out of bounds for array with length [" + list.size() + "] as part of path [" + path + "]");
|
||||
}
|
||||
if (append) {
|
||||
Object object = list.get(index);
|
||||
List<Object> newList = appendValues(object, value);
|
||||
if (newList != object) {
|
||||
list.set(index, newList);
|
||||
}
|
||||
return;
|
||||
}
|
||||
list.set(index, value);
|
||||
} else {
|
||||
throw new IllegalArgumentException("cannot set [" + leafKey + "] with parent object of type [" + context.getClass().getName() + "] as part of path [" + path + "]");
|
||||
}
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
private static List<Object> appendValues(Object maybeList, Object value) {
|
||||
List<Object> list;
|
||||
if (maybeList instanceof List) {
|
||||
//maybeList is already a list, we append the provided values to it
|
||||
list = (List<Object>) maybeList;
|
||||
} else {
|
||||
//maybeList is a scalar, we convert it to a list and append the provided values to it
|
||||
list = new ArrayList<>();
|
||||
list.add(maybeList);
|
||||
}
|
||||
appendValues(list, value);
|
||||
return list;
|
||||
}
|
||||
|
||||
private static void appendValues(List<Object> list, Object value) {
|
||||
if (value instanceof List) {
|
||||
@SuppressWarnings("unchecked")
|
||||
List<?> valueList = (List<?>) value;
|
||||
valueList.stream().forEach(list::add);
|
||||
} else {
|
||||
list.add(value);
|
||||
}
|
||||
}
|
||||
|
||||
private static <T> T cast(String path, Object object, Class<T> clazz) {
|
||||
if (object == null) {
|
||||
return null;
|
||||
}
|
||||
if (clazz.isInstance(object)) {
|
||||
return clazz.cast(object);
|
||||
}
|
||||
throw new IllegalArgumentException("field [" + path + "] of type [" + object.getClass().getName() + "] cannot be cast to [" + clazz.getName() + "]");
|
||||
}
|
||||
|
||||
public String renderTemplate(TemplateService.Template template) {
|
||||
return template.execute(createTemplateModel());
|
||||
}
|
||||
|
||||
private Map<String, Object> createTemplateModel() {
|
||||
Map<String, Object> model = new HashMap<>(sourceAndMetadata);
|
||||
model.put(SourceFieldMapper.NAME, sourceAndMetadata);
|
||||
// If there is a field in the source with the name '_ingest' it gets overwritten here,
|
||||
// if access to that field is required then it get accessed via '_source._ingest'
|
||||
model.put(INGEST_KEY, ingestMetadata);
|
||||
return model;
|
||||
}
|
||||
|
||||
/**
|
||||
* one time operation that extracts the metadata fields from the ingest document and returns them.
|
||||
* Metadata fields that used to be accessible as ordinary top level fields will be removed as part of this call.
|
||||
*/
|
||||
public Map<MetaData, String> extractMetadata() {
|
||||
Map<MetaData, String> metadataMap = new HashMap<>();
|
||||
for (MetaData metaData : MetaData.values()) {
|
||||
metadataMap.put(metaData, cast(metaData.getFieldName(), sourceAndMetadata.remove(metaData.getFieldName()), String.class));
|
||||
}
|
||||
return metadataMap;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the available ingest metadata fields, by default only timestamp, but it is possible to set additional ones.
|
||||
* Use only for reading values, modify them instead using {@link #setFieldValue(String, Object)} and {@link #removeField(String)}
|
||||
*/
|
||||
public Map<String, String> getIngestMetadata() {
|
||||
return this.ingestMetadata;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the document including its metadata fields, unless {@link #extractMetadata()} has been called, in which case the
|
||||
* metadata fields will not be present anymore.
|
||||
* Modify the document instead using {@link #setFieldValue(String, Object)} and {@link #removeField(String)}
|
||||
*/
|
||||
public Map<String, Object> getSourceAndMetadata() {
|
||||
return this.sourceAndMetadata;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object obj) {
|
||||
if (obj == this) { return true; }
|
||||
if (obj == null || getClass() != obj.getClass()) {
|
||||
return false;
|
||||
}
|
||||
|
||||
IngestDocument other = (IngestDocument) obj;
|
||||
return Objects.equals(sourceAndMetadata, other.sourceAndMetadata) &&
|
||||
Objects.equals(ingestMetadata, other.ingestMetadata);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return Objects.hash(sourceAndMetadata, ingestMetadata);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "IngestDocument{" +
|
||||
" sourceAndMetadata=" + sourceAndMetadata +
|
||||
", ingestMetadata=" + ingestMetadata +
|
||||
'}';
|
||||
}
|
||||
|
||||
public enum MetaData {
|
||||
INDEX(IndexFieldMapper.NAME),
|
||||
TYPE(TypeFieldMapper.NAME),
|
||||
ID(IdFieldMapper.NAME),
|
||||
ROUTING(RoutingFieldMapper.NAME),
|
||||
PARENT(ParentFieldMapper.NAME),
|
||||
TIMESTAMP(TimestampFieldMapper.NAME),
|
||||
TTL(TTLFieldMapper.NAME);
|
||||
|
||||
private final String fieldName;
|
||||
|
||||
MetaData(String fieldName) {
|
||||
this.fieldName = fieldName;
|
||||
}
|
||||
|
||||
public String getFieldName() {
|
||||
return fieldName;
|
||||
}
|
||||
}
|
||||
|
||||
private class FieldPath {
|
||||
private final String[] pathElements;
|
||||
private final Object initialContext;
|
||||
|
||||
private FieldPath(String path) {
|
||||
if (Strings.isEmpty(path)) {
|
||||
throw new IllegalArgumentException("path cannot be null nor empty");
|
||||
}
|
||||
String newPath;
|
||||
if (path.startsWith(INGEST_KEY + ".")) {
|
||||
initialContext = ingestMetadata;
|
||||
newPath = path.substring(8, path.length());
|
||||
} else {
|
||||
initialContext = sourceAndMetadata;
|
||||
if (path.startsWith(SourceFieldMapper.NAME + ".")) {
|
||||
newPath = path.substring(8, path.length());
|
||||
} else {
|
||||
newPath = path;
|
||||
}
|
||||
}
|
||||
this.pathElements = Strings.splitStringToArray(newPath, '.');
|
||||
if (pathElements.length == 0) {
|
||||
throw new IllegalArgumentException("path [" + path + "] is not valid");
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
126
core/src/main/java/org/elasticsearch/ingest/core/Pipeline.java
Normal file
126
core/src/main/java/org/elasticsearch/ingest/core/Pipeline.java
Normal file
@ -0,0 +1,126 @@
|
||||
/*
|
||||
* 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.core;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* A pipeline is a list of {@link Processor} instances grouped under a unique id.
|
||||
*/
|
||||
public final class Pipeline {
|
||||
|
||||
final static String DESCRIPTION_KEY = "description";
|
||||
final static String PROCESSORS_KEY = "processors";
|
||||
final static String ON_FAILURE_KEY = "on_failure";
|
||||
|
||||
private final String id;
|
||||
private final String description;
|
||||
private final CompoundProcessor compoundProcessor;
|
||||
|
||||
public Pipeline(String id, String description, CompoundProcessor compoundProcessor) {
|
||||
this.id = id;
|
||||
this.description = description;
|
||||
this.compoundProcessor = compoundProcessor;
|
||||
}
|
||||
|
||||
/**
|
||||
* Modifies the data of a document to be indexed based on the processor this pipeline holds
|
||||
*/
|
||||
public void execute(IngestDocument ingestDocument) throws Exception {
|
||||
compoundProcessor.execute(ingestDocument);
|
||||
}
|
||||
|
||||
/**
|
||||
* The unique id of this pipeline
|
||||
*/
|
||||
public String getId() {
|
||||
return id;
|
||||
}
|
||||
|
||||
/**
|
||||
* An optional description of what this pipeline is doing to the data gets processed by this pipeline.
|
||||
*/
|
||||
public String getDescription() {
|
||||
return description;
|
||||
}
|
||||
|
||||
/**
|
||||
* Unmodifiable list containing each processor that operates on the data.
|
||||
*/
|
||||
public List<Processor> getProcessors() {
|
||||
return compoundProcessor.getProcessors();
|
||||
}
|
||||
|
||||
/**
|
||||
* Unmodifiable list containing each on_failure processor that operates on the data in case of
|
||||
* exception thrown in pipeline processors
|
||||
*/
|
||||
public List<Processor> getOnFailureProcessors() {
|
||||
return compoundProcessor.getOnFailureProcessors();
|
||||
}
|
||||
|
||||
public final static class Factory {
|
||||
|
||||
public Pipeline create(String id, Map<String, Object> config, Map<String, Processor.Factory> processorRegistry) throws Exception {
|
||||
String description = ConfigurationUtils.readOptionalStringProperty(config, DESCRIPTION_KEY);
|
||||
List<Processor> processors = readProcessors(PROCESSORS_KEY, processorRegistry, config);
|
||||
List<Processor> onFailureProcessors = readProcessors(ON_FAILURE_KEY, processorRegistry, config);
|
||||
if (config.isEmpty() == false) {
|
||||
throw new IllegalArgumentException("pipeline [" + id + "] doesn't support one or more provided configuration parameters " + Arrays.toString(config.keySet().toArray()));
|
||||
}
|
||||
CompoundProcessor compoundProcessor = new CompoundProcessor(Collections.unmodifiableList(processors), Collections.unmodifiableList(onFailureProcessors));
|
||||
return new Pipeline(id, description, compoundProcessor);
|
||||
}
|
||||
|
||||
private List<Processor> readProcessors(String fieldName, Map<String, Processor.Factory> processorRegistry, Map<String, Object> config) throws Exception {
|
||||
List<Map<String, Map<String, Object>>> processorConfigs = ConfigurationUtils.readOptionalList(config, fieldName);
|
||||
List<Processor> processors = new ArrayList<>();
|
||||
if (processorConfigs != null) {
|
||||
for (Map<String, Map<String, Object>> processorConfigWithKey : processorConfigs) {
|
||||
for (Map.Entry<String, Map<String, Object>> entry : processorConfigWithKey.entrySet()) {
|
||||
processors.add(readProcessor(processorRegistry, entry.getKey(), entry.getValue()));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return processors;
|
||||
}
|
||||
|
||||
private Processor readProcessor(Map<String, Processor.Factory> processorRegistry, String type, Map<String, Object> config) throws Exception {
|
||||
Processor.Factory factory = processorRegistry.get(type);
|
||||
if (factory != null) {
|
||||
List<Processor> onFailureProcessors = readProcessors(ON_FAILURE_KEY, processorRegistry, config);
|
||||
Processor processor = factory.create(config);
|
||||
if (config.isEmpty() == false) {
|
||||
throw new IllegalArgumentException("processor [" + type + "] doesn't support one or more provided configuration parameters " + Arrays.toString(config.keySet().toArray()));
|
||||
}
|
||||
if (onFailureProcessors.isEmpty()) {
|
||||
return processor;
|
||||
}
|
||||
return new CompoundProcessor(Collections.singletonList(processor), onFailureProcessors);
|
||||
}
|
||||
throw new IllegalArgumentException("No processor type exists with name [" + type + "]");
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,59 @@
|
||||
/*
|
||||
* 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.core;
|
||||
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* A processor implementation may modify the data belonging to a document.
|
||||
* Whether changes are made and what exactly is modified is up to the implementation.
|
||||
*/
|
||||
public interface Processor {
|
||||
|
||||
/**
|
||||
* Introspect and potentially modify the incoming data.
|
||||
*/
|
||||
void execute(IngestDocument ingestDocument) throws Exception;
|
||||
|
||||
/**
|
||||
* Gets the type of a processor
|
||||
*/
|
||||
String getType();
|
||||
|
||||
/**
|
||||
* Gets the tag of a processor.
|
||||
*/
|
||||
String getTag();
|
||||
|
||||
/**
|
||||
* A factory that knows how to construct a processor based on a map of maps.
|
||||
*/
|
||||
interface Factory<P extends Processor> {
|
||||
|
||||
/**
|
||||
* Creates a processor based on the specified map of maps config.
|
||||
*
|
||||
* Implementations are responsible for removing the used keys, so that after creating a pipeline ingest can
|
||||
* verify if all configurations settings have been used.
|
||||
*/
|
||||
P create(Map<String, Object> config) throws Exception;
|
||||
}
|
||||
}
|
@ -0,0 +1,38 @@
|
||||
/*
|
||||
* 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.core;
|
||||
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* Abstraction for the ingest template engine used to decouple {@link IngestDocument} from {@link org.elasticsearch.script.ScriptService}.
|
||||
* Allows to compile a template into an ingest {@link Template} object.
|
||||
* A compiled template can be executed by calling its {@link Template#execute(Map)} method.
|
||||
*/
|
||||
public interface TemplateService {
|
||||
|
||||
Template compile(String template);
|
||||
|
||||
interface Template {
|
||||
|
||||
String execute(Map<String, Object> model);
|
||||
|
||||
String getKey();
|
||||
}
|
||||
}
|
@ -0,0 +1,191 @@
|
||||
/*
|
||||
* 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.core;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
|
||||
/**
|
||||
* Holds a value. If the value is requested a copy is made and optionally template snippets are resolved too.
|
||||
*/
|
||||
public interface ValueSource {
|
||||
|
||||
/**
|
||||
* Returns a copy of the value this ValueSource holds and resolves templates if there're any.
|
||||
*
|
||||
* For immutable values only a copy of the reference to the value is made.
|
||||
*
|
||||
* @param model The model to be used when resolving any templates
|
||||
* @return copy of the wrapped value
|
||||
*/
|
||||
Object copyAndResolve(Map<String, Object> model);
|
||||
|
||||
static ValueSource wrap(Object value, TemplateService templateService) {
|
||||
if (value instanceof Map) {
|
||||
@SuppressWarnings("unchecked")
|
||||
Map<Object, Object> mapValue = (Map) value;
|
||||
Map<ValueSource, ValueSource> valueTypeMap = new HashMap<>(mapValue.size());
|
||||
for (Map.Entry<Object, Object> entry : mapValue.entrySet()) {
|
||||
valueTypeMap.put(wrap(entry.getKey(), templateService), wrap(entry.getValue(), templateService));
|
||||
}
|
||||
return new MapValue(valueTypeMap);
|
||||
} else if (value instanceof List) {
|
||||
@SuppressWarnings("unchecked")
|
||||
List<Object> listValue = (List) value;
|
||||
List<ValueSource> valueSourceList = new ArrayList<>(listValue.size());
|
||||
for (Object item : listValue) {
|
||||
valueSourceList.add(wrap(item, templateService));
|
||||
}
|
||||
return new ListValue(valueSourceList);
|
||||
} else if (value == null || value instanceof Number || value instanceof Boolean) {
|
||||
return new ObjectValue(value);
|
||||
} else if (value instanceof String) {
|
||||
return new TemplatedValue(templateService.compile((String) value));
|
||||
} else {
|
||||
throw new IllegalArgumentException("unexpected value type [" + value.getClass() + "]");
|
||||
}
|
||||
}
|
||||
|
||||
final class MapValue implements ValueSource {
|
||||
|
||||
private final Map<ValueSource, ValueSource> map;
|
||||
|
||||
MapValue(Map<ValueSource, ValueSource> map) {
|
||||
this.map = map;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object copyAndResolve(Map<String, Object> model) {
|
||||
Map<Object, Object> copy = new HashMap<>();
|
||||
for (Map.Entry<ValueSource, ValueSource> entry : this.map.entrySet()) {
|
||||
copy.put(entry.getKey().copyAndResolve(model), entry.getValue().copyAndResolve(model));
|
||||
}
|
||||
return copy;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if (this == o) return true;
|
||||
if (o == null || getClass() != o.getClass()) return false;
|
||||
|
||||
MapValue mapValue = (MapValue) o;
|
||||
return map.equals(mapValue.map);
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return map.hashCode();
|
||||
}
|
||||
}
|
||||
|
||||
final class ListValue implements ValueSource {
|
||||
|
||||
private final List<ValueSource> values;
|
||||
|
||||
ListValue(List<ValueSource> values) {
|
||||
this.values = values;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object copyAndResolve(Map<String, Object> model) {
|
||||
List<Object> copy = new ArrayList<>(values.size());
|
||||
for (ValueSource value : values) {
|
||||
copy.add(value.copyAndResolve(model));
|
||||
}
|
||||
return copy;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if (this == o) return true;
|
||||
if (o == null || getClass() != o.getClass()) return false;
|
||||
|
||||
ListValue listValue = (ListValue) o;
|
||||
return values.equals(listValue.values);
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return values.hashCode();
|
||||
}
|
||||
}
|
||||
|
||||
final class ObjectValue implements ValueSource {
|
||||
|
||||
private final Object value;
|
||||
|
||||
ObjectValue(Object value) {
|
||||
this.value = value;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object copyAndResolve(Map<String, Object> model) {
|
||||
return value;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if (this == o) return true;
|
||||
if (o == null || getClass() != o.getClass()) return false;
|
||||
|
||||
ObjectValue objectValue = (ObjectValue) o;
|
||||
return Objects.equals(value, objectValue.value);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return Objects.hashCode(value);
|
||||
}
|
||||
}
|
||||
|
||||
final class TemplatedValue implements ValueSource {
|
||||
|
||||
private final TemplateService.Template template;
|
||||
|
||||
TemplatedValue(TemplateService.Template template) {
|
||||
this.template = template;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object copyAndResolve(Map<String, Object> model) {
|
||||
return template.execute(model);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if (this == o) return true;
|
||||
if (o == null || getClass() != o.getClass()) return false;
|
||||
|
||||
TemplatedValue templatedValue = (TemplatedValue) o;
|
||||
return Objects.equals(template.getKey(), templatedValue.template.getKey());
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return Objects.hashCode(template.getKey());
|
||||
}
|
||||
}
|
||||
|
||||
}
|
@ -0,0 +1,67 @@
|
||||
/*
|
||||
* 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.processor;
|
||||
|
||||
import org.elasticsearch.ingest.core.AbstractProcessor;
|
||||
import org.elasticsearch.ingest.core.AbstractProcessorFactory;
|
||||
import org.elasticsearch.ingest.core.ConfigurationUtils;
|
||||
import org.elasticsearch.ingest.core.IngestDocument;
|
||||
import org.elasticsearch.ingest.core.Processor;
|
||||
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* Base class for processors that manipulate strings and require a single "fields" array config value, which
|
||||
* holds a list of field names in string format.
|
||||
*/
|
||||
public abstract class AbstractStringProcessor extends AbstractProcessor {
|
||||
private final String field;
|
||||
|
||||
protected AbstractStringProcessor(String tag, String field) {
|
||||
super(tag);
|
||||
this.field = field;
|
||||
}
|
||||
|
||||
public String getField() {
|
||||
return field;
|
||||
}
|
||||
|
||||
@Override
|
||||
public final void execute(IngestDocument document) {
|
||||
String val = document.getFieldValue(field, String.class);
|
||||
if (val == null) {
|
||||
throw new IllegalArgumentException("field [" + field + "] is null, cannot process it.");
|
||||
}
|
||||
document.setFieldValue(field, process(val));
|
||||
}
|
||||
|
||||
protected abstract String process(String value);
|
||||
|
||||
public static abstract class Factory<T extends AbstractStringProcessor> extends AbstractProcessorFactory<T> {
|
||||
|
||||
@Override
|
||||
public T doCreate(String processorTag, Map<String, Object> config) throws Exception {
|
||||
String field = ConfigurationUtils.readStringProperty(config, "field");
|
||||
return newProcessor(processorTag, field);
|
||||
}
|
||||
|
||||
protected abstract T newProcessor(String processorTag, String field);
|
||||
}
|
||||
}
|
@ -0,0 +1,82 @@
|
||||
/*
|
||||
* 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.processor;
|
||||
|
||||
import org.elasticsearch.ingest.core.AbstractProcessor;
|
||||
import org.elasticsearch.ingest.core.AbstractProcessorFactory;
|
||||
import org.elasticsearch.ingest.core.IngestDocument;
|
||||
import org.elasticsearch.ingest.core.TemplateService;
|
||||
import org.elasticsearch.ingest.core.ValueSource;
|
||||
import org.elasticsearch.ingest.core.ConfigurationUtils;
|
||||
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* Processor that appends value or values to existing lists. If the field is not present a new list holding the
|
||||
* provided values will be added. If the field is a scalar it will be converted to a single item list and the provided
|
||||
* values will be added to the newly created list.
|
||||
*/
|
||||
public class AppendProcessor extends AbstractProcessor {
|
||||
|
||||
public static final String TYPE = "append";
|
||||
|
||||
private final TemplateService.Template field;
|
||||
private final ValueSource value;
|
||||
|
||||
AppendProcessor(String tag, TemplateService.Template field, ValueSource value) {
|
||||
super(tag);
|
||||
this.field = field;
|
||||
this.value = value;
|
||||
}
|
||||
|
||||
public TemplateService.Template getField() {
|
||||
return field;
|
||||
}
|
||||
|
||||
public ValueSource getValue() {
|
||||
return value;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void execute(IngestDocument ingestDocument) throws Exception {
|
||||
ingestDocument.appendFieldValue(field, value);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getType() {
|
||||
return TYPE;
|
||||
}
|
||||
|
||||
public static final class Factory extends AbstractProcessorFactory<AppendProcessor> {
|
||||
|
||||
private final TemplateService templateService;
|
||||
|
||||
public Factory(TemplateService templateService) {
|
||||
this.templateService = templateService;
|
||||
}
|
||||
|
||||
@Override
|
||||
public AppendProcessor doCreate(String processorTag, Map<String, Object> config) throws Exception {
|
||||
String field = ConfigurationUtils.readStringProperty(config, "field");
|
||||
Object value = ConfigurationUtils.readObject(config, "value");
|
||||
return new AppendProcessor(processorTag, templateService.compile(field), ValueSource.wrap(value, templateService));
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,145 @@
|
||||
/*
|
||||
* 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.processor;
|
||||
|
||||
import org.elasticsearch.ingest.core.AbstractProcessor;
|
||||
import org.elasticsearch.ingest.core.AbstractProcessorFactory;
|
||||
import org.elasticsearch.ingest.core.IngestDocument;
|
||||
import org.elasticsearch.ingest.core.ConfigurationUtils;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Locale;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* Processor that converts fields content to a different type. Supported types are: integer, float, boolean and string.
|
||||
* Throws exception if the field is not there or the conversion fails.
|
||||
*/
|
||||
public class ConvertProcessor extends AbstractProcessor {
|
||||
|
||||
enum Type {
|
||||
INTEGER {
|
||||
@Override
|
||||
public Object convert(Object value) {
|
||||
try {
|
||||
return Integer.parseInt(value.toString());
|
||||
} catch(NumberFormatException e) {
|
||||
throw new IllegalArgumentException("unable to convert [" + value + "] to integer", e);
|
||||
}
|
||||
|
||||
}
|
||||
}, FLOAT {
|
||||
@Override
|
||||
public Object convert(Object value) {
|
||||
try {
|
||||
return Float.parseFloat(value.toString());
|
||||
} catch(NumberFormatException e) {
|
||||
throw new IllegalArgumentException("unable to convert [" + value + "] to float", e);
|
||||
}
|
||||
}
|
||||
}, BOOLEAN {
|
||||
@Override
|
||||
public Object convert(Object value) {
|
||||
if (value.toString().equalsIgnoreCase("true")) {
|
||||
return true;
|
||||
} else if (value.toString().equalsIgnoreCase("false")) {
|
||||
return false;
|
||||
} else {
|
||||
throw new IllegalArgumentException("[" + value + "] is not a boolean value, cannot convert to boolean");
|
||||
}
|
||||
}
|
||||
}, STRING {
|
||||
@Override
|
||||
public Object convert(Object value) {
|
||||
return value.toString();
|
||||
}
|
||||
};
|
||||
|
||||
@Override
|
||||
public final String toString() {
|
||||
return name().toLowerCase(Locale.ROOT);
|
||||
}
|
||||
|
||||
public abstract Object convert(Object value);
|
||||
|
||||
public static Type fromString(String type) {
|
||||
try {
|
||||
return Type.valueOf(type.toUpperCase(Locale.ROOT));
|
||||
} catch(IllegalArgumentException e) {
|
||||
throw new IllegalArgumentException("type [" + type + "] not supported, cannot convert field.", e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public static final String TYPE = "convert";
|
||||
|
||||
private final String field;
|
||||
private final Type convertType;
|
||||
|
||||
ConvertProcessor(String tag, String field, Type convertType) {
|
||||
super(tag);
|
||||
this.field = field;
|
||||
this.convertType = convertType;
|
||||
}
|
||||
|
||||
String getField() {
|
||||
return field;
|
||||
}
|
||||
|
||||
Type getConvertType() {
|
||||
return convertType;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void execute(IngestDocument document) {
|
||||
Object oldValue = document.getFieldValue(field, Object.class);
|
||||
Object newValue;
|
||||
if (oldValue == null) {
|
||||
throw new IllegalArgumentException("Field [" + field + "] is null, cannot be converted to type [" + convertType + "]");
|
||||
}
|
||||
|
||||
if (oldValue instanceof List) {
|
||||
List<?> list = (List<?>) oldValue;
|
||||
List<Object> newList = new ArrayList<>();
|
||||
for (Object value : list) {
|
||||
newList.add(convertType.convert(value));
|
||||
}
|
||||
newValue = newList;
|
||||
} else {
|
||||
newValue = convertType.convert(oldValue);
|
||||
}
|
||||
document.setFieldValue(field, newValue);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getType() {
|
||||
return TYPE;
|
||||
}
|
||||
|
||||
public static class Factory extends AbstractProcessorFactory<ConvertProcessor> {
|
||||
@Override
|
||||
public ConvertProcessor doCreate(String processorTag, Map<String, Object> config) throws Exception {
|
||||
String field = ConfigurationUtils.readStringProperty(config, "field");
|
||||
Type convertType = Type.fromString(ConfigurationUtils.readStringProperty(config, "type"));
|
||||
return new ConvertProcessor(processorTag, field, convertType);
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,90 @@
|
||||
/*
|
||||
* 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.processor;
|
||||
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.DateTimeZone;
|
||||
import org.joda.time.format.DateTimeFormat;
|
||||
import org.joda.time.format.ISODateTimeFormat;
|
||||
|
||||
import java.util.Locale;
|
||||
import java.util.function.Function;
|
||||
|
||||
enum DateFormat {
|
||||
Iso8601 {
|
||||
@Override
|
||||
Function<String, DateTime> getFunction(String format, DateTimeZone timezone, Locale locale) {
|
||||
return ISODateTimeFormat.dateTimeParser().withZone(timezone)::parseDateTime;
|
||||
}
|
||||
},
|
||||
Unix {
|
||||
@Override
|
||||
Function<String, DateTime> getFunction(String format, DateTimeZone timezone, Locale locale) {
|
||||
return (date) -> new DateTime((long)(Float.parseFloat(date) * 1000), timezone);
|
||||
}
|
||||
},
|
||||
UnixMs {
|
||||
@Override
|
||||
Function<String, DateTime> getFunction(String format, DateTimeZone timezone, Locale locale) {
|
||||
return (date) -> new DateTime(Long.parseLong(date), timezone);
|
||||
}
|
||||
},
|
||||
Tai64n {
|
||||
@Override
|
||||
Function<String, DateTime> getFunction(String format, DateTimeZone timezone, Locale locale) {
|
||||
return (date) -> new DateTime(parseMillis(date), timezone);
|
||||
}
|
||||
|
||||
private long parseMillis(String date) {
|
||||
if (date.startsWith("@")) {
|
||||
date = date.substring(1);
|
||||
}
|
||||
long base = Long.parseLong(date.substring(1, 16), 16);
|
||||
// 1356138046000
|
||||
long rest = Long.parseLong(date.substring(16, 24), 16);
|
||||
return ((base * 1000) - 10000) + (rest/1000000);
|
||||
}
|
||||
},
|
||||
Joda {
|
||||
@Override
|
||||
Function<String, DateTime> getFunction(String format, DateTimeZone timezone, Locale locale) {
|
||||
return DateTimeFormat.forPattern(format)
|
||||
.withDefaultYear((new DateTime(DateTimeZone.UTC)).getYear())
|
||||
.withZone(timezone).withLocale(locale)::parseDateTime;
|
||||
}
|
||||
};
|
||||
|
||||
abstract Function<String, DateTime> getFunction(String format, DateTimeZone timezone, Locale locale);
|
||||
|
||||
static DateFormat fromString(String format) {
|
||||
switch (format) {
|
||||
case "ISO8601":
|
||||
return Iso8601;
|
||||
case "UNIX":
|
||||
return Unix;
|
||||
case "UNIX_MS":
|
||||
return UnixMs;
|
||||
case "TAI64N":
|
||||
return Tai64n;
|
||||
default:
|
||||
return Joda;
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,132 @@
|
||||
/*
|
||||
* 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.processor;
|
||||
|
||||
import org.elasticsearch.ExceptionsHelper;
|
||||
import org.elasticsearch.ingest.core.AbstractProcessor;
|
||||
import org.elasticsearch.ingest.core.AbstractProcessorFactory;
|
||||
import org.elasticsearch.ingest.core.ConfigurationUtils;
|
||||
import org.elasticsearch.ingest.core.IngestDocument;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.DateTimeZone;
|
||||
import org.joda.time.format.ISODateTimeFormat;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.IllformedLocaleException;
|
||||
import java.util.List;
|
||||
import java.util.Locale;
|
||||
import java.util.Map;
|
||||
import java.util.function.Function;
|
||||
|
||||
public final class DateProcessor extends AbstractProcessor {
|
||||
|
||||
public static final String TYPE = "date";
|
||||
static final String DEFAULT_TARGET_FIELD = "@timestamp";
|
||||
|
||||
private final DateTimeZone timezone;
|
||||
private final Locale locale;
|
||||
private final String matchField;
|
||||
private final String targetField;
|
||||
private final List<String> matchFormats;
|
||||
private final List<Function<String, DateTime>> dateParsers;
|
||||
|
||||
DateProcessor(String tag, DateTimeZone timezone, Locale locale, String matchField, List<String> matchFormats, String targetField) {
|
||||
super(tag);
|
||||
this.timezone = timezone;
|
||||
this.locale = locale;
|
||||
this.matchField = matchField;
|
||||
this.targetField = targetField;
|
||||
this.matchFormats = matchFormats;
|
||||
this.dateParsers = new ArrayList<>();
|
||||
for (String matchFormat : matchFormats) {
|
||||
DateFormat dateFormat = DateFormat.fromString(matchFormat);
|
||||
dateParsers.add(dateFormat.getFunction(matchFormat, timezone, locale));
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void execute(IngestDocument ingestDocument) {
|
||||
String value = ingestDocument.getFieldValue(matchField, String.class);
|
||||
|
||||
DateTime dateTime = null;
|
||||
Exception lastException = null;
|
||||
for (Function<String, DateTime> dateParser : dateParsers) {
|
||||
try {
|
||||
dateTime = dateParser.apply(value);
|
||||
} catch (Exception e) {
|
||||
//try the next parser and keep track of the exceptions
|
||||
lastException = ExceptionsHelper.useOrSuppress(lastException, e);
|
||||
}
|
||||
}
|
||||
|
||||
if (dateTime == null) {
|
||||
throw new IllegalArgumentException("unable to parse date [" + value + "]", lastException);
|
||||
}
|
||||
|
||||
ingestDocument.setFieldValue(targetField, ISODateTimeFormat.dateTime().print(dateTime));
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getType() {
|
||||
return TYPE;
|
||||
}
|
||||
|
||||
DateTimeZone getTimezone() {
|
||||
return timezone;
|
||||
}
|
||||
|
||||
Locale getLocale() {
|
||||
return locale;
|
||||
}
|
||||
|
||||
String getMatchField() {
|
||||
return matchField;
|
||||
}
|
||||
|
||||
String getTargetField() {
|
||||
return targetField;
|
||||
}
|
||||
|
||||
List<String> getMatchFormats() {
|
||||
return matchFormats;
|
||||
}
|
||||
|
||||
public static class Factory extends AbstractProcessorFactory<DateProcessor> {
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
public DateProcessor doCreate(String processorTag, Map<String, Object> config) throws Exception {
|
||||
String matchField = ConfigurationUtils.readStringProperty(config, "match_field");
|
||||
String targetField = ConfigurationUtils.readStringProperty(config, "target_field", DEFAULT_TARGET_FIELD);
|
||||
String timezoneString = ConfigurationUtils.readOptionalStringProperty(config, "timezone");
|
||||
DateTimeZone timezone = timezoneString == null ? DateTimeZone.UTC : DateTimeZone.forID(timezoneString);
|
||||
String localeString = ConfigurationUtils.readOptionalStringProperty(config, "locale");
|
||||
Locale locale = Locale.ENGLISH;
|
||||
if (localeString != null) {
|
||||
try {
|
||||
locale = (new Locale.Builder()).setLanguageTag(localeString).build();
|
||||
} catch (IllformedLocaleException e) {
|
||||
throw new IllegalArgumentException("Invalid language tag specified: " + localeString);
|
||||
}
|
||||
}
|
||||
List<String> matchFormats = ConfigurationUtils.readList(config, "match_formats");
|
||||
return new DateProcessor(processorTag, timezone, locale, matchField, matchFormats, targetField);
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,107 @@
|
||||
/*
|
||||
* 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.processor;
|
||||
|
||||
import org.elasticsearch.ingest.core.AbstractProcessor;
|
||||
import org.elasticsearch.ingest.core.AbstractProcessorFactory;
|
||||
import org.elasticsearch.ingest.core.ConfigurationUtils;
|
||||
import org.elasticsearch.ingest.core.IngestDocument;
|
||||
import org.elasticsearch.ingest.core.Processor;
|
||||
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* Processor that replaces dots in document field names with a
|
||||
* specified separator.
|
||||
*/
|
||||
public class DeDotProcessor extends AbstractProcessor {
|
||||
|
||||
public static final String TYPE = "dedot";
|
||||
static final String DEFAULT_SEPARATOR = "_";
|
||||
|
||||
private final String separator;
|
||||
|
||||
DeDotProcessor(String tag, String separator) {
|
||||
super(tag);
|
||||
this.separator = separator;
|
||||
}
|
||||
|
||||
public String getSeparator() {
|
||||
return separator;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void execute(IngestDocument document) {
|
||||
deDot(document.getSourceAndMetadata());
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getType() {
|
||||
return TYPE;
|
||||
}
|
||||
|
||||
/**
|
||||
* Recursively iterates through Maps and Lists in search of map entries with
|
||||
* keys containing dots. The dots in these fields are replaced with {@link #separator}.
|
||||
*
|
||||
* @param obj The current object in context to be checked for dots in its fields.
|
||||
*/
|
||||
private void deDot(Object obj) {
|
||||
if (obj instanceof Map) {
|
||||
@SuppressWarnings("unchecked")
|
||||
Map<String, Object> doc = (Map) obj;
|
||||
Iterator<Map.Entry<String, Object>> it = doc.entrySet().iterator();
|
||||
Map<String, Object> deDottedFields = new HashMap<>();
|
||||
while (it.hasNext()) {
|
||||
Map.Entry<String, Object> entry = it.next();
|
||||
deDot(entry.getValue());
|
||||
String fieldName = entry.getKey();
|
||||
if (fieldName.contains(".")) {
|
||||
String deDottedFieldName = fieldName.replaceAll("\\.", separator);
|
||||
deDottedFields.put(deDottedFieldName, entry.getValue());
|
||||
it.remove();
|
||||
}
|
||||
}
|
||||
doc.putAll(deDottedFields);
|
||||
} else if (obj instanceof List) {
|
||||
@SuppressWarnings("unchecked")
|
||||
List<Object> list = (List) obj;
|
||||
for (Object value : list) {
|
||||
deDot(value);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public static class Factory extends AbstractProcessorFactory<DeDotProcessor> {
|
||||
|
||||
@Override
|
||||
public DeDotProcessor doCreate(String processorTag, Map<String, Object> config) throws Exception {
|
||||
String separator = ConfigurationUtils.readOptionalStringProperty(config, "separator");
|
||||
if (separator == null) {
|
||||
separator = DEFAULT_SEPARATOR;
|
||||
}
|
||||
return new DeDotProcessor(processorTag, separator);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -0,0 +1,75 @@
|
||||
/*
|
||||
* 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.processor;
|
||||
|
||||
import org.elasticsearch.ingest.core.AbstractProcessor;
|
||||
import org.elasticsearch.ingest.core.AbstractProcessorFactory;
|
||||
import org.elasticsearch.ingest.core.ConfigurationUtils;
|
||||
import org.elasticsearch.ingest.core.IngestDocument;
|
||||
import org.elasticsearch.ingest.core.Processor;
|
||||
import org.elasticsearch.ingest.core.TemplateService;
|
||||
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* Processor that raises a runtime exception with a provided
|
||||
* error message.
|
||||
*/
|
||||
public class FailProcessor extends AbstractProcessor {
|
||||
|
||||
public static final String TYPE = "fail";
|
||||
|
||||
private final TemplateService.Template message;
|
||||
|
||||
FailProcessor(String tag, TemplateService.Template message) {
|
||||
super(tag);
|
||||
this.message = message;
|
||||
}
|
||||
|
||||
public TemplateService.Template getMessage() {
|
||||
return message;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void execute(IngestDocument document) {
|
||||
throw new FailProcessorException(document.renderTemplate(message));
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getType() {
|
||||
return TYPE;
|
||||
}
|
||||
|
||||
public static class Factory extends AbstractProcessorFactory<FailProcessor> {
|
||||
|
||||
private final TemplateService templateService;
|
||||
|
||||
public Factory(TemplateService templateService) {
|
||||
this.templateService = templateService;
|
||||
}
|
||||
|
||||
@Override
|
||||
public FailProcessor doCreate(String processorTag, Map<String, Object> config) throws Exception {
|
||||
String message = ConfigurationUtils.readStringProperty(config, "message");
|
||||
return new FailProcessor(processorTag, templateService.compile(message));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -0,0 +1,35 @@
|
||||
/*
|
||||
* 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.processor;
|
||||
|
||||
/**
|
||||
* Exception class thrown by {@link FailProcessor}.
|
||||
*
|
||||
* This exception is caught in the {@link org.elasticsearch.ingest.core.CompoundProcessor} and
|
||||
* then changes the state of {@link org.elasticsearch.ingest.core.IngestDocument}. This
|
||||
* exception should get serialized.
|
||||
*/
|
||||
public class FailProcessorException extends RuntimeException {
|
||||
|
||||
public FailProcessorException(String message) {
|
||||
super(message);
|
||||
}
|
||||
}
|
||||
|
@ -0,0 +1,90 @@
|
||||
/*
|
||||
* 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.processor;
|
||||
|
||||
import org.elasticsearch.ingest.core.AbstractProcessor;
|
||||
import org.elasticsearch.ingest.core.AbstractProcessorFactory;
|
||||
import org.elasticsearch.ingest.core.IngestDocument;
|
||||
import org.elasticsearch.ingest.core.ConfigurationUtils;
|
||||
import org.elasticsearch.ingest.core.Processor;
|
||||
|
||||
import java.util.Map;
|
||||
import java.util.regex.Matcher;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
/**
|
||||
* Processor that allows to search for patterns in field content and replace them with corresponding string replacement.
|
||||
* Support fields of string type only, throws exception if a field is of a different type.
|
||||
*/
|
||||
public class GsubProcessor extends AbstractProcessor {
|
||||
|
||||
public static final String TYPE = "gsub";
|
||||
|
||||
private final String field;
|
||||
private final Pattern pattern;
|
||||
private final String replacement;
|
||||
|
||||
GsubProcessor(String tag, String field, Pattern pattern, String replacement) {
|
||||
super(tag);
|
||||
this.field = field;
|
||||
this.pattern = pattern;
|
||||
this.replacement = replacement;
|
||||
}
|
||||
|
||||
String getField() {
|
||||
return field;
|
||||
}
|
||||
|
||||
Pattern getPattern() {
|
||||
return pattern;
|
||||
}
|
||||
|
||||
String getReplacement() {
|
||||
return replacement;
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public void execute(IngestDocument document) {
|
||||
String oldVal = document.getFieldValue(field, String.class);
|
||||
if (oldVal == null) {
|
||||
throw new IllegalArgumentException("field [" + field + "] is null, cannot match pattern.");
|
||||
}
|
||||
Matcher matcher = pattern.matcher(oldVal);
|
||||
String newVal = matcher.replaceAll(replacement);
|
||||
document.setFieldValue(field, newVal);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getType() {
|
||||
return TYPE;
|
||||
}
|
||||
|
||||
public static class Factory extends AbstractProcessorFactory<GsubProcessor> {
|
||||
@Override
|
||||
public GsubProcessor doCreate(String processorTag, Map<String, Object> config) throws Exception {
|
||||
String field = ConfigurationUtils.readStringProperty(config, "field");
|
||||
String pattern = ConfigurationUtils.readStringProperty(config, "pattern");
|
||||
String replacement = ConfigurationUtils.readStringProperty(config, "replacement");
|
||||
Pattern searchPattern = Pattern.compile(pattern);
|
||||
return new GsubProcessor(processorTag, field, searchPattern, replacement);
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,83 @@
|
||||
/*
|
||||
* 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.processor;
|
||||
|
||||
import org.elasticsearch.ingest.core.AbstractProcessor;
|
||||
import org.elasticsearch.ingest.core.AbstractProcessorFactory;
|
||||
import org.elasticsearch.ingest.core.IngestDocument;
|
||||
import org.elasticsearch.ingest.core.ConfigurationUtils;
|
||||
import org.elasticsearch.ingest.core.Processor;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
* Processor that joins the different items of an array into a single string value using a separator between each item.
|
||||
* Throws exception is the specified field is not an array.
|
||||
*/
|
||||
public class JoinProcessor extends AbstractProcessor {
|
||||
|
||||
public static final String TYPE = "join";
|
||||
|
||||
private final String field;
|
||||
private final String separator;
|
||||
|
||||
JoinProcessor(String tag, String field, String separator) {
|
||||
super(tag);
|
||||
this.field = field;
|
||||
this.separator = separator;
|
||||
}
|
||||
|
||||
String getField() {
|
||||
return field;
|
||||
}
|
||||
|
||||
String getSeparator() {
|
||||
return separator;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void execute(IngestDocument document) {
|
||||
List<?> list = document.getFieldValue(field, List.class);
|
||||
if (list == null) {
|
||||
throw new IllegalArgumentException("field [" + field + "] is null, cannot join.");
|
||||
}
|
||||
String joined = list.stream()
|
||||
.map(Object::toString)
|
||||
.collect(Collectors.joining(separator));
|
||||
document.setFieldValue(field, joined);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getType() {
|
||||
return TYPE;
|
||||
}
|
||||
|
||||
public static class Factory extends AbstractProcessorFactory<JoinProcessor> {
|
||||
@Override
|
||||
public JoinProcessor doCreate(String processorTag, Map<String, Object> config) throws Exception {
|
||||
String field = ConfigurationUtils.readStringProperty(config, "field");
|
||||
String separator = ConfigurationUtils.readStringProperty(config, "separator");
|
||||
return new JoinProcessor(processorTag, field, separator);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -0,0 +1,53 @@
|
||||
/*
|
||||
* 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.processor;
|
||||
|
||||
import java.util.Locale;
|
||||
|
||||
/**
|
||||
* Processor that converts the content of string fields to lowercase.
|
||||
* Throws exception is the field is not of type string.
|
||||
*/
|
||||
|
||||
public class LowercaseProcessor extends AbstractStringProcessor {
|
||||
|
||||
public static final String TYPE = "lowercase";
|
||||
|
||||
LowercaseProcessor(String processorTag, String field) {
|
||||
super(processorTag, field);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected String process(String value) {
|
||||
return value.toLowerCase(Locale.ROOT);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getType() {
|
||||
return TYPE;
|
||||
}
|
||||
|
||||
public static class Factory extends AbstractStringProcessor.Factory<LowercaseProcessor> {
|
||||
@Override
|
||||
protected LowercaseProcessor newProcessor(String tag, String field) {
|
||||
return new LowercaseProcessor(tag, field);
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,74 @@
|
||||
/*
|
||||
* 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.processor;
|
||||
|
||||
import org.elasticsearch.ingest.core.AbstractProcessor;
|
||||
import org.elasticsearch.ingest.core.AbstractProcessorFactory;
|
||||
import org.elasticsearch.ingest.core.IngestDocument;
|
||||
import org.elasticsearch.ingest.core.TemplateService;
|
||||
import org.elasticsearch.ingest.core.ConfigurationUtils;
|
||||
import org.elasticsearch.ingest.core.Processor;
|
||||
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* Processor that removes existing fields. Nothing happens if the field is not present.
|
||||
*/
|
||||
public class RemoveProcessor extends AbstractProcessor {
|
||||
|
||||
public static final String TYPE = "remove";
|
||||
|
||||
private final TemplateService.Template field;
|
||||
|
||||
RemoveProcessor(String tag, TemplateService.Template field) {
|
||||
super(tag);
|
||||
this.field = field;
|
||||
}
|
||||
|
||||
public TemplateService.Template getField() {
|
||||
return field;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void execute(IngestDocument document) {
|
||||
document.removeField(field);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getType() {
|
||||
return TYPE;
|
||||
}
|
||||
|
||||
public static class Factory extends AbstractProcessorFactory<RemoveProcessor> {
|
||||
|
||||
private final TemplateService templateService;
|
||||
|
||||
public Factory(TemplateService templateService) {
|
||||
this.templateService = templateService;
|
||||
}
|
||||
|
||||
@Override
|
||||
public RemoveProcessor doCreate(String processorTag, Map<String, Object> config) throws Exception {
|
||||
String field = ConfigurationUtils.readStringProperty(config, "field");
|
||||
return new RemoveProcessor(processorTag, templateService.compile(field));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -0,0 +1,87 @@
|
||||
/*
|
||||
* 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.processor;
|
||||
|
||||
import org.elasticsearch.ingest.core.AbstractProcessor;
|
||||
import org.elasticsearch.ingest.core.AbstractProcessorFactory;
|
||||
import org.elasticsearch.ingest.core.IngestDocument;
|
||||
import org.elasticsearch.ingest.core.ConfigurationUtils;
|
||||
import org.elasticsearch.ingest.core.Processor;
|
||||
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* Processor that allows to rename existing fields. Will throw exception if the field is not present.
|
||||
*/
|
||||
public class RenameProcessor extends AbstractProcessor {
|
||||
|
||||
public static final String TYPE = "rename";
|
||||
|
||||
private final String oldFieldName;
|
||||
private final String newFieldName;
|
||||
|
||||
RenameProcessor(String tag, String oldFieldName, String newFieldName) {
|
||||
super(tag);
|
||||
this.oldFieldName = oldFieldName;
|
||||
this.newFieldName = newFieldName;
|
||||
}
|
||||
|
||||
String getOldFieldName() {
|
||||
return oldFieldName;
|
||||
}
|
||||
|
||||
String getNewFieldName() {
|
||||
return newFieldName;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void execute(IngestDocument document) {
|
||||
if (document.hasField(oldFieldName) == false) {
|
||||
throw new IllegalArgumentException("field [" + oldFieldName + "] doesn't exist");
|
||||
}
|
||||
if (document.hasField(newFieldName)) {
|
||||
throw new IllegalArgumentException("field [" + newFieldName + "] already exists");
|
||||
}
|
||||
|
||||
Object oldValue = document.getFieldValue(oldFieldName, Object.class);
|
||||
document.setFieldValue(newFieldName, oldValue);
|
||||
try {
|
||||
document.removeField(oldFieldName);
|
||||
} catch (Exception e) {
|
||||
//remove the new field if the removal of the old one failed
|
||||
document.removeField(newFieldName);
|
||||
throw e;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getType() {
|
||||
return TYPE;
|
||||
}
|
||||
|
||||
public static class Factory extends AbstractProcessorFactory<RenameProcessor> {
|
||||
@Override
|
||||
public RenameProcessor doCreate(String processorTag, Map<String, Object> config) throws Exception {
|
||||
String field = ConfigurationUtils.readStringProperty(config, "field");
|
||||
String newField = ConfigurationUtils.readStringProperty(config, "to");
|
||||
return new RenameProcessor(processorTag, field, newField);
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,81 @@
|
||||
/*
|
||||
* 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.processor;
|
||||
|
||||
import org.elasticsearch.ingest.core.AbstractProcessor;
|
||||
import org.elasticsearch.ingest.core.AbstractProcessorFactory;
|
||||
import org.elasticsearch.ingest.core.IngestDocument;
|
||||
import org.elasticsearch.ingest.core.TemplateService;
|
||||
import org.elasticsearch.ingest.core.ValueSource;
|
||||
import org.elasticsearch.ingest.core.ConfigurationUtils;
|
||||
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* Processor that adds new fields with their corresponding values. If the field is already present, its value
|
||||
* will be replaced with the provided one.
|
||||
*/
|
||||
public class SetProcessor extends AbstractProcessor {
|
||||
|
||||
public static final String TYPE = "set";
|
||||
|
||||
private final TemplateService.Template field;
|
||||
private final ValueSource value;
|
||||
|
||||
SetProcessor(String tag, TemplateService.Template field, ValueSource value) {
|
||||
super(tag);
|
||||
this.field = field;
|
||||
this.value = value;
|
||||
}
|
||||
|
||||
public TemplateService.Template getField() {
|
||||
return field;
|
||||
}
|
||||
|
||||
public ValueSource getValue() {
|
||||
return value;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void execute(IngestDocument document) {
|
||||
document.setFieldValue(field, value);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getType() {
|
||||
return TYPE;
|
||||
}
|
||||
|
||||
public static final class Factory extends AbstractProcessorFactory<SetProcessor> {
|
||||
|
||||
private final TemplateService templateService;
|
||||
|
||||
public Factory(TemplateService templateService) {
|
||||
this.templateService = templateService;
|
||||
}
|
||||
|
||||
@Override
|
||||
public SetProcessor doCreate(String processorTag, Map<String, Object> config) throws Exception {
|
||||
String field = ConfigurationUtils.readStringProperty(config, "field");
|
||||
Object value = ConfigurationUtils.readObject(config, "value");
|
||||
return new SetProcessor(processorTag, templateService.compile(field), ValueSource.wrap(value, templateService));
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,82 @@
|
||||
/*
|
||||
* 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.processor;
|
||||
|
||||
import org.elasticsearch.ingest.core.AbstractProcessor;
|
||||
import org.elasticsearch.ingest.core.AbstractProcessorFactory;
|
||||
import org.elasticsearch.ingest.core.ConfigurationUtils;
|
||||
import org.elasticsearch.ingest.core.IngestDocument;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* Processor that splits fields content into different items based on the occurrence of a specified separator.
|
||||
* New field value will be an array containing all of the different extracted items.
|
||||
* Throws exception if the field is null or a type other than string.
|
||||
*/
|
||||
public class SplitProcessor extends AbstractProcessor {
|
||||
|
||||
public static final String TYPE = "split";
|
||||
|
||||
private final String field;
|
||||
private final String separator;
|
||||
|
||||
SplitProcessor(String tag, String field, String separator) {
|
||||
super(tag);
|
||||
this.field = field;
|
||||
this.separator = separator;
|
||||
}
|
||||
|
||||
String getField() {
|
||||
return field;
|
||||
}
|
||||
|
||||
String getSeparator() {
|
||||
return separator;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void execute(IngestDocument document) {
|
||||
String oldVal = document.getFieldValue(field, String.class);
|
||||
if (oldVal == null) {
|
||||
throw new IllegalArgumentException("field [" + field + "] is null, cannot split.");
|
||||
}
|
||||
String[] strings = oldVal.split(separator);
|
||||
List<String> splitList = new ArrayList<>(strings.length);
|
||||
Collections.addAll(splitList, strings);
|
||||
document.setFieldValue(field, splitList);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getType() {
|
||||
return TYPE;
|
||||
}
|
||||
|
||||
public static class Factory extends AbstractProcessorFactory<SplitProcessor> {
|
||||
@Override
|
||||
public SplitProcessor doCreate(String processorTag, Map<String, Object> config) throws Exception {
|
||||
String field = ConfigurationUtils.readStringProperty(config, "field");
|
||||
return new SplitProcessor(processorTag, field, ConfigurationUtils.readStringProperty(config, "separator"));
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,51 @@
|
||||
/*
|
||||
* 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.processor;
|
||||
|
||||
/**
|
||||
* Processor that trims the content of string fields.
|
||||
* Throws exception is the field is not of type string.
|
||||
*/
|
||||
public class TrimProcessor extends AbstractStringProcessor {
|
||||
|
||||
public static final String TYPE = "trim";
|
||||
|
||||
TrimProcessor(String processorTag, String field) {
|
||||
super(processorTag, field);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected String process(String value) {
|
||||
return value.trim();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getType() {
|
||||
return TYPE;
|
||||
}
|
||||
|
||||
public static class Factory extends AbstractStringProcessor.Factory<TrimProcessor> {
|
||||
@Override
|
||||
protected TrimProcessor newProcessor(String tag, String field) {
|
||||
return new TrimProcessor(tag, field);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -0,0 +1,53 @@
|
||||
/*
|
||||
* 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.processor;
|
||||
|
||||
import java.util.Locale;
|
||||
|
||||
/**
|
||||
* Processor that converts the content of string fields to uppercase.
|
||||
* Throws exception is the field is not of type string.
|
||||
*/
|
||||
public class UppercaseProcessor extends AbstractStringProcessor {
|
||||
|
||||
public static final String TYPE = "uppercase";
|
||||
|
||||
UppercaseProcessor(String processorTag, String field) {
|
||||
super(processorTag, field);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected String process(String value) {
|
||||
return value.toUpperCase(Locale.ROOT);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getType() {
|
||||
return TYPE;
|
||||
}
|
||||
|
||||
public static class Factory extends AbstractStringProcessor.Factory<UppercaseProcessor> {
|
||||
@Override
|
||||
protected UppercaseProcessor newProcessor(String tag, String field) {
|
||||
return new UppercaseProcessor(tag, field);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -30,6 +30,7 @@ import org.elasticsearch.cluster.ClusterModule;
|
||||
import org.elasticsearch.cluster.ClusterNameModule;
|
||||
import org.elasticsearch.cluster.ClusterService;
|
||||
import org.elasticsearch.cluster.action.index.MappingUpdatedAction;
|
||||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||
import org.elasticsearch.cluster.routing.RoutingService;
|
||||
import org.elasticsearch.common.StopWatch;
|
||||
import org.elasticsearch.common.component.Lifecycle;
|
||||
@ -76,6 +77,7 @@ import org.elasticsearch.indices.ttl.IndicesTTLService;
|
||||
import org.elasticsearch.monitor.MonitorService;
|
||||
import org.elasticsearch.monitor.jvm.JvmInfo;
|
||||
import org.elasticsearch.node.internal.InternalSettingsPreparer;
|
||||
import org.elasticsearch.node.service.NodeService;
|
||||
import org.elasticsearch.percolator.PercolatorModule;
|
||||
import org.elasticsearch.percolator.PercolatorService;
|
||||
import org.elasticsearch.plugins.Plugin;
|
||||
@ -119,6 +121,7 @@ import static org.elasticsearch.common.settings.Settings.settingsBuilder;
|
||||
*/
|
||||
public class Node implements Releasable {
|
||||
|
||||
public static final Setting<Boolean> NODE_INGEST_SETTING = Setting.boolSetting("node.ingest", true, false, Setting.Scope.CLUSTER);
|
||||
private static final String CLIENT_TYPE = "node";
|
||||
public static final Setting<Boolean> WRITE_PORTS_FIELD_SETTING = Setting.boolSetting("node.portsfile", false, false, Setting.Scope.CLUSTER);
|
||||
private final Lifecycle lifecycle = new Lifecycle();
|
||||
@ -190,7 +193,7 @@ public class Node implements Releasable {
|
||||
modules.add(new ClusterModule(this.settings));
|
||||
modules.add(new IndicesModule());
|
||||
modules.add(new SearchModule(settings, namedWriteableRegistry));
|
||||
modules.add(new ActionModule(false));
|
||||
modules.add(new ActionModule(DiscoveryNode.ingestNode(settings), false));
|
||||
modules.add(new GatewayModule(settings));
|
||||
modules.add(new NodeClientModule());
|
||||
modules.add(new PercolatorModule());
|
||||
@ -232,6 +235,13 @@ public class Node implements Releasable {
|
||||
return client;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the environment of the node
|
||||
*/
|
||||
public Environment getEnvironment() {
|
||||
return environment;
|
||||
}
|
||||
|
||||
/**
|
||||
* Start the node. If the node is already started, this method is no-op.
|
||||
*/
|
||||
@ -347,6 +357,12 @@ public class Node implements Releasable {
|
||||
StopWatch stopWatch = new StopWatch("node_close");
|
||||
stopWatch.start("tribe");
|
||||
injector.getInstance(TribeService.class).close();
|
||||
stopWatch.stop().start("node_service");
|
||||
try {
|
||||
injector.getInstance(NodeService.class).close();
|
||||
} catch (IOException e) {
|
||||
logger.warn("NodeService close failed", e);
|
||||
}
|
||||
stopWatch.stop().start("http");
|
||||
if (settings.getAsBoolean("http.enabled", true)) {
|
||||
injector.getInstance(HttpServer.class).close();
|
||||
|
@ -22,9 +22,28 @@ package org.elasticsearch.node;
|
||||
import org.elasticsearch.cache.recycler.PageCacheRecycler;
|
||||
import org.elasticsearch.common.inject.AbstractModule;
|
||||
import org.elasticsearch.common.util.BigArrays;
|
||||
import org.elasticsearch.ingest.ProcessorsRegistry;
|
||||
import org.elasticsearch.ingest.core.Processor;
|
||||
import org.elasticsearch.ingest.core.TemplateService;
|
||||
import org.elasticsearch.ingest.processor.AppendProcessor;
|
||||
import org.elasticsearch.ingest.processor.ConvertProcessor;
|
||||
import org.elasticsearch.ingest.processor.DateProcessor;
|
||||
import org.elasticsearch.ingest.processor.DeDotProcessor;
|
||||
import org.elasticsearch.ingest.processor.FailProcessor;
|
||||
import org.elasticsearch.ingest.processor.GsubProcessor;
|
||||
import org.elasticsearch.ingest.processor.JoinProcessor;
|
||||
import org.elasticsearch.ingest.processor.LowercaseProcessor;
|
||||
import org.elasticsearch.ingest.processor.RemoveProcessor;
|
||||
import org.elasticsearch.ingest.processor.RenameProcessor;
|
||||
import org.elasticsearch.ingest.processor.SetProcessor;
|
||||
import org.elasticsearch.ingest.processor.SplitProcessor;
|
||||
import org.elasticsearch.ingest.processor.TrimProcessor;
|
||||
import org.elasticsearch.ingest.processor.UppercaseProcessor;
|
||||
import org.elasticsearch.monitor.MonitorService;
|
||||
import org.elasticsearch.node.service.NodeService;
|
||||
|
||||
import java.util.function.Function;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
@ -32,6 +51,7 @@ public class NodeModule extends AbstractModule {
|
||||
|
||||
private final Node node;
|
||||
private final MonitorService monitorService;
|
||||
private final ProcessorsRegistry processorsRegistry;
|
||||
|
||||
// pkg private so tests can mock
|
||||
Class<? extends PageCacheRecycler> pageCacheRecyclerImpl = PageCacheRecycler.class;
|
||||
@ -40,6 +60,22 @@ public class NodeModule extends AbstractModule {
|
||||
public NodeModule(Node node, MonitorService monitorService) {
|
||||
this.node = node;
|
||||
this.monitorService = monitorService;
|
||||
this.processorsRegistry = new ProcessorsRegistry();
|
||||
|
||||
registerProcessor(DateProcessor.TYPE, (templateService) -> new DateProcessor.Factory());
|
||||
registerProcessor(SetProcessor.TYPE, SetProcessor.Factory::new);
|
||||
registerProcessor(AppendProcessor.TYPE, AppendProcessor.Factory::new);
|
||||
registerProcessor(RenameProcessor.TYPE, (templateService) -> new RenameProcessor.Factory());
|
||||
registerProcessor(RemoveProcessor.TYPE, RemoveProcessor.Factory::new);
|
||||
registerProcessor(SplitProcessor.TYPE, (templateService) -> new SplitProcessor.Factory());
|
||||
registerProcessor(JoinProcessor.TYPE, (templateService) -> new JoinProcessor.Factory());
|
||||
registerProcessor(UppercaseProcessor.TYPE, (templateService) -> new UppercaseProcessor.Factory());
|
||||
registerProcessor(LowercaseProcessor.TYPE, (templateService) -> new LowercaseProcessor.Factory());
|
||||
registerProcessor(TrimProcessor.TYPE, (templateService) -> new TrimProcessor.Factory());
|
||||
registerProcessor(ConvertProcessor.TYPE, (templateService) -> new ConvertProcessor.Factory());
|
||||
registerProcessor(GsubProcessor.TYPE, (templateService) -> new GsubProcessor.Factory());
|
||||
registerProcessor(FailProcessor.TYPE, FailProcessor.Factory::new);
|
||||
registerProcessor(DeDotProcessor.TYPE, (templateService) -> new DeDotProcessor.Factory());
|
||||
}
|
||||
|
||||
@Override
|
||||
@ -58,5 +94,20 @@ public class NodeModule extends AbstractModule {
|
||||
bind(Node.class).toInstance(node);
|
||||
bind(MonitorService.class).toInstance(monitorService);
|
||||
bind(NodeService.class).asEagerSingleton();
|
||||
bind(ProcessorsRegistry.class).toInstance(processorsRegistry);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the node
|
||||
*/
|
||||
public Node getNode() {
|
||||
return node;
|
||||
}
|
||||
|
||||
/**
|
||||
* Adds a processor factory under a specific type name.
|
||||
*/
|
||||
public void registerProcessor(String type, Function<TemplateService, Processor.Factory<?>> processorFactoryProvider) {
|
||||
processorsRegistry.registerProcessor(type, processorFactoryProvider);
|
||||
}
|
||||
}
|
||||
|
@ -24,20 +24,25 @@ import org.elasticsearch.Version;
|
||||
import org.elasticsearch.action.admin.cluster.node.info.NodeInfo;
|
||||
import org.elasticsearch.action.admin.cluster.node.stats.NodeStats;
|
||||
import org.elasticsearch.action.admin.indices.stats.CommonStatsFlags;
|
||||
import org.elasticsearch.cluster.ClusterService;
|
||||
import org.elasticsearch.common.Nullable;
|
||||
import org.elasticsearch.common.component.AbstractComponent;
|
||||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.discovery.Discovery;
|
||||
import org.elasticsearch.env.Environment;
|
||||
import org.elasticsearch.http.HttpServer;
|
||||
import org.elasticsearch.indices.IndicesService;
|
||||
import org.elasticsearch.indices.breaker.CircuitBreakerService;
|
||||
import org.elasticsearch.ingest.IngestService;
|
||||
import org.elasticsearch.ingest.ProcessorsRegistry;
|
||||
import org.elasticsearch.monitor.MonitorService;
|
||||
import org.elasticsearch.plugins.PluginsService;
|
||||
import org.elasticsearch.script.ScriptService;
|
||||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
import org.elasticsearch.transport.TransportService;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
@ -47,7 +52,7 @@ import static java.util.Collections.unmodifiableMap;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class NodeService extends AbstractComponent {
|
||||
public class NodeService extends AbstractComponent implements Closeable {
|
||||
|
||||
private final ThreadPool threadPool;
|
||||
private final MonitorService monitorService;
|
||||
@ -55,6 +60,7 @@ public class NodeService extends AbstractComponent {
|
||||
private final IndicesService indicesService;
|
||||
private final PluginsService pluginService;
|
||||
private final CircuitBreakerService circuitBreakerService;
|
||||
private final IngestService ingestService;
|
||||
private ScriptService scriptService;
|
||||
|
||||
@Nullable
|
||||
@ -67,10 +73,10 @@ public class NodeService extends AbstractComponent {
|
||||
private final Discovery discovery;
|
||||
|
||||
@Inject
|
||||
public NodeService(Settings settings, ThreadPool threadPool, MonitorService monitorService, Discovery discovery,
|
||||
TransportService transportService, IndicesService indicesService,
|
||||
PluginsService pluginService, CircuitBreakerService circuitBreakerService,
|
||||
Version version) {
|
||||
public NodeService(Settings settings, Environment environment, ThreadPool threadPool, MonitorService monitorService,
|
||||
Discovery discovery, TransportService transportService, IndicesService indicesService,
|
||||
PluginsService pluginService, CircuitBreakerService circuitBreakerService, Version version,
|
||||
ProcessorsRegistry processorsRegistry, ClusterService clusterService) {
|
||||
super(settings);
|
||||
this.threadPool = threadPool;
|
||||
this.monitorService = monitorService;
|
||||
@ -81,12 +87,15 @@ public class NodeService extends AbstractComponent {
|
||||
this.version = version;
|
||||
this.pluginService = pluginService;
|
||||
this.circuitBreakerService = circuitBreakerService;
|
||||
this.ingestService = new IngestService(settings, threadPool, processorsRegistry);
|
||||
clusterService.add(ingestService.getPipelineStore());
|
||||
}
|
||||
|
||||
// can not use constructor injection or there will be a circular dependency
|
||||
@Inject(optional = true)
|
||||
public void setScriptService(ScriptService scriptService) {
|
||||
this.scriptService = scriptService;
|
||||
this.ingestService.setScriptService(scriptService);
|
||||
}
|
||||
|
||||
public void setHttpServer(@Nullable HttpServer httpServer) {
|
||||
@ -176,4 +185,13 @@ public class NodeService extends AbstractComponent {
|
||||
discoveryStats ? discovery.stats() : null
|
||||
);
|
||||
}
|
||||
|
||||
public IngestService getIngestService() {
|
||||
return ingestService;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
indicesService.close();
|
||||
}
|
||||
}
|
||||
|
@ -101,6 +101,7 @@ public class PluginManager {
|
||||
"discovery-ec2",
|
||||
"discovery-gce",
|
||||
"discovery-multicast",
|
||||
"ingest-geoip",
|
||||
"lang-javascript",
|
||||
"lang-plan-a",
|
||||
"lang-python",
|
||||
|
@ -77,6 +77,7 @@ public class RestBulkAction extends BaseRestHandler {
|
||||
String defaultType = request.param("type");
|
||||
String defaultRouting = request.param("routing");
|
||||
String fieldsParam = request.param("fields");
|
||||
String defaultPipeline = request.param("pipeline");
|
||||
String[] defaultFields = fieldsParam != null ? Strings.commaDelimitedListToStringArray(fieldsParam) : null;
|
||||
|
||||
String consistencyLevel = request.param("consistency");
|
||||
@ -85,7 +86,7 @@ public class RestBulkAction extends BaseRestHandler {
|
||||
}
|
||||
bulkRequest.timeout(request.paramAsTime("timeout", BulkShardRequest.DEFAULT_TIMEOUT));
|
||||
bulkRequest.refresh(request.paramAsBoolean("refresh", bulkRequest.refresh()));
|
||||
bulkRequest.add(request.content(), defaultIndex, defaultType, defaultRouting, defaultFields, null, allowExplicitIndex);
|
||||
bulkRequest.add(request.content(), defaultIndex, defaultType, defaultRouting, defaultFields, defaultPipeline, null, allowExplicitIndex);
|
||||
|
||||
client.bulk(bulkRequest, new RestBuilderListener<BulkResponse>(channel) {
|
||||
@Override
|
||||
|
@ -77,6 +77,7 @@ public class RestIndexAction extends BaseRestHandler {
|
||||
if (request.hasParam("ttl")) {
|
||||
indexRequest.ttl(request.param("ttl"));
|
||||
}
|
||||
indexRequest.setPipeline(request.param("pipeline"));
|
||||
indexRequest.source(request.content());
|
||||
indexRequest.timeout(request.paramAsTime("timeout", IndexRequest.DEFAULT_TIMEOUT));
|
||||
indexRequest.refresh(request.paramAsBoolean("refresh", indexRequest.refresh()));
|
||||
|
@ -0,0 +1,47 @@
|
||||
/*
|
||||
* 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.rest.action.ingest;
|
||||
|
||||
import org.elasticsearch.action.ingest.DeletePipelineRequest;
|
||||
import org.elasticsearch.client.Client;
|
||||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.rest.BaseRestHandler;
|
||||
import org.elasticsearch.rest.RestChannel;
|
||||
import org.elasticsearch.rest.RestController;
|
||||
import org.elasticsearch.rest.RestRequest;
|
||||
import org.elasticsearch.rest.action.support.AcknowledgedRestListener;
|
||||
|
||||
public class RestDeletePipelineAction extends BaseRestHandler {
|
||||
|
||||
@Inject
|
||||
public RestDeletePipelineAction(Settings settings, RestController controller, Client client) {
|
||||
super(settings, controller, client);
|
||||
controller.registerHandler(RestRequest.Method.DELETE, "/_ingest/pipeline/{id}", this);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void handleRequest(RestRequest restRequest, RestChannel channel, Client client) throws Exception {
|
||||
DeletePipelineRequest request = new DeletePipelineRequest(restRequest.param("id"));
|
||||
request.masterNodeTimeout(restRequest.paramAsTime("master_timeout", request.masterNodeTimeout()));
|
||||
request.timeout(restRequest.paramAsTime("timeout", request.timeout()));
|
||||
client.deletePipeline(request, new AcknowledgedRestListener<>(channel));
|
||||
}
|
||||
}
|
@ -0,0 +1,47 @@
|
||||
/*
|
||||
* 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.rest.action.ingest;
|
||||
|
||||
import org.elasticsearch.action.ingest.GetPipelineRequest;
|
||||
import org.elasticsearch.client.Client;
|
||||
import org.elasticsearch.common.Strings;
|
||||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.rest.BaseRestHandler;
|
||||
import org.elasticsearch.rest.RestChannel;
|
||||
import org.elasticsearch.rest.RestController;
|
||||
import org.elasticsearch.rest.RestRequest;
|
||||
import org.elasticsearch.rest.action.support.RestStatusToXContentListener;
|
||||
|
||||
public class RestGetPipelineAction extends BaseRestHandler {
|
||||
|
||||
@Inject
|
||||
public RestGetPipelineAction(Settings settings, RestController controller, Client client) {
|
||||
super(settings, controller, client);
|
||||
controller.registerHandler(RestRequest.Method.GET, "/_ingest/pipeline/{id}", this);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void handleRequest(RestRequest restRequest, RestChannel channel, Client client) throws Exception {
|
||||
GetPipelineRequest request = new GetPipelineRequest(Strings.splitStringByCommaToArray(restRequest.param("id")));
|
||||
request.masterNodeTimeout(restRequest.paramAsTime("master_timeout", request.masterNodeTimeout()));
|
||||
client.getPipeline(request, new RestStatusToXContentListener<>(channel));
|
||||
}
|
||||
}
|
@ -0,0 +1,48 @@
|
||||
/*
|
||||
* 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.rest.action.ingest;
|
||||
|
||||
import org.elasticsearch.action.ingest.PutPipelineRequest;
|
||||
import org.elasticsearch.client.Client;
|
||||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.rest.BaseRestHandler;
|
||||
import org.elasticsearch.rest.RestChannel;
|
||||
import org.elasticsearch.rest.RestController;
|
||||
import org.elasticsearch.rest.RestRequest;
|
||||
import org.elasticsearch.rest.action.support.AcknowledgedRestListener;
|
||||
import org.elasticsearch.rest.action.support.RestActions;
|
||||
|
||||
public class RestPutPipelineAction extends BaseRestHandler {
|
||||
|
||||
@Inject
|
||||
public RestPutPipelineAction(Settings settings, RestController controller, Client client) {
|
||||
super(settings, controller, client);
|
||||
controller.registerHandler(RestRequest.Method.PUT, "/_ingest/pipeline/{id}", this);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void handleRequest(RestRequest restRequest, RestChannel channel, Client client) throws Exception {
|
||||
PutPipelineRequest request = new PutPipelineRequest(restRequest.param("id"), RestActions.getRestContent(restRequest));
|
||||
request.masterNodeTimeout(restRequest.paramAsTime("master_timeout", request.masterNodeTimeout()));
|
||||
request.timeout(restRequest.paramAsTime("timeout", request.timeout()));
|
||||
client.putPipeline(request, new AcknowledgedRestListener<>(channel));
|
||||
}
|
||||
}
|
@ -0,0 +1,51 @@
|
||||
/*
|
||||
* 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.rest.action.ingest;
|
||||
|
||||
import org.elasticsearch.action.ingest.SimulatePipelineRequest;
|
||||
import org.elasticsearch.client.Client;
|
||||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.rest.BaseRestHandler;
|
||||
import org.elasticsearch.rest.RestChannel;
|
||||
import org.elasticsearch.rest.RestController;
|
||||
import org.elasticsearch.rest.RestRequest;
|
||||
import org.elasticsearch.rest.action.support.RestActions;
|
||||
import org.elasticsearch.rest.action.support.RestToXContentListener;
|
||||
|
||||
public class RestSimulatePipelineAction extends BaseRestHandler {
|
||||
|
||||
@Inject
|
||||
public RestSimulatePipelineAction(Settings settings, RestController controller, Client client) {
|
||||
super(settings, controller, client);
|
||||
controller.registerHandler(RestRequest.Method.POST, "/_ingest/pipeline/{id}/_simulate", this);
|
||||
controller.registerHandler(RestRequest.Method.GET, "/_ingest/pipeline/{id}/_simulate", this);
|
||||
controller.registerHandler(RestRequest.Method.POST, "/_ingest/pipeline/_simulate", this);
|
||||
controller.registerHandler(RestRequest.Method.GET, "/_ingest/pipeline/_simulate", this);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void handleRequest(RestRequest restRequest, RestChannel channel, Client client) throws Exception {
|
||||
SimulatePipelineRequest request = new SimulatePipelineRequest(RestActions.getRestContent(restRequest));
|
||||
request.setId(restRequest.param("id"));
|
||||
request.setVerbose(restRequest.paramAsBoolean("verbose", false));
|
||||
client.simulatePipeline(request, new RestToXContentListener<>(channel));
|
||||
}
|
||||
}
|
@ -37,7 +37,7 @@ public interface ScriptContext {
|
||||
*/
|
||||
enum Standard implements ScriptContext {
|
||||
|
||||
AGGS("aggs"), SEARCH("search"), UPDATE("update");
|
||||
AGGS("aggs"), SEARCH("search"), UPDATE("update"), INGEST("ingest");
|
||||
|
||||
private final String key;
|
||||
|
||||
|
@ -43,6 +43,7 @@ OFFICIAL PLUGINS
|
||||
- discovery-ec2
|
||||
- discovery-gce
|
||||
- discovery-multicast
|
||||
- ingest-geoip
|
||||
- lang-javascript
|
||||
- lang-plan-a
|
||||
- lang-python
|
||||
|
@ -0,0 +1,165 @@
|
||||
package org.elasticsearch.action.ingest;
|
||||
|
||||
/*
|
||||
* 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.
|
||||
*/
|
||||
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.action.ActionRequest;
|
||||
import org.elasticsearch.action.bulk.BulkItemResponse;
|
||||
import org.elasticsearch.action.bulk.BulkRequest;
|
||||
import org.elasticsearch.action.bulk.BulkResponse;
|
||||
import org.elasticsearch.action.index.IndexRequest;
|
||||
import org.elasticsearch.action.index.IndexResponse;
|
||||
import org.elasticsearch.index.shard.ShardId;
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
import org.hamcrest.Matchers;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
|
||||
import static org.hamcrest.Matchers.equalTo;
|
||||
import static org.hamcrest.Matchers.is;
|
||||
import static org.hamcrest.Matchers.nullValue;
|
||||
import static org.mockito.Mockito.mock;
|
||||
|
||||
public class BulkRequestModifierTests extends ESTestCase {
|
||||
|
||||
public void testBulkRequestModifier() {
|
||||
int numRequests = scaledRandomIntBetween(8, 64);
|
||||
BulkRequest bulkRequest = new BulkRequest();
|
||||
for (int i = 0; i < numRequests; i++) {
|
||||
bulkRequest.add(new IndexRequest("_index", "_type", String.valueOf(i)).source("{}"));
|
||||
}
|
||||
CaptureActionListener actionListener = new CaptureActionListener();
|
||||
IngestActionFilter.BulkRequestModifier bulkRequestModifier = new IngestActionFilter.BulkRequestModifier(bulkRequest);
|
||||
|
||||
int i = 0;
|
||||
Set<Integer> failedSlots = new HashSet<>();
|
||||
while (bulkRequestModifier.hasNext()) {
|
||||
bulkRequestModifier.next();
|
||||
if (randomBoolean()) {
|
||||
bulkRequestModifier.markCurrentItemAsFailed(new RuntimeException());
|
||||
failedSlots.add(i);
|
||||
}
|
||||
i++;
|
||||
}
|
||||
|
||||
assertThat(bulkRequestModifier.getBulkRequest().requests().size(), equalTo(numRequests - failedSlots.size()));
|
||||
// simulate that we actually executed the modified bulk request:
|
||||
ActionListener<BulkResponse> result = bulkRequestModifier.wrapActionListenerIfNeeded(actionListener);
|
||||
result.onResponse(new BulkResponse(new BulkItemResponse[numRequests - failedSlots.size()], 0));
|
||||
|
||||
BulkResponse bulkResponse = actionListener.getResponse();
|
||||
for (int j = 0; j < bulkResponse.getItems().length; j++) {
|
||||
if (failedSlots.contains(j)) {
|
||||
BulkItemResponse item = bulkResponse.getItems()[j];
|
||||
assertThat(item.isFailed(), is(true));
|
||||
assertThat(item.getFailure().getIndex(), equalTo("_index"));
|
||||
assertThat(item.getFailure().getType(), equalTo("_type"));
|
||||
assertThat(item.getFailure().getId(), equalTo(String.valueOf(j)));
|
||||
assertThat(item.getFailure().getMessage(), equalTo("java.lang.RuntimeException"));
|
||||
} else {
|
||||
assertThat(bulkResponse.getItems()[j], nullValue());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public void testPipelineFailures() {
|
||||
BulkRequest originalBulkRequest = new BulkRequest();
|
||||
for (int i = 0; i < 32; i++) {
|
||||
originalBulkRequest.add(new IndexRequest("index", "type", String.valueOf(i)));
|
||||
}
|
||||
|
||||
IngestActionFilter.BulkRequestModifier modifier = new IngestActionFilter.BulkRequestModifier(originalBulkRequest);
|
||||
for (int i = 0; modifier.hasNext(); i++) {
|
||||
modifier.next();
|
||||
if (i % 2 == 0) {
|
||||
modifier.markCurrentItemAsFailed(new RuntimeException());
|
||||
}
|
||||
}
|
||||
|
||||
// So half of the requests have "failed", so only the successful requests are left:
|
||||
BulkRequest bulkRequest = modifier.getBulkRequest();
|
||||
assertThat(bulkRequest.requests().size(), Matchers.equalTo(16));
|
||||
|
||||
List<BulkItemResponse> responses = new ArrayList<>();
|
||||
ActionListener<BulkResponse> bulkResponseListener = modifier.wrapActionListenerIfNeeded(new ActionListener<BulkResponse>() {
|
||||
@Override
|
||||
public void onResponse(BulkResponse bulkItemResponses) {
|
||||
responses.addAll(Arrays.asList(bulkItemResponses.getItems()));
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onFailure(Throwable e) {
|
||||
}
|
||||
});
|
||||
|
||||
List<BulkItemResponse> originalResponses = new ArrayList<>();
|
||||
for (ActionRequest actionRequest : bulkRequest.requests()) {
|
||||
IndexRequest indexRequest = (IndexRequest) actionRequest;
|
||||
IndexResponse indexResponse = new IndexResponse(new ShardId("index", 0), indexRequest.type(), indexRequest.id(), 1, true);
|
||||
originalResponses.add(new BulkItemResponse(Integer.parseInt(indexRequest.id()), indexRequest.opType().lowercase(), indexResponse));
|
||||
}
|
||||
bulkResponseListener.onResponse(new BulkResponse(originalResponses.toArray(new BulkItemResponse[originalResponses.size()]), 0));
|
||||
|
||||
assertThat(responses.size(), Matchers.equalTo(32));
|
||||
for (int i = 0; i < 32; i++) {
|
||||
assertThat(responses.get(i).getId(), Matchers.equalTo(String.valueOf(i)));
|
||||
}
|
||||
}
|
||||
|
||||
public void testNoFailures() {
|
||||
BulkRequest originalBulkRequest = new BulkRequest();
|
||||
for (int i = 0; i < 32; i++) {
|
||||
originalBulkRequest.add(new IndexRequest("index", "type", String.valueOf(i)));
|
||||
}
|
||||
|
||||
IngestActionFilter.BulkRequestModifier modifier = new IngestActionFilter.BulkRequestModifier(originalBulkRequest);
|
||||
while (modifier.hasNext()) {
|
||||
modifier.next();
|
||||
}
|
||||
|
||||
BulkRequest bulkRequest = modifier.getBulkRequest();
|
||||
assertThat(bulkRequest, Matchers.sameInstance(originalBulkRequest));
|
||||
@SuppressWarnings("unchecked")
|
||||
ActionListener<BulkResponse> actionListener = mock(ActionListener.class);
|
||||
assertThat(modifier.wrapActionListenerIfNeeded(actionListener), Matchers.sameInstance(actionListener));
|
||||
}
|
||||
|
||||
private static class CaptureActionListener implements ActionListener<BulkResponse> {
|
||||
|
||||
private BulkResponse response;
|
||||
|
||||
@Override
|
||||
public void onResponse(BulkResponse bulkItemResponses) {
|
||||
this.response = bulkItemResponses ;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onFailure(Throwable e) {
|
||||
}
|
||||
|
||||
public BulkResponse getResponse() {
|
||||
return response;
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,249 @@
|
||||
/*
|
||||
* 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.action.ingest;
|
||||
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.action.ActionRequest;
|
||||
import org.elasticsearch.action.bulk.BulkAction;
|
||||
import org.elasticsearch.action.bulk.BulkRequest;
|
||||
import org.elasticsearch.action.delete.DeleteRequest;
|
||||
import org.elasticsearch.action.index.IndexAction;
|
||||
import org.elasticsearch.action.index.IndexRequest;
|
||||
import org.elasticsearch.action.support.ActionFilterChain;
|
||||
import org.elasticsearch.action.update.UpdateRequest;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.ingest.IngestService;
|
||||
import org.elasticsearch.ingest.PipelineExecutionService;
|
||||
import org.elasticsearch.ingest.PipelineStore;
|
||||
import org.elasticsearch.ingest.core.CompoundProcessor;
|
||||
import org.elasticsearch.ingest.core.IngestDocument;
|
||||
import org.elasticsearch.ingest.core.Pipeline;
|
||||
import org.elasticsearch.ingest.core.Processor;
|
||||
import org.elasticsearch.node.service.NodeService;
|
||||
import org.elasticsearch.tasks.Task;
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
import org.junit.Before;
|
||||
import org.mockito.stubbing.Answer;
|
||||
|
||||
import java.util.function.Consumer;
|
||||
|
||||
import static org.hamcrest.Matchers.equalTo;
|
||||
import static org.hamcrest.Matchers.nullValue;
|
||||
import static org.mockito.Matchers.same;
|
||||
import static org.mockito.Mockito.any;
|
||||
import static org.mockito.Mockito.doAnswer;
|
||||
import static org.mockito.Mockito.mock;
|
||||
import static org.mockito.Mockito.times;
|
||||
import static org.mockito.Mockito.verify;
|
||||
import static org.mockito.Mockito.verifyZeroInteractions;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
public class IngestActionFilterTests extends ESTestCase {
|
||||
|
||||
private IngestActionFilter filter;
|
||||
private PipelineExecutionService executionService;
|
||||
|
||||
@Before
|
||||
public void setup() {
|
||||
executionService = mock(PipelineExecutionService.class);
|
||||
IngestService ingestService = mock(IngestService.class);
|
||||
when(ingestService.getPipelineExecutionService()).thenReturn(executionService);
|
||||
NodeService nodeService = mock(NodeService.class);
|
||||
when(nodeService.getIngestService()).thenReturn(ingestService);
|
||||
filter = new IngestActionFilter(Settings.EMPTY, nodeService);
|
||||
}
|
||||
|
||||
public void testApplyNoPipelineId() throws Exception {
|
||||
IndexRequest indexRequest = new IndexRequest();
|
||||
Task task = mock(Task.class);
|
||||
ActionListener actionListener = mock(ActionListener.class);
|
||||
ActionFilterChain actionFilterChain = mock(ActionFilterChain.class);
|
||||
|
||||
filter.apply(task, IndexAction.NAME, indexRequest, actionListener, actionFilterChain);
|
||||
|
||||
verify(actionFilterChain).proceed(task, IndexAction.NAME, indexRequest, actionListener);
|
||||
verifyZeroInteractions(executionService, actionFilterChain);
|
||||
}
|
||||
|
||||
public void testApplyBulkNoPipelineId() throws Exception {
|
||||
BulkRequest bulkRequest = new BulkRequest();
|
||||
bulkRequest.add(new IndexRequest());
|
||||
Task task = mock(Task.class);
|
||||
ActionListener actionListener = mock(ActionListener.class);
|
||||
ActionFilterChain actionFilterChain = mock(ActionFilterChain.class);
|
||||
|
||||
filter.apply(task, BulkAction.NAME, bulkRequest, actionListener, actionFilterChain);
|
||||
|
||||
verify(actionFilterChain).proceed(task, BulkAction.NAME, bulkRequest, actionListener);
|
||||
verifyZeroInteractions(executionService, actionFilterChain);
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
public void testApplyIngestIdViaRequestParam() throws Exception {
|
||||
Task task = mock(Task.class);
|
||||
IndexRequest indexRequest = new IndexRequest("_index", "_type", "_id").setPipeline("_id");
|
||||
indexRequest.source("field", "value");
|
||||
ActionListener actionListener = mock(ActionListener.class);
|
||||
ActionFilterChain actionFilterChain = mock(ActionFilterChain.class);
|
||||
|
||||
filter.apply(task, IndexAction.NAME, indexRequest, actionListener, actionFilterChain);
|
||||
|
||||
verify(executionService).execute(same(indexRequest), any(Consumer.class), any(Consumer.class));
|
||||
verifyZeroInteractions(actionFilterChain);
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
public void testApplyExecuted() throws Exception {
|
||||
Task task = mock(Task.class);
|
||||
IndexRequest indexRequest = new IndexRequest("_index", "_type", "_id").setPipeline("_id");
|
||||
indexRequest.source("field", "value");
|
||||
ActionListener actionListener = mock(ActionListener.class);
|
||||
ActionFilterChain actionFilterChain = mock(ActionFilterChain.class);
|
||||
|
||||
Answer answer = invocationOnMock -> {
|
||||
@SuppressWarnings("unchecked")
|
||||
Consumer<Boolean> listener = (Consumer) invocationOnMock.getArguments()[2];
|
||||
listener.accept(true);
|
||||
return null;
|
||||
};
|
||||
doAnswer(answer).when(executionService).execute(any(IndexRequest.class), any(Consumer.class), any(Consumer.class));
|
||||
filter.apply(task, IndexAction.NAME, indexRequest, actionListener, actionFilterChain);
|
||||
|
||||
verify(executionService).execute(same(indexRequest), any(Consumer.class), any(Consumer.class));
|
||||
verify(actionFilterChain).proceed(task, IndexAction.NAME, indexRequest, actionListener);
|
||||
verifyZeroInteractions(actionListener);
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
public void testApplyFailed() throws Exception {
|
||||
Task task = mock(Task.class);
|
||||
IndexRequest indexRequest = new IndexRequest("_index", "_type", "_id").setPipeline("_id");
|
||||
indexRequest.source("field", "value");
|
||||
ActionListener actionListener = mock(ActionListener.class);
|
||||
ActionFilterChain actionFilterChain = mock(ActionFilterChain.class);
|
||||
|
||||
RuntimeException exception = new RuntimeException();
|
||||
Answer answer = invocationOnMock -> {
|
||||
Consumer<Throwable> handler = (Consumer) invocationOnMock.getArguments()[1];
|
||||
handler.accept(exception);
|
||||
return null;
|
||||
};
|
||||
doAnswer(answer).when(executionService).execute(same(indexRequest), any(Consumer.class), any(Consumer.class));
|
||||
filter.apply(task, IndexAction.NAME, indexRequest, actionListener, actionFilterChain);
|
||||
|
||||
verify(executionService).execute(same(indexRequest), any(Consumer.class), any(Consumer.class));
|
||||
verify(actionListener).onFailure(exception);
|
||||
verifyZeroInteractions(actionFilterChain);
|
||||
}
|
||||
|
||||
public void testApplyWithBulkRequest() throws Exception {
|
||||
Task task = mock(Task.class);
|
||||
ThreadPool threadPool = mock(ThreadPool.class);
|
||||
when(threadPool.executor(any())).thenReturn(Runnable::run);
|
||||
PipelineStore store = mock(PipelineStore.class);
|
||||
|
||||
Processor processor = new Processor() {
|
||||
@Override
|
||||
public void execute(IngestDocument ingestDocument) {
|
||||
ingestDocument.setFieldValue("field2", "value2");
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getType() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getTag() {
|
||||
return null;
|
||||
}
|
||||
};
|
||||
when(store.get("_id")).thenReturn(new Pipeline("_id", "_description", new CompoundProcessor(processor)));
|
||||
executionService = new PipelineExecutionService(store, threadPool);
|
||||
IngestService ingestService = mock(IngestService.class);
|
||||
when(ingestService.getPipelineExecutionService()).thenReturn(executionService);
|
||||
NodeService nodeService = mock(NodeService.class);
|
||||
when(nodeService.getIngestService()).thenReturn(ingestService);
|
||||
filter = new IngestActionFilter(Settings.EMPTY, nodeService);
|
||||
|
||||
BulkRequest bulkRequest = new BulkRequest();
|
||||
int numRequest = scaledRandomIntBetween(8, 64);
|
||||
for (int i = 0; i < numRequest; i++) {
|
||||
if (rarely()) {
|
||||
ActionRequest request;
|
||||
if (randomBoolean()) {
|
||||
request = new DeleteRequest("_index", "_type", "_id");
|
||||
} else {
|
||||
request = new UpdateRequest("_index", "_type", "_id");
|
||||
}
|
||||
bulkRequest.add(request);
|
||||
} else {
|
||||
IndexRequest indexRequest = new IndexRequest("_index", "_type", "_id").setPipeline("_id");
|
||||
indexRequest.source("field1", "value1");
|
||||
bulkRequest.add(indexRequest);
|
||||
}
|
||||
}
|
||||
|
||||
ActionListener actionListener = mock(ActionListener.class);
|
||||
ActionFilterChain actionFilterChain = mock(ActionFilterChain.class);
|
||||
|
||||
filter.apply(task, BulkAction.NAME, bulkRequest, actionListener, actionFilterChain);
|
||||
|
||||
assertBusy(() -> {
|
||||
verify(actionFilterChain).proceed(task, BulkAction.NAME, bulkRequest, actionListener);
|
||||
verifyZeroInteractions(actionListener);
|
||||
|
||||
int assertedRequests = 0;
|
||||
for (ActionRequest actionRequest : bulkRequest.requests()) {
|
||||
if (actionRequest instanceof IndexRequest) {
|
||||
IndexRequest indexRequest = (IndexRequest) actionRequest;
|
||||
assertThat(indexRequest.sourceAsMap().size(), equalTo(2));
|
||||
assertThat(indexRequest.sourceAsMap().get("field1"), equalTo("value1"));
|
||||
assertThat(indexRequest.sourceAsMap().get("field2"), equalTo("value2"));
|
||||
}
|
||||
assertedRequests++;
|
||||
}
|
||||
assertThat(assertedRequests, equalTo(numRequest));
|
||||
});
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
public void testIndexApiSinglePipelineExecution() {
|
||||
Answer answer = invocationOnMock -> {
|
||||
@SuppressWarnings("unchecked")
|
||||
Consumer<Boolean> listener = (Consumer) invocationOnMock.getArguments()[2];
|
||||
listener.accept(true);
|
||||
return null;
|
||||
};
|
||||
doAnswer(answer).when(executionService).execute(any(IndexRequest.class), any(Consumer.class), any(Consumer.class));
|
||||
|
||||
Task task = mock(Task.class);
|
||||
ActionListener actionListener = mock(ActionListener.class);
|
||||
ActionFilterChain actionFilterChain = mock(ActionFilterChain.class);
|
||||
|
||||
IndexRequest indexRequest = new IndexRequest("_index", "_type", "_id").setPipeline("_id").source("field", "value");
|
||||
filter.apply(task, IndexAction.NAME, indexRequest, actionListener, actionFilterChain);
|
||||
assertThat(indexRequest.getPipeline(), nullValue());
|
||||
filter.apply(task, IndexAction.NAME, indexRequest, actionListener, actionFilterChain);
|
||||
verify(executionService, times(1)).execute(same(indexRequest), any(Consumer.class), any(Consumer.class));
|
||||
verify(actionFilterChain, times(2)).proceed(task, IndexAction.NAME, indexRequest, actionListener);
|
||||
}
|
||||
}
|
@ -0,0 +1,251 @@
|
||||
/*
|
||||
* 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.action.ingest;
|
||||
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.action.ActionRequest;
|
||||
import org.elasticsearch.action.bulk.BulkAction;
|
||||
import org.elasticsearch.action.bulk.BulkRequest;
|
||||
import org.elasticsearch.action.bulk.BulkResponse;
|
||||
import org.elasticsearch.action.index.IndexAction;
|
||||
import org.elasticsearch.action.index.IndexRequest;
|
||||
import org.elasticsearch.action.index.IndexResponse;
|
||||
import org.elasticsearch.action.support.ActionFilterChain;
|
||||
import org.elasticsearch.cluster.ClusterName;
|
||||
import org.elasticsearch.cluster.ClusterService;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||
import org.elasticsearch.cluster.node.DiscoveryNodes;
|
||||
import org.elasticsearch.common.transport.DummyTransportAddress;
|
||||
import org.elasticsearch.tasks.Task;
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
import org.elasticsearch.test.VersionUtils;
|
||||
import org.elasticsearch.transport.TransportException;
|
||||
import org.elasticsearch.transport.TransportRequest;
|
||||
import org.elasticsearch.transport.TransportResponse;
|
||||
import org.elasticsearch.transport.TransportResponseHandler;
|
||||
import org.elasticsearch.transport.TransportService;
|
||||
import org.hamcrest.CustomTypeSafeMatcher;
|
||||
import org.mockito.stubbing.Answer;
|
||||
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
import static org.hamcrest.CoreMatchers.equalTo;
|
||||
import static org.mockito.Matchers.any;
|
||||
import static org.mockito.Matchers.argThat;
|
||||
import static org.mockito.Matchers.eq;
|
||||
import static org.mockito.Matchers.same;
|
||||
import static org.mockito.Mockito.doAnswer;
|
||||
import static org.mockito.Mockito.mock;
|
||||
import static org.mockito.Mockito.never;
|
||||
import static org.mockito.Mockito.verify;
|
||||
import static org.mockito.Mockito.verifyZeroInteractions;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
public class IngestProxyActionFilterTests extends ESTestCase {
|
||||
|
||||
private TransportService transportService;
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
private IngestProxyActionFilter buildFilter(int ingestNodes, int totalNodes) {
|
||||
ClusterState.Builder clusterState = new ClusterState.Builder(new ClusterName("_name"));
|
||||
DiscoveryNodes.Builder builder = new DiscoveryNodes.Builder();
|
||||
DiscoveryNode localNode = null;
|
||||
for (int i = 0; i < totalNodes; i++) {
|
||||
String nodeId = "node" + i;
|
||||
Map<String, String> attributes = new HashMap<>();
|
||||
if (i >= ingestNodes) {
|
||||
attributes.put("ingest", "false");
|
||||
} else if (randomBoolean()) {
|
||||
attributes.put("ingest", "true");
|
||||
}
|
||||
DiscoveryNode node = new DiscoveryNode(nodeId, nodeId, DummyTransportAddress.INSTANCE, attributes, VersionUtils.randomVersion(random()));
|
||||
builder.put(node);
|
||||
if (i == totalNodes - 1) {
|
||||
localNode = node;
|
||||
}
|
||||
}
|
||||
clusterState.nodes(builder);
|
||||
ClusterService clusterService = mock(ClusterService.class);
|
||||
when(clusterService.localNode()).thenReturn(localNode);
|
||||
when(clusterService.state()).thenReturn(clusterState.build());
|
||||
transportService = mock(TransportService.class);
|
||||
return new IngestProxyActionFilter(clusterService, transportService);
|
||||
}
|
||||
|
||||
public void testApplyNoIngestNodes() {
|
||||
Task task = mock(Task.class);
|
||||
ActionListener actionListener = mock(ActionListener.class);
|
||||
ActionFilterChain actionFilterChain = mock(ActionFilterChain.class);
|
||||
int totalNodes = randomIntBetween(1, 5);
|
||||
IngestProxyActionFilter filter = buildFilter(0, totalNodes);
|
||||
|
||||
String action;
|
||||
ActionRequest request;
|
||||
if (randomBoolean()) {
|
||||
action = IndexAction.NAME;
|
||||
request = new IndexRequest().setPipeline("_id");
|
||||
} else {
|
||||
action = BulkAction.NAME;
|
||||
request = new BulkRequest().add(new IndexRequest().setPipeline("_id"));
|
||||
}
|
||||
try {
|
||||
filter.apply(task, action, request, actionListener, actionFilterChain);
|
||||
fail("should have failed because there are no ingest nodes");
|
||||
} catch(IllegalStateException e) {
|
||||
assertThat(e.getMessage(), equalTo("There are no ingest nodes in this cluster, unable to forward request to an ingest node."));
|
||||
}
|
||||
verifyZeroInteractions(transportService);
|
||||
verifyZeroInteractions(actionFilterChain);
|
||||
verifyZeroInteractions(actionListener);
|
||||
}
|
||||
|
||||
public void testApplyNoPipelineId() {
|
||||
Task task = mock(Task.class);
|
||||
ActionListener actionListener = mock(ActionListener.class);
|
||||
ActionFilterChain actionFilterChain = mock(ActionFilterChain.class);
|
||||
int totalNodes = randomIntBetween(1, 5);
|
||||
IngestProxyActionFilter filter = buildFilter(randomIntBetween(0, totalNodes - 1), totalNodes);
|
||||
|
||||
String action;
|
||||
ActionRequest request;
|
||||
if (randomBoolean()) {
|
||||
action = IndexAction.NAME;
|
||||
request = new IndexRequest();
|
||||
} else {
|
||||
action = BulkAction.NAME;
|
||||
request = new BulkRequest().add(new IndexRequest());
|
||||
}
|
||||
filter.apply(task, action, request, actionListener, actionFilterChain);
|
||||
verifyZeroInteractions(transportService);
|
||||
verify(actionFilterChain).proceed(any(Task.class), eq(action), same(request), same(actionListener));
|
||||
verifyZeroInteractions(actionListener);
|
||||
}
|
||||
|
||||
public void testApplyAnyAction() {
|
||||
Task task = mock(Task.class);
|
||||
ActionListener actionListener = mock(ActionListener.class);
|
||||
ActionFilterChain actionFilterChain = mock(ActionFilterChain.class);
|
||||
ActionRequest request = mock(ActionRequest.class);
|
||||
int totalNodes = randomIntBetween(1, 5);
|
||||
IngestProxyActionFilter filter = buildFilter(randomIntBetween(0, totalNodes - 1), totalNodes);
|
||||
|
||||
String action = randomAsciiOfLengthBetween(1, 20);
|
||||
filter.apply(task, action, request, actionListener, actionFilterChain);
|
||||
verifyZeroInteractions(transportService);
|
||||
verify(actionFilterChain).proceed(any(Task.class), eq(action), same(request), same(actionListener));
|
||||
verifyZeroInteractions(actionListener);
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
public void testApplyIndexRedirect() {
|
||||
Task task = mock(Task.class);
|
||||
ActionListener actionListener = mock(ActionListener.class);
|
||||
ActionFilterChain actionFilterChain = mock(ActionFilterChain.class);
|
||||
int totalNodes = randomIntBetween(2, 5);
|
||||
IngestProxyActionFilter filter = buildFilter(randomIntBetween(1, totalNodes - 1), totalNodes);
|
||||
Answer<Void> answer = invocationOnMock -> {
|
||||
TransportResponseHandler transportResponseHandler = (TransportResponseHandler) invocationOnMock.getArguments()[3];
|
||||
transportResponseHandler.handleResponse(new IndexResponse());
|
||||
return null;
|
||||
};
|
||||
doAnswer(answer).when(transportService).sendRequest(any(DiscoveryNode.class), any(String.class), any(TransportRequest.class), any(TransportResponseHandler.class));
|
||||
|
||||
IndexRequest indexRequest = new IndexRequest().setPipeline("_id");
|
||||
filter.apply(task, IndexAction.NAME, indexRequest, actionListener, actionFilterChain);
|
||||
|
||||
verify(transportService).sendRequest(argThat(new IngestNodeMatcher()), eq(IndexAction.NAME), same(indexRequest), any(TransportResponseHandler.class));
|
||||
verifyZeroInteractions(actionFilterChain);
|
||||
verify(actionListener).onResponse(any(IndexResponse.class));
|
||||
verify(actionListener, never()).onFailure(any(TransportException.class));
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
public void testApplyBulkRedirect() {
|
||||
Task task = mock(Task.class);
|
||||
ActionListener actionListener = mock(ActionListener.class);
|
||||
ActionFilterChain actionFilterChain = mock(ActionFilterChain.class);
|
||||
int totalNodes = randomIntBetween(2, 5);
|
||||
IngestProxyActionFilter filter = buildFilter(randomIntBetween(1, totalNodes - 1), totalNodes);
|
||||
Answer<Void> answer = invocationOnMock -> {
|
||||
TransportResponseHandler transportResponseHandler = (TransportResponseHandler) invocationOnMock.getArguments()[3];
|
||||
transportResponseHandler.handleResponse(new BulkResponse(null, -1));
|
||||
return null;
|
||||
};
|
||||
doAnswer(answer).when(transportService).sendRequest(any(DiscoveryNode.class), any(String.class), any(TransportRequest.class), any(TransportResponseHandler.class));
|
||||
|
||||
BulkRequest bulkRequest = new BulkRequest();
|
||||
bulkRequest.add(new IndexRequest().setPipeline("_id"));
|
||||
int numNoPipelineRequests = randomIntBetween(0, 10);
|
||||
for (int i = 0; i < numNoPipelineRequests; i++) {
|
||||
bulkRequest.add(new IndexRequest());
|
||||
}
|
||||
filter.apply(task, BulkAction.NAME, bulkRequest, actionListener, actionFilterChain);
|
||||
|
||||
verify(transportService).sendRequest(argThat(new IngestNodeMatcher()), eq(BulkAction.NAME), same(bulkRequest), any(TransportResponseHandler.class));
|
||||
verifyZeroInteractions(actionFilterChain);
|
||||
verify(actionListener).onResponse(any(BulkResponse.class));
|
||||
verify(actionListener, never()).onFailure(any(TransportException.class));
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
public void testApplyFailures() {
|
||||
Task task = mock(Task.class);
|
||||
ActionListener actionListener = mock(ActionListener.class);
|
||||
ActionFilterChain actionFilterChain = mock(ActionFilterChain.class);
|
||||
int totalNodes = randomIntBetween(2, 5);
|
||||
IngestProxyActionFilter filter = buildFilter(randomIntBetween(1, totalNodes - 1), totalNodes);
|
||||
Answer<Void> answer = invocationOnMock -> {
|
||||
TransportResponseHandler transportResponseHandler = (TransportResponseHandler) invocationOnMock.getArguments()[3];
|
||||
transportResponseHandler.handleException(new TransportException(new IllegalArgumentException()));
|
||||
return null;
|
||||
};
|
||||
doAnswer(answer).when(transportService).sendRequest(any(DiscoveryNode.class), any(String.class), any(TransportRequest.class), any(TransportResponseHandler.class));
|
||||
|
||||
String action;
|
||||
ActionRequest request;
|
||||
if (randomBoolean()) {
|
||||
action = IndexAction.NAME;
|
||||
request = new IndexRequest().setPipeline("_id");
|
||||
} else {
|
||||
action = BulkAction.NAME;
|
||||
request = new BulkRequest().add(new IndexRequest().setPipeline("_id"));
|
||||
}
|
||||
|
||||
filter.apply(task, action, request, actionListener, actionFilterChain);
|
||||
|
||||
verify(transportService).sendRequest(argThat(new IngestNodeMatcher()), eq(action), same(request), any(TransportResponseHandler.class));
|
||||
verifyZeroInteractions(actionFilterChain);
|
||||
verify(actionListener).onFailure(any(TransportException.class));
|
||||
verify(actionListener, never()).onResponse(any(TransportResponse.class));
|
||||
}
|
||||
|
||||
private static class IngestNodeMatcher extends CustomTypeSafeMatcher<DiscoveryNode> {
|
||||
private IngestNodeMatcher() {
|
||||
super("discovery node should be an ingest node");
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean matchesSafely(DiscoveryNode node) {
|
||||
return node.isIngestNode();
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,57 @@
|
||||
/*
|
||||
* 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.action.ingest;
|
||||
|
||||
import org.elasticsearch.common.io.stream.BytesStreamOutput;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.ingest.core.IngestDocument;
|
||||
import org.elasticsearch.ingest.RandomDocumentPicks;
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import static org.hamcrest.CoreMatchers.equalTo;
|
||||
import static org.hamcrest.CoreMatchers.instanceOf;
|
||||
|
||||
public class SimulateDocumentSimpleResultTests extends ESTestCase {
|
||||
|
||||
public void testSerialization() throws IOException {
|
||||
boolean isFailure = randomBoolean();
|
||||
SimulateDocumentBaseResult simulateDocumentBaseResult;
|
||||
if (isFailure) {
|
||||
simulateDocumentBaseResult = new SimulateDocumentBaseResult(new IllegalArgumentException("test"));
|
||||
} else {
|
||||
IngestDocument ingestDocument = RandomDocumentPicks.randomIngestDocument(random());
|
||||
simulateDocumentBaseResult = new SimulateDocumentBaseResult(ingestDocument);
|
||||
}
|
||||
|
||||
BytesStreamOutput out = new BytesStreamOutput();
|
||||
simulateDocumentBaseResult.writeTo(out);
|
||||
StreamInput streamInput = StreamInput.wrap(out.bytes());
|
||||
SimulateDocumentBaseResult otherSimulateDocumentBaseResult = SimulateDocumentBaseResult.readSimulateDocumentSimpleResult(streamInput);
|
||||
|
||||
assertThat(otherSimulateDocumentBaseResult.getIngestDocument(), equalTo(simulateDocumentBaseResult.getIngestDocument()));
|
||||
if (isFailure) {
|
||||
assertThat(otherSimulateDocumentBaseResult.getFailure(), instanceOf(IllegalArgumentException.class));
|
||||
IllegalArgumentException e = (IllegalArgumentException) otherSimulateDocumentBaseResult.getFailure();
|
||||
assertThat(e.getMessage(), equalTo("test"));
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,206 @@
|
||||
/*
|
||||
* 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.action.ingest;
|
||||
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.ingest.RandomDocumentPicks;
|
||||
import org.elasticsearch.ingest.TestProcessor;
|
||||
import org.elasticsearch.ingest.core.CompoundProcessor;
|
||||
import org.elasticsearch.ingest.core.Processor;
|
||||
import org.elasticsearch.ingest.core.IngestDocument;
|
||||
import org.elasticsearch.ingest.core.Pipeline;
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
import org.junit.After;
|
||||
import org.junit.Before;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
||||
import static org.hamcrest.Matchers.equalTo;
|
||||
import static org.hamcrest.Matchers.instanceOf;
|
||||
import static org.hamcrest.Matchers.not;
|
||||
import static org.hamcrest.Matchers.nullValue;
|
||||
import static org.hamcrest.Matchers.sameInstance;
|
||||
|
||||
public class SimulateExecutionServiceTests extends ESTestCase {
|
||||
|
||||
private ThreadPool threadPool;
|
||||
private SimulateExecutionService executionService;
|
||||
private Pipeline pipeline;
|
||||
private Processor processor;
|
||||
private IngestDocument ingestDocument;
|
||||
|
||||
@Before
|
||||
public void setup() {
|
||||
threadPool = new ThreadPool(
|
||||
Settings.builder()
|
||||
.put("name", getClass().getName())
|
||||
.build()
|
||||
);
|
||||
executionService = new SimulateExecutionService(threadPool);
|
||||
processor = new TestProcessor("id", "mock", ingestDocument -> {});
|
||||
pipeline = new Pipeline("_id", "_description", new CompoundProcessor(processor, processor));
|
||||
ingestDocument = RandomDocumentPicks.randomIngestDocument(random());
|
||||
}
|
||||
|
||||
@After
|
||||
public void destroy() {
|
||||
threadPool.shutdown();
|
||||
}
|
||||
|
||||
public void testExecuteVerboseDocumentSimple() throws Exception {
|
||||
List<SimulateProcessorResult> processorResultList = new ArrayList<>();
|
||||
executionService.executeVerboseDocument(processor, ingestDocument, processorResultList);
|
||||
SimulateProcessorResult result = new SimulateProcessorResult("id", ingestDocument);
|
||||
assertThat(processorResultList.size(), equalTo(1));
|
||||
assertThat(processorResultList.get(0).getProcessorTag(), equalTo(result.getProcessorTag()));
|
||||
assertThat(processorResultList.get(0).getIngestDocument(), equalTo(result.getIngestDocument()));
|
||||
assertThat(processorResultList.get(0).getFailure(), nullValue());
|
||||
}
|
||||
|
||||
public void testExecuteVerboseDocumentSimpleException() throws Exception {
|
||||
RuntimeException exception = new RuntimeException("mock_exception");
|
||||
TestProcessor processor = new TestProcessor("id", "mock", ingestDocument -> { throw exception; });
|
||||
List<SimulateProcessorResult> processorResultList = new ArrayList<>();
|
||||
try {
|
||||
executionService.executeVerboseDocument(processor, ingestDocument, processorResultList);
|
||||
fail("should throw exception");
|
||||
} catch (RuntimeException e) {
|
||||
assertThat(e.getMessage(), equalTo("mock_exception"));
|
||||
}
|
||||
SimulateProcessorResult result = new SimulateProcessorResult("id", exception);
|
||||
assertThat(processorResultList.size(), equalTo(1));
|
||||
assertThat(processorResultList.get(0).getProcessorTag(), equalTo(result.getProcessorTag()));
|
||||
assertThat(processorResultList.get(0).getFailure(), equalTo(result.getFailure()));
|
||||
}
|
||||
|
||||
public void testExecuteVerboseDocumentCompoundSuccess() throws Exception {
|
||||
TestProcessor processor1 = new TestProcessor("p1", "mock", ingestDocument -> { });
|
||||
TestProcessor processor2 = new TestProcessor("p2", "mock", ingestDocument -> { });
|
||||
|
||||
Processor compoundProcessor = new CompoundProcessor(processor1, processor2);
|
||||
List<SimulateProcessorResult> processorResultList = new ArrayList<>();
|
||||
executionService.executeVerboseDocument(compoundProcessor, ingestDocument, processorResultList);
|
||||
assertThat(processor1.getInvokedCounter(), equalTo(1));
|
||||
assertThat(processor2.getInvokedCounter(), equalTo(1));
|
||||
assertThat(processorResultList.size(), equalTo(2));
|
||||
assertThat(processorResultList.get(0).getProcessorTag(), equalTo("p1"));
|
||||
assertThat(processorResultList.get(0).getIngestDocument(), equalTo(ingestDocument));
|
||||
assertThat(processorResultList.get(0).getFailure(), nullValue());
|
||||
assertThat(processorResultList.get(1).getProcessorTag(), equalTo("p2"));
|
||||
assertThat(processorResultList.get(1).getIngestDocument(), equalTo(ingestDocument));
|
||||
assertThat(processorResultList.get(1).getFailure(), nullValue());
|
||||
}
|
||||
|
||||
public void testExecuteVerboseDocumentCompoundOnFailure() throws Exception {
|
||||
TestProcessor processor1 = new TestProcessor("p1", "mock", ingestDocument -> { });
|
||||
TestProcessor processor2 = new TestProcessor("p2", "mock", ingestDocument -> { throw new RuntimeException("p2_exception"); });
|
||||
TestProcessor onFailureProcessor1 = new TestProcessor("fail_p1", "mock", ingestDocument -> { });
|
||||
TestProcessor onFailureProcessor2 = new TestProcessor("fail_p2", "mock", ingestDocument -> { throw new RuntimeException("fail_p2_exception"); });
|
||||
TestProcessor onFailureProcessor3 = new TestProcessor("fail_p3", "mock", ingestDocument -> { });
|
||||
CompoundProcessor onFailureCompoundProcessor = new CompoundProcessor(Collections.singletonList(onFailureProcessor2), Collections.singletonList(onFailureProcessor3));
|
||||
|
||||
Processor compoundProcessor = new CompoundProcessor(Arrays.asList(processor1, processor2), Arrays.asList(onFailureProcessor1, onFailureCompoundProcessor));
|
||||
List<SimulateProcessorResult> processorResultList = new ArrayList<>();
|
||||
executionService.executeVerboseDocument(compoundProcessor, ingestDocument, processorResultList);
|
||||
assertThat(processor1.getInvokedCounter(), equalTo(1));
|
||||
assertThat(processor2.getInvokedCounter(), equalTo(1));
|
||||
assertThat(onFailureProcessor1.getInvokedCounter(), equalTo(1));
|
||||
assertThat(onFailureProcessor2.getInvokedCounter(), equalTo(1));
|
||||
assertThat(onFailureProcessor3.getInvokedCounter(), equalTo(1));
|
||||
assertThat(processorResultList.size(), equalTo(5));
|
||||
assertThat(processorResultList.get(0).getProcessorTag(), equalTo("p1"));
|
||||
assertThat(processorResultList.get(1).getProcessorTag(), equalTo("p2"));
|
||||
assertThat(processorResultList.get(2).getProcessorTag(), equalTo("fail_p1"));
|
||||
assertThat(processorResultList.get(3).getProcessorTag(), equalTo("fail_p2"));
|
||||
assertThat(processorResultList.get(4).getProcessorTag(), equalTo("fail_p3"));
|
||||
}
|
||||
|
||||
public void testExecuteVerboseItem() throws Exception {
|
||||
TestProcessor processor = new TestProcessor("test-id", "mock", ingestDocument -> {});
|
||||
Pipeline pipeline = new Pipeline("_id", "_description", new CompoundProcessor(processor, processor));
|
||||
SimulateDocumentResult actualItemResponse = executionService.executeDocument(pipeline, ingestDocument, true);
|
||||
assertThat(processor.getInvokedCounter(), equalTo(2));
|
||||
assertThat(actualItemResponse, instanceOf(SimulateDocumentVerboseResult.class));
|
||||
SimulateDocumentVerboseResult simulateDocumentVerboseResult = (SimulateDocumentVerboseResult) actualItemResponse;
|
||||
assertThat(simulateDocumentVerboseResult.getProcessorResults().size(), equalTo(2));
|
||||
assertThat(simulateDocumentVerboseResult.getProcessorResults().get(0).getProcessorTag(), equalTo("test-id"));
|
||||
assertThat(simulateDocumentVerboseResult.getProcessorResults().get(0).getIngestDocument(), not(sameInstance(ingestDocument)));
|
||||
assertThat(simulateDocumentVerboseResult.getProcessorResults().get(0).getIngestDocument(), equalTo(ingestDocument));
|
||||
assertThat(simulateDocumentVerboseResult.getProcessorResults().get(0).getIngestDocument().getSourceAndMetadata(), not(sameInstance(ingestDocument.getSourceAndMetadata())));
|
||||
|
||||
assertThat(simulateDocumentVerboseResult.getProcessorResults().get(0).getFailure(), nullValue());
|
||||
assertThat(simulateDocumentVerboseResult.getProcessorResults().get(1).getProcessorTag(), equalTo("test-id"));
|
||||
assertThat(simulateDocumentVerboseResult.getProcessorResults().get(1).getIngestDocument(), not(sameInstance(ingestDocument)));
|
||||
assertThat(simulateDocumentVerboseResult.getProcessorResults().get(1).getIngestDocument(), equalTo(ingestDocument));
|
||||
assertThat(simulateDocumentVerboseResult.getProcessorResults().get(1).getIngestDocument().getSourceAndMetadata(), not(sameInstance(ingestDocument.getSourceAndMetadata())));
|
||||
assertThat(simulateDocumentVerboseResult.getProcessorResults().get(1).getIngestDocument().getSourceAndMetadata(),
|
||||
not(sameInstance(simulateDocumentVerboseResult.getProcessorResults().get(0).getIngestDocument().getSourceAndMetadata())));
|
||||
assertThat(simulateDocumentVerboseResult.getProcessorResults().get(1).getFailure(), nullValue());
|
||||
}
|
||||
|
||||
public void testExecuteItem() throws Exception {
|
||||
TestProcessor processor = new TestProcessor("processor_0", "mock", ingestDocument -> {});
|
||||
Pipeline pipeline = new Pipeline("_id", "_description", new CompoundProcessor(processor, processor));
|
||||
SimulateDocumentResult actualItemResponse = executionService.executeDocument(pipeline, ingestDocument, false);
|
||||
assertThat(processor.getInvokedCounter(), equalTo(2));
|
||||
assertThat(actualItemResponse, instanceOf(SimulateDocumentBaseResult.class));
|
||||
SimulateDocumentBaseResult simulateDocumentBaseResult = (SimulateDocumentBaseResult) actualItemResponse;
|
||||
assertThat(simulateDocumentBaseResult.getIngestDocument(), equalTo(ingestDocument));
|
||||
assertThat(simulateDocumentBaseResult.getFailure(), nullValue());
|
||||
}
|
||||
|
||||
public void testExecuteVerboseItemWithFailure() throws Exception {
|
||||
TestProcessor processor1 = new TestProcessor("processor_0", "mock", ingestDocument -> { throw new RuntimeException("processor failed"); });
|
||||
TestProcessor processor2 = new TestProcessor("processor_1", "mock", ingestDocument -> {});
|
||||
Pipeline pipeline = new Pipeline("_id", "_description", new CompoundProcessor(Collections.singletonList(processor1), Collections.singletonList(processor2)));
|
||||
SimulateDocumentResult actualItemResponse = executionService.executeDocument(pipeline, ingestDocument, true);
|
||||
assertThat(processor1.getInvokedCounter(), equalTo(1));
|
||||
assertThat(processor2.getInvokedCounter(), equalTo(1));
|
||||
assertThat(actualItemResponse, instanceOf(SimulateDocumentVerboseResult.class));
|
||||
SimulateDocumentVerboseResult simulateDocumentVerboseResult = (SimulateDocumentVerboseResult) actualItemResponse;
|
||||
assertThat(simulateDocumentVerboseResult.getProcessorResults().size(), equalTo(2));
|
||||
assertThat(simulateDocumentVerboseResult.getProcessorResults().get(0).getProcessorTag(), equalTo("processor_0"));
|
||||
assertThat(simulateDocumentVerboseResult.getProcessorResults().get(0).getIngestDocument(), nullValue());
|
||||
assertThat(simulateDocumentVerboseResult.getProcessorResults().get(0).getFailure(), instanceOf(RuntimeException.class));
|
||||
RuntimeException runtimeException = (RuntimeException) simulateDocumentVerboseResult.getProcessorResults().get(0).getFailure();
|
||||
assertThat(runtimeException.getMessage(), equalTo("processor failed"));
|
||||
assertThat(simulateDocumentVerboseResult.getProcessorResults().get(1).getProcessorTag(), equalTo("processor_1"));
|
||||
assertThat(simulateDocumentVerboseResult.getProcessorResults().get(1).getIngestDocument(), not(sameInstance(ingestDocument)));
|
||||
assertThat(simulateDocumentVerboseResult.getProcessorResults().get(1).getIngestDocument(), equalTo(ingestDocument));
|
||||
assertThat(simulateDocumentVerboseResult.getProcessorResults().get(1).getFailure(), nullValue());
|
||||
}
|
||||
|
||||
public void testExecuteItemWithFailure() throws Exception {
|
||||
TestProcessor processor = new TestProcessor(ingestDocument -> { throw new RuntimeException("processor failed"); });
|
||||
Pipeline pipeline = new Pipeline("_id", "_description", new CompoundProcessor(processor, processor));
|
||||
SimulateDocumentResult actualItemResponse = executionService.executeDocument(pipeline, ingestDocument, false);
|
||||
assertThat(processor.getInvokedCounter(), equalTo(1));
|
||||
assertThat(actualItemResponse, instanceOf(SimulateDocumentBaseResult.class));
|
||||
SimulateDocumentBaseResult simulateDocumentBaseResult = (SimulateDocumentBaseResult) actualItemResponse;
|
||||
assertThat(simulateDocumentBaseResult.getIngestDocument(), nullValue());
|
||||
assertThat(simulateDocumentBaseResult.getFailure(), instanceOf(RuntimeException.class));
|
||||
RuntimeException runtimeException = (RuntimeException) simulateDocumentBaseResult.getFailure();
|
||||
assertThat(runtimeException.getMessage(), equalTo("processor failed"));
|
||||
}
|
||||
}
|
@ -0,0 +1,181 @@
|
||||
/*
|
||||
* 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.action.ingest;
|
||||
|
||||
import org.elasticsearch.ingest.PipelineStore;
|
||||
import org.elasticsearch.ingest.TestProcessor;
|
||||
import org.elasticsearch.ingest.core.CompoundProcessor;
|
||||
import org.elasticsearch.ingest.core.IngestDocument;
|
||||
import org.elasticsearch.ingest.core.Pipeline;
|
||||
import org.elasticsearch.ingest.core.Processor;
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
import org.junit.Before;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import static org.elasticsearch.action.ingest.SimulatePipelineRequest.Fields;
|
||||
import static org.elasticsearch.ingest.core.IngestDocument.MetaData.ID;
|
||||
import static org.elasticsearch.ingest.core.IngestDocument.MetaData.INDEX;
|
||||
import static org.elasticsearch.ingest.core.IngestDocument.MetaData.TYPE;
|
||||
import static org.hamcrest.Matchers.equalTo;
|
||||
import static org.hamcrest.Matchers.nullValue;
|
||||
import static org.mockito.Mockito.mock;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
public class SimulatePipelineRequestParsingTests extends ESTestCase {
|
||||
|
||||
private PipelineStore store;
|
||||
|
||||
@Before
|
||||
public void init() throws IOException {
|
||||
TestProcessor processor = new TestProcessor(ingestDocument -> {});
|
||||
CompoundProcessor pipelineCompoundProcessor = new CompoundProcessor(processor);
|
||||
Pipeline pipeline = new Pipeline(SimulatePipelineRequest.SIMULATED_PIPELINE_ID, null, pipelineCompoundProcessor);
|
||||
Map<String, Processor.Factory> processorRegistry = new HashMap<>();
|
||||
processorRegistry.put("mock_processor", mock(Processor.Factory.class));
|
||||
store = mock(PipelineStore.class);
|
||||
when(store.get(SimulatePipelineRequest.SIMULATED_PIPELINE_ID)).thenReturn(pipeline);
|
||||
when(store.getProcessorFactoryRegistry()).thenReturn(processorRegistry);
|
||||
}
|
||||
|
||||
public void testParseUsingPipelineStore() throws Exception {
|
||||
int numDocs = randomIntBetween(1, 10);
|
||||
|
||||
Map<String, Object> requestContent = new HashMap<>();
|
||||
List<Map<String, Object>> docs = new ArrayList<>();
|
||||
List<Map<String, Object>> expectedDocs = new ArrayList<>();
|
||||
requestContent.put(Fields.DOCS, docs);
|
||||
for (int i = 0; i < numDocs; i++) {
|
||||
Map<String, Object> doc = new HashMap<>();
|
||||
String index = randomAsciiOfLengthBetween(1, 10);
|
||||
String type = randomAsciiOfLengthBetween(1, 10);
|
||||
String id = randomAsciiOfLengthBetween(1, 10);
|
||||
doc.put(INDEX.getFieldName(), index);
|
||||
doc.put(TYPE.getFieldName(), type);
|
||||
doc.put(ID.getFieldName(), id);
|
||||
String fieldName = randomAsciiOfLengthBetween(1, 10);
|
||||
String fieldValue = randomAsciiOfLengthBetween(1, 10);
|
||||
doc.put(Fields.SOURCE, Collections.singletonMap(fieldName, fieldValue));
|
||||
docs.add(doc);
|
||||
Map<String, Object> expectedDoc = new HashMap<>();
|
||||
expectedDoc.put(INDEX.getFieldName(), index);
|
||||
expectedDoc.put(TYPE.getFieldName(), type);
|
||||
expectedDoc.put(ID.getFieldName(), id);
|
||||
expectedDoc.put(Fields.SOURCE, Collections.singletonMap(fieldName, fieldValue));
|
||||
expectedDocs.add(expectedDoc);
|
||||
}
|
||||
|
||||
SimulatePipelineRequest.Parsed actualRequest = SimulatePipelineRequest.parseWithPipelineId(SimulatePipelineRequest.SIMULATED_PIPELINE_ID, requestContent, false, store);
|
||||
assertThat(actualRequest.isVerbose(), equalTo(false));
|
||||
assertThat(actualRequest.getDocuments().size(), equalTo(numDocs));
|
||||
Iterator<Map<String, Object>> expectedDocsIterator = expectedDocs.iterator();
|
||||
for (IngestDocument ingestDocument : actualRequest.getDocuments()) {
|
||||
Map<String, Object> expectedDocument = expectedDocsIterator.next();
|
||||
Map<IngestDocument.MetaData, String> metadataMap = ingestDocument.extractMetadata();
|
||||
assertThat(metadataMap.get(INDEX), equalTo(expectedDocument.get(INDEX.getFieldName())));
|
||||
assertThat(metadataMap.get(TYPE), equalTo(expectedDocument.get(TYPE.getFieldName())));
|
||||
assertThat(metadataMap.get(ID), equalTo(expectedDocument.get(ID.getFieldName())));
|
||||
assertThat(ingestDocument.getSourceAndMetadata(), equalTo(expectedDocument.get(Fields.SOURCE)));
|
||||
}
|
||||
|
||||
assertThat(actualRequest.getPipeline().getId(), equalTo(SimulatePipelineRequest.SIMULATED_PIPELINE_ID));
|
||||
assertThat(actualRequest.getPipeline().getDescription(), nullValue());
|
||||
assertThat(actualRequest.getPipeline().getProcessors().size(), equalTo(1));
|
||||
}
|
||||
|
||||
public void testParseWithProvidedPipeline() throws Exception {
|
||||
int numDocs = randomIntBetween(1, 10);
|
||||
|
||||
Map<String, Object> requestContent = new HashMap<>();
|
||||
List<Map<String, Object>> docs = new ArrayList<>();
|
||||
List<Map<String, Object>> expectedDocs = new ArrayList<>();
|
||||
requestContent.put(Fields.DOCS, docs);
|
||||
for (int i = 0; i < numDocs; i++) {
|
||||
Map<String, Object> doc = new HashMap<>();
|
||||
String index = randomAsciiOfLengthBetween(1, 10);
|
||||
String type = randomAsciiOfLengthBetween(1, 10);
|
||||
String id = randomAsciiOfLengthBetween(1, 10);
|
||||
doc.put(INDEX.getFieldName(), index);
|
||||
doc.put(TYPE.getFieldName(), type);
|
||||
doc.put(ID.getFieldName(), id);
|
||||
String fieldName = randomAsciiOfLengthBetween(1, 10);
|
||||
String fieldValue = randomAsciiOfLengthBetween(1, 10);
|
||||
doc.put(Fields.SOURCE, Collections.singletonMap(fieldName, fieldValue));
|
||||
docs.add(doc);
|
||||
Map<String, Object> expectedDoc = new HashMap<>();
|
||||
expectedDoc.put(INDEX.getFieldName(), index);
|
||||
expectedDoc.put(TYPE.getFieldName(), type);
|
||||
expectedDoc.put(ID.getFieldName(), id);
|
||||
expectedDoc.put(Fields.SOURCE, Collections.singletonMap(fieldName, fieldValue));
|
||||
expectedDocs.add(expectedDoc);
|
||||
}
|
||||
|
||||
Map<String, Object> pipelineConfig = new HashMap<>();
|
||||
List<Map<String, Object>> processors = new ArrayList<>();
|
||||
int numProcessors = randomIntBetween(1, 10);
|
||||
for (int i = 0; i < numProcessors; i++) {
|
||||
Map<String, Object> processorConfig = new HashMap<>();
|
||||
List<Map<String, Object>> onFailureProcessors = new ArrayList<>();
|
||||
int numOnFailureProcessors = randomIntBetween(0, 1);
|
||||
for (int j = 0; j < numOnFailureProcessors; j++) {
|
||||
onFailureProcessors.add(Collections.singletonMap("mock_processor", Collections.emptyMap()));
|
||||
}
|
||||
if (numOnFailureProcessors > 0) {
|
||||
processorConfig.put("on_failure", onFailureProcessors);
|
||||
}
|
||||
processors.add(Collections.singletonMap("mock_processor", processorConfig));
|
||||
}
|
||||
pipelineConfig.put("processors", processors);
|
||||
|
||||
List<Map<String, Object>> onFailureProcessors = new ArrayList<>();
|
||||
int numOnFailureProcessors = randomIntBetween(0, 1);
|
||||
for (int i = 0; i < numOnFailureProcessors; i++) {
|
||||
onFailureProcessors.add(Collections.singletonMap("mock_processor", Collections.emptyMap()));
|
||||
}
|
||||
if (numOnFailureProcessors > 0) {
|
||||
pipelineConfig.put("on_failure", onFailureProcessors);
|
||||
}
|
||||
|
||||
requestContent.put(Fields.PIPELINE, pipelineConfig);
|
||||
|
||||
SimulatePipelineRequest.Parsed actualRequest = SimulatePipelineRequest.parse(requestContent, false, store);
|
||||
assertThat(actualRequest.isVerbose(), equalTo(false));
|
||||
assertThat(actualRequest.getDocuments().size(), equalTo(numDocs));
|
||||
Iterator<Map<String, Object>> expectedDocsIterator = expectedDocs.iterator();
|
||||
for (IngestDocument ingestDocument : actualRequest.getDocuments()) {
|
||||
Map<String, Object> expectedDocument = expectedDocsIterator.next();
|
||||
Map<IngestDocument.MetaData, String> metadataMap = ingestDocument.extractMetadata();
|
||||
assertThat(metadataMap.get(INDEX), equalTo(expectedDocument.get(INDEX.getFieldName())));
|
||||
assertThat(metadataMap.get(TYPE), equalTo(expectedDocument.get(TYPE.getFieldName())));
|
||||
assertThat(metadataMap.get(ID), equalTo(expectedDocument.get(ID.getFieldName())));
|
||||
assertThat(ingestDocument.getSourceAndMetadata(), equalTo(expectedDocument.get(Fields.SOURCE)));
|
||||
}
|
||||
|
||||
assertThat(actualRequest.getPipeline().getId(), equalTo(SimulatePipelineRequest.SIMULATED_PIPELINE_ID));
|
||||
assertThat(actualRequest.getPipeline().getDescription(), nullValue());
|
||||
assertThat(actualRequest.getPipeline().getProcessors().size(), equalTo(numProcessors));
|
||||
}
|
||||
}
|
@ -0,0 +1,117 @@
|
||||
/*
|
||||
* 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.action.ingest;
|
||||
|
||||
import org.elasticsearch.common.io.stream.BytesStreamOutput;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.ingest.core.IngestDocument;
|
||||
import org.elasticsearch.ingest.RandomDocumentPicks;
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
|
||||
import static org.hamcrest.CoreMatchers.equalTo;
|
||||
import static org.hamcrest.CoreMatchers.instanceOf;
|
||||
import static org.hamcrest.CoreMatchers.nullValue;
|
||||
|
||||
public class SimulatePipelineResponseTests extends ESTestCase {
|
||||
|
||||
public void testSerialization() throws IOException {
|
||||
boolean isVerbose = randomBoolean();
|
||||
int numResults = randomIntBetween(1, 10);
|
||||
List<SimulateDocumentResult> results = new ArrayList<>(numResults);
|
||||
for (int i = 0; i < numResults; i++) {
|
||||
boolean isFailure = randomBoolean();
|
||||
IngestDocument ingestDocument = RandomDocumentPicks.randomIngestDocument(random());
|
||||
if (isVerbose) {
|
||||
int numProcessors = randomIntBetween(1, 10);
|
||||
List<SimulateProcessorResult> processorResults = new ArrayList<>(numProcessors);
|
||||
for (int j = 0; j < numProcessors; j++) {
|
||||
String processorTag = randomAsciiOfLengthBetween(1, 10);
|
||||
SimulateProcessorResult processorResult;
|
||||
if (isFailure) {
|
||||
processorResult = new SimulateProcessorResult(processorTag, new IllegalArgumentException("test"));
|
||||
} else {
|
||||
processorResult = new SimulateProcessorResult(processorTag, ingestDocument);
|
||||
}
|
||||
processorResults.add(processorResult);
|
||||
}
|
||||
results.add(new SimulateDocumentVerboseResult(processorResults));
|
||||
} else {
|
||||
results.add(new SimulateDocumentBaseResult(ingestDocument));
|
||||
SimulateDocumentBaseResult simulateDocumentBaseResult;
|
||||
if (isFailure) {
|
||||
simulateDocumentBaseResult = new SimulateDocumentBaseResult(new IllegalArgumentException("test"));
|
||||
} else {
|
||||
simulateDocumentBaseResult = new SimulateDocumentBaseResult(ingestDocument);
|
||||
}
|
||||
results.add(simulateDocumentBaseResult);
|
||||
}
|
||||
}
|
||||
|
||||
SimulatePipelineResponse response = new SimulatePipelineResponse(randomAsciiOfLengthBetween(1, 10), isVerbose, results);
|
||||
BytesStreamOutput out = new BytesStreamOutput();
|
||||
response.writeTo(out);
|
||||
StreamInput streamInput = StreamInput.wrap(out.bytes());
|
||||
SimulatePipelineResponse otherResponse = new SimulatePipelineResponse();
|
||||
otherResponse.readFrom(streamInput);
|
||||
|
||||
assertThat(otherResponse.getPipelineId(), equalTo(response.getPipelineId()));
|
||||
assertThat(otherResponse.getResults().size(), equalTo(response.getResults().size()));
|
||||
|
||||
Iterator<SimulateDocumentResult> expectedResultIterator = response.getResults().iterator();
|
||||
for (SimulateDocumentResult result : otherResponse.getResults()) {
|
||||
if (isVerbose) {
|
||||
SimulateDocumentVerboseResult expectedSimulateDocumentVerboseResult = (SimulateDocumentVerboseResult) expectedResultIterator.next();
|
||||
assertThat(result, instanceOf(SimulateDocumentVerboseResult.class));
|
||||
SimulateDocumentVerboseResult simulateDocumentVerboseResult = (SimulateDocumentVerboseResult) result;
|
||||
assertThat(simulateDocumentVerboseResult.getProcessorResults().size(), equalTo(expectedSimulateDocumentVerboseResult.getProcessorResults().size()));
|
||||
Iterator<SimulateProcessorResult> expectedProcessorResultIterator = expectedSimulateDocumentVerboseResult.getProcessorResults().iterator();
|
||||
for (SimulateProcessorResult simulateProcessorResult : simulateDocumentVerboseResult.getProcessorResults()) {
|
||||
SimulateProcessorResult expectedProcessorResult = expectedProcessorResultIterator.next();
|
||||
assertThat(simulateProcessorResult.getProcessorTag(), equalTo(expectedProcessorResult.getProcessorTag()));
|
||||
assertThat(simulateProcessorResult.getIngestDocument(), equalTo(expectedProcessorResult.getIngestDocument()));
|
||||
if (expectedProcessorResult.getFailure() == null) {
|
||||
assertThat(simulateProcessorResult.getFailure(), nullValue());
|
||||
} else {
|
||||
assertThat(simulateProcessorResult.getFailure(), instanceOf(IllegalArgumentException.class));
|
||||
IllegalArgumentException e = (IllegalArgumentException) simulateProcessorResult.getFailure();
|
||||
assertThat(e.getMessage(), equalTo("test"));
|
||||
}
|
||||
}
|
||||
} else {
|
||||
SimulateDocumentBaseResult expectedSimulateDocumentBaseResult = (SimulateDocumentBaseResult) expectedResultIterator.next();
|
||||
assertThat(result, instanceOf(SimulateDocumentBaseResult.class));
|
||||
SimulateDocumentBaseResult simulateDocumentBaseResult = (SimulateDocumentBaseResult) result;
|
||||
assertThat(simulateDocumentBaseResult.getIngestDocument(), equalTo(expectedSimulateDocumentBaseResult.getIngestDocument()));
|
||||
if (expectedSimulateDocumentBaseResult.getFailure() == null) {
|
||||
assertThat(simulateDocumentBaseResult.getFailure(), nullValue());
|
||||
} else {
|
||||
assertThat(simulateDocumentBaseResult.getFailure(), instanceOf(IllegalArgumentException.class));
|
||||
IllegalArgumentException e = (IllegalArgumentException) simulateDocumentBaseResult.getFailure();
|
||||
assertThat(e.getMessage(), equalTo("test"));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,58 @@
|
||||
/*
|
||||
* 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.action.ingest;
|
||||
|
||||
import org.elasticsearch.common.io.stream.BytesStreamOutput;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.ingest.core.IngestDocument;
|
||||
import org.elasticsearch.ingest.RandomDocumentPicks;
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import static org.hamcrest.Matchers.equalTo;
|
||||
import static org.hamcrest.Matchers.instanceOf;
|
||||
|
||||
public class SimulateProcessorResultTests extends ESTestCase {
|
||||
|
||||
public void testSerialization() throws IOException {
|
||||
String processorTag = randomAsciiOfLengthBetween(1, 10);
|
||||
boolean isFailure = randomBoolean();
|
||||
SimulateProcessorResult simulateProcessorResult;
|
||||
if (isFailure) {
|
||||
simulateProcessorResult = new SimulateProcessorResult(processorTag, new IllegalArgumentException("test"));
|
||||
} else {
|
||||
IngestDocument ingestDocument = RandomDocumentPicks.randomIngestDocument(random());
|
||||
simulateProcessorResult = new SimulateProcessorResult(processorTag, ingestDocument);
|
||||
}
|
||||
|
||||
BytesStreamOutput out = new BytesStreamOutput();
|
||||
simulateProcessorResult.writeTo(out);
|
||||
StreamInput streamInput = StreamInput.wrap(out.bytes());
|
||||
SimulateProcessorResult otherSimulateProcessorResult = new SimulateProcessorResult(streamInput);
|
||||
assertThat(otherSimulateProcessorResult.getProcessorTag(), equalTo(simulateProcessorResult.getProcessorTag()));
|
||||
assertThat(otherSimulateProcessorResult.getIngestDocument(), equalTo(simulateProcessorResult.getIngestDocument()));
|
||||
if (isFailure) {
|
||||
assertThat(otherSimulateProcessorResult.getFailure(), instanceOf(IllegalArgumentException.class));
|
||||
IllegalArgumentException e = (IllegalArgumentException) otherSimulateProcessorResult.getFailure();
|
||||
assertThat(e.getMessage(), equalTo("test"));
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,114 @@
|
||||
/*
|
||||
* 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.action.ingest;
|
||||
|
||||
import org.elasticsearch.common.io.stream.BytesStreamOutput;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.ingest.RandomDocumentPicks;
|
||||
import org.elasticsearch.ingest.core.IngestDocument;
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
import static org.hamcrest.Matchers.equalTo;
|
||||
import static org.hamcrest.Matchers.not;
|
||||
|
||||
public class WriteableIngestDocumentTests extends ESTestCase {
|
||||
|
||||
public void testEqualsAndHashcode() throws Exception {
|
||||
Map<String, Object> sourceAndMetadata = RandomDocumentPicks.randomSource(random());
|
||||
int numFields = randomIntBetween(1, IngestDocument.MetaData.values().length);
|
||||
for (int i = 0; i < numFields; i++) {
|
||||
sourceAndMetadata.put(randomFrom(IngestDocument.MetaData.values()).getFieldName(), randomAsciiOfLengthBetween(5, 10));
|
||||
}
|
||||
Map<String, String> ingestMetadata = new HashMap<>();
|
||||
numFields = randomIntBetween(1, 5);
|
||||
for (int i = 0; i < numFields; i++) {
|
||||
ingestMetadata.put(randomAsciiOfLengthBetween(5, 10), randomAsciiOfLengthBetween(5, 10));
|
||||
}
|
||||
WriteableIngestDocument ingestDocument = new WriteableIngestDocument(new IngestDocument(sourceAndMetadata, ingestMetadata));
|
||||
|
||||
boolean changed = false;
|
||||
Map<String, Object> otherSourceAndMetadata;
|
||||
if (randomBoolean()) {
|
||||
otherSourceAndMetadata = RandomDocumentPicks.randomSource(random());
|
||||
changed = true;
|
||||
} else {
|
||||
otherSourceAndMetadata = new HashMap<>(sourceAndMetadata);
|
||||
}
|
||||
if (randomBoolean()) {
|
||||
numFields = randomIntBetween(1, IngestDocument.MetaData.values().length);
|
||||
for (int i = 0; i < numFields; i++) {
|
||||
otherSourceAndMetadata.put(randomFrom(IngestDocument.MetaData.values()).getFieldName(), randomAsciiOfLengthBetween(5, 10));
|
||||
}
|
||||
changed = true;
|
||||
}
|
||||
|
||||
Map<String, String> otherIngestMetadata;
|
||||
if (randomBoolean()) {
|
||||
otherIngestMetadata = new HashMap<>();
|
||||
numFields = randomIntBetween(1, 5);
|
||||
for (int i = 0; i < numFields; i++) {
|
||||
otherIngestMetadata.put(randomAsciiOfLengthBetween(5, 10), randomAsciiOfLengthBetween(5, 10));
|
||||
}
|
||||
changed = true;
|
||||
} else {
|
||||
otherIngestMetadata = Collections.unmodifiableMap(ingestMetadata);
|
||||
}
|
||||
|
||||
WriteableIngestDocument otherIngestDocument = new WriteableIngestDocument(new IngestDocument(otherSourceAndMetadata, otherIngestMetadata));
|
||||
if (changed) {
|
||||
assertThat(ingestDocument, not(equalTo(otherIngestDocument)));
|
||||
assertThat(otherIngestDocument, not(equalTo(ingestDocument)));
|
||||
} else {
|
||||
assertThat(ingestDocument, equalTo(otherIngestDocument));
|
||||
assertThat(otherIngestDocument, equalTo(ingestDocument));
|
||||
assertThat(ingestDocument.hashCode(), equalTo(otherIngestDocument.hashCode()));
|
||||
WriteableIngestDocument thirdIngestDocument = new WriteableIngestDocument(new IngestDocument(Collections.unmodifiableMap(sourceAndMetadata), Collections.unmodifiableMap(ingestMetadata)));
|
||||
assertThat(thirdIngestDocument, equalTo(ingestDocument));
|
||||
assertThat(ingestDocument, equalTo(thirdIngestDocument));
|
||||
assertThat(ingestDocument.hashCode(), equalTo(thirdIngestDocument.hashCode()));
|
||||
}
|
||||
}
|
||||
|
||||
public void testSerialization() throws IOException {
|
||||
Map<String, Object> sourceAndMetadata = RandomDocumentPicks.randomSource(random());
|
||||
int numFields = randomIntBetween(1, IngestDocument.MetaData.values().length);
|
||||
for (int i = 0; i < numFields; i++) {
|
||||
sourceAndMetadata.put(randomFrom(IngestDocument.MetaData.values()).getFieldName(), randomAsciiOfLengthBetween(5, 10));
|
||||
}
|
||||
Map<String, String> ingestMetadata = new HashMap<>();
|
||||
numFields = randomIntBetween(1, 5);
|
||||
for (int i = 0; i < numFields; i++) {
|
||||
ingestMetadata.put(randomAsciiOfLengthBetween(5, 10), randomAsciiOfLengthBetween(5, 10));
|
||||
}
|
||||
Map<String, Object> document = RandomDocumentPicks.randomSource(random());
|
||||
WriteableIngestDocument writeableIngestDocument = new WriteableIngestDocument(new IngestDocument(sourceAndMetadata, ingestMetadata));
|
||||
|
||||
BytesStreamOutput out = new BytesStreamOutput();
|
||||
writeableIngestDocument.writeTo(out);
|
||||
StreamInput streamInput = StreamInput.wrap(out.bytes());
|
||||
WriteableIngestDocument otherWriteableIngestDocument = new WriteableIngestDocument(streamInput);
|
||||
assertThat(otherWriteableIngestDocument, equalTo(writeableIngestDocument));
|
||||
}
|
||||
}
|
@ -63,6 +63,7 @@ import static org.hamcrest.Matchers.nullValue;
|
||||
*
|
||||
*/
|
||||
public class SimpleIndexTemplateIT extends ESIntegTestCase {
|
||||
|
||||
public void testSimpleIndexTemplateTests() throws Exception {
|
||||
// clean all templates setup by the framework.
|
||||
client().admin().indices().prepareDeleteTemplate("*").get();
|
||||
|
231
core/src/test/java/org/elasticsearch/ingest/IngestClientIT.java
Normal file
231
core/src/test/java/org/elasticsearch/ingest/IngestClientIT.java
Normal file
@ -0,0 +1,231 @@
|
||||
/*
|
||||
* 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.action.bulk.BulkItemResponse;
|
||||
import org.elasticsearch.action.bulk.BulkRequest;
|
||||
import org.elasticsearch.action.bulk.BulkResponse;
|
||||
import org.elasticsearch.action.index.IndexRequest;
|
||||
import org.elasticsearch.action.index.IndexResponse;
|
||||
import org.elasticsearch.action.ingest.DeletePipelineRequest;
|
||||
import org.elasticsearch.action.ingest.GetPipelineRequest;
|
||||
import org.elasticsearch.action.ingest.GetPipelineResponse;
|
||||
import org.elasticsearch.action.ingest.PutPipelineRequest;
|
||||
import org.elasticsearch.action.ingest.SimulateDocumentBaseResult;
|
||||
import org.elasticsearch.action.ingest.SimulatePipelineRequest;
|
||||
import org.elasticsearch.action.ingest.SimulatePipelineResponse;
|
||||
import org.elasticsearch.action.ingest.WritePipelineResponse;
|
||||
import org.elasticsearch.common.bytes.BytesReference;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.ingest.core.IngestDocument;
|
||||
import org.elasticsearch.node.NodeModule;
|
||||
import org.elasticsearch.plugins.Plugin;
|
||||
import org.elasticsearch.test.ESIntegTestCase;
|
||||
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
|
||||
import static org.hamcrest.Matchers.equalTo;
|
||||
import static org.hamcrest.Matchers.instanceOf;
|
||||
import static org.hamcrest.Matchers.nullValue;
|
||||
import static org.hamcrest.core.Is.is;
|
||||
|
||||
@ESIntegTestCase.ClusterScope(minNumDataNodes = 2)
|
||||
public class IngestClientIT extends ESIntegTestCase {
|
||||
|
||||
@Override
|
||||
protected Settings nodeSettings(int nodeOrdinal) {
|
||||
// TODO: Remove this method once gets in: https://github.com/elastic/elasticsearch/issues/16019
|
||||
if (nodeOrdinal % 2 == 0) {
|
||||
return Settings.builder().put("node.ingest", false).put(super.nodeSettings(nodeOrdinal)).build();
|
||||
}
|
||||
return super.nodeSettings(nodeOrdinal);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Collection<Class<? extends Plugin>> nodePlugins() {
|
||||
return pluginList(IngestPlugin.class);
|
||||
}
|
||||
|
||||
public void testSimulate() throws Exception {
|
||||
BytesReference pipelineSource = jsonBuilder().startObject()
|
||||
.field("description", "my_pipeline")
|
||||
.startArray("processors")
|
||||
.startObject()
|
||||
.startObject("test")
|
||||
.endObject()
|
||||
.endObject()
|
||||
.endArray()
|
||||
.endObject().bytes();
|
||||
client().preparePutPipeline("_id", pipelineSource)
|
||||
.get();
|
||||
GetPipelineResponse getResponse = client().prepareGetPipeline("_id")
|
||||
.get();
|
||||
assertThat(getResponse.isFound(), is(true));
|
||||
assertThat(getResponse.pipelines().size(), equalTo(1));
|
||||
assertThat(getResponse.pipelines().get(0).getId(), equalTo("_id"));
|
||||
|
||||
BytesReference bytes = jsonBuilder().startObject()
|
||||
.startArray("docs")
|
||||
.startObject()
|
||||
.field("_index", "index")
|
||||
.field("_type", "type")
|
||||
.field("_id", "id")
|
||||
.startObject("_source")
|
||||
.field("foo", "bar")
|
||||
.field("fail", false)
|
||||
.endObject()
|
||||
.endObject()
|
||||
.endArray()
|
||||
.endObject().bytes();
|
||||
SimulatePipelineResponse response;
|
||||
if (randomBoolean()) {
|
||||
response = client().prepareSimulatePipeline(bytes)
|
||||
.setId("_id").get();
|
||||
} else {
|
||||
SimulatePipelineRequest request = new SimulatePipelineRequest(bytes);
|
||||
request.setId("_id");
|
||||
response = client().simulatePipeline(request).get();
|
||||
}
|
||||
assertThat(response.isVerbose(), equalTo(false));
|
||||
assertThat(response.getPipelineId(), equalTo("_id"));
|
||||
assertThat(response.getResults().size(), equalTo(1));
|
||||
assertThat(response.getResults().get(0), instanceOf(SimulateDocumentBaseResult.class));
|
||||
SimulateDocumentBaseResult simulateDocumentBaseResult = (SimulateDocumentBaseResult) response.getResults().get(0);
|
||||
Map<String, Object> source = new HashMap<>();
|
||||
source.put("foo", "bar");
|
||||
source.put("fail", false);
|
||||
source.put("processed", true);
|
||||
IngestDocument ingestDocument = new IngestDocument("index", "type", "id", null, null, null, null, source);
|
||||
assertThat(simulateDocumentBaseResult.getIngestDocument().getSourceAndMetadata(), equalTo(ingestDocument.getSourceAndMetadata()));
|
||||
assertThat(simulateDocumentBaseResult.getFailure(), nullValue());
|
||||
}
|
||||
|
||||
public void testBulkWithIngestFailures() throws Exception {
|
||||
createIndex("index");
|
||||
|
||||
BytesReference source = jsonBuilder().startObject()
|
||||
.field("description", "my_pipeline")
|
||||
.startArray("processors")
|
||||
.startObject()
|
||||
.startObject("test")
|
||||
.endObject()
|
||||
.endObject()
|
||||
.endArray()
|
||||
.endObject().bytes();
|
||||
PutPipelineRequest putPipelineRequest = new PutPipelineRequest("_id", source);
|
||||
client().putPipeline(putPipelineRequest).get();
|
||||
|
||||
int numRequests = scaledRandomIntBetween(32, 128);
|
||||
BulkRequest bulkRequest = new BulkRequest();
|
||||
for (int i = 0; i < numRequests; i++) {
|
||||
IndexRequest indexRequest = new IndexRequest("index", "type", Integer.toString(i)).setPipeline("_id");
|
||||
indexRequest.source("field", "value", "fail", i % 2 == 0);
|
||||
bulkRequest.add(indexRequest);
|
||||
}
|
||||
|
||||
BulkResponse response = client().bulk(bulkRequest).actionGet();
|
||||
assertThat(response.getItems().length, equalTo(bulkRequest.requests().size()));
|
||||
for (int i = 0; i < bulkRequest.requests().size(); i++) {
|
||||
BulkItemResponse itemResponse = response.getItems()[i];
|
||||
if (i % 2 == 0) {
|
||||
BulkItemResponse.Failure failure = itemResponse.getFailure();
|
||||
assertThat(failure.getMessage(), equalTo("java.lang.IllegalArgumentException: test processor failed"));
|
||||
} else {
|
||||
IndexResponse indexResponse = itemResponse.getResponse();
|
||||
assertThat(indexResponse.getId(), equalTo(Integer.toString(i)));
|
||||
assertThat(indexResponse.isCreated(), is(true));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public void test() throws Exception {
|
||||
BytesReference source = jsonBuilder().startObject()
|
||||
.field("description", "my_pipeline")
|
||||
.startArray("processors")
|
||||
.startObject()
|
||||
.startObject("test")
|
||||
.endObject()
|
||||
.endObject()
|
||||
.endArray()
|
||||
.endObject().bytes();
|
||||
PutPipelineRequest putPipelineRequest = new PutPipelineRequest("_id", source);
|
||||
client().putPipeline(putPipelineRequest).get();
|
||||
|
||||
GetPipelineRequest getPipelineRequest = new GetPipelineRequest("_id");
|
||||
GetPipelineResponse getResponse = client().getPipeline(getPipelineRequest).get();
|
||||
assertThat(getResponse.isFound(), is(true));
|
||||
assertThat(getResponse.pipelines().size(), equalTo(1));
|
||||
assertThat(getResponse.pipelines().get(0).getId(), equalTo("_id"));
|
||||
|
||||
client().prepareIndex("test", "type", "1").setPipeline("_id").setSource("field", "value", "fail", false).get();
|
||||
|
||||
Map<String, Object> doc = client().prepareGet("test", "type", "1")
|
||||
.get().getSourceAsMap();
|
||||
assertThat(doc.get("field"), equalTo("value"));
|
||||
assertThat(doc.get("processed"), equalTo(true));
|
||||
|
||||
client().prepareBulk().add(
|
||||
client().prepareIndex("test", "type", "2").setSource("field", "value2", "fail", false).setPipeline("_id")).get();
|
||||
doc = client().prepareGet("test", "type", "2").get().getSourceAsMap();
|
||||
assertThat(doc.get("field"), equalTo("value2"));
|
||||
assertThat(doc.get("processed"), equalTo(true));
|
||||
|
||||
DeletePipelineRequest deletePipelineRequest = new DeletePipelineRequest("_id");
|
||||
WritePipelineResponse response = client().deletePipeline(deletePipelineRequest).get();
|
||||
assertThat(response.isAcknowledged(), is(true));
|
||||
|
||||
getResponse = client().prepareGetPipeline("_id").get();
|
||||
assertThat(getResponse.isFound(), is(false));
|
||||
assertThat(getResponse.pipelines().size(), equalTo(0));
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Collection<Class<? extends Plugin>> getMockPlugins() {
|
||||
return Collections.singletonList(TestSeedPlugin.class);
|
||||
}
|
||||
|
||||
public static class IngestPlugin extends Plugin {
|
||||
|
||||
@Override
|
||||
public String name() {
|
||||
return "ingest";
|
||||
}
|
||||
|
||||
@Override
|
||||
public String description() {
|
||||
return "ingest mock";
|
||||
}
|
||||
|
||||
public void onModule(NodeModule nodeModule) {
|
||||
nodeModule.registerProcessor("test", templateService -> config ->
|
||||
new TestProcessor("id", "test", ingestDocument -> {
|
||||
ingestDocument.setFieldValue("processed", true);
|
||||
if (ingestDocument.getFieldValue("fail", Boolean.class)) {
|
||||
throw new IllegalArgumentException("test processor failed");
|
||||
}
|
||||
})
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,64 @@
|
||||
/*
|
||||
* 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.cluster.metadata.MetaData;
|
||||
import org.elasticsearch.common.bytes.BytesArray;
|
||||
import org.elasticsearch.common.xcontent.ToXContent;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.common.xcontent.XContentFactory;
|
||||
import org.elasticsearch.common.xcontent.XContentParser;
|
||||
import org.elasticsearch.common.xcontent.XContentType;
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
public class IngestMetadataTests extends ESTestCase {
|
||||
|
||||
public void testFromXContent() throws IOException {
|
||||
PipelineConfiguration pipeline = new PipelineConfiguration(
|
||||
"1",new BytesArray("{\"processors\": [{\"set\" : {\"field\": \"_field\", \"value\": \"_value\"}}]}")
|
||||
);
|
||||
PipelineConfiguration pipeline2 = new PipelineConfiguration(
|
||||
"2",new BytesArray("{\"processors\": [{\"set\" : {\"field\": \"_field1\", \"value\": \"_value1\"}}]}")
|
||||
);
|
||||
Map<String, PipelineConfiguration> map = new HashMap<>();
|
||||
map.put(pipeline.getId(), pipeline);
|
||||
map.put(pipeline2.getId(), pipeline2);
|
||||
IngestMetadata ingestMetadata = new IngestMetadata(map);
|
||||
XContentBuilder builder = XContentFactory.jsonBuilder();
|
||||
builder.prettyPrint();
|
||||
builder.startObject();
|
||||
ingestMetadata.toXContent(builder, ToXContent.EMPTY_PARAMS);
|
||||
builder.endObject();
|
||||
String string = builder.string();
|
||||
final XContentParser parser = XContentFactory.xContent(XContentType.JSON).createParser(string);
|
||||
MetaData.Custom custom = ingestMetadata.fromXContent(parser);
|
||||
assertTrue(custom instanceof IngestMetadata);
|
||||
IngestMetadata m = (IngestMetadata) custom;
|
||||
assertEquals(2, m.getPipelines().size());
|
||||
assertEquals("1", m.getPipelines().get("1").getId());
|
||||
assertEquals("2", m.getPipelines().get("2").getId());
|
||||
assertEquals(pipeline.getConfigAsMap(), m.getPipelines().get("1").getConfigAsMap());
|
||||
assertEquals(pipeline2.getConfigAsMap(), m.getPipelines().get("2").getConfigAsMap());
|
||||
}
|
||||
}
|
@ -0,0 +1,366 @@
|
||||
/*
|
||||
* 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.ElasticsearchParseException;
|
||||
import org.elasticsearch.action.ActionRequest;
|
||||
import org.elasticsearch.action.bulk.BulkRequest;
|
||||
import org.elasticsearch.action.delete.DeleteRequest;
|
||||
import org.elasticsearch.action.index.IndexRequest;
|
||||
import org.elasticsearch.action.update.UpdateRequest;
|
||||
import org.elasticsearch.common.unit.TimeValue;
|
||||
import org.elasticsearch.ingest.core.CompoundProcessor;
|
||||
import org.elasticsearch.ingest.core.IngestDocument;
|
||||
import org.elasticsearch.ingest.core.Pipeline;
|
||||
import org.elasticsearch.ingest.core.Processor;
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
import org.hamcrest.CustomTypeSafeMatcher;
|
||||
import org.junit.Before;
|
||||
import org.mockito.ArgumentMatcher;
|
||||
import org.mockito.invocation.InvocationOnMock;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
import java.util.function.BiConsumer;
|
||||
import java.util.function.Consumer;
|
||||
|
||||
import static org.hamcrest.Matchers.equalTo;
|
||||
import static org.mockito.Matchers.eq;
|
||||
import static org.mockito.Matchers.any;
|
||||
import static org.mockito.Matchers.anyBoolean;
|
||||
import static org.mockito.Matchers.anyString;
|
||||
import static org.mockito.Matchers.argThat;
|
||||
import static org.mockito.Mockito.doAnswer;
|
||||
import static org.mockito.Mockito.doThrow;
|
||||
import static org.mockito.Mockito.mock;
|
||||
import static org.mockito.Mockito.never;
|
||||
import static org.mockito.Mockito.times;
|
||||
import static org.mockito.Mockito.verify;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
public class PipelineExecutionServiceTests extends ESTestCase {
|
||||
|
||||
private PipelineStore store;
|
||||
private PipelineExecutionService executionService;
|
||||
|
||||
@Before
|
||||
public void setup() {
|
||||
store = mock(PipelineStore.class);
|
||||
ThreadPool threadPool = mock(ThreadPool.class);
|
||||
when(threadPool.executor(anyString())).thenReturn(Runnable::run);
|
||||
executionService = new PipelineExecutionService(store, threadPool);
|
||||
}
|
||||
|
||||
public void testExecuteIndexPipelineDoesNotExist() {
|
||||
IndexRequest indexRequest = new IndexRequest("_index", "_type", "_id").source(Collections.emptyMap()).setPipeline("_id");
|
||||
@SuppressWarnings("unchecked")
|
||||
Consumer<Throwable> failureHandler = mock(Consumer.class);
|
||||
@SuppressWarnings("unchecked")
|
||||
Consumer<Boolean> completionHandler = mock(Consumer.class);
|
||||
try {
|
||||
executionService.execute(indexRequest, failureHandler, completionHandler);
|
||||
fail("IllegalArgumentException expected");
|
||||
} catch (IllegalArgumentException e) {
|
||||
assertThat(e.getMessage(), equalTo("pipeline with id [_id] does not exist"));
|
||||
}
|
||||
verify(failureHandler, never()).accept(any(Throwable.class));
|
||||
verify(completionHandler, never()).accept(anyBoolean());
|
||||
}
|
||||
|
||||
public void testExecuteBulkPipelineDoesNotExist() {
|
||||
CompoundProcessor processor = mock(CompoundProcessor.class);
|
||||
when(store.get("_id")).thenReturn(new Pipeline("_id", "_description", processor));
|
||||
BulkRequest bulkRequest = new BulkRequest();
|
||||
|
||||
IndexRequest indexRequest1 = new IndexRequest("_index", "_type", "_id").source(Collections.emptyMap()).setPipeline("_id");
|
||||
bulkRequest.add(indexRequest1);
|
||||
IndexRequest indexRequest2 = new IndexRequest("_index", "_type", "_id").source(Collections.emptyMap()).setPipeline("does_not_exist");
|
||||
bulkRequest.add(indexRequest2);
|
||||
@SuppressWarnings("unchecked")
|
||||
BiConsumer<IndexRequest, Throwable> failureHandler = mock(BiConsumer.class);
|
||||
@SuppressWarnings("unchecked")
|
||||
Consumer<Throwable> completionHandler = mock(Consumer.class);
|
||||
executionService.execute(bulkRequest.requests(), failureHandler, completionHandler);
|
||||
verify(failureHandler, times(1)).accept(
|
||||
argThat(new CustomTypeSafeMatcher<IndexRequest>("failure handler was not called with the expected arguments") {
|
||||
@Override
|
||||
protected boolean matchesSafely(IndexRequest item) {
|
||||
return item == indexRequest2;
|
||||
}
|
||||
|
||||
}),
|
||||
argThat(new CustomTypeSafeMatcher<IllegalArgumentException>("failure handler was not called with the expected arguments") {
|
||||
@Override
|
||||
protected boolean matchesSafely(IllegalArgumentException iae) {
|
||||
return "pipeline with id [does_not_exist] does not exist".equals(iae.getMessage());
|
||||
}
|
||||
})
|
||||
);
|
||||
verify(completionHandler, times(1)).accept(null);
|
||||
}
|
||||
|
||||
public void testExecuteSuccess() throws Exception {
|
||||
CompoundProcessor processor = mock(CompoundProcessor.class);
|
||||
when(store.get("_id")).thenReturn(new Pipeline("_id", "_description", processor));
|
||||
|
||||
IndexRequest indexRequest = new IndexRequest("_index", "_type", "_id").source(Collections.emptyMap()).setPipeline("_id");
|
||||
@SuppressWarnings("unchecked")
|
||||
Consumer<Throwable> failureHandler = mock(Consumer.class);
|
||||
@SuppressWarnings("unchecked")
|
||||
Consumer<Boolean> completionHandler = mock(Consumer.class);
|
||||
executionService.execute(indexRequest, failureHandler, completionHandler);
|
||||
verify(failureHandler, never()).accept(any());
|
||||
verify(completionHandler, times(1)).accept(true);
|
||||
}
|
||||
|
||||
public void testExecutePropagateAllMetaDataUpdates() throws Exception {
|
||||
CompoundProcessor processor = mock(CompoundProcessor.class);
|
||||
doAnswer((InvocationOnMock invocationOnMock) -> {
|
||||
IngestDocument ingestDocument = (IngestDocument) invocationOnMock.getArguments()[0];
|
||||
for (IngestDocument.MetaData metaData : IngestDocument.MetaData.values()) {
|
||||
if (metaData == IngestDocument.MetaData.TTL) {
|
||||
ingestDocument.setFieldValue(IngestDocument.MetaData.TTL.getFieldName(), "5w");
|
||||
} else {
|
||||
ingestDocument.setFieldValue(metaData.getFieldName(), "update" + metaData.getFieldName());
|
||||
}
|
||||
|
||||
}
|
||||
return null;
|
||||
}).when(processor).execute(any());
|
||||
when(store.get("_id")).thenReturn(new Pipeline("_id", "_description", processor));
|
||||
|
||||
IndexRequest indexRequest = new IndexRequest("_index", "_type", "_id").source(Collections.emptyMap()).setPipeline("_id");
|
||||
@SuppressWarnings("unchecked")
|
||||
Consumer<Throwable> failureHandler = mock(Consumer.class);
|
||||
@SuppressWarnings("unchecked")
|
||||
Consumer<Boolean> completionHandler = mock(Consumer.class);
|
||||
executionService.execute(indexRequest, failureHandler, completionHandler);
|
||||
verify(processor).execute(any());
|
||||
verify(failureHandler, never()).accept(any());
|
||||
verify(completionHandler, times(1)).accept(true);
|
||||
|
||||
assertThat(indexRequest.index(), equalTo("update_index"));
|
||||
assertThat(indexRequest.type(), equalTo("update_type"));
|
||||
assertThat(indexRequest.id(), equalTo("update_id"));
|
||||
assertThat(indexRequest.routing(), equalTo("update_routing"));
|
||||
assertThat(indexRequest.parent(), equalTo("update_parent"));
|
||||
assertThat(indexRequest.timestamp(), equalTo("update_timestamp"));
|
||||
assertThat(indexRequest.ttl(), equalTo(new TimeValue(3024000000L)));
|
||||
}
|
||||
|
||||
public void testExecuteFailure() throws Exception {
|
||||
CompoundProcessor processor = mock(CompoundProcessor.class);
|
||||
when(store.get("_id")).thenReturn(new Pipeline("_id", "_description", processor));
|
||||
IndexRequest indexRequest = new IndexRequest("_index", "_type", "_id").source(Collections.emptyMap()).setPipeline("_id");
|
||||
doThrow(new RuntimeException()).when(processor).execute(eqID("_index", "_type", "_id", Collections.emptyMap()));
|
||||
@SuppressWarnings("unchecked")
|
||||
Consumer<Throwable> failureHandler = mock(Consumer.class);
|
||||
@SuppressWarnings("unchecked")
|
||||
Consumer<Boolean> completionHandler = mock(Consumer.class);
|
||||
executionService.execute(indexRequest, failureHandler, completionHandler);
|
||||
verify(processor).execute(eqID("_index", "_type", "_id", Collections.emptyMap()));
|
||||
verify(failureHandler, times(1)).accept(any(RuntimeException.class));
|
||||
verify(completionHandler, never()).accept(anyBoolean());
|
||||
}
|
||||
|
||||
public void testExecuteSuccessWithOnFailure() throws Exception {
|
||||
Processor processor = mock(Processor.class);
|
||||
Processor onFailureProcessor = mock(Processor.class);
|
||||
CompoundProcessor compoundProcessor = new CompoundProcessor(Collections.singletonList(processor), Collections.singletonList(new CompoundProcessor(onFailureProcessor)));
|
||||
when(store.get("_id")).thenReturn(new Pipeline("_id", "_description", compoundProcessor));
|
||||
IndexRequest indexRequest = new IndexRequest("_index", "_type", "_id").source(Collections.emptyMap()).setPipeline("_id");
|
||||
doThrow(new RuntimeException()).when(processor).execute(eqID("_index", "_type", "_id", Collections.emptyMap()));
|
||||
@SuppressWarnings("unchecked")
|
||||
Consumer<Throwable> failureHandler = mock(Consumer.class);
|
||||
@SuppressWarnings("unchecked")
|
||||
Consumer<Boolean> completionHandler = mock(Consumer.class);
|
||||
executionService.execute(indexRequest, failureHandler, completionHandler);
|
||||
verify(failureHandler, never()).accept(any(RuntimeException.class));
|
||||
verify(completionHandler, times(1)).accept(true);
|
||||
}
|
||||
|
||||
public void testExecuteFailureWithOnFailure() throws Exception {
|
||||
Processor processor = mock(Processor.class);
|
||||
Processor onFailureProcessor = mock(Processor.class);
|
||||
CompoundProcessor compoundProcessor = new CompoundProcessor(Collections.singletonList(processor), Collections.singletonList(new CompoundProcessor(onFailureProcessor)));
|
||||
when(store.get("_id")).thenReturn(new Pipeline("_id", "_description", compoundProcessor));
|
||||
IndexRequest indexRequest = new IndexRequest("_index", "_type", "_id").source(Collections.emptyMap()).setPipeline("_id");
|
||||
doThrow(new RuntimeException()).when(processor).execute(eqID("_index", "_type", "_id", Collections.emptyMap()));
|
||||
doThrow(new RuntimeException()).when(onFailureProcessor).execute(eqID("_index", "_type", "_id", Collections.emptyMap()));
|
||||
@SuppressWarnings("unchecked")
|
||||
Consumer<Throwable> failureHandler = mock(Consumer.class);
|
||||
@SuppressWarnings("unchecked")
|
||||
Consumer<Boolean> completionHandler = mock(Consumer.class);
|
||||
executionService.execute(indexRequest, failureHandler, completionHandler);
|
||||
verify(processor).execute(eqID("_index", "_type", "_id", Collections.emptyMap()));
|
||||
verify(failureHandler, times(1)).accept(any(RuntimeException.class));
|
||||
verify(completionHandler, never()).accept(anyBoolean());
|
||||
}
|
||||
|
||||
public void testExecuteFailureWithNestedOnFailure() throws Exception {
|
||||
Processor processor = mock(Processor.class);
|
||||
Processor onFailureProcessor = mock(Processor.class);
|
||||
Processor onFailureOnFailureProcessor = mock(Processor.class);
|
||||
CompoundProcessor compoundProcessor = new CompoundProcessor(Collections.singletonList(processor),
|
||||
Collections.singletonList(new CompoundProcessor(Collections.singletonList(onFailureProcessor), Collections.singletonList(onFailureOnFailureProcessor))));
|
||||
when(store.get("_id")).thenReturn(new Pipeline("_id", "_description", compoundProcessor));
|
||||
IndexRequest indexRequest = new IndexRequest("_index", "_type", "_id").source(Collections.emptyMap()).setPipeline("_id");
|
||||
doThrow(new RuntimeException()).when(onFailureOnFailureProcessor).execute(eqID("_index", "_type", "_id", Collections.emptyMap()));
|
||||
doThrow(new RuntimeException()).when(onFailureProcessor).execute(eqID("_index", "_type", "_id", Collections.emptyMap()));
|
||||
doThrow(new RuntimeException()).when(processor).execute(eqID("_index", "_type", "_id", Collections.emptyMap()));
|
||||
@SuppressWarnings("unchecked")
|
||||
Consumer<Throwable> failureHandler = mock(Consumer.class);
|
||||
@SuppressWarnings("unchecked")
|
||||
Consumer<Boolean> completionHandler = mock(Consumer.class);
|
||||
executionService.execute(indexRequest, failureHandler, completionHandler);
|
||||
verify(processor).execute(eqID("_index", "_type", "_id", Collections.emptyMap()));
|
||||
verify(failureHandler, times(1)).accept(any(RuntimeException.class));
|
||||
verify(completionHandler, never()).accept(anyBoolean());
|
||||
}
|
||||
|
||||
public void testExecuteSetTTL() throws Exception {
|
||||
Processor processor = new TestProcessor(ingestDocument -> ingestDocument.setFieldValue("_ttl", "5d"));
|
||||
when(store.get("_id")).thenReturn(new Pipeline("_id", "_description", new CompoundProcessor(processor)));
|
||||
|
||||
IndexRequest indexRequest = new IndexRequest("_index", "_type", "_id").source(Collections.emptyMap()).setPipeline("_id");
|
||||
@SuppressWarnings("unchecked")
|
||||
Consumer<Throwable> failureHandler = mock(Consumer.class);
|
||||
@SuppressWarnings("unchecked")
|
||||
Consumer<Boolean> completionHandler = mock(Consumer.class);
|
||||
executionService.execute(indexRequest, failureHandler, completionHandler);
|
||||
|
||||
assertThat(indexRequest.ttl(), equalTo(TimeValue.parseTimeValue("5d", null, "ttl")));
|
||||
verify(failureHandler, never()).accept(any());
|
||||
verify(completionHandler, times(1)).accept(true);
|
||||
}
|
||||
|
||||
public void testExecuteSetInvalidTTL() throws Exception {
|
||||
Processor processor = new TestProcessor(ingestDocument -> ingestDocument.setFieldValue("_ttl", "abc"));
|
||||
when(store.get("_id")).thenReturn(new Pipeline("_id", "_description", new CompoundProcessor(processor)));
|
||||
|
||||
IndexRequest indexRequest = new IndexRequest("_index", "_type", "_id").source(Collections.emptyMap()).setPipeline("_id");
|
||||
@SuppressWarnings("unchecked")
|
||||
Consumer<Throwable> failureHandler = mock(Consumer.class);
|
||||
@SuppressWarnings("unchecked")
|
||||
Consumer<Boolean> completionHandler = mock(Consumer.class);
|
||||
executionService.execute(indexRequest, failureHandler, completionHandler);
|
||||
verify(failureHandler, times(1)).accept(any(ElasticsearchParseException.class));
|
||||
verify(completionHandler, never()).accept(anyBoolean());
|
||||
}
|
||||
|
||||
public void testExecuteProvidedTTL() throws Exception {
|
||||
when(store.get("_id")).thenReturn(new Pipeline("_id", "_description", mock(CompoundProcessor.class)));
|
||||
|
||||
IndexRequest indexRequest = new IndexRequest("_index", "_type", "_id").setPipeline("_id")
|
||||
.source(Collections.emptyMap())
|
||||
.ttl(1000L);
|
||||
Consumer<Throwable> failureHandler = mock(Consumer.class);
|
||||
Consumer<Boolean> completionHandler = mock(Consumer.class);
|
||||
executionService.execute(indexRequest, failureHandler, completionHandler);
|
||||
|
||||
assertThat(indexRequest.ttl(), equalTo(new TimeValue(1000L)));
|
||||
verify(failureHandler, never()).accept(any());
|
||||
verify(completionHandler, times(1)).accept(true);
|
||||
}
|
||||
|
||||
public void testBulkRequestExecutionWithFailures() throws Exception {
|
||||
BulkRequest bulkRequest = new BulkRequest();
|
||||
String pipelineId = "_id";
|
||||
|
||||
int numRequest = scaledRandomIntBetween(8, 64);
|
||||
int numIndexRequests = 0;
|
||||
for (int i = 0; i < numRequest; i++) {
|
||||
ActionRequest request;
|
||||
if (randomBoolean()) {
|
||||
if (randomBoolean()) {
|
||||
request = new DeleteRequest("_index", "_type", "_id");
|
||||
} else {
|
||||
request = new UpdateRequest("_index", "_type", "_id");
|
||||
}
|
||||
} else {
|
||||
IndexRequest indexRequest = new IndexRequest("_index", "_type", "_id").setPipeline(pipelineId);
|
||||
indexRequest.source("field1", "value1");
|
||||
request = indexRequest;
|
||||
numIndexRequests++;
|
||||
}
|
||||
bulkRequest.add(request);
|
||||
}
|
||||
|
||||
CompoundProcessor processor = mock(CompoundProcessor.class);
|
||||
Exception error = new RuntimeException();
|
||||
doThrow(error).when(processor).execute(any());
|
||||
when(store.get(pipelineId)).thenReturn(new Pipeline(pipelineId, null, processor));
|
||||
|
||||
BiConsumer<IndexRequest, Throwable> requestItemErrorHandler = mock(BiConsumer.class);
|
||||
Consumer<Throwable> completionHandler = mock(Consumer.class);
|
||||
executionService.execute(bulkRequest.requests(), requestItemErrorHandler, completionHandler);
|
||||
|
||||
verify(requestItemErrorHandler, times(numIndexRequests)).accept(any(IndexRequest.class), eq(error));
|
||||
verify(completionHandler, times(1)).accept(null);
|
||||
}
|
||||
|
||||
public void testBulkRequestExecution() throws Exception {
|
||||
BulkRequest bulkRequest = new BulkRequest();
|
||||
String pipelineId = "_id";
|
||||
|
||||
int numRequest = scaledRandomIntBetween(8, 64);
|
||||
for (int i = 0; i < numRequest; i++) {
|
||||
IndexRequest indexRequest = new IndexRequest("_index", "_type", "_id").setPipeline(pipelineId);
|
||||
indexRequest.source("field1", "value1");
|
||||
bulkRequest.add(indexRequest);
|
||||
}
|
||||
|
||||
when(store.get(pipelineId)).thenReturn(new Pipeline(pipelineId, null, new CompoundProcessor()));
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
BiConsumer<IndexRequest, Throwable> requestItemErrorHandler = mock(BiConsumer.class);
|
||||
@SuppressWarnings("unchecked")
|
||||
Consumer<Throwable> completionHandler = mock(Consumer.class);
|
||||
executionService.execute(bulkRequest.requests(), requestItemErrorHandler, completionHandler);
|
||||
|
||||
verify(requestItemErrorHandler, never()).accept(any(), any());
|
||||
verify(completionHandler, times(1)).accept(null);
|
||||
}
|
||||
|
||||
private IngestDocument eqID(String index, String type, String id, Map<String, Object> source) {
|
||||
return argThat(new IngestDocumentMatcher(index, type, id, source));
|
||||
}
|
||||
|
||||
private class IngestDocumentMatcher extends ArgumentMatcher<IngestDocument> {
|
||||
|
||||
private final IngestDocument ingestDocument;
|
||||
|
||||
public IngestDocumentMatcher(String index, String type, String id, Map<String, Object> source) {
|
||||
this.ingestDocument = new IngestDocument(index, type, id, null, null, null, null, source);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean matches(Object o) {
|
||||
if (o.getClass() == IngestDocument.class) {
|
||||
IngestDocument otherIngestDocument = (IngestDocument) o;
|
||||
//ingest metadata will not be the same (timestamp differs every time)
|
||||
return Objects.equals(ingestDocument.getSourceAndMetadata(), otherIngestDocument.getSourceAndMetadata());
|
||||
}
|
||||
return false;
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,182 @@
|
||||
/*
|
||||
* 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.ResourceNotFoundException;
|
||||
import org.elasticsearch.action.ingest.DeletePipelineRequest;
|
||||
import org.elasticsearch.action.ingest.PutPipelineRequest;
|
||||
import org.elasticsearch.cluster.ClusterName;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
import org.elasticsearch.cluster.metadata.MetaData;
|
||||
import org.elasticsearch.common.bytes.BytesArray;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.ingest.core.Pipeline;
|
||||
import org.elasticsearch.ingest.processor.SetProcessor;
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
import org.junit.Before;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import static org.hamcrest.Matchers.equalTo;
|
||||
import static org.hamcrest.Matchers.is;
|
||||
import static org.hamcrest.Matchers.notNullValue;
|
||||
import static org.hamcrest.Matchers.nullValue;
|
||||
import static org.mockito.Mockito.mock;
|
||||
|
||||
public class PipelineStoreTests extends ESTestCase {
|
||||
|
||||
private PipelineStore store;
|
||||
|
||||
@Before
|
||||
public void init() throws Exception {
|
||||
store = new PipelineStore(Settings.EMPTY);
|
||||
ProcessorsRegistry registry = new ProcessorsRegistry();
|
||||
registry.registerProcessor("set", (templateService) -> new SetProcessor.Factory(TestTemplateService.instance()));
|
||||
store.buildProcessorFactoryRegistry(registry, null);
|
||||
}
|
||||
|
||||
public void testUpdatePipelines() {
|
||||
ClusterState clusterState = ClusterState.builder(new ClusterName("_name")).build();
|
||||
store.innerUpdatePipelines(clusterState);
|
||||
assertThat(store.pipelines.size(), is(0));
|
||||
|
||||
PipelineConfiguration pipeline = new PipelineConfiguration(
|
||||
"_id",new BytesArray("{\"processors\": [{\"set\" : {\"field\": \"_field\", \"value\": \"_value\"}}]}")
|
||||
);
|
||||
IngestMetadata ingestMetadata = new IngestMetadata(Collections.singletonMap("_id", pipeline));
|
||||
clusterState = ClusterState.builder(clusterState)
|
||||
.metaData(MetaData.builder().putCustom(IngestMetadata.TYPE, ingestMetadata))
|
||||
.build();
|
||||
store.innerUpdatePipelines(clusterState);
|
||||
assertThat(store.pipelines.size(), is(1));
|
||||
assertThat(store.pipelines.get("_id").getId(), equalTo("_id"));
|
||||
assertThat(store.pipelines.get("_id").getDescription(), nullValue());
|
||||
assertThat(store.pipelines.get("_id").getProcessors().size(), equalTo(1));
|
||||
assertThat(store.pipelines.get("_id").getProcessors().get(0).getType(), equalTo("set"));
|
||||
}
|
||||
|
||||
public void testPut() {
|
||||
String id = "_id";
|
||||
Pipeline pipeline = store.get(id);
|
||||
assertThat(pipeline, nullValue());
|
||||
ClusterState clusterState = ClusterState.builder(new ClusterName("_name")).build();
|
||||
|
||||
// add a new pipeline:
|
||||
PutPipelineRequest putRequest = new PutPipelineRequest(id, new BytesArray("{\"processors\": []}"));
|
||||
clusterState = store.innerPut(putRequest, clusterState);
|
||||
store.innerUpdatePipelines(clusterState);
|
||||
pipeline = store.get(id);
|
||||
assertThat(pipeline, notNullValue());
|
||||
assertThat(pipeline.getId(), equalTo(id));
|
||||
assertThat(pipeline.getDescription(), nullValue());
|
||||
assertThat(pipeline.getProcessors().size(), equalTo(0));
|
||||
|
||||
// overwrite existing pipeline:
|
||||
putRequest = new PutPipelineRequest(id, new BytesArray("{\"processors\": [], \"description\": \"_description\"}"));
|
||||
clusterState = store.innerPut(putRequest, clusterState);
|
||||
store.innerUpdatePipelines(clusterState);
|
||||
pipeline = store.get(id);
|
||||
assertThat(pipeline, notNullValue());
|
||||
assertThat(pipeline.getId(), equalTo(id));
|
||||
assertThat(pipeline.getDescription(), equalTo("_description"));
|
||||
assertThat(pipeline.getProcessors().size(), equalTo(0));
|
||||
}
|
||||
|
||||
public void testDelete() {
|
||||
PipelineConfiguration config = new PipelineConfiguration(
|
||||
"_id",new BytesArray("{\"processors\": [{\"set\" : {\"field\": \"_field\", \"value\": \"_value\"}}]}")
|
||||
);
|
||||
IngestMetadata ingestMetadata = new IngestMetadata(Collections.singletonMap("_id", config));
|
||||
ClusterState clusterState = ClusterState.builder(new ClusterName("_name"))
|
||||
.metaData(MetaData.builder().putCustom(IngestMetadata.TYPE, ingestMetadata))
|
||||
.build();
|
||||
store.innerUpdatePipelines(clusterState);
|
||||
assertThat(store.get("_id"), notNullValue());
|
||||
|
||||
// Delete pipeline:
|
||||
DeletePipelineRequest deleteRequest = new DeletePipelineRequest("_id");
|
||||
clusterState = store.innerDelete(deleteRequest, clusterState);
|
||||
store.innerUpdatePipelines(clusterState);
|
||||
assertThat(store.get("_id"), nullValue());
|
||||
|
||||
// Delete existing pipeline:
|
||||
try {
|
||||
store.innerDelete(deleteRequest, clusterState);
|
||||
fail("exception expected");
|
||||
} catch (ResourceNotFoundException e) {
|
||||
assertThat(e.getMessage(), equalTo("pipeline [_id] is missing"));
|
||||
}
|
||||
}
|
||||
|
||||
public void testGetPipelines() {
|
||||
Map<String, PipelineConfiguration> configs = new HashMap<>();
|
||||
configs.put("_id1", new PipelineConfiguration(
|
||||
"_id1", new BytesArray("{\"processors\": []}")
|
||||
));
|
||||
configs.put("_id2", new PipelineConfiguration(
|
||||
"_id2", new BytesArray("{\"processors\": []}")
|
||||
));
|
||||
|
||||
assertThat(store.innerGetPipelines(null, "_id1").isEmpty(), is(true));
|
||||
|
||||
IngestMetadata ingestMetadata = new IngestMetadata(configs);
|
||||
List<PipelineConfiguration> pipelines = store.innerGetPipelines(ingestMetadata, "_id1");
|
||||
assertThat(pipelines.size(), equalTo(1));
|
||||
assertThat(pipelines.get(0).getId(), equalTo("_id1"));
|
||||
|
||||
pipelines = store.innerGetPipelines(ingestMetadata, "_id1", "_id2");
|
||||
assertThat(pipelines.size(), equalTo(2));
|
||||
assertThat(pipelines.get(0).getId(), equalTo("_id1"));
|
||||
assertThat(pipelines.get(1).getId(), equalTo("_id2"));
|
||||
|
||||
pipelines = store.innerGetPipelines(ingestMetadata, "_id*");
|
||||
pipelines.sort((o1, o2) -> o1.getId().compareTo(o2.getId()));
|
||||
assertThat(pipelines.size(), equalTo(2));
|
||||
assertThat(pipelines.get(0).getId(), equalTo("_id1"));
|
||||
assertThat(pipelines.get(1).getId(), equalTo("_id2"));
|
||||
}
|
||||
|
||||
public void testCrud() throws Exception {
|
||||
String id = "_id";
|
||||
Pipeline pipeline = store.get(id);
|
||||
assertThat(pipeline, nullValue());
|
||||
ClusterState clusterState = ClusterState.builder(new ClusterName("_name")).build(); // Start empty
|
||||
|
||||
PutPipelineRequest putRequest = new PutPipelineRequest(id, new BytesArray("{\"processors\": [{\"set\" : {\"field\": \"_field\", \"value\": \"_value\"}}]}"));
|
||||
clusterState = store.innerPut(putRequest, clusterState);
|
||||
store.innerUpdatePipelines(clusterState);
|
||||
pipeline = store.get(id);
|
||||
assertThat(pipeline, notNullValue());
|
||||
assertThat(pipeline.getId(), equalTo(id));
|
||||
assertThat(pipeline.getDescription(), nullValue());
|
||||
assertThat(pipeline.getProcessors().size(), equalTo(1));
|
||||
assertThat(pipeline.getProcessors().get(0).getType(), equalTo("set"));
|
||||
|
||||
DeletePipelineRequest deleteRequest = new DeletePipelineRequest(id);
|
||||
clusterState = store.innerDelete(deleteRequest, clusterState);
|
||||
store.innerUpdatePipelines(clusterState);
|
||||
pipeline = store.get(id);
|
||||
assertThat(pipeline, nullValue());
|
||||
}
|
||||
|
||||
}
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
x
Reference in New Issue
Block a user