Merge branch 'master' into feature/aggs-refactoring

# Conflicts:
#	core/src/test/java/org/elasticsearch/percolator/PercolateDocumentParserTests.java
This commit is contained in:
Colin Goodheart-Smithe 2016-01-26 13:17:53 +00:00
commit 3b35754f59
244 changed files with 19117 additions and 281 deletions

View File

@ -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);

View File

@ -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;
}

View File

@ -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;

View File

@ -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

View File

@ -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;
}
}

View File

@ -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();
}
}

View File

@ -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);
}
}

View File

@ -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));
}
}

View File

@ -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);
}
}

View File

@ -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();
}
}

View File

@ -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);
}
}

View File

@ -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));
}
}

View File

@ -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;
}
}

View File

@ -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);
}
}

View File

@ -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);
}
}
}

View File

@ -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;
}
}

View File

@ -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();
}
}

View File

@ -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);
}
}

View File

@ -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));
}
}

View File

@ -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);
}
}

View File

@ -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;
}
}

View File

@ -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 {
}

View File

@ -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;
}
}

View File

@ -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));
}
});
}
}

View File

@ -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();
}
}

View File

@ -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;
}
}

View File

@ -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;
}
}

View File

@ -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");
}
}

View File

@ -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);
}
}

View File

@ -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;
}
}

View File

@ -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);
}
}

View File

@ -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();
}
}

View File

@ -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
*/

View File

@ -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;

View File

@ -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);

View File

@ -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);
}
/**

View File

@ -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;
}

View File

@ -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 {

View File

@ -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(

View File

@ -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)));
}

View File

@ -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));

View 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;
}
}

View File

@ -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();
}
}

View File

@ -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;
}
}
}

View File

@ -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);
}
}

View File

@ -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;
}
}

View 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);
}
}

View File

@ -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();
}
}

View File

@ -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;
}
}

View File

@ -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;
}

View File

@ -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);
}
}
}

View File

@ -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;
}
}

View File

@ -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");
}
}
}
}

View 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 + "]");
}
}
}

View File

@ -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;
}
}

View File

@ -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();
}
}

View File

@ -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());
}
}
}

View File

@ -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);
}
}

View File

@ -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));
}
}
}

View File

@ -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);
}
}
}

View File

@ -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;
}
}
}

View File

@ -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);
}
}
}

View File

@ -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);
}
}
}

View File

@ -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));
}
}
}

View File

@ -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);
}
}

View File

@ -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);
}
}
}

View File

@ -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);
}
}
}

View File

@ -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);
}
}
}

View File

@ -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));
}
}
}

View File

@ -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);
}
}
}

View File

@ -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));
}
}
}

View File

@ -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"));
}
}
}

View File

@ -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);
}
}
}

View File

@ -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);
}
}
}

View File

@ -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();

View File

@ -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);
}
}

View File

@ -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();
}
}

View File

@ -39,6 +39,8 @@ import org.apache.lucene.util.CloseableThreadLocal;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.common.lucene.search.Queries;
import org.elasticsearch.index.engine.Engine;
import org.elasticsearch.index.mapper.DocumentMapper;
import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.mapper.ParseContext;
import org.elasticsearch.index.mapper.ParsedDocument;
import org.elasticsearch.index.mapper.internal.UidFieldMapper;
@ -76,8 +78,7 @@ class MultiDocumentPercolatorIndex implements PercolatorIndex {
} else {
memoryIndex = new MemoryIndex(true);
}
Analyzer analyzer = context.mapperService().documentMapper(parsedDocument.type()).mappers().indexAnalyzer();
memoryIndices[i] = indexDoc(d, analyzer, memoryIndex).createSearcher().getIndexReader();
memoryIndices[i] = indexDoc(d, memoryIndex, context, parsedDocument).createSearcher().getIndexReader();
}
try {
MultiReader mReader = new MultiReader(memoryIndices, true);
@ -101,8 +102,13 @@ class MultiDocumentPercolatorIndex implements PercolatorIndex {
}
}
MemoryIndex indexDoc(ParseContext.Document d, Analyzer analyzer, MemoryIndex memoryIndex) {
MemoryIndex indexDoc(ParseContext.Document d, MemoryIndex memoryIndex, PercolateContext context, ParsedDocument parsedDocument) {
for (IndexableField field : d.getFields()) {
Analyzer analyzer = context.analysisService().defaultIndexAnalyzer();
DocumentMapper documentMapper = context.mapperService().documentMapper(parsedDocument.type());
if (documentMapper != null && documentMapper.mappers().getMapper(field.name()) != null) {
analyzer = documentMapper.mappers().indexAnalyzer();
}
if (field.fieldType().indexOptions() == IndexOptions.NONE && field.name().equals(UidFieldMapper.NAME)) {
continue;
}

View File

@ -49,14 +49,13 @@ public class PercolateDocumentParser {
private final HighlightPhase highlightPhase;
private final SortParseElement sortParseElement;
private final AggregationPhase aggregationPhase;
private final MappingUpdatedAction mappingUpdatedAction;
@Inject
public PercolateDocumentParser(HighlightPhase highlightPhase, SortParseElement sortParseElement, AggregationPhase aggregationPhase, MappingUpdatedAction mappingUpdatedAction) {
public PercolateDocumentParser(HighlightPhase highlightPhase, SortParseElement sortParseElement,
AggregationPhase aggregationPhase) {
this.highlightPhase = highlightPhase;
this.sortParseElement = sortParseElement;
this.aggregationPhase = aggregationPhase;
this.mappingUpdatedAction = mappingUpdatedAction;
}
public ParsedDocument parse(PercolateShardRequest request, PercolateContext context, MapperService mapperService, QueryShardContext queryShardContext) {
@ -98,9 +97,6 @@ public class PercolateDocumentParser {
if (docMapper.getMapping() != null) {
doc.addDynamicMappingsUpdate(docMapper.getMapping());
}
if (doc.dynamicMappingsUpdate() != null) {
mappingUpdatedAction.updateMappingOnMasterSynchronously(request.shardId().getIndex(), request.documentType(), doc.dynamicMappingsUpdate());
}
// the document parsing exists the "doc" object, so we need to set the new current field.
currentFieldName = parser.currentName();
}

View File

@ -53,6 +53,7 @@ import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.fieldvisitor.SingleFieldsVisitor;
import org.elasticsearch.index.mapper.DocumentMapper;
import org.elasticsearch.index.mapper.ParsedDocument;
import org.elasticsearch.index.mapper.internal.UidFieldMapper;
import org.elasticsearch.index.percolator.PercolatorFieldMapper;
@ -205,7 +206,8 @@ public class PercolatorService extends AbstractComponent {
// parse the source either into one MemoryIndex, if it is a single document or index multiple docs if nested
PercolatorIndex percolatorIndex;
boolean isNested = indexShard.mapperService().documentMapper(request.documentType()).hasNestedObjects();
DocumentMapper documentMapper = indexShard.mapperService().documentMapper(request.documentType());
boolean isNested = documentMapper != null && documentMapper.hasNestedObjects();
if (parsedDocument.docs().size() > 1) {
assert isNested;
percolatorIndex = multi;

View File

@ -28,6 +28,7 @@ import org.apache.lucene.index.memory.MemoryIndex;
import org.apache.lucene.util.CloseableThreadLocal;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.index.engine.Engine;
import org.elasticsearch.index.mapper.DocumentMapper;
import org.elasticsearch.index.mapper.ParsedDocument;
import org.elasticsearch.index.mapper.internal.UidFieldMapper;
@ -49,11 +50,15 @@ class SingleDocumentPercolatorIndex implements PercolatorIndex {
public void prepare(PercolateContext context, ParsedDocument parsedDocument) {
MemoryIndex memoryIndex = cache.get();
for (IndexableField field : parsedDocument.rootDoc().getFields()) {
Analyzer analyzer = context.analysisService().defaultIndexAnalyzer();
DocumentMapper documentMapper = context.mapperService().documentMapper(parsedDocument.type());
if (documentMapper != null && documentMapper.mappers().getMapper(field.name()) != null) {
analyzer = documentMapper.mappers().indexAnalyzer();
}
if (field.fieldType().indexOptions() == IndexOptions.NONE && field.name().equals(UidFieldMapper.NAME)) {
continue;
}
try {
Analyzer analyzer = context.mapperService().documentMapper(parsedDocument.type()).mappers().indexAnalyzer();
// TODO: instead of passing null here, we can have a CTL<Map<String,TokenStream>> and pass previous,
// like the indexer does
try (TokenStream tokenStream = field.tokenStream(analyzer, null)) {

View File

@ -101,6 +101,7 @@ public class PluginManager {
"discovery-ec2",
"discovery-gce",
"discovery-multicast",
"ingest-geoip",
"lang-javascript",
"lang-plan-a",
"lang-python",

View File

@ -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

View File

@ -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()));

View File

@ -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));
}
}

View File

@ -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));
}
}

View File

@ -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));
}
}

View File

@ -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));
}
}

View File

@ -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;

View File

@ -22,11 +22,8 @@ package org.elasticsearch.search.sort;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.index.query.QueryBuilder;
import org.elasticsearch.script.Script;
import org.elasticsearch.script.ScriptService.ScriptType;
import java.io.IOException;
import java.util.HashMap;
import java.util.Map;
/**
* Script sort builder allows to sort based on a custom script expression.
@ -35,17 +32,8 @@ public class ScriptSortBuilder extends SortBuilder {
private Script script;
@Deprecated
private String scriptString;
private final String type;
@Deprecated
private String lang;
@Deprecated
private Map<String, Object> params;
private SortOrder order;
private String sortMode;
@ -65,66 +53,6 @@ public class ScriptSortBuilder extends SortBuilder {
this.type = type;
}
/**
* Constructs a script sort builder with the script and the type.
*
* @param script
* The script to use.
* @param type
* The type, can either be "string" or "number".
*
* @deprecated Use {@link #ScriptSortBuilder(Script, String)} instead.
*/
@Deprecated
public ScriptSortBuilder(String script, String type) {
this.scriptString = script;
this.type = type;
}
/**
* Adds a parameter to the script.
*
* @param name
* The name of the parameter.
* @param value
* The value of the parameter.
*
* @deprecated Use {@link #ScriptSortBuilder(Script, String)} instead.
*/
@Deprecated
public ScriptSortBuilder param(String name, Object value) {
if (params == null) {
params = new HashMap<>();
}
params.put(name, value);
return this;
}
/**
* Sets parameters for the script.
*
* @param params
* The script parameters
*
* @deprecated Use {@link #ScriptSortBuilder(Script, String)} instead.
*/
@Deprecated
public ScriptSortBuilder setParams(Map<String, Object> params) {
this.params = params;
return this;
}
/**
* The language of the script.
*
* @deprecated Use {@link #ScriptSortBuilder(Script, String)} instead.
*/
@Deprecated
public ScriptSortBuilder lang(String lang) {
this.lang = lang;
return this;
}
/**
* Sets the sort order.
*/
@ -172,12 +100,7 @@ public class ScriptSortBuilder extends SortBuilder {
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params builderParams) throws IOException {
builder.startObject("_script");
if (script == null) {
builder.field("script", new Script(scriptString, ScriptType.INLINE, lang, params));
} else {
builder.field("script", script);
}
builder.field("script", script);
builder.field("type", type);
if (order == SortOrder.DESC) {
builder.field("reverse", true);
@ -189,7 +112,7 @@ public class ScriptSortBuilder extends SortBuilder {
builder.field("nested_path", nestedPath);
}
if (nestedFilter != null) {
builder.field("nested_filter", nestedFilter, params);
builder.field("nested_filter", nestedFilter, builderParams);
}
builder.endObject();
return builder;

View File

@ -54,20 +54,6 @@ public class SortBuilders {
return new ScriptSortBuilder(script, type);
}
/**
* Constructs a new script based sort.
*
* @param script
* The script to use.
* @param type
* The type, can either be "string" or "number".
* @deprecated Use {@link #scriptSort(Script, String)} instead.
*/
@Deprecated
public static ScriptSortBuilder scriptSort(String script, String type) {
return new ScriptSortBuilder(script, type);
}
/**
* A geo distance based sort.
*

View File

@ -43,6 +43,7 @@ OFFICIAL PLUGINS
- discovery-ec2
- discovery-gce
- discovery-multicast
- ingest-geoip
- lang-javascript
- lang-plan-a
- lang-python

View File

@ -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;
}
}
}

View File

@ -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);
}
}

View File

@ -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();
}
}
}

View File

@ -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"));
}
}
}

View File

@ -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"));
}
}

View File

@ -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));
}
}

View File

@ -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"));
}
}
}
}
}

View File

@ -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"));
}
}
}

Some files were not shown because too many files have changed in this diff Show More