From bce7f6c7ad2b22b56a5a7b2d2e8dbf0516311c4e Mon Sep 17 00:00:00 2001 From: Tal Levy Date: Wed, 4 Nov 2015 19:48:14 -0800 Subject: [PATCH 01/14] Add simulate endpoint --- .../java/org/elasticsearch/ingest/Data.java | 24 +++- .../plugin/ingest/IngestModule.java | 1 + .../plugin/ingest/IngestPlugin.java | 8 +- .../ingest/PipelineExecutionService.java | 25 +++- .../plugin/ingest/PipelineStore.java | 4 + .../rest/RestSimulatePipelineAction.java | 51 +++++++ .../simulate/SimulatePipelineAction.java | 43 ++++++ .../simulate/SimulatePipelineRequest.java | 75 ++++++++++ .../SimulatePipelineRequestBuilder.java | 42 ++++++ .../SimulatePipelineRequestPayload.java | 89 ++++++++++++ .../simulate/SimulatePipelineResponse.java | 96 +++++++++++++ .../SimulatePipelineTransportAction.java | 101 ++++++++++++++ .../simulate/SimulatedItemResponse.java | 129 ++++++++++++++++++ .../elasticsearch/ingest/IngestClientIT.java | 64 ++++++++- .../rest-api-spec/api/ingest.simulate.json | 23 ++++ .../test/ingest/80_simulate.yaml | 51 +++++++ 16 files changed, 814 insertions(+), 12 deletions(-) create mode 100644 plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/rest/RestSimulatePipelineAction.java create mode 100644 plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineAction.java create mode 100644 plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineRequest.java create mode 100644 plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineRequestBuilder.java create mode 100644 plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineRequestPayload.java create mode 100644 plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineResponse.java create mode 100644 plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineTransportAction.java create mode 100644 plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatedItemResponse.java create mode 100644 plugins/ingest/src/test/resources/rest-api-spec/api/ingest.simulate.json create mode 100644 plugins/ingest/src/test/resources/rest-api-spec/test/ingest/80_simulate.yaml diff --git a/plugins/ingest/src/main/java/org/elasticsearch/ingest/Data.java b/plugins/ingest/src/main/java/org/elasticsearch/ingest/Data.java index 879acfcbf8e..407c1b894f1 100644 --- a/plugins/ingest/src/main/java/org/elasticsearch/ingest/Data.java +++ b/plugins/ingest/src/main/java/org/elasticsearch/ingest/Data.java @@ -23,10 +23,7 @@ import org.elasticsearch.common.Strings; import org.elasticsearch.common.xcontent.support.XContentMapValues; import java.lang.reflect.Array; -import java.util.Arrays; -import java.util.HashMap; -import java.util.List; -import java.util.Map; +import java.util.*; /** * Represents the data and meta data (like id and type) of a single document that is going to be indexed. @@ -129,4 +126,23 @@ public final class Data { public boolean isModified() { return modified; } + + @Override + public boolean equals(Object obj) { + if (obj == this) { return true; } + if (obj == null || getClass() != obj.getClass()) { + return false; + } + + Data other = (Data) obj; + return Objects.equals(document, other.document) && + Objects.equals(index, other.index) && + Objects.equals(type, other.type) && + Objects.equals(id, other.id); + } + + @Override + public int hashCode() { + return Objects.hash(index, type, id, document); + } } diff --git a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/IngestModule.java b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/IngestModule.java index 94dfb4fb690..15c2b7769fb 100644 --- a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/IngestModule.java +++ b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/IngestModule.java @@ -39,6 +39,7 @@ public class IngestModule extends AbstractModule { protected void configure() { binder().bind(IngestRestFilter.class).asEagerSingleton(); binder().bind(PipelineExecutionService.class).asEagerSingleton(); + // TODO(talevy): write it! binder().bind(PipelineSimulateService.class).asEagerSingleton(); binder().bind(PipelineStore.class).asEagerSingleton(); binder().bind(PipelineStoreClient.class).asEagerSingleton(); diff --git a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/IngestPlugin.java b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/IngestPlugin.java index c5efe74db26..34411fd60e7 100644 --- a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/IngestPlugin.java +++ b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/IngestPlugin.java @@ -20,7 +20,6 @@ package org.elasticsearch.plugin.ingest; -import org.elasticsearch.SpecialPermission; import org.elasticsearch.action.ActionModule; import org.elasticsearch.client.Client; import org.elasticsearch.client.transport.TransportClient; @@ -30,6 +29,7 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.plugin.ingest.rest.RestDeletePipelineAction; import org.elasticsearch.plugin.ingest.rest.RestGetPipelineAction; import org.elasticsearch.plugin.ingest.rest.RestPutPipelineAction; +import org.elasticsearch.plugin.ingest.rest.RestSimulatePipelineAction; import org.elasticsearch.plugin.ingest.transport.IngestActionFilter; import org.elasticsearch.plugin.ingest.transport.delete.DeletePipelineAction; import org.elasticsearch.plugin.ingest.transport.delete.DeletePipelineTransportAction; @@ -37,11 +37,11 @@ import org.elasticsearch.plugin.ingest.transport.get.GetPipelineAction; import org.elasticsearch.plugin.ingest.transport.get.GetPipelineTransportAction; import org.elasticsearch.plugin.ingest.transport.put.PutPipelineAction; import org.elasticsearch.plugin.ingest.transport.put.PutPipelineTransportAction; +import org.elasticsearch.plugin.ingest.transport.simulate.SimulatePipelineAction; +import org.elasticsearch.plugin.ingest.transport.simulate.SimulatePipelineTransportAction; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.rest.RestModule; -import java.security.AccessController; -import java.security.PrivilegedAction; import java.util.Arrays; import java.util.Collection; import java.util.Collections; @@ -105,11 +105,13 @@ public class IngestPlugin extends Plugin { module.registerAction(PutPipelineAction.INSTANCE, PutPipelineTransportAction.class); module.registerAction(GetPipelineAction.INSTANCE, GetPipelineTransportAction.class); module.registerAction(DeletePipelineAction.INSTANCE, DeletePipelineTransportAction.class); + module.registerAction(SimulatePipelineAction.INSTANCE, SimulatePipelineTransportAction.class); } public void onModule(RestModule restModule) { restModule.addRestAction(RestPutPipelineAction.class); restModule.addRestAction(RestGetPipelineAction.class); restModule.addRestAction(RestDeletePipelineAction.class); + restModule.addRestAction(RestSimulatePipelineAction.class); } } diff --git a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/PipelineExecutionService.java b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/PipelineExecutionService.java index 18d656813ec..c6314ae6a6c 100644 --- a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/PipelineExecutionService.java +++ b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/PipelineExecutionService.java @@ -25,8 +25,11 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.concurrent.EsExecutors; import org.elasticsearch.ingest.Data; import org.elasticsearch.ingest.Pipeline; +import org.elasticsearch.ingest.processor.Processor; import org.elasticsearch.threadpool.ThreadPool; +import java.util.Map; + public class PipelineExecutionService { static final String THREAD_POOL_NAME = IngestPlugin.NAME; @@ -40,13 +43,29 @@ public class PipelineExecutionService { this.threadPool = threadPool; } - public void execute(Data data, String pipelineId, Listener listener) { + public Pipeline getPipeline(String pipelineId) { Pipeline pipeline = store.get(pipelineId); + if (pipeline == null) { - listener.failed(new IllegalArgumentException(LoggerMessageFormat.format("pipeline with id [{}] does not exist", pipelineId))); - return; + throw new IllegalArgumentException(LoggerMessageFormat.format("pipeline with id [{}] does not exist", pipelineId)); } + return pipeline; + } + + public Map getProcessorFactoryRegistry() { + return store.getProcessorFactoryRegistry(); + } + + public void execute(Data data, String pipelineId, Listener listener) { + try { + execute(data, getPipeline(pipelineId), listener); + } catch (IllegalArgumentException e) { + listener.failed(e); + } + } + + public void execute(Data data, Pipeline pipeline, Listener listener) { threadPool.executor(THREAD_POOL_NAME).execute(new Runnable() { @Override public void run() { diff --git a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/PipelineStore.java b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/PipelineStore.java index 8fc0e4d2d2c..3bbddb1b842 100644 --- a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/PipelineStore.java +++ b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/PipelineStore.java @@ -95,6 +95,10 @@ public class PipelineStore extends AbstractLifecycleComponent { } } + public Map getProcessorFactoryRegistry() { + return processorFactoryRegistry; + } + public List getReference(String... ids) { List result = new ArrayList<>(ids.length); for (String id : ids) { diff --git a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/rest/RestSimulatePipelineAction.java b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/rest/RestSimulatePipelineAction.java new file mode 100644 index 00000000000..983f43d6a1a --- /dev/null +++ b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/rest/RestSimulatePipelineAction.java @@ -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.plugin.ingest.rest; + +import org.elasticsearch.client.Client; +import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.plugin.ingest.transport.simulate.SimulatePipelineAction; +import org.elasticsearch.plugin.ingest.transport.simulate.SimulatePipelineRequest; +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 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.POST, "/_ingest/pipeline/_simulate", this); + } + + @Override + protected void handleRequest(RestRequest restRequest, RestChannel channel, Client client) throws Exception { + SimulatePipelineRequest request = new SimulatePipelineRequest(); + request.id(restRequest.param("id")); + if (restRequest.hasContent()) { + request.source(restRequest.content()); + } + client.execute(SimulatePipelineAction.INSTANCE, request, new RestStatusToXContentListener<>(channel)); + } +} diff --git a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineAction.java b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineAction.java new file mode 100644 index 00000000000..7c671a442f6 --- /dev/null +++ b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineAction.java @@ -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.plugin.ingest.transport.simulate; + +import org.elasticsearch.action.Action; +import org.elasticsearch.client.ElasticsearchClient; + +public class SimulatePipelineAction extends Action { + + 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(); + } +} diff --git a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineRequest.java b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineRequest.java new file mode 100644 index 00000000000..2bf7c01bf24 --- /dev/null +++ b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineRequest.java @@ -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.plugin.ingest.transport.simulate; + +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 java.io.IOException; + +import static org.elasticsearch.action.ValidateActions.addValidationError; + +public class SimulatePipelineRequest extends ActionRequest { + + private String id; + private BytesReference source; + + @Override + public ActionRequestValidationException validate() { + ActionRequestValidationException validationException = null; + if (source == null) { + validationException = addValidationError("source is missing", validationException); + } + return validationException; + } + + public String id() { + return id; + } + + public void id(String id) { + this.id = id; + } + + public BytesReference source() { + return source; + } + + public void source(BytesReference source) { + this.source = 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); + } +} diff --git a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineRequestBuilder.java b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineRequestBuilder.java new file mode 100644 index 00000000000..8f446b75238 --- /dev/null +++ b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineRequestBuilder.java @@ -0,0 +1,42 @@ +/* + * 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.plugin.ingest.transport.simulate; + +import org.elasticsearch.action.ActionRequestBuilder; +import org.elasticsearch.client.ElasticsearchClient; +import org.elasticsearch.common.bytes.BytesReference; + +public class SimulatePipelineRequestBuilder extends ActionRequestBuilder { + + public SimulatePipelineRequestBuilder(ElasticsearchClient client, SimulatePipelineAction action) { + super(client, action, new SimulatePipelineRequest()); + } + + public SimulatePipelineRequestBuilder setId(String id) { + request.id(id); + return this; + } + + public SimulatePipelineRequestBuilder setSource(BytesReference source) { + request.source(source); + return this; + } + +} diff --git a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineRequestPayload.java b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineRequestPayload.java new file mode 100644 index 00000000000..b376c67d006 --- /dev/null +++ b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineRequestPayload.java @@ -0,0 +1,89 @@ +/* + * 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.plugin.ingest.transport.simulate; + +import org.elasticsearch.ingest.Data; +import org.elasticsearch.ingest.Pipeline; +import org.elasticsearch.ingest.processor.ConfigurationUtils; +import org.elasticsearch.plugin.ingest.PipelineExecutionService; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; + +public class SimulatePipelineRequestPayload { + + private final List documents; + private final Pipeline pipeline; + + public SimulatePipelineRequestPayload(Pipeline pipeline, List documents) { + this.pipeline = pipeline; + this.documents = Collections.unmodifiableList(documents); + } + + public String pipelineId() { + return pipeline.getId(); + } + + public Pipeline pipeline() { + return pipeline; + } + + + public Data getDocument(int i) { + return documents.get(i); + } + + public int size() { + return documents.size(); + } + + public static class Factory { + + public SimulatePipelineRequestPayload create(String pipelineId, Map config, PipelineExecutionService executionService) throws IOException { + Pipeline pipeline; + // if pipeline `id` passed to request, fetch pipeline from store. + if (pipelineId != null) { + pipeline = executionService.getPipeline(pipelineId); + } else { + Map pipelineConfig = (Map) config.get("pipeline"); + pipeline = (new Pipeline.Factory()).create("_pipeline_id", pipelineConfig, executionService.getProcessorFactoryRegistry()); + } + + // distribute docs by shard key to SimulateShardPipelineResponse + List> docs = (List>) config.get("docs"); + + List dataList = new ArrayList<>(); + + for (int i = 0; i < docs.size(); i++) { + Map dataMap = docs.get(i); + Map document = (Map) dataMap.get("_source"); + Data data = new Data(ConfigurationUtils.readStringProperty(dataMap, "_index", null), + ConfigurationUtils.readStringProperty(dataMap, "_type", null), + ConfigurationUtils.readStringProperty(dataMap, "_id", null), + document); + dataList.add(data); + } + + return new SimulatePipelineRequestPayload(pipeline, dataList); + } + } +} diff --git a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineResponse.java b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineResponse.java new file mode 100644 index 00000000000..92e936dd245 --- /dev/null +++ b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineResponse.java @@ -0,0 +1,96 @@ +/* + * 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.plugin.ingest.transport.simulate; + +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.ToXContent; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentBuilderString; +import org.elasticsearch.ingest.Data; +import org.elasticsearch.rest.RestStatus; + +import java.io.IOException; + +public class SimulatePipelineResponse extends ActionResponse implements StatusToXContent { + + private String pipelineId; + private SimulatedItemResponse[] responses; + + public String pipelineId() { + return pipelineId; + } + + public SimulatePipelineResponse pipelineId(String pipelineId) { + this.pipelineId = pipelineId; + return this; + } + + public SimulatePipelineResponse responses(SimulatedItemResponse[] responses) { + this.responses = responses; + return this; + } + + public SimulatedItemResponse[] responses() { + return responses; + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + out.writeString(pipelineId); + out.writeVInt(responses.length); + for (SimulatedItemResponse response : responses) { + response.writeTo(out); + } + } + + @Override + public void readFrom(StreamInput in) throws IOException { + super.readFrom(in); + this.pipelineId = in.readString(); + int responsesLength = in.readVInt(); + responses = new SimulatedItemResponse[responsesLength]; + for (int i = 0; i < responsesLength; i++) { + SimulatedItemResponse response = new SimulatedItemResponse(); + response.readFrom(in); + responses[i] = response; + } + + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startArray("docs"); + for (SimulatedItemResponse response : responses) { + builder.value(response); + } + builder.endArray(); + + return builder; + } + + @Override + public RestStatus status() { + return RestStatus.OK; + } +} diff --git a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineTransportAction.java b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineTransportAction.java new file mode 100644 index 00000000000..04c26d3c98f --- /dev/null +++ b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineTransportAction.java @@ -0,0 +1,101 @@ +/* + * 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.plugin.ingest.transport.simulate; + +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.util.concurrent.AtomicArray; +import org.elasticsearch.common.xcontent.XContentHelper; +import org.elasticsearch.ingest.Data; +import org.elasticsearch.plugin.ingest.PipelineExecutionService; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.TransportService; + +import java.io.IOException; +import java.util.Map; +import java.util.concurrent.atomic.AtomicInteger; + +public class SimulatePipelineTransportAction extends HandledTransportAction { + + private final PipelineExecutionService executionService; + + @Inject + public SimulatePipelineTransportAction(Settings settings, ThreadPool threadPool, TransportService transportService, ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver, PipelineExecutionService executionService) { + super(settings, SimulatePipelineAction.NAME, threadPool, transportService, actionFilters, indexNameExpressionResolver, SimulatePipelineRequest::new); + this.executionService = executionService; + } + + @Override + protected void doExecute(SimulatePipelineRequest request, ActionListener listener) { + Map source = XContentHelper.convertToMap(request.source(), false).v2(); + + SimulatePipelineRequestPayload payload; + SimulatePipelineRequestPayload.Factory factory = new SimulatePipelineRequestPayload.Factory(); + try { + payload = factory.create(request.id(), source, executionService); + } catch (IOException e) { + listener.onFailure(e); + return; + } + + final AtomicArray responses = new AtomicArray<>(payload.size()); + final AtomicInteger counter = new AtomicInteger(payload.size()); + + for (int i = 0; i < payload.size(); i++) { + final int index = i; + + executionService.execute(payload.getDocument(index), payload.pipeline(), new PipelineExecutionService.Listener() { + @Override + public void executed(Data data) { + responses.set(index, new SimulatedItemResponse(data)); + + if (counter.decrementAndGet() == 0) { + finishHim(); + } + } + + @Override + public void failed(Exception e) { + logger.error("failed to execute pipeline [{}]", e, payload.pipelineId()); + responses.set(index, new SimulatedItemResponse(e)); + + if (counter.decrementAndGet() == 0) { + finishHim(); + } + } + + public void finishHim() { + SimulatedItemResponse[] responseArray = new SimulatedItemResponse[responses.length()]; + responses.toArray(responseArray); + + SimulatePipelineResponse response = new SimulatePipelineResponse() + .pipelineId(payload.pipelineId()) + .responses(responseArray); + + listener.onResponse(response); + } + }); + } + } +} diff --git a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatedItemResponse.java b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatedItemResponse.java new file mode 100644 index 00000000000..a41661c0685 --- /dev/null +++ b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatedItemResponse.java @@ -0,0 +1,129 @@ +/* + * 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.plugin.ingest.transport.simulate; + +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.io.stream.Streamable; +import org.elasticsearch.common.xcontent.StatusToXContent; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentBuilderString; +import org.elasticsearch.ingest.Data; +import org.elasticsearch.rest.RestStatus; + +import java.io.IOException; +import java.util.Map; +import java.util.Objects; + +public class SimulatedItemResponse implements Streamable, StatusToXContent { + + private Data data; + private Throwable failure; + + public SimulatedItemResponse() { + + } + + public SimulatedItemResponse(Data data) { + this.data = data; + } + + public SimulatedItemResponse(Throwable failure) { + this.failure = failure; + } + + public boolean failed() { + return this.failure != null; + } + + public Data getData() { + return data; + } + + @Override + public void readFrom(StreamInput in) throws IOException { + boolean failed = in.readBoolean(); + + if (failed) { + this.failure = in.readThrowable(); + // TODO(talevy): check out mget for throwable limitations + } else { + String index = in.readString(); + String type = in.readString(); + String id = in.readString(); + Map doc = in.readMap(); + this.data = new Data(index, type, id, doc); + } + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeBoolean(failed()); + + if (failed()) { + out.writeThrowable(failure); + } else { + out.writeString(data.getIndex()); + out.writeString(data.getType()); + out.writeString(data.getId()); + out.writeMap(data.getDocument()); + } + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject(); + builder.field(Fields.ERROR, failed()); + if (failed()) { + builder.field(Fields.FAILURE, failure.toString()); + } else { + builder.field(Fields.MODIFIED, data.isModified()); + builder.field(Fields.DOCUMENT, data.getDocument()); + } + builder.endObject(); + return builder; + } + + @Override + public RestStatus status() { + return null; + } + + @Override + public boolean equals(Object obj) { + if (obj == this) { return true; } + if (obj == null || getClass() != obj.getClass()) { + return false; + } + SimulatedItemResponse other = (SimulatedItemResponse) obj; + return Objects.equals(data, other.data) && Objects.equals(failure, other.failure); + } + + @Override + public int hashCode() { + return Objects.hash(data, failure); + } + + static final class Fields { + static final XContentBuilderString DOCUMENT = new XContentBuilderString("doc"); + static final XContentBuilderString ERROR = new XContentBuilderString("error"); + static final XContentBuilderString FAILURE = new XContentBuilderString("failure"); + static final XContentBuilderString MODIFIED = new XContentBuilderString("modified"); + } +} diff --git a/plugins/ingest/src/test/java/org/elasticsearch/ingest/IngestClientIT.java b/plugins/ingest/src/test/java/org/elasticsearch/ingest/IngestClientIT.java index fe860b742d6..f055d92f2d2 100644 --- a/plugins/ingest/src/test/java/org/elasticsearch/ingest/IngestClientIT.java +++ b/plugins/ingest/src/test/java/org/elasticsearch/ingest/IngestClientIT.java @@ -30,10 +30,14 @@ import org.elasticsearch.plugin.ingest.transport.get.GetPipelineRequestBuilder; import org.elasticsearch.plugin.ingest.transport.get.GetPipelineResponse; import org.elasticsearch.plugin.ingest.transport.put.PutPipelineAction; import org.elasticsearch.plugin.ingest.transport.put.PutPipelineRequestBuilder; +import org.elasticsearch.plugin.ingest.transport.simulate.SimulatePipelineAction; +import org.elasticsearch.plugin.ingest.transport.simulate.SimulatePipelineRequestBuilder; +import org.elasticsearch.plugin.ingest.transport.simulate.SimulatePipelineResponse; +import org.elasticsearch.plugin.ingest.transport.simulate.SimulatedItemResponse; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.ESIntegTestCase; -import java.util.Collection; -import java.util.Map; + +import java.util.*; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; @@ -55,6 +59,62 @@ public class IngestClientIT extends ESIntegTestCase { } + public void testSimulate() throws Exception { + new PutPipelineRequestBuilder(client(), PutPipelineAction.INSTANCE) + .setId("_id") + .setSource(jsonBuilder().startObject() + .field("description", "my_pipeline") + .startArray("processors") + .startObject() + .startObject("grok") + .field("field", "field1") + .field("pattern", "%{NUMBER:val:float} %{NUMBER:status:int} <%{WORD:msg}>") + .endObject() + .endObject() + .endArray() + .endObject().bytes()) + .get(); + assertBusy(new Runnable() { + @Override + public void run() { + GetPipelineResponse response = new GetPipelineRequestBuilder(client(), GetPipelineAction.INSTANCE) + .setIds("_id") + .get(); + assertThat(response.isFound(), is(true)); + assertThat(response.pipelines().get("_id"), notNullValue()); + } + }); + + SimulatePipelineResponse response = new SimulatePipelineRequestBuilder(client(), SimulatePipelineAction.INSTANCE) + .setId("_id") + .setSource(jsonBuilder().startObject() + .startArray("docs") + .startObject() + .field("_index", "index") + .field("_type", "type") + .field("_id", "id") + .startObject("_source") + .field("foo", "bar") + .endObject() + .endObject() + .endArray() + .endObject().bytes()) + .get(); + + Map expectedDoc = new HashMap<>(); + expectedDoc.put("foo", "bar"); + Data expectedData = new Data("index", "type", "id", expectedDoc); + SimulatedItemResponse expectedResponse = new SimulatedItemResponse(expectedData); + SimulatedItemResponse[] expectedResponses = new SimulatedItemResponse[] { expectedResponse }; + + assertThat(response.responses().length, equalTo(1)); + assertThat(response.responses()[0].getData().getIndex(), equalTo(expectedResponse.getData().getIndex())); + assertThat(response.responses()[0].getData(), equalTo(expectedResponse.getData())); + assertThat(response.responses()[0], equalTo(expectedResponse)); + assertThat(response.responses(), equalTo(expectedResponses)); + assertThat(response.pipelineId(), equalTo("_id")); + } + public void test() throws Exception { new PutPipelineRequestBuilder(client(), PutPipelineAction.INSTANCE) .setId("_id") diff --git a/plugins/ingest/src/test/resources/rest-api-spec/api/ingest.simulate.json b/plugins/ingest/src/test/resources/rest-api-spec/api/ingest.simulate.json new file mode 100644 index 00000000000..33007e3be87 --- /dev/null +++ b/plugins/ingest/src/test/resources/rest-api-spec/api/ingest.simulate.json @@ -0,0 +1,23 @@ +{ + "ingest.simulate": { + "documentation": "https://www.elastic.co/guide/en/elasticsearch/plugins/master/ingest.html", + "methods": [ "GET", "POST" ], + "url": { + "path": "/_ingest/pipeline/_simulate", + "paths": [ "/_ingest/pipeline/_simulate", "/_ingest/pipeline/{id}/_simulate" ], + "parts": { + "id": { + "type" : "string", + "description" : "Pipeline ID", + "required" : false + } + }, + "params": { + } + }, + "body": { + "description" : "The simulate definition", + "required" : true + } + } +} diff --git a/plugins/ingest/src/test/resources/rest-api-spec/test/ingest/80_simulate.yaml b/plugins/ingest/src/test/resources/rest-api-spec/test/ingest/80_simulate.yaml new file mode 100644 index 00000000000..eb68a439bcb --- /dev/null +++ b/plugins/ingest/src/test/resources/rest-api-spec/test/ingest/80_simulate.yaml @@ -0,0 +1,51 @@ +--- +"Test simulate with stored ingest pipeline": + - do: + cluster.health: + wait_for_status: green + + - do: + ingest.put_pipeline: + id: "my_pipeline" + body: > + { + "description": "_description", + "processors": [ + { + "simple" : { + "path" : "field1", + "expected_value" : "_value", + "add_field" : "field2", + "add_field_value" : "_value" + } + } + ] + } + - match: { _id: "my_pipeline" } + + # Simulate a Thread.sleep(), because pipeline are updated in the background + - do: + catch: request_timeout + cluster.health: + wait_for_nodes: 99 + timeout: 2s + - match: { "timed_out": true } + + - do: + ingest.simulate: + id: "my_pipeline" + body: > + { + "docs": [ + { + "_index": "index", + "_type": "type", + "_id": "id", + "_source": { + "foo": "bar" + } + } + ] + } + - length: { docs: 1 } + From 1f29fa4fe94558fc60a36ff7f7e054933ff684ed Mon Sep 17 00:00:00 2001 From: Tal Levy Date: Thu, 5 Nov 2015 16:23:30 -0800 Subject: [PATCH 02/14] update rest status --- .../org/elasticsearch/plugin/ingest/IngestModule.java | 1 - .../transport/simulate/SimulatePipelineResponse.java | 8 +++++--- .../ingest/transport/simulate/SimulatedItemResponse.java | 6 +++++- 3 files changed, 10 insertions(+), 5 deletions(-) diff --git a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/IngestModule.java b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/IngestModule.java index 15c2b7769fb..94dfb4fb690 100644 --- a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/IngestModule.java +++ b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/IngestModule.java @@ -39,7 +39,6 @@ public class IngestModule extends AbstractModule { protected void configure() { binder().bind(IngestRestFilter.class).asEagerSingleton(); binder().bind(PipelineExecutionService.class).asEagerSingleton(); - // TODO(talevy): write it! binder().bind(PipelineSimulateService.class).asEagerSingleton(); binder().bind(PipelineStore.class).asEagerSingleton(); binder().bind(PipelineStoreClient.class).asEagerSingleton(); diff --git a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineResponse.java b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineResponse.java index 92e936dd245..d152424abe5 100644 --- a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineResponse.java +++ b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineResponse.java @@ -23,10 +23,7 @@ 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.ToXContent; import org.elasticsearch.common.xcontent.XContentBuilder; -import org.elasticsearch.common.xcontent.XContentBuilderString; -import org.elasticsearch.ingest.Data; import org.elasticsearch.rest.RestStatus; import java.io.IOException; @@ -91,6 +88,11 @@ public class SimulatePipelineResponse extends ActionResponse implements StatusTo @Override public RestStatus status() { + for (SimulatedItemResponse response : responses) { + if (response.failed()) { + return response.status(); + } + } return RestStatus.OK; } } diff --git a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatedItemResponse.java b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatedItemResponse.java index a41661c0685..5377b8e0ea0 100644 --- a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatedItemResponse.java +++ b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatedItemResponse.java @@ -102,7 +102,11 @@ public class SimulatedItemResponse implements Streamable, StatusToXContent { @Override public RestStatus status() { - return null; + if (failed()) { + return RestStatus.BAD_REQUEST; + } else { + return RestStatus.OK; + } } @Override From c22c1e0f54d8894934c46ff414635ba80fdb4d8a Mon Sep 17 00:00:00 2001 From: Tal Levy Date: Thu, 5 Nov 2015 21:46:48 -0800 Subject: [PATCH 03/14] remove simulate executor service call and move to simple execution --- .../ingest/PipelineExecutionService.java | 4 -- .../SimulatePipelineRequestPayload.java | 24 +++++--- .../simulate/SimulatePipelineResponse.java | 51 ++++++++++++----- .../SimulatePipelineTransportAction.java | 57 ++++--------------- .../elasticsearch/ingest/IngestClientIT.java | 10 +--- .../rest-api-spec/api/ingest.simulate.json | 2 +- 6 files changed, 70 insertions(+), 78 deletions(-) diff --git a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/PipelineExecutionService.java b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/PipelineExecutionService.java index c6314ae6a6c..3b2f71aa142 100644 --- a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/PipelineExecutionService.java +++ b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/PipelineExecutionService.java @@ -53,10 +53,6 @@ public class PipelineExecutionService { return pipeline; } - public Map getProcessorFactoryRegistry() { - return store.getProcessorFactoryRegistry(); - } - public void execute(Data data, String pipelineId, Listener listener) { try { execute(data, getPipeline(pipelineId), listener); diff --git a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineRequestPayload.java b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineRequestPayload.java index b376c67d006..03ab8253583 100644 --- a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineRequestPayload.java +++ b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineRequestPayload.java @@ -22,6 +22,7 @@ import org.elasticsearch.ingest.Data; import org.elasticsearch.ingest.Pipeline; import org.elasticsearch.ingest.processor.ConfigurationUtils; import org.elasticsearch.plugin.ingest.PipelineExecutionService; +import org.elasticsearch.plugin.ingest.PipelineStore; import java.io.IOException; import java.util.ArrayList; @@ -48,24 +49,33 @@ public class SimulatePipelineRequestPayload { } - public Data getDocument(int i) { - return documents.get(i); + public List documents() { + return documents; } - public int size() { - return documents.size(); + public SimulatePipelineResponse execute() { + List responses = new ArrayList<>(); + for (Data data : documents) { + try { + pipeline.execute(data); + responses.add(new SimulatedItemResponse(data)); + } catch (Exception e) { + responses.add(new SimulatedItemResponse(e)); + } + } + return new SimulatePipelineResponse(pipeline.getId(), responses); } public static class Factory { - public SimulatePipelineRequestPayload create(String pipelineId, Map config, PipelineExecutionService executionService) throws IOException { + public SimulatePipelineRequestPayload create(String pipelineId, Map config, PipelineStore pipelineStore) throws IOException { Pipeline pipeline; // if pipeline `id` passed to request, fetch pipeline from store. if (pipelineId != null) { - pipeline = executionService.getPipeline(pipelineId); + pipeline = pipelineStore.get(pipelineId); } else { Map pipelineConfig = (Map) config.get("pipeline"); - pipeline = (new Pipeline.Factory()).create("_pipeline_id", pipelineConfig, executionService.getProcessorFactoryRegistry()); + pipeline = (new Pipeline.Factory()).create("_pipeline_id", pipelineConfig, pipelineStore.getProcessorFactoryRegistry()); } // distribute docs by shard key to SimulateShardPipelineResponse diff --git a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineResponse.java b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineResponse.java index d152424abe5..aeed148d74a 100644 --- a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineResponse.java +++ b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineResponse.java @@ -27,35 +27,46 @@ import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.rest.RestStatus; import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Objects; public class SimulatePipelineResponse extends ActionResponse implements StatusToXContent { private String pipelineId; - private SimulatedItemResponse[] responses; + private List responses; + + public SimulatePipelineResponse() { + + } + + public SimulatePipelineResponse(String pipelineId, List responses) { + this.pipelineId = pipelineId; + this.responses = Collections.unmodifiableList(responses); + } public String pipelineId() { return pipelineId; } - public SimulatePipelineResponse pipelineId(String pipelineId) { + public void pipelineId(String pipelineId) { this.pipelineId = pipelineId; - return this; } - public SimulatePipelineResponse responses(SimulatedItemResponse[] responses) { - this.responses = responses; - return this; - } - - public SimulatedItemResponse[] responses() { + public List responses() { return responses; } + public void responses(List responses) { + this.responses = responses; + } + @Override public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); out.writeString(pipelineId); - out.writeVInt(responses.length); + out.writeVInt(responses.size()); for (SimulatedItemResponse response : responses) { response.writeTo(out); } @@ -66,11 +77,11 @@ public class SimulatePipelineResponse extends ActionResponse implements StatusTo super.readFrom(in); this.pipelineId = in.readString(); int responsesLength = in.readVInt(); - responses = new SimulatedItemResponse[responsesLength]; + responses = new ArrayList<>(); for (int i = 0; i < responsesLength; i++) { SimulatedItemResponse response = new SimulatedItemResponse(); response.readFrom(in); - responses[i] = response; + responses.add(response); } } @@ -90,9 +101,23 @@ public class SimulatePipelineResponse extends ActionResponse implements StatusTo public RestStatus status() { for (SimulatedItemResponse response : responses) { if (response.failed()) { - return response.status(); + return RestStatus.BAD_REQUEST; } } return RestStatus.OK; } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + SimulatePipelineResponse that = (SimulatePipelineResponse) o; + return Objects.equals(pipelineId, that.pipelineId) && + Objects.equals(responses, that.responses); + } + + @Override + public int hashCode() { + return Objects.hash(pipelineId, responses); + } } diff --git a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineTransportAction.java b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineTransportAction.java index 04c26d3c98f..c4f3484ca82 100644 --- a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineTransportAction.java +++ b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineTransportAction.java @@ -25,25 +25,22 @@ 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.util.concurrent.AtomicArray; import org.elasticsearch.common.xcontent.XContentHelper; -import org.elasticsearch.ingest.Data; -import org.elasticsearch.plugin.ingest.PipelineExecutionService; +import org.elasticsearch.plugin.ingest.PipelineStore; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; import java.io.IOException; import java.util.Map; -import java.util.concurrent.atomic.AtomicInteger; public class SimulatePipelineTransportAction extends HandledTransportAction { - private final PipelineExecutionService executionService; + private final PipelineStore pipelineStore; @Inject - public SimulatePipelineTransportAction(Settings settings, ThreadPool threadPool, TransportService transportService, ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver, PipelineExecutionService executionService) { + public SimulatePipelineTransportAction(Settings settings, ThreadPool threadPool, TransportService transportService, ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver, PipelineStore pipelineStore) { super(settings, SimulatePipelineAction.NAME, threadPool, transportService, actionFilters, indexNameExpressionResolver, SimulatePipelineRequest::new); - this.executionService = executionService; + this.pipelineStore = pipelineStore; } @Override @@ -53,49 +50,17 @@ public class SimulatePipelineTransportAction extends HandledTransportAction responses = new AtomicArray<>(payload.size()); - final AtomicInteger counter = new AtomicInteger(payload.size()); - - for (int i = 0; i < payload.size(); i++) { - final int index = i; - - executionService.execute(payload.getDocument(index), payload.pipeline(), new PipelineExecutionService.Listener() { - @Override - public void executed(Data data) { - responses.set(index, new SimulatedItemResponse(data)); - - if (counter.decrementAndGet() == 0) { - finishHim(); - } - } - - @Override - public void failed(Exception e) { - logger.error("failed to execute pipeline [{}]", e, payload.pipelineId()); - responses.set(index, new SimulatedItemResponse(e)); - - if (counter.decrementAndGet() == 0) { - finishHim(); - } - } - - public void finishHim() { - SimulatedItemResponse[] responseArray = new SimulatedItemResponse[responses.length()]; - responses.toArray(responseArray); - - SimulatePipelineResponse response = new SimulatePipelineResponse() - .pipelineId(payload.pipelineId()) - .responses(responseArray); - - listener.onResponse(response); - } - }); - } + threadPool.executor(ThreadPool.Names.MANAGEMENT).execute(new Runnable() { + @Override + public void run() { + listener.onResponse(payload.execute()); + } + }); } } diff --git a/plugins/ingest/src/test/java/org/elasticsearch/ingest/IngestClientIT.java b/plugins/ingest/src/test/java/org/elasticsearch/ingest/IngestClientIT.java index f055d92f2d2..d0d11ae60a0 100644 --- a/plugins/ingest/src/test/java/org/elasticsearch/ingest/IngestClientIT.java +++ b/plugins/ingest/src/test/java/org/elasticsearch/ingest/IngestClientIT.java @@ -105,14 +105,10 @@ public class IngestClientIT extends ESIntegTestCase { expectedDoc.put("foo", "bar"); Data expectedData = new Data("index", "type", "id", expectedDoc); SimulatedItemResponse expectedResponse = new SimulatedItemResponse(expectedData); - SimulatedItemResponse[] expectedResponses = new SimulatedItemResponse[] { expectedResponse }; + List expectedResponses = Arrays.asList(expectedResponse); + SimulatePipelineResponse expected = new SimulatePipelineResponse("_id", expectedResponses); - assertThat(response.responses().length, equalTo(1)); - assertThat(response.responses()[0].getData().getIndex(), equalTo(expectedResponse.getData().getIndex())); - assertThat(response.responses()[0].getData(), equalTo(expectedResponse.getData())); - assertThat(response.responses()[0], equalTo(expectedResponse)); - assertThat(response.responses(), equalTo(expectedResponses)); - assertThat(response.pipelineId(), equalTo("_id")); + assertThat(response, equalTo(expected)); } public void test() throws Exception { diff --git a/plugins/ingest/src/test/resources/rest-api-spec/api/ingest.simulate.json b/plugins/ingest/src/test/resources/rest-api-spec/api/ingest.simulate.json index 33007e3be87..bf08435eb8e 100644 --- a/plugins/ingest/src/test/resources/rest-api-spec/api/ingest.simulate.json +++ b/plugins/ingest/src/test/resources/rest-api-spec/api/ingest.simulate.json @@ -1,7 +1,7 @@ { "ingest.simulate": { "documentation": "https://www.elastic.co/guide/en/elasticsearch/plugins/master/ingest.html", - "methods": [ "GET", "POST" ], + "methods": [ "POST" ], "url": { "path": "/_ingest/pipeline/_simulate", "paths": [ "/_ingest/pipeline/_simulate", "/_ingest/pipeline/{id}/_simulate" ], From b40af1bcfd735df7dd73aaf2d58449b5cabc2a0e Mon Sep 17 00:00:00 2001 From: Tal Levy Date: Wed, 11 Nov 2015 18:20:40 -0800 Subject: [PATCH 04/14] updates, moar verbose --- .../java/org/elasticsearch/ingest/Data.java | 4 + .../org/elasticsearch/ingest/Pipeline.java | 15 ++ .../ingest/processor/ConfigurationUtils.java | 20 +- .../ingest/processor/Processor.java | 6 + .../ingest/processor/date/DateProcessor.java | 7 +- .../processor/geoip/GeoIpProcessor.java | 9 +- .../ingest/processor/grok/GrokProcessor.java | 5 + .../processor/mutate/MutateProcessor.java | 35 +++- .../plugin/ingest/IngestModule.java | 2 + .../ingest/PipelineExecutionService.java | 23 +-- .../rest/RestSimulatePipelineAction.java | 10 +- .../simulate/ParsedSimulateRequest.java | 101 +++++++++++ .../ProcessedData.java} | 44 +++-- .../simulate/SimulateExecutionService.java | 94 ++++++++++ .../simulate/SimulatedItemResponse.java | 171 ++++++++++++++++++ .../simulate/SimulatePipelineRequest.java | 11 ++ .../SimulatePipelineRequestBuilder.java | 5 + .../SimulatePipelineRequestPayload.java | 99 ---------- .../simulate/SimulatePipelineResponse.java | 3 +- .../SimulatePipelineTransportAction.java | 20 +- .../elasticsearch/ingest/IngestClientIT.java | 2 +- .../processor/ConfigurationUtilsTests.java | 2 +- .../ParsedSimulateRequestParserTests.java | 97 ++++++++++ .../SimulateExecutionServiceTests.java | 130 +++++++++++++ .../rest-api-spec/api/ingest.simulate.json | 5 + .../test/ingest/80_simulate.yaml | 149 ++++++++++++++- 26 files changed, 899 insertions(+), 170 deletions(-) create mode 100644 plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/simulate/ParsedSimulateRequest.java rename plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/{transport/simulate/SimulatedItemResponse.java => simulate/ProcessedData.java} (75%) create mode 100644 plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/simulate/SimulateExecutionService.java create mode 100644 plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/simulate/SimulatedItemResponse.java delete mode 100644 plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineRequestPayload.java create mode 100644 plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/simulate/ParsedSimulateRequestParserTests.java create mode 100644 plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/simulate/SimulateExecutionServiceTests.java diff --git a/plugins/ingest/src/main/java/org/elasticsearch/ingest/Data.java b/plugins/ingest/src/main/java/org/elasticsearch/ingest/Data.java index 407c1b894f1..dd1bfde7923 100644 --- a/plugins/ingest/src/main/java/org/elasticsearch/ingest/Data.java +++ b/plugins/ingest/src/main/java/org/elasticsearch/ingest/Data.java @@ -44,6 +44,10 @@ public final class Data { this.document = document; } + public Data(Data other) { + this(other.index, other.type, other.id, new HashMap<>(other.document)); + } + @SuppressWarnings("unchecked") public T getProperty(String path) { // TODO: we should not rely on any core class, so we should have custom map extract value logic: diff --git a/plugins/ingest/src/main/java/org/elasticsearch/ingest/Pipeline.java b/plugins/ingest/src/main/java/org/elasticsearch/ingest/Pipeline.java index a719c4ec727..844a6889e74 100644 --- a/plugins/ingest/src/main/java/org/elasticsearch/ingest/Pipeline.java +++ b/plugins/ingest/src/main/java/org/elasticsearch/ingest/Pipeline.java @@ -70,6 +70,21 @@ public final class Pipeline { return processors; } + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + Pipeline pipeline = (Pipeline) o; + return Objects.equals(id, pipeline.id) && + Objects.equals(description, pipeline.description) && + Objects.equals(processors, pipeline.processors); + } + + @Override + public int hashCode() { + return Objects.hash(id, description, processors); + } + public final static class Factory { public Pipeline create(String id, Map config, Map processorRegistry) throws IOException { diff --git a/plugins/ingest/src/main/java/org/elasticsearch/ingest/processor/ConfigurationUtils.java b/plugins/ingest/src/main/java/org/elasticsearch/ingest/processor/ConfigurationUtils.java index e3f77b4a141..3a8dbbf448d 100644 --- a/plugins/ingest/src/main/java/org/elasticsearch/ingest/processor/ConfigurationUtils.java +++ b/plugins/ingest/src/main/java/org/elasticsearch/ingest/processor/ConfigurationUtils.java @@ -77,39 +77,41 @@ public final class ConfigurationUtils { * 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 List readStringList(Map configuration, String propertyName) { + public static List readOptionalList(Map configuration, String propertyName) { Object value = configuration.remove(propertyName); if (value == null) { - throw new IllegalArgumentException("required property [" + propertyName + "] is missing"); + return null; } - return readStringList(propertyName, value); + 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 List readOptionalStringList(Map configuration, String propertyName) { + public static List readList(Map configuration, String propertyName) { Object value = configuration.remove(propertyName); if (value == null) { - return null; + throw new IllegalArgumentException("required property [" + propertyName + "] is missing"); } - return readStringList(propertyName, value); + + return readList(propertyName, value); } - private static List readStringList(String propertyName, Object value) { + private static List readList(String propertyName, Object value) { if (value instanceof List) { @SuppressWarnings("unchecked") - List stringList = (List) value; + List stringList = (List) 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. * diff --git a/plugins/ingest/src/main/java/org/elasticsearch/ingest/processor/Processor.java b/plugins/ingest/src/main/java/org/elasticsearch/ingest/processor/Processor.java index a023b78ade4..46a9d43e280 100644 --- a/plugins/ingest/src/main/java/org/elasticsearch/ingest/processor/Processor.java +++ b/plugins/ingest/src/main/java/org/elasticsearch/ingest/processor/Processor.java @@ -38,6 +38,11 @@ public interface Processor { */ void execute(Data data); + /** + * Gets the type of a processor + */ + String getType(); + /** * A factory that knows how to construct a processor based on a map of maps. */ @@ -54,6 +59,7 @@ public interface Processor { default void setConfigDirectory(Path configDirectory) { } + @Override default void close() throws IOException { } diff --git a/plugins/ingest/src/main/java/org/elasticsearch/ingest/processor/date/DateProcessor.java b/plugins/ingest/src/main/java/org/elasticsearch/ingest/processor/date/DateProcessor.java index 4c61cc8ee7a..d19433b10ae 100644 --- a/plugins/ingest/src/main/java/org/elasticsearch/ingest/processor/date/DateProcessor.java +++ b/plugins/ingest/src/main/java/org/elasticsearch/ingest/processor/date/DateProcessor.java @@ -78,6 +78,11 @@ public final class DateProcessor implements Processor { data.addField(targetField, ISODateTimeFormat.dateTime().print(dateTime)); } + @Override + public String getType() { + return TYPE; + } + DateTimeZone getTimezone() { return timezone; } @@ -108,7 +113,7 @@ public final class DateProcessor implements Processor { DateTimeZone timezone = timezoneString == null ? DateTimeZone.UTC : DateTimeZone.forID(timezoneString); String localeString = ConfigurationUtils.readOptionalStringProperty(config, "locale"); Locale locale = localeString == null ? Locale.ENGLISH : Locale.forLanguageTag(localeString); - List matchFormats = ConfigurationUtils.readStringList(config, "match_formats"); + List matchFormats = ConfigurationUtils.readList(config, "match_formats"); return new DateProcessor(timezone, locale, matchField, matchFormats, targetField); } } diff --git a/plugins/ingest/src/main/java/org/elasticsearch/ingest/processor/geoip/GeoIpProcessor.java b/plugins/ingest/src/main/java/org/elasticsearch/ingest/processor/geoip/GeoIpProcessor.java index ae63efd05cb..a50bde0e6cb 100644 --- a/plugins/ingest/src/main/java/org/elasticsearch/ingest/processor/geoip/GeoIpProcessor.java +++ b/plugins/ingest/src/main/java/org/elasticsearch/ingest/processor/geoip/GeoIpProcessor.java @@ -40,7 +40,7 @@ import java.security.AccessController; import java.security.PrivilegedAction; import java.util.*; -import static org.elasticsearch.ingest.processor.ConfigurationUtils.readStringList; +import static org.elasticsearch.ingest.processor.ConfigurationUtils.readList; import static org.elasticsearch.ingest.processor.ConfigurationUtils.readStringProperty; public final class GeoIpProcessor implements Processor { @@ -91,6 +91,11 @@ public final class GeoIpProcessor implements Processor { data.addField(targetField, geoData); } + @Override + public String getType() { + return TYPE; + } + String getSourceField() { return sourceField; } @@ -222,7 +227,7 @@ public final class GeoIpProcessor implements Processor { final Set fields; if (config.containsKey("fields")) { fields = EnumSet.noneOf(Field.class); - List fieldNames = readStringList(config, "fields"); + List fieldNames = readList(config, "fields"); for (String fieldName : fieldNames) { try { fields.add(Field.parse(fieldName)); diff --git a/plugins/ingest/src/main/java/org/elasticsearch/ingest/processor/grok/GrokProcessor.java b/plugins/ingest/src/main/java/org/elasticsearch/ingest/processor/grok/GrokProcessor.java index bdba25c7c78..c0688c42220 100644 --- a/plugins/ingest/src/main/java/org/elasticsearch/ingest/processor/grok/GrokProcessor.java +++ b/plugins/ingest/src/main/java/org/elasticsearch/ingest/processor/grok/GrokProcessor.java @@ -56,6 +56,11 @@ public final class GrokProcessor implements Processor { } } + @Override + public String getType() { + return TYPE; + } + String getMatchField() { return matchField; } diff --git a/plugins/ingest/src/main/java/org/elasticsearch/ingest/processor/mutate/MutateProcessor.java b/plugins/ingest/src/main/java/org/elasticsearch/ingest/processor/mutate/MutateProcessor.java index d09454b1016..4a950bea083 100644 --- a/plugins/ingest/src/main/java/org/elasticsearch/ingest/processor/mutate/MutateProcessor.java +++ b/plugins/ingest/src/main/java/org/elasticsearch/ingest/processor/mutate/MutateProcessor.java @@ -134,6 +134,11 @@ public final class MutateProcessor implements Processor { } } + @Override + public String getType() { + return TYPE; + } + private void doUpdate(Data data) { for(Map.Entry entry : update.entrySet()) { data.addField(entry.getKey(), entry.getValue()); @@ -272,6 +277,28 @@ public final class MutateProcessor implements Processor { } } + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + MutateProcessor that = (MutateProcessor) o; + return Objects.equals(update, that.update) && + Objects.equals(rename, that.rename) && + Objects.equals(convert, that.convert) && + Objects.equals(split, that.split) && + Objects.equals(gsub, that.gsub) && + Objects.equals(join, that.join) && + Objects.equals(remove, that.remove) && + Objects.equals(trim, that.trim) && + Objects.equals(uppercase, that.uppercase) && + Objects.equals(lowercase, that.lowercase); + } + + @Override + public int hashCode() { + return Objects.hash(update, rename, convert, split, gsub, join, remove, trim, uppercase, lowercase); + } + public static final class Factory implements Processor.Factory { @Override public MutateProcessor create(Map config) throws IOException { @@ -281,10 +308,10 @@ public final class MutateProcessor implements Processor { Map split = ConfigurationUtils.readOptionalMap(config, "split"); Map> gsubConfig = ConfigurationUtils.readOptionalMap(config, "gsub"); Map join = ConfigurationUtils.readOptionalMap(config, "join"); - List remove = ConfigurationUtils.readOptionalStringList(config, "remove"); - List trim = ConfigurationUtils.readOptionalStringList(config, "trim"); - List uppercase = ConfigurationUtils.readOptionalStringList(config, "uppercase"); - List lowercase = ConfigurationUtils.readOptionalStringList(config, "lowercase"); + List remove = ConfigurationUtils.readOptionalList(config, "remove"); + List trim = ConfigurationUtils.readOptionalList(config, "trim"); + List uppercase = ConfigurationUtils.readOptionalList(config, "uppercase"); + List lowercase = ConfigurationUtils.readOptionalList(config, "lowercase"); // pre-compile regex patterns List gsubExpressions = null; diff --git a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/IngestModule.java b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/IngestModule.java index 94dfb4fb690..e9f7cdd1f88 100644 --- a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/IngestModule.java +++ b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/IngestModule.java @@ -27,6 +27,7 @@ import org.elasticsearch.ingest.processor.geoip.GeoIpProcessor; import org.elasticsearch.ingest.processor.grok.GrokProcessor; import org.elasticsearch.ingest.processor.mutate.MutateProcessor; import org.elasticsearch.plugin.ingest.rest.IngestRestFilter; +import org.elasticsearch.plugin.ingest.simulate.SimulateExecutionService; import java.util.HashMap; import java.util.Map; @@ -41,6 +42,7 @@ public class IngestModule extends AbstractModule { binder().bind(PipelineExecutionService.class).asEagerSingleton(); binder().bind(PipelineStore.class).asEagerSingleton(); binder().bind(PipelineStoreClient.class).asEagerSingleton(); + binder().bind(SimulateExecutionService.class).asEagerSingleton(); addProcessor(GeoIpProcessor.TYPE, new GeoIpProcessor.Factory()); addProcessor(GrokProcessor.TYPE, new GrokProcessor.Factory()); diff --git a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/PipelineExecutionService.java b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/PipelineExecutionService.java index 3b2f71aa142..18d656813ec 100644 --- a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/PipelineExecutionService.java +++ b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/PipelineExecutionService.java @@ -25,11 +25,8 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.concurrent.EsExecutors; import org.elasticsearch.ingest.Data; import org.elasticsearch.ingest.Pipeline; -import org.elasticsearch.ingest.processor.Processor; import org.elasticsearch.threadpool.ThreadPool; -import java.util.Map; - public class PipelineExecutionService { static final String THREAD_POOL_NAME = IngestPlugin.NAME; @@ -43,25 +40,13 @@ public class PipelineExecutionService { this.threadPool = threadPool; } - public Pipeline getPipeline(String pipelineId) { - Pipeline pipeline = store.get(pipelineId); - - if (pipeline == null) { - throw new IllegalArgumentException(LoggerMessageFormat.format("pipeline with id [{}] does not exist", pipelineId)); - } - - return pipeline; - } - public void execute(Data data, String pipelineId, Listener listener) { - try { - execute(data, getPipeline(pipelineId), listener); - } catch (IllegalArgumentException e) { - listener.failed(e); + Pipeline pipeline = store.get(pipelineId); + if (pipeline == null) { + listener.failed(new IllegalArgumentException(LoggerMessageFormat.format("pipeline with id [{}] does not exist", pipelineId))); + return; } - } - public void execute(Data data, Pipeline pipeline, Listener listener) { threadPool.executor(THREAD_POOL_NAME).execute(new Runnable() { @Override public void run() { diff --git a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/rest/RestSimulatePipelineAction.java b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/rest/RestSimulatePipelineAction.java index 983f43d6a1a..c110da28b99 100644 --- a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/rest/RestSimulatePipelineAction.java +++ b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/rest/RestSimulatePipelineAction.java @@ -28,6 +28,7 @@ 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.RestStatusToXContentListener; public class RestSimulatePipelineAction extends BaseRestHandler { @@ -37,15 +38,20 @@ public class RestSimulatePipelineAction extends BaseRestHandler { super(settings, controller, client); controller.registerHandler(RestRequest.Method.POST, "/_ingest/pipeline/{id}/_simulate", this); controller.registerHandler(RestRequest.Method.POST, "/_ingest/pipeline/_simulate", this); + // controller.registerHandler(RestRequest.Method.GET, "/_ingest/pipeline/{id}/_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(); request.id(restRequest.param("id")); - if (restRequest.hasContent()) { - request.source(restRequest.content()); + request.verbose(restRequest.paramAsBoolean("verbose", false)); + + if (RestActions.hasBodyContent(restRequest)) { + request.source(RestActions.getRestContent(restRequest)); } + client.execute(SimulatePipelineAction.INSTANCE, request, new RestStatusToXContentListener<>(channel)); } } diff --git a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/simulate/ParsedSimulateRequest.java b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/simulate/ParsedSimulateRequest.java new file mode 100644 index 00000000000..d9f2dc4dc0c --- /dev/null +++ b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/simulate/ParsedSimulateRequest.java @@ -0,0 +1,101 @@ +/* + * 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.plugin.ingest.simulate; + +import org.elasticsearch.ingest.Data; +import org.elasticsearch.ingest.Pipeline; +import org.elasticsearch.ingest.processor.ConfigurationUtils; +import org.elasticsearch.plugin.ingest.PipelineStore; + +import java.io.IOException; +import java.util.*; + +public class ParsedSimulateRequest { + private final List documents; + private final Pipeline pipeline; + private final boolean verbose; + + ParsedSimulateRequest(Pipeline pipeline, List documents, boolean verbose) { + this.pipeline = pipeline; + this.documents = Collections.unmodifiableList(documents); + this.verbose = verbose; + } + + public Pipeline getPipeline() { + return pipeline; + } + + public List getDocuments() { + return documents; + } + + public boolean isVerbose() { + return verbose; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + ParsedSimulateRequest that = (ParsedSimulateRequest) o; + return Objects.equals(verbose, that.verbose) && + Objects.equals(documents, that.documents) && + Objects.equals(pipeline, that.pipeline); + } + + @Override + public int hashCode() { + return Objects.hash(documents, pipeline, verbose); + } + + public static class Parser { + private static final Pipeline.Factory PIPELINE_FACTORY = new Pipeline.Factory(); + public static final String SIMULATED_PIPELINE_ID = "_simulate_pipeline"; + + public ParsedSimulateRequest parse(String pipelineId, Map config, boolean verbose, PipelineStore pipelineStore) throws IOException { + Pipeline pipeline; + // if pipeline `id` passed to request, fetch pipeline from store. + if (pipelineId != null) { + pipeline = pipelineStore.get(pipelineId); + } else { + Map pipelineConfig = ConfigurationUtils.readOptionalMap(config, "pipeline"); + pipeline = PIPELINE_FACTORY.create(SIMULATED_PIPELINE_ID, pipelineConfig, pipelineStore.getProcessorFactoryRegistry()); + } + + List> docs = ConfigurationUtils.readList(config, "docs"); + + List dataList = new ArrayList<>(); + + for (int i = 0; i < docs.size(); i++) { + Map dataMap = docs.get(i); + Map document = ConfigurationUtils.readOptionalMap(dataMap, "_source"); + if (document == null) { + document = Collections.emptyMap(); + } + Data data = new Data(ConfigurationUtils.readOptionalStringProperty(dataMap, "_index"), + ConfigurationUtils.readOptionalStringProperty(dataMap, "_type"), + ConfigurationUtils.readOptionalStringProperty(dataMap, "_id"), + document); + dataList.add(data); + } + + return new ParsedSimulateRequest(pipeline, dataList, verbose); + } + } +} diff --git a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatedItemResponse.java b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/simulate/ProcessedData.java similarity index 75% rename from plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatedItemResponse.java rename to plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/simulate/ProcessedData.java index 5377b8e0ea0..238a1e79d40 100644 --- a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatedItemResponse.java +++ b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/simulate/ProcessedData.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.elasticsearch.plugin.ingest.transport.simulate; +package org.elasticsearch.plugin.ingest.simulate; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; @@ -31,24 +31,26 @@ import java.io.IOException; import java.util.Map; import java.util.Objects; -public class SimulatedItemResponse implements Streamable, StatusToXContent { +public class ProcessedData implements Streamable, StatusToXContent { + private String processorId; private Data data; private Throwable failure; - public SimulatedItemResponse() { + public ProcessedData() { } - public SimulatedItemResponse(Data data) { + public ProcessedData(String processorId, Data data) { + this.processorId = processorId; this.data = data; } - public SimulatedItemResponse(Throwable failure) { + public ProcessedData(Throwable failure) { this.failure = failure; } - public boolean failed() { + public boolean isFailed() { return this.failure != null; } @@ -56,14 +58,18 @@ public class SimulatedItemResponse implements Streamable, StatusToXContent { return data; } + public String getProcessorId() { + return processorId; + } + @Override public void readFrom(StreamInput in) throws IOException { - boolean failed = in.readBoolean(); - - if (failed) { + boolean isFailure = in.readBoolean(); + if (isFailure) { this.failure = in.readThrowable(); // TODO(talevy): check out mget for throwable limitations } else { + this.processorId = in.readString(); String index = in.readString(); String type = in.readString(); String id = in.readString(); @@ -74,11 +80,11 @@ public class SimulatedItemResponse implements Streamable, StatusToXContent { @Override public void writeTo(StreamOutput out) throws IOException { - out.writeBoolean(failed()); - - if (failed()) { + out.writeBoolean(isFailed()); + if (isFailed()) { out.writeThrowable(failure); } else { + out.writeString(processorId); out.writeString(data.getIndex()); out.writeString(data.getType()); out.writeString(data.getId()); @@ -89,8 +95,9 @@ public class SimulatedItemResponse implements Streamable, StatusToXContent { @Override public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { builder.startObject(); - builder.field(Fields.ERROR, failed()); - if (failed()) { + builder.field(Fields.PROCESSOR_ID, processorId); + builder.field(Fields.ERROR, isFailed()); + if (isFailed()) { builder.field(Fields.FAILURE, failure.toString()); } else { builder.field(Fields.MODIFIED, data.isModified()); @@ -102,7 +109,7 @@ public class SimulatedItemResponse implements Streamable, StatusToXContent { @Override public RestStatus status() { - if (failed()) { + if (isFailed()) { return RestStatus.BAD_REQUEST; } else { return RestStatus.OK; @@ -115,17 +122,18 @@ public class SimulatedItemResponse implements Streamable, StatusToXContent { if (obj == null || getClass() != obj.getClass()) { return false; } - SimulatedItemResponse other = (SimulatedItemResponse) obj; - return Objects.equals(data, other.data) && Objects.equals(failure, other.failure); + ProcessedData other = (ProcessedData) obj; + return Objects.equals(processorId, other.processorId) && Objects.equals(data, other.data) && Objects.equals(failure, other.failure); } @Override public int hashCode() { - return Objects.hash(data, failure); + return Objects.hash(processorId, data, failure); } static final class Fields { static final XContentBuilderString DOCUMENT = new XContentBuilderString("doc"); + static final XContentBuilderString PROCESSOR_ID = new XContentBuilderString("processor_id"); static final XContentBuilderString ERROR = new XContentBuilderString("error"); static final XContentBuilderString FAILURE = new XContentBuilderString("failure"); static final XContentBuilderString MODIFIED = new XContentBuilderString("modified"); diff --git a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/simulate/SimulateExecutionService.java b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/simulate/SimulateExecutionService.java new file mode 100644 index 00000000000..d7949532c0b --- /dev/null +++ b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/simulate/SimulateExecutionService.java @@ -0,0 +1,94 @@ +/* + * 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.plugin.ingest.simulate; + +import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.ingest.Data; +import org.elasticsearch.ingest.Pipeline; +import org.elasticsearch.ingest.processor.Processor; +import org.elasticsearch.plugin.ingest.transport.simulate.SimulatePipelineResponse; +import org.elasticsearch.threadpool.ThreadPool; + +import java.util.ArrayList; +import java.util.List; + +public class SimulateExecutionService { + + static final String THREAD_POOL_NAME = ThreadPool.Names.MANAGEMENT; + + private final ThreadPool threadPool; + + @Inject + public SimulateExecutionService(ThreadPool threadPool) { + this.threadPool = threadPool; + } + + + SimulatedItemResponse executeItem(Pipeline pipeline, Data data, boolean verbose) { + try { + if (verbose) { + return executeVerboseItem(pipeline, data); + } else { + pipeline.execute(data); + return new SimulatedItemResponse(data); + } + } catch (Exception e) { + return new SimulatedItemResponse(e); + } + + } + + SimulatedItemResponse executeVerboseItem(Pipeline pipeline, Data data) { + List processedDataList = new ArrayList<>(); + Data currentData = new Data(data); + for (int i = 0; i < pipeline.getProcessors().size(); i++) { + Processor processor = pipeline.getProcessors().get(i); + String processorId = "processor[" + processor.getType() + "]-" + i; + + processor.execute(currentData); + processedDataList.add(new ProcessedData(processorId, currentData)); + + currentData = new Data(currentData); + } + return new SimulatedItemResponse(processedDataList); + } + + SimulatePipelineResponse execute(ParsedSimulateRequest request) { + List responses = new ArrayList<>(); + for (Data data : request.getDocuments()) { + responses.add(executeItem(request.getPipeline(), data, request.isVerbose())); + } + return new SimulatePipelineResponse(request.getPipeline().getId(), responses); + } + + public void execute(ParsedSimulateRequest request, Listener listener) { + threadPool.executor(THREAD_POOL_NAME).execute(new Runnable() { + @Override + public void run() { + SimulatePipelineResponse response = execute(request); + listener.onResponse(response); + } + }); + } + + public interface Listener { + void onResponse(SimulatePipelineResponse response); + } +} diff --git a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/simulate/SimulatedItemResponse.java b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/simulate/SimulatedItemResponse.java new file mode 100644 index 00000000000..0fff29ed0c4 --- /dev/null +++ b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/simulate/SimulatedItemResponse.java @@ -0,0 +1,171 @@ +/* + * 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.plugin.ingest.simulate; + +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.io.stream.Streamable; +import org.elasticsearch.common.xcontent.StatusToXContent; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentBuilderString; +import org.elasticsearch.ingest.Data; +import org.elasticsearch.rest.RestStatus; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Objects; + +public class SimulatedItemResponse implements Streamable, StatusToXContent { + + private Data data; + private List processedDataList; + private Throwable failure; + + public SimulatedItemResponse() { + + } + + public SimulatedItemResponse(Data data) { + this.data = data; + } + + public SimulatedItemResponse(List processedDataList) { + this.processedDataList = processedDataList; + } + + public SimulatedItemResponse(Throwable failure) { + this.failure = failure; + } + + public boolean isFailed() { + return this.failure != null; + } + + public boolean isVerbose() { + return this.processedDataList != null; + } + + public Data getData() { + return data; + } + + public List getProcessedDataList() { + return processedDataList; + } + + @Override + public void readFrom(StreamInput in) throws IOException { + boolean isFailed = in.readBoolean(); + boolean isVerbose = in.readBoolean(); + if (isFailed) { + this.failure = in.readThrowable(); + // TODO(talevy): check out mget for throwable limitations + } else if (isVerbose) { + int size = in.readVInt(); + processedDataList = new ArrayList<>(); + for (int i = 0; i < size; i++) { + ProcessedData processedData = new ProcessedData(); + processedData.readFrom(in); + processedDataList.add(processedData); + } + } else { + String index = in.readString(); + String type = in.readString(); + String id = in.readString(); + Map doc = in.readMap(); + this.data = new Data(index, type, id, doc); + } + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeBoolean(isFailed()); + out.writeBoolean(isVerbose()); + + if (isFailed()) { + out.writeThrowable(failure); + } else if (isVerbose()) { + out.writeVInt(processedDataList.size()); + for (ProcessedData p : processedDataList) { + p.writeTo(out); + } + } else { + out.writeString(data.getIndex()); + out.writeString(data.getType()); + out.writeString(data.getId()); + out.writeMap(data.getDocument()); + } + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject(); + builder.field(Fields.ERROR, isFailed()); + builder.field(Fields.VERBOSE, isVerbose()); + if (isFailed()) { + builder.field(Fields.FAILURE, failure.toString()); + } else if (isVerbose()) { + builder.startArray(Fields.PROCESSOR_STEPS); + for (ProcessedData processedData : processedDataList) { + builder.value(processedData); + } + builder.endArray(); + } else { + builder.field(Fields.MODIFIED, data.isModified()); + builder.field(Fields.DOCUMENT, data.getDocument()); + } + builder.endObject(); + return builder; + } + + @Override + public RestStatus status() { + if (isFailed()) { + return RestStatus.BAD_REQUEST; + } else { + return RestStatus.OK; + } + } + + @Override + public boolean equals(Object obj) { + if (obj == this) { return true; } + if (obj == null || getClass() != obj.getClass()) { + return false; + } + SimulatedItemResponse other = (SimulatedItemResponse) obj; + return Objects.equals(data, other.data) && Objects.equals(processedDataList, other.processedDataList) && Objects.equals(failure, other.failure); + } + + @Override + public int hashCode() { + return Objects.hash(data, processedDataList, failure); + } + + static final class Fields { + static final XContentBuilderString DOCUMENT = new XContentBuilderString("doc"); + static final XContentBuilderString ERROR = new XContentBuilderString("error"); + static final XContentBuilderString VERBOSE = new XContentBuilderString("verbose"); + static final XContentBuilderString FAILURE = new XContentBuilderString("failure"); + static final XContentBuilderString MODIFIED = new XContentBuilderString("modified"); + static final XContentBuilderString PROCESSOR_STEPS = new XContentBuilderString("processor_steps"); + } +} diff --git a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineRequest.java b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineRequest.java index 2bf7c01bf24..8c5a5f1ef71 100644 --- a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineRequest.java +++ b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineRequest.java @@ -32,6 +32,7 @@ import static org.elasticsearch.action.ValidateActions.addValidationError; public class SimulatePipelineRequest extends ActionRequest { private String id; + private boolean verbose; private BytesReference source; @Override @@ -51,6 +52,14 @@ public class SimulatePipelineRequest extends ActionRequest { this.id = id; } + public boolean verbose() { + return verbose; + } + + public void verbose(boolean verbose) { + this.verbose = verbose; + } + public BytesReference source() { return source; } @@ -63,6 +72,7 @@ public class SimulatePipelineRequest extends ActionRequest { public void readFrom(StreamInput in) throws IOException { super.readFrom(in); id = in.readString(); + verbose = in.readBoolean(); source = in.readBytesReference(); } @@ -70,6 +80,7 @@ public class SimulatePipelineRequest extends ActionRequest { public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); out.writeString(id); + out.writeBoolean(verbose); out.writeBytesReference(source); } } diff --git a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineRequestBuilder.java b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineRequestBuilder.java index 8f446b75238..7b140b345f4 100644 --- a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineRequestBuilder.java +++ b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineRequestBuilder.java @@ -34,6 +34,11 @@ public class SimulatePipelineRequestBuilder extends ActionRequestBuilder documents; - private final Pipeline pipeline; - - public SimulatePipelineRequestPayload(Pipeline pipeline, List documents) { - this.pipeline = pipeline; - this.documents = Collections.unmodifiableList(documents); - } - - public String pipelineId() { - return pipeline.getId(); - } - - public Pipeline pipeline() { - return pipeline; - } - - - public List documents() { - return documents; - } - - public SimulatePipelineResponse execute() { - List responses = new ArrayList<>(); - for (Data data : documents) { - try { - pipeline.execute(data); - responses.add(new SimulatedItemResponse(data)); - } catch (Exception e) { - responses.add(new SimulatedItemResponse(e)); - } - } - return new SimulatePipelineResponse(pipeline.getId(), responses); - } - - public static class Factory { - - public SimulatePipelineRequestPayload create(String pipelineId, Map config, PipelineStore pipelineStore) throws IOException { - Pipeline pipeline; - // if pipeline `id` passed to request, fetch pipeline from store. - if (pipelineId != null) { - pipeline = pipelineStore.get(pipelineId); - } else { - Map pipelineConfig = (Map) config.get("pipeline"); - pipeline = (new Pipeline.Factory()).create("_pipeline_id", pipelineConfig, pipelineStore.getProcessorFactoryRegistry()); - } - - // distribute docs by shard key to SimulateShardPipelineResponse - List> docs = (List>) config.get("docs"); - - List dataList = new ArrayList<>(); - - for (int i = 0; i < docs.size(); i++) { - Map dataMap = docs.get(i); - Map document = (Map) dataMap.get("_source"); - Data data = new Data(ConfigurationUtils.readStringProperty(dataMap, "_index", null), - ConfigurationUtils.readStringProperty(dataMap, "_type", null), - ConfigurationUtils.readStringProperty(dataMap, "_id", null), - document); - dataList.add(data); - } - - return new SimulatePipelineRequestPayload(pipeline, dataList); - } - } -} diff --git a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineResponse.java b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineResponse.java index aeed148d74a..6659635f7b0 100644 --- a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineResponse.java +++ b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineResponse.java @@ -24,6 +24,7 @@ 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.plugin.ingest.simulate.SimulatedItemResponse; import org.elasticsearch.rest.RestStatus; import java.io.IOException; @@ -100,7 +101,7 @@ public class SimulatePipelineResponse extends ActionResponse implements StatusTo @Override public RestStatus status() { for (SimulatedItemResponse response : responses) { - if (response.failed()) { + if (response.isFailed()) { return RestStatus.BAD_REQUEST; } } diff --git a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineTransportAction.java b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineTransportAction.java index c4f3484ca82..f43d0ec03d0 100644 --- a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineTransportAction.java +++ b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineTransportAction.java @@ -26,7 +26,10 @@ 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.Pipeline; +import org.elasticsearch.plugin.ingest.simulate.ParsedSimulateRequest; import org.elasticsearch.plugin.ingest.PipelineStore; +import org.elasticsearch.plugin.ingest.simulate.SimulateExecutionService; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; @@ -34,32 +37,33 @@ import java.io.IOException; import java.util.Map; public class SimulatePipelineTransportAction extends HandledTransportAction { - private final PipelineStore pipelineStore; + private final SimulateExecutionService executionService; @Inject - public SimulatePipelineTransportAction(Settings settings, ThreadPool threadPool, TransportService transportService, ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver, PipelineStore pipelineStore) { + public SimulatePipelineTransportAction(Settings settings, ThreadPool threadPool, TransportService transportService, ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver, PipelineStore pipelineStore, SimulateExecutionService executionService) { super(settings, SimulatePipelineAction.NAME, threadPool, transportService, actionFilters, indexNameExpressionResolver, SimulatePipelineRequest::new); this.pipelineStore = pipelineStore; + this.executionService = executionService; } @Override protected void doExecute(SimulatePipelineRequest request, ActionListener listener) { Map source = XContentHelper.convertToMap(request.source(), false).v2(); - SimulatePipelineRequestPayload payload; - SimulatePipelineRequestPayload.Factory factory = new SimulatePipelineRequestPayload.Factory(); + ParsedSimulateRequest payload; + ParsedSimulateRequest.Parser parser = new ParsedSimulateRequest.Parser(); try { - payload = factory.create(request.id(), source, pipelineStore); + payload = parser.parse(request.id(), source, request.verbose(), pipelineStore); } catch (IOException e) { listener.onFailure(e); return; } - threadPool.executor(ThreadPool.Names.MANAGEMENT).execute(new Runnable() { + executionService.execute(payload, new SimulateExecutionService.Listener() { @Override - public void run() { - listener.onResponse(payload.execute()); + public void onResponse(SimulatePipelineResponse response) { + listener.onResponse(response); } }); } diff --git a/plugins/ingest/src/test/java/org/elasticsearch/ingest/IngestClientIT.java b/plugins/ingest/src/test/java/org/elasticsearch/ingest/IngestClientIT.java index d0d11ae60a0..61fcad9a3c3 100644 --- a/plugins/ingest/src/test/java/org/elasticsearch/ingest/IngestClientIT.java +++ b/plugins/ingest/src/test/java/org/elasticsearch/ingest/IngestClientIT.java @@ -33,7 +33,7 @@ import org.elasticsearch.plugin.ingest.transport.put.PutPipelineRequestBuilder; import org.elasticsearch.plugin.ingest.transport.simulate.SimulatePipelineAction; import org.elasticsearch.plugin.ingest.transport.simulate.SimulatePipelineRequestBuilder; import org.elasticsearch.plugin.ingest.transport.simulate.SimulatePipelineResponse; -import org.elasticsearch.plugin.ingest.transport.simulate.SimulatedItemResponse; +import org.elasticsearch.plugin.ingest.simulate.SimulatedItemResponse; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.ESIntegTestCase; diff --git a/plugins/ingest/src/test/java/org/elasticsearch/ingest/processor/ConfigurationUtilsTests.java b/plugins/ingest/src/test/java/org/elasticsearch/ingest/processor/ConfigurationUtilsTests.java index 274a952f935..38c4e2493a2 100644 --- a/plugins/ingest/src/test/java/org/elasticsearch/ingest/processor/ConfigurationUtilsTests.java +++ b/plugins/ingest/src/test/java/org/elasticsearch/ingest/processor/ConfigurationUtilsTests.java @@ -59,7 +59,7 @@ public class ConfigurationUtilsTests extends ESTestCase { // TODO(talevy): Issue with generics. This test should fail, "int" is of type List public void testOptional_InvalidType() { - List val = ConfigurationUtils.readStringList(config, "int"); + List val = ConfigurationUtils.readList(config, "int"); assertThat(val, equalTo(Arrays.asList(2))); } } diff --git a/plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/simulate/ParsedSimulateRequestParserTests.java b/plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/simulate/ParsedSimulateRequestParserTests.java new file mode 100644 index 00000000000..01f3beee391 --- /dev/null +++ b/plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/simulate/ParsedSimulateRequestParserTests.java @@ -0,0 +1,97 @@ +/* + * 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.plugin.ingest.simulate; + +import org.elasticsearch.ingest.Data; +import org.elasticsearch.ingest.Pipeline; +import org.elasticsearch.ingest.processor.Processor; +import org.elasticsearch.ingest.processor.mutate.MutateProcessor; +import org.elasticsearch.plugin.ingest.PipelineStore; +import org.elasticsearch.test.ESTestCase; +import org.junit.Before; + +import java.io.IOException; +import java.util.*; + +import static org.hamcrest.Matchers.equalTo; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class ParsedSimulateRequestParserTests extends ESTestCase { + private static final ParsedSimulateRequest.Parser PARSER = new ParsedSimulateRequest.Parser(); + + private Map processorRegistry; + private PipelineStore store; + private Processor processor; + private Pipeline pipeline; + private Data data; + + @Before + public void init() throws IOException { + List uppercase = Collections.unmodifiableList(Collections.singletonList("foo")); + processor = new MutateProcessor(null, null, null, null, null, null, null, null, uppercase, null); + pipeline = new Pipeline(ParsedSimulateRequest.Parser.SIMULATED_PIPELINE_ID, null, Collections.unmodifiableList(Arrays.asList(processor))); + data = new Data("_index", "_type", "_id", Collections.emptyMap()); + processorRegistry = new HashMap<>(); + processorRegistry.put("mutate", new MutateProcessor.Factory()); + store = mock(PipelineStore.class); + when(store.get("_id")).thenReturn(pipeline); + when(store.getProcessorFactoryRegistry()).thenReturn(processorRegistry); + } + + public void testParse_UsingPipelineStore() throws Exception { + ParsedSimulateRequest expectedRequest = new ParsedSimulateRequest(pipeline, Collections.singletonList(data), false); + + Map raw = new HashMap<>(); + List> docs = new ArrayList<>(); + Map doc = new HashMap<>(); + doc.put("_index", "_index"); + doc.put("_type", "_type"); + doc.put("_id", "_id"); + docs.add(doc); + raw.put("docs", docs); + + ParsedSimulateRequest actualRequest = PARSER.parse("_id", raw, false, store); + assertThat(actualRequest, equalTo(expectedRequest)); + } + + public void testParse_ProvidedPipeline() throws Exception { + ParsedSimulateRequest expectedRequest = new ParsedSimulateRequest(pipeline, Collections.singletonList(data), false); + + Map raw = new HashMap<>(); + List> docs = new ArrayList<>(); + Map doc = new HashMap<>(); + doc.put("_index", "_index"); + doc.put("_type", "_type"); + doc.put("_id", "_id"); + docs.add(doc); + + Map processorConfig = new HashMap<>(); + processorConfig.put("uppercase", Arrays.asList("foo")); + Map pipelineConfig = new HashMap<>(); + pipelineConfig.put("processors", Collections.singletonList(Collections.singletonMap("mutate", processorConfig))); + + raw.put("docs", docs); + raw.put("pipeline", pipelineConfig); + + ParsedSimulateRequest actualRequest = PARSER.parse(null, raw, false, store); + assertThat(actualRequest, equalTo(expectedRequest)); + } +} diff --git a/plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/simulate/SimulateExecutionServiceTests.java b/plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/simulate/SimulateExecutionServiceTests.java new file mode 100644 index 00000000000..f622db68d95 --- /dev/null +++ b/plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/simulate/SimulateExecutionServiceTests.java @@ -0,0 +1,130 @@ +/* + * 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.plugin.ingest.simulate; + +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.ingest.Data; +import org.elasticsearch.ingest.Pipeline; +import org.elasticsearch.ingest.processor.Processor; +import org.elasticsearch.plugin.ingest.PipelineStore; +import org.elasticsearch.plugin.ingest.transport.simulate.SimulatePipelineResponse; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.threadpool.ThreadPool; +import org.junit.After; +import org.junit.Before; + +import java.util.Arrays; +import java.util.Collections; + +import static org.hamcrest.Matchers.equalTo; +import static org.mockito.Mockito.*; + +public class SimulateExecutionServiceTests extends ESTestCase { + + private PipelineStore store; + private ThreadPool threadPool; + private SimulateExecutionService executionService; + private Pipeline pipeline; + private Processor processor; + private Data data; + + @Before + public void setup() { + store = mock(PipelineStore.class); + threadPool = new ThreadPool( + Settings.builder() + .put("name", "_name") + .build() + ); + executionService = new SimulateExecutionService(threadPool); + processor = mock(Processor.class); + when(processor.getType()).thenReturn("mock"); + pipeline = new Pipeline("_id", "_description", Arrays.asList(processor, processor)); + data = new Data("_index", "_type", "_id", Collections.emptyMap()); + } + + @After + public void destroy() { + threadPool.shutdown(); + } + + public void testExecuteVerboseItem() throws Exception { + SimulatedItemResponse expectedItemResponse = new SimulatedItemResponse( + Arrays.asList(new ProcessedData("processor[mock]-0", data), new ProcessedData("processor[mock]-1", data))); + SimulatedItemResponse actualItemResponse = executionService.executeVerboseItem(pipeline, data); + verify(processor, times(2)).execute(data); + assertThat(actualItemResponse, equalTo(expectedItemResponse)); + } + + public void testExecuteItem_verboseSuccessful() throws Exception { + SimulatedItemResponse expectedItemResponse = new SimulatedItemResponse( + Arrays.asList(new ProcessedData("processor[mock]-0", data), new ProcessedData("processor[mock]-1", data))); + SimulatedItemResponse actualItemResponse = executionService.executeItem(pipeline, data, true); + verify(processor, times(2)).execute(data); + assertThat(actualItemResponse, equalTo(expectedItemResponse)); + } + + public void testExecuteItem_Simple() throws Exception { + SimulatedItemResponse expectedItemResponse = new SimulatedItemResponse(data); + SimulatedItemResponse actualItemResponse = executionService.executeItem(pipeline, data, false); + verify(processor, times(2)).execute(data); + assertThat(actualItemResponse, equalTo(expectedItemResponse)); + } + + public void testExecuteItem_Failure() throws Exception { + Exception e = new RuntimeException("processor failed"); + SimulatedItemResponse expectedItemResponse = new SimulatedItemResponse(e); + doThrow(e).when(processor).execute(data); + SimulatedItemResponse actualItemResponse = executionService.executeItem(pipeline, data, false); + verify(processor, times(1)).execute(data); + assertThat(actualItemResponse, equalTo(expectedItemResponse)); + } + + public void testExecute() throws Exception { + SimulateExecutionService.Listener listener = mock(SimulateExecutionService.Listener.class); + SimulatedItemResponse itemResponse = new SimulatedItemResponse(data); + ParsedSimulateRequest request = new ParsedSimulateRequest(pipeline, Collections.singletonList(data), false); + executionService.execute(request, listener); + SimulatePipelineResponse response = new SimulatePipelineResponse("_id", Collections.singletonList(itemResponse)); + assertBusy(new Runnable() { + @Override + public void run() { + verify(processor, times(2)).execute(data); + verify(listener).onResponse(response); + } + }); + } + + public void testExecute_Verbose() throws Exception { + SimulateExecutionService.Listener listener = mock(SimulateExecutionService.Listener.class); + ParsedSimulateRequest request = new ParsedSimulateRequest(pipeline, Collections.singletonList(data), true); + SimulatedItemResponse itemResponse = new SimulatedItemResponse( + Arrays.asList(new ProcessedData("processor[mock]-0", data), new ProcessedData("processor[mock]-1", data))); + executionService.execute(request, listener); + SimulatePipelineResponse response = new SimulatePipelineResponse("_id", Collections.singletonList(itemResponse)); + assertBusy(new Runnable() { + @Override + public void run() { + verify(processor, times(2)).execute(data); + verify(listener).onResponse(response); + } + }); + } +} diff --git a/plugins/ingest/src/test/resources/rest-api-spec/api/ingest.simulate.json b/plugins/ingest/src/test/resources/rest-api-spec/api/ingest.simulate.json index bf08435eb8e..bef89fed54d 100644 --- a/plugins/ingest/src/test/resources/rest-api-spec/api/ingest.simulate.json +++ b/plugins/ingest/src/test/resources/rest-api-spec/api/ingest.simulate.json @@ -13,6 +13,11 @@ } }, "params": { + "verbose": { + "type" : "boolean", + "description" : "Verbose mode. Display data output for each processor in executed pipeline", + "default" : false + } } }, "body": { diff --git a/plugins/ingest/src/test/resources/rest-api-spec/test/ingest/80_simulate.yaml b/plugins/ingest/src/test/resources/rest-api-spec/test/ingest/80_simulate.yaml index eb68a439bcb..7c8764472e0 100644 --- a/plugins/ingest/src/test/resources/rest-api-spec/test/ingest/80_simulate.yaml +++ b/plugins/ingest/src/test/resources/rest-api-spec/test/ingest/80_simulate.yaml @@ -12,11 +12,10 @@ "description": "_description", "processors": [ { - "simple" : { - "path" : "field1", - "expected_value" : "_value", - "add_field" : "field2", - "add_field_value" : "_value" + "mutate" : { + "update" : { + "field2" : "_value" + } } } ] @@ -48,4 +47,144 @@ ] } - length: { docs: 1 } + - is_false: docs.0.error + - is_true: docs.0.modified + - match: { docs.0.foo: "bar" } + - match: { docs.0.field2: "_value" } + +--- +"Test simulate with provided pipeline definition": + - do: + cluster.health: + wait_for_status: green + + - do: + ingest.simulate: + body: > + { + "pipeline": { + "description": "_description", + "processors": [ + { + "mutate" : { + "update" : { + "field2" : "_value" + } + } + } + ] + }, + "docs": [ + { + "_index": "index", + "_type": "type", + "_id": "id", + "_source": { + "foo": "bar" + } + } + ] + } + - length: { docs: 1 } + +--- +"Test simulate with verbose flag": + - do: + cluster.health: + wait_for_status: green + + - do: + ingest.simulate: + verbose: true + body: > + { + "pipeline": { + "description": "_description", + "processors": [ + { + "mutate" : { + "update" : { + "field2" : "_value" + } + } + }, + { + "mutate" : { + "update" : { + "field3" : "third_val" + } + } + } + ] + }, + "docs": [ + { + "_index": "index", + "_type": "type", + "_id": "id", + "_source": { + "foo": "bar" + } + } + ] + } + - length: { docs: 1 } + - length: { docs.0.processor_steps: 2 } + - match: { docs.0.processor_steps.0.processor_id: "processor[mutate]-0" } + - is_false: docs.0.processor_steps.0.error + - is_true: docs.0.processor_steps.0.modified + - length: { docs.0.processor_steps.0.doc: 2 } + - match: { docs.0.processor_steps.0.doc.foo: "bar" } + - match: { docs.0.processor_steps.0.doc.field2: "_value" } + - length: { docs.0.processor_steps.1.doc: 3 } + - match: { docs.0.processor_steps.1.doc.foo: "bar" } + - match: { docs.0.processor_steps.1.doc.field2: "_value" } + - match: { docs.0.processor_steps.1.doc.field3: "third_val" } + +--- +"Test simulate with exception thrown": + - do: + cluster.health: + wait_for_status: green + + - do: + catch: request + ingest.simulate: + body: > + { + "pipeline": { + "description": "_description", + "processors": [ + { + "mutate" : { + "uppercase" : ["foo"] + } + } + ] + }, + "docs": [ + { + "_index": "index", + "_type": "type", + "_id": "id", + "_source": { + "not_foo": "bar" + } + }, + { + "_index": "index", + "_type": "type", + "_id": "id2", + "_source": { + "foo": "bar" + } + } + ] + } + - length: { docs: 2 } + - is_true: docs.0.error + - match: { docs.0.failure: "java.lang.NullPointerException" } + - is_false: docs.1.error + - is_true: docs.1.modified + - match: { docs.1.doc.foo: "BAR" } From 674084973dbdb5147c2d3e448dc1faf2ef861d7e Mon Sep 17 00:00:00 2001 From: Tal Levy Date: Wed, 11 Nov 2015 21:51:45 -0800 Subject: [PATCH 05/14] moar updates --- .../java/org/elasticsearch/ingest/Data.java | 11 ++- .../plugin/ingest/IngestModule.java | 2 +- .../rest/RestSimulatePipelineAction.java | 6 +- .../simulate/ParsedSimulateRequest.java | 2 +- .../simulate/ProcessorResult.java} | 33 +++---- .../simulate/SimulateExecutionService.java | 31 +++--- .../simulate/SimulatePipelineResponse.java | 1 - .../SimulatePipelineTransportAction.java | 9 +- .../simulate/SimulatedItemResponse.java | 78 ++++++++------- .../elasticsearch/ingest/IngestClientIT.java | 2 +- .../ParsedSimulateRequestParserTests.java | 2 +- .../SimulateExecutionServiceTests.java | 26 ++--- .../rest-api-spec/api/ingest.simulate.json | 4 +- .../test/ingest/80_simulate.yaml | 94 ++++++++++++++++--- 14 files changed, 182 insertions(+), 119 deletions(-) rename plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/{ => transport}/simulate/ParsedSimulateRequest.java (98%) rename plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/{simulate/ProcessedData.java => transport/simulate/ProcessorResult.java} (81%) rename plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/{ => transport}/simulate/SimulateExecutionService.java (78%) rename plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/{ => transport}/simulate/SimulatedItemResponse.java (66%) rename plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/{ => transport}/simulate/ParsedSimulateRequestParserTests.java (98%) rename plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/{ => transport}/simulate/SimulateExecutionServiceTests.java (86%) diff --git a/plugins/ingest/src/main/java/org/elasticsearch/ingest/Data.java b/plugins/ingest/src/main/java/org/elasticsearch/ingest/Data.java index dd1bfde7923..543fd9c1c4b 100644 --- a/plugins/ingest/src/main/java/org/elasticsearch/ingest/Data.java +++ b/plugins/ingest/src/main/java/org/elasticsearch/ingest/Data.java @@ -22,7 +22,6 @@ package org.elasticsearch.ingest; import org.elasticsearch.common.Strings; import org.elasticsearch.common.xcontent.support.XContentMapValues; -import java.lang.reflect.Array; import java.util.*; /** @@ -131,6 +130,16 @@ public final class Data { return modified; } + public Map asMap() { + Map dataMap = new HashMap<>(); + dataMap.put("_index", index); + dataMap.put("_type", type); + dataMap.put("_id", id); + dataMap.put("_source", document); + + return dataMap; + } + @Override public boolean equals(Object obj) { if (obj == this) { return true; } diff --git a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/IngestModule.java b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/IngestModule.java index e9f7cdd1f88..ae685c75d5a 100644 --- a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/IngestModule.java +++ b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/IngestModule.java @@ -27,7 +27,7 @@ import org.elasticsearch.ingest.processor.geoip.GeoIpProcessor; import org.elasticsearch.ingest.processor.grok.GrokProcessor; import org.elasticsearch.ingest.processor.mutate.MutateProcessor; import org.elasticsearch.plugin.ingest.rest.IngestRestFilter; -import org.elasticsearch.plugin.ingest.simulate.SimulateExecutionService; +import org.elasticsearch.plugin.ingest.transport.simulate.SimulateExecutionService; import java.util.HashMap; import java.util.Map; diff --git a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/rest/RestSimulatePipelineAction.java b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/rest/RestSimulatePipelineAction.java index c110da28b99..ac18c2dfeef 100644 --- a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/rest/RestSimulatePipelineAction.java +++ b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/rest/RestSimulatePipelineAction.java @@ -36,10 +36,10 @@ 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.POST, "/_ingest/pipeline/_simulate/{id}", this); + controller.registerHandler(RestRequest.Method.GET, "/_ingest/pipeline/_simulate/{id}", this); controller.registerHandler(RestRequest.Method.POST, "/_ingest/pipeline/_simulate", this); - // controller.registerHandler(RestRequest.Method.GET, "/_ingest/pipeline/{id}/_simulate", this); - // controller.registerHandler(RestRequest.Method.GET, "/_ingest/pipeline/_simulate", this); + controller.registerHandler(RestRequest.Method.GET, "/_ingest/pipeline/_simulate", this); } @Override diff --git a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/simulate/ParsedSimulateRequest.java b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/ParsedSimulateRequest.java similarity index 98% rename from plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/simulate/ParsedSimulateRequest.java rename to plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/ParsedSimulateRequest.java index d9f2dc4dc0c..e39596c777e 100644 --- a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/simulate/ParsedSimulateRequest.java +++ b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/ParsedSimulateRequest.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.elasticsearch.plugin.ingest.simulate; +package org.elasticsearch.plugin.ingest.transport.simulate; import org.elasticsearch.ingest.Data; import org.elasticsearch.ingest.Pipeline; diff --git a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/simulate/ProcessedData.java b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/ProcessorResult.java similarity index 81% rename from plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/simulate/ProcessedData.java rename to plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/ProcessorResult.java index 238a1e79d40..3f2073e480b 100644 --- a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/simulate/ProcessedData.java +++ b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/ProcessorResult.java @@ -16,37 +16,38 @@ * specific language governing permissions and limitations * under the License. */ -package org.elasticsearch.plugin.ingest.simulate; +package org.elasticsearch.plugin.ingest.transport.simulate; +import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.Streamable; -import org.elasticsearch.common.xcontent.StatusToXContent; +import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentBuilderString; import org.elasticsearch.ingest.Data; -import org.elasticsearch.rest.RestStatus; import java.io.IOException; import java.util.Map; import java.util.Objects; -public class ProcessedData implements Streamable, StatusToXContent { +public class ProcessorResult implements Streamable, ToXContent { private String processorId; private Data data; private Throwable failure; - public ProcessedData() { + public ProcessorResult() { } - public ProcessedData(String processorId, Data data) { + public ProcessorResult(String processorId, Data data) { this.processorId = processorId; this.data = data; } - public ProcessedData(Throwable failure) { + public ProcessorResult(String processorId, Throwable failure) { + this.processorId = processorId; this.failure = failure; } @@ -67,7 +68,6 @@ public class ProcessedData implements Streamable, StatusToXContent { boolean isFailure = in.readBoolean(); if (isFailure) { this.failure = in.readThrowable(); - // TODO(talevy): check out mget for throwable limitations } else { this.processorId = in.readString(); String index = in.readString(); @@ -98,31 +98,22 @@ public class ProcessedData implements Streamable, StatusToXContent { builder.field(Fields.PROCESSOR_ID, processorId); builder.field(Fields.ERROR, isFailed()); if (isFailed()) { - builder.field(Fields.FAILURE, failure.toString()); + builder.field(Fields.ERROR_MESSAGE, ExceptionsHelper.detailedMessage(failure)); } else { builder.field(Fields.MODIFIED, data.isModified()); - builder.field(Fields.DOCUMENT, data.getDocument()); + builder.field(Fields.DOCUMENT, data.asMap()); } builder.endObject(); return builder; } - @Override - public RestStatus status() { - if (isFailed()) { - return RestStatus.BAD_REQUEST; - } else { - return RestStatus.OK; - } - } - @Override public boolean equals(Object obj) { if (obj == this) { return true; } if (obj == null || getClass() != obj.getClass()) { return false; } - ProcessedData other = (ProcessedData) obj; + ProcessorResult other = (ProcessorResult) obj; return Objects.equals(processorId, other.processorId) && Objects.equals(data, other.data) && Objects.equals(failure, other.failure); } @@ -135,7 +126,7 @@ public class ProcessedData implements Streamable, StatusToXContent { static final XContentBuilderString DOCUMENT = new XContentBuilderString("doc"); static final XContentBuilderString PROCESSOR_ID = new XContentBuilderString("processor_id"); static final XContentBuilderString ERROR = new XContentBuilderString("error"); - static final XContentBuilderString FAILURE = new XContentBuilderString("failure"); + static final XContentBuilderString ERROR_MESSAGE = new XContentBuilderString("error_message"); static final XContentBuilderString MODIFIED = new XContentBuilderString("modified"); } } diff --git a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/simulate/SimulateExecutionService.java b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateExecutionService.java similarity index 78% rename from plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/simulate/SimulateExecutionService.java rename to plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateExecutionService.java index d7949532c0b..119bbc91ea8 100644 --- a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/simulate/SimulateExecutionService.java +++ b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateExecutionService.java @@ -17,13 +17,12 @@ * under the License. */ -package org.elasticsearch.plugin.ingest.simulate; +package org.elasticsearch.plugin.ingest.transport.simulate; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.ingest.Data; import org.elasticsearch.ingest.Pipeline; import org.elasticsearch.ingest.processor.Processor; -import org.elasticsearch.plugin.ingest.transport.simulate.SimulatePipelineResponse; import org.elasticsearch.threadpool.ThreadPool; import java.util.ArrayList; @@ -41,14 +40,10 @@ public class SimulateExecutionService { } - SimulatedItemResponse executeItem(Pipeline pipeline, Data data, boolean verbose) { + SimulatedItemResponse executeItem(Pipeline pipeline, Data data) { try { - if (verbose) { - return executeVerboseItem(pipeline, data); - } else { - pipeline.execute(data); - return new SimulatedItemResponse(data); - } + pipeline.execute(data); + return new SimulatedItemResponse(data); } catch (Exception e) { return new SimulatedItemResponse(e); } @@ -56,24 +51,32 @@ public class SimulateExecutionService { } SimulatedItemResponse executeVerboseItem(Pipeline pipeline, Data data) { - List processedDataList = new ArrayList<>(); + List processorResultList = new ArrayList<>(); Data currentData = new Data(data); for (int i = 0; i < pipeline.getProcessors().size(); i++) { Processor processor = pipeline.getProcessors().get(i); String processorId = "processor[" + processor.getType() + "]-" + i; - processor.execute(currentData); - processedDataList.add(new ProcessedData(processorId, currentData)); + try { + processor.execute(currentData); + processorResultList.add(new ProcessorResult(processorId, currentData)); + } catch (Exception e) { + processorResultList.add(new ProcessorResult(processorId, e)); + } currentData = new Data(currentData); } - return new SimulatedItemResponse(processedDataList); + return new SimulatedItemResponse(processorResultList); } SimulatePipelineResponse execute(ParsedSimulateRequest request) { List responses = new ArrayList<>(); for (Data data : request.getDocuments()) { - responses.add(executeItem(request.getPipeline(), data, request.isVerbose())); + if (request.isVerbose()) { + responses.add(executeVerboseItem(request.getPipeline(), data)); + } else { + responses.add(executeItem(request.getPipeline(), data)); + } } return new SimulatePipelineResponse(request.getPipeline().getId(), responses); } diff --git a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineResponse.java b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineResponse.java index 6659635f7b0..04b988a3ce3 100644 --- a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineResponse.java +++ b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineResponse.java @@ -24,7 +24,6 @@ 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.plugin.ingest.simulate.SimulatedItemResponse; import org.elasticsearch.rest.RestStatus; import java.io.IOException; diff --git a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineTransportAction.java b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineTransportAction.java index f43d0ec03d0..d52a92715a3 100644 --- a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineTransportAction.java +++ b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineTransportAction.java @@ -26,10 +26,7 @@ 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.Pipeline; -import org.elasticsearch.plugin.ingest.simulate.ParsedSimulateRequest; import org.elasticsearch.plugin.ingest.PipelineStore; -import org.elasticsearch.plugin.ingest.simulate.SimulateExecutionService; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; @@ -51,16 +48,16 @@ public class SimulatePipelineTransportAction extends HandledTransportAction listener) { Map source = XContentHelper.convertToMap(request.source(), false).v2(); - ParsedSimulateRequest payload; + ParsedSimulateRequest simulateRequest; ParsedSimulateRequest.Parser parser = new ParsedSimulateRequest.Parser(); try { - payload = parser.parse(request.id(), source, request.verbose(), pipelineStore); + simulateRequest = parser.parse(request.id(), source, request.verbose(), pipelineStore); } catch (IOException e) { listener.onFailure(e); return; } - executionService.execute(payload, new SimulateExecutionService.Listener() { + executionService.execute(simulateRequest, new SimulateExecutionService.Listener() { @Override public void onResponse(SimulatePipelineResponse response) { listener.onResponse(response); diff --git a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/simulate/SimulatedItemResponse.java b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatedItemResponse.java similarity index 66% rename from plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/simulate/SimulatedItemResponse.java rename to plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatedItemResponse.java index 0fff29ed0c4..d18e16bd56b 100644 --- a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/simulate/SimulatedItemResponse.java +++ b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatedItemResponse.java @@ -16,16 +16,16 @@ * specific language governing permissions and limitations * under the License. */ -package org.elasticsearch.plugin.ingest.simulate; +package org.elasticsearch.plugin.ingest.transport.simulate; +import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.Streamable; -import org.elasticsearch.common.xcontent.StatusToXContent; +import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentBuilderString; import org.elasticsearch.ingest.Data; -import org.elasticsearch.rest.RestStatus; import java.io.IOException; import java.util.ArrayList; @@ -33,10 +33,10 @@ import java.util.List; import java.util.Map; import java.util.Objects; -public class SimulatedItemResponse implements Streamable, StatusToXContent { +public class SimulatedItemResponse implements Streamable, ToXContent { private Data data; - private List processedDataList; + private List processorResultList; private Throwable failure; public SimulatedItemResponse() { @@ -47,8 +47,8 @@ public class SimulatedItemResponse implements Streamable, StatusToXContent { this.data = data; } - public SimulatedItemResponse(List processedDataList) { - this.processedDataList = processedDataList; + public SimulatedItemResponse(List processorResultList) { + this.processorResultList = processorResultList; } public SimulatedItemResponse(Throwable failure) { @@ -56,19 +56,29 @@ public class SimulatedItemResponse implements Streamable, StatusToXContent { } public boolean isFailed() { - return this.failure != null; + if (failure != null) { + return true; + } else if (processorResultList != null) { + for (ProcessorResult result : processorResultList) { + if (result.isFailed()) { + return true; + } + } + } + + return false; } public boolean isVerbose() { - return this.processedDataList != null; + return this.processorResultList != null; } public Data getData() { return data; } - public List getProcessedDataList() { - return processedDataList; + public List getProcessorResultList() { + return processorResultList; } @Override @@ -77,14 +87,13 @@ public class SimulatedItemResponse implements Streamable, StatusToXContent { boolean isVerbose = in.readBoolean(); if (isFailed) { this.failure = in.readThrowable(); - // TODO(talevy): check out mget for throwable limitations } else if (isVerbose) { int size = in.readVInt(); - processedDataList = new ArrayList<>(); + processorResultList = new ArrayList<>(); for (int i = 0; i < size; i++) { - ProcessedData processedData = new ProcessedData(); - processedData.readFrom(in); - processedDataList.add(processedData); + ProcessorResult processorResult = new ProcessorResult(); + processorResult.readFrom(in); + processorResultList.add(processorResult); } } else { String index = in.readString(); @@ -100,11 +109,11 @@ public class SimulatedItemResponse implements Streamable, StatusToXContent { out.writeBoolean(isFailed()); out.writeBoolean(isVerbose()); - if (isFailed()) { + if (failure != null) { out.writeThrowable(failure); } else if (isVerbose()) { - out.writeVInt(processedDataList.size()); - for (ProcessedData p : processedDataList) { + out.writeVInt(processorResultList.size()); + for (ProcessorResult p : processorResultList) { p.writeTo(out); } } else { @@ -119,32 +128,22 @@ public class SimulatedItemResponse implements Streamable, StatusToXContent { public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { builder.startObject(); builder.field(Fields.ERROR, isFailed()); - builder.field(Fields.VERBOSE, isVerbose()); - if (isFailed()) { - builder.field(Fields.FAILURE, failure.toString()); + if (failure != null) { + builder.field(Fields.ERROR_MESSAGE, ExceptionsHelper.detailedMessage(failure)); } else if (isVerbose()) { - builder.startArray(Fields.PROCESSOR_STEPS); - for (ProcessedData processedData : processedDataList) { - builder.value(processedData); + builder.startArray(Fields.PROCESSOR_RESULTS); + for (ProcessorResult processorResult : processorResultList) { + builder.value(processorResult); } builder.endArray(); } else { builder.field(Fields.MODIFIED, data.isModified()); - builder.field(Fields.DOCUMENT, data.getDocument()); + builder.field(Fields.DOCUMENT, data.asMap()); } builder.endObject(); return builder; } - @Override - public RestStatus status() { - if (isFailed()) { - return RestStatus.BAD_REQUEST; - } else { - return RestStatus.OK; - } - } - @Override public boolean equals(Object obj) { if (obj == this) { return true; } @@ -152,20 +151,19 @@ public class SimulatedItemResponse implements Streamable, StatusToXContent { return false; } SimulatedItemResponse other = (SimulatedItemResponse) obj; - return Objects.equals(data, other.data) && Objects.equals(processedDataList, other.processedDataList) && Objects.equals(failure, other.failure); + return Objects.equals(data, other.data) && Objects.equals(processorResultList, other.processorResultList) && Objects.equals(failure, other.failure); } @Override public int hashCode() { - return Objects.hash(data, processedDataList, failure); + return Objects.hash(data, processorResultList, failure); } static final class Fields { static final XContentBuilderString DOCUMENT = new XContentBuilderString("doc"); static final XContentBuilderString ERROR = new XContentBuilderString("error"); - static final XContentBuilderString VERBOSE = new XContentBuilderString("verbose"); - static final XContentBuilderString FAILURE = new XContentBuilderString("failure"); + static final XContentBuilderString ERROR_MESSAGE = new XContentBuilderString("error_message"); static final XContentBuilderString MODIFIED = new XContentBuilderString("modified"); - static final XContentBuilderString PROCESSOR_STEPS = new XContentBuilderString("processor_steps"); + static final XContentBuilderString PROCESSOR_RESULTS = new XContentBuilderString("processor_results"); } } diff --git a/plugins/ingest/src/test/java/org/elasticsearch/ingest/IngestClientIT.java b/plugins/ingest/src/test/java/org/elasticsearch/ingest/IngestClientIT.java index 61fcad9a3c3..d0d11ae60a0 100644 --- a/plugins/ingest/src/test/java/org/elasticsearch/ingest/IngestClientIT.java +++ b/plugins/ingest/src/test/java/org/elasticsearch/ingest/IngestClientIT.java @@ -33,7 +33,7 @@ import org.elasticsearch.plugin.ingest.transport.put.PutPipelineRequestBuilder; import org.elasticsearch.plugin.ingest.transport.simulate.SimulatePipelineAction; import org.elasticsearch.plugin.ingest.transport.simulate.SimulatePipelineRequestBuilder; import org.elasticsearch.plugin.ingest.transport.simulate.SimulatePipelineResponse; -import org.elasticsearch.plugin.ingest.simulate.SimulatedItemResponse; +import org.elasticsearch.plugin.ingest.transport.simulate.SimulatedItemResponse; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.ESIntegTestCase; diff --git a/plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/simulate/ParsedSimulateRequestParserTests.java b/plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/transport/simulate/ParsedSimulateRequestParserTests.java similarity index 98% rename from plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/simulate/ParsedSimulateRequestParserTests.java rename to plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/transport/simulate/ParsedSimulateRequestParserTests.java index 01f3beee391..d88bf7a5467 100644 --- a/plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/simulate/ParsedSimulateRequestParserTests.java +++ b/plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/transport/simulate/ParsedSimulateRequestParserTests.java @@ -17,7 +17,7 @@ * under the License. */ -package org.elasticsearch.plugin.ingest.simulate; +package org.elasticsearch.plugin.ingest.transport.simulate; import org.elasticsearch.ingest.Data; import org.elasticsearch.ingest.Pipeline; diff --git a/plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/simulate/SimulateExecutionServiceTests.java b/plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateExecutionServiceTests.java similarity index 86% rename from plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/simulate/SimulateExecutionServiceTests.java rename to plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateExecutionServiceTests.java index f622db68d95..8ed2cc6abec 100644 --- a/plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/simulate/SimulateExecutionServiceTests.java +++ b/plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateExecutionServiceTests.java @@ -17,14 +17,13 @@ * under the License. */ -package org.elasticsearch.plugin.ingest.simulate; +package org.elasticsearch.plugin.ingest.transport.simulate; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.ingest.Data; import org.elasticsearch.ingest.Pipeline; import org.elasticsearch.ingest.processor.Processor; import org.elasticsearch.plugin.ingest.PipelineStore; -import org.elasticsearch.plugin.ingest.transport.simulate.SimulatePipelineResponse; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.threadpool.ThreadPool; import org.junit.After; @@ -67,23 +66,26 @@ public class SimulateExecutionServiceTests extends ESTestCase { public void testExecuteVerboseItem() throws Exception { SimulatedItemResponse expectedItemResponse = new SimulatedItemResponse( - Arrays.asList(new ProcessedData("processor[mock]-0", data), new ProcessedData("processor[mock]-1", data))); + Arrays.asList(new ProcessorResult("processor[mock]-0", data), new ProcessorResult("processor[mock]-1", data))); SimulatedItemResponse actualItemResponse = executionService.executeVerboseItem(pipeline, data); verify(processor, times(2)).execute(data); assertThat(actualItemResponse, equalTo(expectedItemResponse)); } - public void testExecuteItem_verboseSuccessful() throws Exception { - SimulatedItemResponse expectedItemResponse = new SimulatedItemResponse( - Arrays.asList(new ProcessedData("processor[mock]-0", data), new ProcessedData("processor[mock]-1", data))); - SimulatedItemResponse actualItemResponse = executionService.executeItem(pipeline, data, true); + public void testExecuteItem() throws Exception { + SimulatedItemResponse expectedItemResponse = new SimulatedItemResponse(data); + SimulatedItemResponse actualItemResponse = executionService.executeItem(pipeline, data); verify(processor, times(2)).execute(data); assertThat(actualItemResponse, equalTo(expectedItemResponse)); } - public void testExecuteItem_Simple() throws Exception { - SimulatedItemResponse expectedItemResponse = new SimulatedItemResponse(data); - SimulatedItemResponse actualItemResponse = executionService.executeItem(pipeline, data, false); + public void testExecuteVerboseItem_Failure() throws Exception { + Exception e = new RuntimeException("processor failed"); + SimulatedItemResponse expectedItemResponse = new SimulatedItemResponse( + Arrays.asList(new ProcessorResult("processor[mock]-0", e), new ProcessorResult("processor[mock]-1", data)) + ); + doThrow(e).doNothing().when(processor).execute(data); + SimulatedItemResponse actualItemResponse = executionService.executeVerboseItem(pipeline, data); verify(processor, times(2)).execute(data); assertThat(actualItemResponse, equalTo(expectedItemResponse)); } @@ -92,7 +94,7 @@ public class SimulateExecutionServiceTests extends ESTestCase { Exception e = new RuntimeException("processor failed"); SimulatedItemResponse expectedItemResponse = new SimulatedItemResponse(e); doThrow(e).when(processor).execute(data); - SimulatedItemResponse actualItemResponse = executionService.executeItem(pipeline, data, false); + SimulatedItemResponse actualItemResponse = executionService.executeItem(pipeline, data); verify(processor, times(1)).execute(data); assertThat(actualItemResponse, equalTo(expectedItemResponse)); } @@ -116,7 +118,7 @@ public class SimulateExecutionServiceTests extends ESTestCase { SimulateExecutionService.Listener listener = mock(SimulateExecutionService.Listener.class); ParsedSimulateRequest request = new ParsedSimulateRequest(pipeline, Collections.singletonList(data), true); SimulatedItemResponse itemResponse = new SimulatedItemResponse( - Arrays.asList(new ProcessedData("processor[mock]-0", data), new ProcessedData("processor[mock]-1", data))); + Arrays.asList(new ProcessorResult("processor[mock]-0", data), new ProcessorResult("processor[mock]-1", data))); executionService.execute(request, listener); SimulatePipelineResponse response = new SimulatePipelineResponse("_id", Collections.singletonList(itemResponse)); assertBusy(new Runnable() { diff --git a/plugins/ingest/src/test/resources/rest-api-spec/api/ingest.simulate.json b/plugins/ingest/src/test/resources/rest-api-spec/api/ingest.simulate.json index bef89fed54d..8431b245ef8 100644 --- a/plugins/ingest/src/test/resources/rest-api-spec/api/ingest.simulate.json +++ b/plugins/ingest/src/test/resources/rest-api-spec/api/ingest.simulate.json @@ -1,10 +1,10 @@ { "ingest.simulate": { "documentation": "https://www.elastic.co/guide/en/elasticsearch/plugins/master/ingest.html", - "methods": [ "POST" ], + "methods": [ "GET", "POST" ], "url": { "path": "/_ingest/pipeline/_simulate", - "paths": [ "/_ingest/pipeline/_simulate", "/_ingest/pipeline/{id}/_simulate" ], + "paths": [ "/_ingest/pipeline/_simulate", "/_ingest/pipeline/_simulate/{id}" ], "parts": { "id": { "type" : "string", diff --git a/plugins/ingest/src/test/resources/rest-api-spec/test/ingest/80_simulate.yaml b/plugins/ingest/src/test/resources/rest-api-spec/test/ingest/80_simulate.yaml index 7c8764472e0..7596469b7f2 100644 --- a/plugins/ingest/src/test/resources/rest-api-spec/test/ingest/80_simulate.yaml +++ b/plugins/ingest/src/test/resources/rest-api-spec/test/ingest/80_simulate.yaml @@ -49,8 +49,8 @@ - length: { docs: 1 } - is_false: docs.0.error - is_true: docs.0.modified - - match: { docs.0.foo: "bar" } - - match: { docs.0.field2: "_value" } + - match: { docs.0.doc._source.foo: "bar" } + - match: { docs.0.doc._source.field2: "_value" } --- @@ -130,17 +130,17 @@ ] } - length: { docs: 1 } - - length: { docs.0.processor_steps: 2 } - - match: { docs.0.processor_steps.0.processor_id: "processor[mutate]-0" } - - is_false: docs.0.processor_steps.0.error - - is_true: docs.0.processor_steps.0.modified - - length: { docs.0.processor_steps.0.doc: 2 } - - match: { docs.0.processor_steps.0.doc.foo: "bar" } - - match: { docs.0.processor_steps.0.doc.field2: "_value" } - - length: { docs.0.processor_steps.1.doc: 3 } - - match: { docs.0.processor_steps.1.doc.foo: "bar" } - - match: { docs.0.processor_steps.1.doc.field2: "_value" } - - match: { docs.0.processor_steps.1.doc.field3: "third_val" } + - length: { docs.0.processor_results: 2 } + - match: { docs.0.processor_results.0.processor_id: "processor[mutate]-0" } + - is_false: docs.0.processor_results.0.error + - is_true: docs.0.processor_results.0.modified + - length: { docs.0.processor_results.0.doc._source: 2 } + - match: { docs.0.processor_results.0.doc._source.foo: "bar" } + - match: { docs.0.processor_results.0.doc._source.field2: "_value" } + - length: { docs.0.processor_results.1.doc._source: 3 } + - match: { docs.0.processor_results.1.doc._source.foo: "bar" } + - match: { docs.0.processor_results.1.doc._source.field2: "_value" } + - match: { docs.0.processor_results.1.doc._source..field3: "third_val" } --- "Test simulate with exception thrown": @@ -184,7 +184,71 @@ } - length: { docs: 2 } - is_true: docs.0.error - - match: { docs.0.failure: "java.lang.NullPointerException" } + - match: { docs.0.error_message: "NullPointerException[null]" } - is_false: docs.1.error - is_true: docs.1.modified - - match: { docs.1.doc.foo: "BAR" } + - match: { docs.1.doc._source.foo: "BAR" } + +--- +"Test verbose simulate with exception thrown": + - do: + cluster.health: + wait_for_status: green + + - do: + catch: request + ingest.simulate: + verbose: true + body: > + { + "pipeline": { + "description": "_description", + "processors": [ + { + "mutate" : { + "convert" : { + "foo": "integer" + } + } + }, + { + "mutate" : { + "uppercase" : ["bar"] + } + } + ] + }, + "docs": [ + { + "_index": "index", + "_type": "type", + "_id": "id", + "_source": { + "foo": "bar", + "bar": "hello" + } + }, + { + "_index": "index", + "_type": "type", + "_id": "id2", + "_source": { + "foo": "5", + "bar": "hello" + } + } + ] + } + - length: { docs: 2 } + - is_true: docs.0.error + - is_false: docs.1.error + - length: { docs.0.processor_results: 2 } + - is_false: docs.1.processor_results.0.error + - match: { docs.0.processor_results.0.error_message: "NumberFormatException[For input string: \"bar\"]" } + - is_false: docs.1.processor_results.1.error + - match: { docs.0.processor_results.1.doc._source.foo: "bar" } + - match: { docs.1.processor_results.1.doc._source.bar: "HELLO" } + - match: { docs.1.processor_results.0.doc._source.foo: 5 } + - match: { docs.1.processor_results.0.doc._source.bar: "hello" } + - match: { docs.1.processor_results.1.doc._source.foo: 5 } + - match: { docs.1.processor_results.1.doc._source.bar: "HELLO" } From 75371b23813608399fd295357a049f5628a43d41 Mon Sep 17 00:00:00 2001 From: javanna Date: Thu, 12 Nov 2015 15:45:17 +0100 Subject: [PATCH 06/14] restore initial simulate endpoint url, adapt get pipeline param name --- .../plugin/ingest/rest/RestGetPipelineAction.java | 7 ++----- .../plugin/ingest/rest/RestSimulatePipelineAction.java | 4 ++-- .../test/resources/rest-api-spec/api/ingest.simulate.json | 2 +- 3 files changed, 5 insertions(+), 8 deletions(-) diff --git a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/rest/RestGetPipelineAction.java b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/rest/RestGetPipelineAction.java index a58366eedb0..6d444739900 100644 --- a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/rest/RestGetPipelineAction.java +++ b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/rest/RestGetPipelineAction.java @@ -25,27 +25,24 @@ import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.plugin.ingest.transport.get.GetPipelineAction; import org.elasticsearch.plugin.ingest.transport.get.GetPipelineRequest; -import org.elasticsearch.plugin.ingest.transport.put.PutPipelineAction; -import org.elasticsearch.plugin.ingest.transport.put.PutPipelineRequest; 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; -import org.elasticsearch.rest.action.support.RestToXContentListener; 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/{ids}", this); + 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(); - request.ids(Strings.splitStringByCommaToArray(restRequest.param("ids"))); + request.ids(Strings.splitStringByCommaToArray(restRequest.param("id"))); client.execute(GetPipelineAction.INSTANCE, request, new RestStatusToXContentListener<>(channel)); } } diff --git a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/rest/RestSimulatePipelineAction.java b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/rest/RestSimulatePipelineAction.java index ac18c2dfeef..ed543b7a1eb 100644 --- a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/rest/RestSimulatePipelineAction.java +++ b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/rest/RestSimulatePipelineAction.java @@ -36,8 +36,8 @@ 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/_simulate/{id}", this); - controller.registerHandler(RestRequest.Method.GET, "/_ingest/pipeline/_simulate/{id}", this); + 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); } diff --git a/plugins/ingest/src/test/resources/rest-api-spec/api/ingest.simulate.json b/plugins/ingest/src/test/resources/rest-api-spec/api/ingest.simulate.json index 8431b245ef8..a4904cef80a 100644 --- a/plugins/ingest/src/test/resources/rest-api-spec/api/ingest.simulate.json +++ b/plugins/ingest/src/test/resources/rest-api-spec/api/ingest.simulate.json @@ -4,7 +4,7 @@ "methods": [ "GET", "POST" ], "url": { "path": "/_ingest/pipeline/_simulate", - "paths": [ "/_ingest/pipeline/_simulate", "/_ingest/pipeline/_simulate/{id}" ], + "paths": [ "/_ingest/pipeline/_simulate", "/_ingest/pipeline/{id}/_simulate/" ], "parts": { "id": { "type" : "string", From 979fa816180a91291bc447c61d57a7d55dc7ae32 Mon Sep 17 00:00:00 2001 From: javanna Date: Thu, 12 Nov 2015 15:45:35 +0100 Subject: [PATCH 07/14] make description optional as part of a Pipeline --- .../ingest/src/main/java/org/elasticsearch/ingest/Pipeline.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/plugins/ingest/src/main/java/org/elasticsearch/ingest/Pipeline.java b/plugins/ingest/src/main/java/org/elasticsearch/ingest/Pipeline.java index 7d7959c2921..b98a469d3c3 100644 --- a/plugins/ingest/src/main/java/org/elasticsearch/ingest/Pipeline.java +++ b/plugins/ingest/src/main/java/org/elasticsearch/ingest/Pipeline.java @@ -89,7 +89,7 @@ public final class Pipeline { public final static class Factory { public Pipeline create(String id, Map config, Map processorRegistry) throws IOException { - String description = ConfigurationUtils.readStringProperty(config, "description"); + String description = ConfigurationUtils.readOptionalStringProperty(config, "description"); List processors = new ArrayList<>(); @SuppressWarnings("unchecked") List>> processorConfigs = (List>>) config.get("processors"); From c4951ef74fa29c8bd47ddb6c076afeac9d974d85 Mon Sep 17 00:00:00 2001 From: javanna Date: Thu, 12 Nov 2015 17:40:14 +0100 Subject: [PATCH 08/14] update get pipeline param names to id for consistency --- .../resources/rest-api-spec/api/ingest.get_pipeline.json | 6 +++--- .../test/resources/rest-api-spec/test/ingest/20_crud.yaml | 4 ++-- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/plugins/ingest/src/test/resources/rest-api-spec/api/ingest.get_pipeline.json b/plugins/ingest/src/test/resources/rest-api-spec/api/ingest.get_pipeline.json index 246c6535e92..71772a28a76 100644 --- a/plugins/ingest/src/test/resources/rest-api-spec/api/ingest.get_pipeline.json +++ b/plugins/ingest/src/test/resources/rest-api-spec/api/ingest.get_pipeline.json @@ -3,10 +3,10 @@ "documentation": "https://www.elastic.co/guide/en/elasticsearch/plugins/master/ingest.html", "methods": [ "GET" ], "url": { - "path": "/_ingest/pipeline/{ids}", - "paths": [ "/_ingest/pipeline/{ids}" ], + "path": "/_ingest/pipeline/{id}", + "paths": [ "/_ingest/pipeline/{id}" ], "parts": { - "ids": { + "id": { "type" : "string", "description" : "Comma separated list of pipeline ids. Wildcards supported", "required" : true diff --git a/plugins/ingest/src/test/resources/rest-api-spec/test/ingest/20_crud.yaml b/plugins/ingest/src/test/resources/rest-api-spec/test/ingest/20_crud.yaml index 82177c615ab..daf13a34c1b 100644 --- a/plugins/ingest/src/test/resources/rest-api-spec/test/ingest/20_crud.yaml +++ b/plugins/ingest/src/test/resources/rest-api-spec/test/ingest/20_crud.yaml @@ -32,7 +32,7 @@ - do: ingest.get_pipeline: - ids: "my_pipeline" + id: "my_pipeline" - match: { my_pipeline._source.description: "_description" } - match: { my_pipeline._version: 1 } @@ -53,7 +53,7 @@ - do: catch: missing ingest.get_pipeline: - ids: "my_pipeline" + id: "my_pipeline" --- "Test invalid config": From af1de8e1cc9fc439722d76e3bcfece0305164d33 Mon Sep 17 00:00:00 2001 From: Tal Levy Date: Thu, 12 Nov 2015 11:37:17 -0800 Subject: [PATCH 09/14] updated with cosmetic changes --- .../ingest/processor/ConfigurationUtils.java | 20 ++++ .../rest/RestSimulatePipelineAction.java | 10 +- .../ingest/transport/TransportData.java | 99 +++++++++++++++++++ .../simulate/ParsedSimulateRequest.java | 54 +++++----- ...ponse.java => SimulateDocumentResult.java} | 65 +++++------- .../simulate/SimulateExecutionService.java | 48 ++++----- .../simulate/SimulatePipelineRequest.java | 12 +-- .../SimulatePipelineRequestBuilder.java | 6 +- .../simulate/SimulatePipelineResponse.java | 42 ++++---- .../SimulatePipelineTransportAction.java | 15 ++- ...sult.java => SimulateProcessorResult.java} | 47 ++++----- .../org/elasticsearch/ingest/DataTests.java | 26 +++++ .../elasticsearch/ingest/IngestClientIT.java | 6 +- .../ingest/transport/TransportDataTests.java | 43 ++++++++ .../ParsedSimulateRequestParserTests.java | 22 ++--- .../SimulateExecutionServiceTests.java | 47 ++++----- .../test/ingest/80_simulate.yaml | 23 ++--- 17 files changed, 366 insertions(+), 219 deletions(-) create mode 100644 plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/TransportData.java rename plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/{SimulatedItemResponse.java => SimulateDocumentResult.java} (63%) rename plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/{ProcessorResult.java => SimulateProcessorResult.java} (66%) create mode 100644 plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/transport/TransportDataTests.java diff --git a/plugins/ingest/src/main/java/org/elasticsearch/ingest/processor/ConfigurationUtils.java b/plugins/ingest/src/main/java/org/elasticsearch/ingest/processor/ConfigurationUtils.java index 3a8dbbf448d..49fd90e7afe 100644 --- a/plugins/ingest/src/main/java/org/elasticsearch/ingest/processor/ConfigurationUtils.java +++ b/plugins/ingest/src/main/java/org/elasticsearch/ingest/processor/ConfigurationUtils.java @@ -111,6 +111,20 @@ public final class ConfigurationUtils { } } + /** + * 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 Map readMap(Map 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. @@ -122,6 +136,11 @@ public final class ConfigurationUtils { if (value == null) { return null; } + + return readMap(propertyName, value); + } + + private static Map readMap(String propertyName, Object value) { if (value instanceof Map) { @SuppressWarnings("unchecked") Map map = (Map) value; @@ -130,4 +149,5 @@ public final class ConfigurationUtils { throw new IllegalArgumentException("property [" + propertyName + "] isn't a map, but of type [" + value.getClass().getName() + "]"); } } + } diff --git a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/rest/RestSimulatePipelineAction.java b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/rest/RestSimulatePipelineAction.java index ed543b7a1eb..0b86e35b522 100644 --- a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/rest/RestSimulatePipelineAction.java +++ b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/rest/RestSimulatePipelineAction.java @@ -29,7 +29,7 @@ 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.RestStatusToXContentListener; +import org.elasticsearch.rest.action.support.RestToXContentListener; public class RestSimulatePipelineAction extends BaseRestHandler { @@ -45,13 +45,13 @@ public class RestSimulatePipelineAction extends BaseRestHandler { @Override protected void handleRequest(RestRequest restRequest, RestChannel channel, Client client) throws Exception { SimulatePipelineRequest request = new SimulatePipelineRequest(); - request.id(restRequest.param("id")); - request.verbose(restRequest.paramAsBoolean("verbose", false)); + request.setId(restRequest.param("id")); + request.setVerbose(restRequest.paramAsBoolean("verbose", false)); if (RestActions.hasBodyContent(restRequest)) { - request.source(RestActions.getRestContent(restRequest)); + request.setSource(RestActions.getRestContent(restRequest)); } - client.execute(SimulatePipelineAction.INSTANCE, request, new RestStatusToXContentListener<>(channel)); + client.execute(SimulatePipelineAction.INSTANCE, request, new RestToXContentListener<>(channel)); } } diff --git a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/TransportData.java b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/TransportData.java new file mode 100644 index 00000000000..b9fa46fe939 --- /dev/null +++ b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/TransportData.java @@ -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.plugin.ingest.transport; + +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.io.stream.Streamable; +import org.elasticsearch.common.xcontent.ToXContent; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentBuilderString; +import org.elasticsearch.ingest.Data; + +import java.io.IOException; +import java.util.Map; +import java.util.Objects; + +public class TransportData implements Streamable, ToXContent { + private Data data; + + public TransportData() { + + } + + public TransportData(Data data) { + this.data = data; + } + + public Data get() { + return data; + } + + @Override + public void readFrom(StreamInput in) throws IOException { + String index = in.readString(); + String type = in.readString(); + String id = in.readString(); + Map doc = in.readMap(); + this.data = new Data(index, type, id, doc); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeString(data.getIndex()); + out.writeString(data.getType()); + out.writeString(data.getId()); + out.writeMap(data.getDocument()); + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject(Fields.DOCUMENT); + builder.field(Fields.MODIFIED, data.isModified()); + builder.field(Fields.INDEX, data.getIndex()); + builder.field(Fields.TYPE, data.getType()); + builder.field(Fields.ID, data.getId()); + builder.field(Fields.SOURCE, data.getDocument()); + builder.endObject(); + return builder; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + TransportData that = (TransportData) o; + return Objects.equals(data, that.data); + } + + @Override + public int hashCode() { + return Objects.hash(data); + } + + static final class Fields { + static final XContentBuilderString DOCUMENT = new XContentBuilderString("doc"); + static final XContentBuilderString MODIFIED = new XContentBuilderString("modified"); + static final XContentBuilderString INDEX = new XContentBuilderString("_index"); + static final XContentBuilderString TYPE = new XContentBuilderString("_type"); + static final XContentBuilderString ID = new XContentBuilderString("_id"); + static final XContentBuilderString SOURCE = new XContentBuilderString("_source"); + } +} diff --git a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/ParsedSimulateRequest.java b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/ParsedSimulateRequest.java index e39596c777e..7d02f686469 100644 --- a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/ParsedSimulateRequest.java +++ b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/ParsedSimulateRequest.java @@ -68,34 +68,44 @@ public class ParsedSimulateRequest { private static final Pipeline.Factory PIPELINE_FACTORY = new Pipeline.Factory(); public static final String SIMULATED_PIPELINE_ID = "_simulate_pipeline"; - public ParsedSimulateRequest parse(String pipelineId, Map config, boolean verbose, PipelineStore pipelineStore) throws IOException { - Pipeline pipeline; - // if pipeline `id` passed to request, fetch pipeline from store. - if (pipelineId != null) { - pipeline = pipelineStore.get(pipelineId); - } else { - Map pipelineConfig = ConfigurationUtils.readOptionalMap(config, "pipeline"); - pipeline = PIPELINE_FACTORY.create(SIMULATED_PIPELINE_ID, pipelineConfig, pipelineStore.getProcessorFactoryRegistry()); - } - - List> docs = ConfigurationUtils.readList(config, "docs"); - + private List parseDocs(Map config) { + List> docs = ConfigurationUtils.readList(config, Fields.DOCS); List dataList = new ArrayList<>(); - - for (int i = 0; i < docs.size(); i++) { - Map dataMap = docs.get(i); - Map document = ConfigurationUtils.readOptionalMap(dataMap, "_source"); - if (document == null) { - document = Collections.emptyMap(); - } - Data data = new Data(ConfigurationUtils.readOptionalStringProperty(dataMap, "_index"), - ConfigurationUtils.readOptionalStringProperty(dataMap, "_type"), - ConfigurationUtils.readOptionalStringProperty(dataMap, "_id"), + for (Map dataMap : docs) { + Map document = ConfigurationUtils.readMap(dataMap, Fields.SOURCE); + Data data = new Data(ConfigurationUtils.readStringProperty(dataMap, Fields.INDEX), + ConfigurationUtils.readStringProperty(dataMap, Fields.TYPE), + ConfigurationUtils.readStringProperty(dataMap, Fields.ID), document); dataList.add(data); } + return dataList; + } + public ParsedSimulateRequest parseWithPipelineId(String pipelineId, Map config, boolean verbose, PipelineStore pipelineStore) { + if (pipelineId == null) { + throw new IllegalArgumentException("param [pipeline] is null"); + } + Pipeline pipeline = pipelineStore.get(pipelineId); + List dataList = parseDocs(config); return new ParsedSimulateRequest(pipeline, dataList, verbose); + + } + + public ParsedSimulateRequest parse(Map config, boolean verbose, PipelineStore pipelineStore) throws IOException { + Map pipelineConfig = ConfigurationUtils.readMap(config, Fields.PIPELINE); + Pipeline pipeline = PIPELINE_FACTORY.create(SIMULATED_PIPELINE_ID, pipelineConfig, pipelineStore.getProcessorFactoryRegistry()); + List dataList = parseDocs(config); + return new ParsedSimulateRequest(pipeline, dataList, verbose); + } + + static final class Fields { + static final String PIPELINE = "pipeline"; + static final String DOCS = "docs"; + static final String SOURCE = "_source"; + static final String INDEX = "_index"; + static final String TYPE = "_type"; + static final String ID = "_id"; } } } diff --git a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatedItemResponse.java b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateDocumentResult.java similarity index 63% rename from plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatedItemResponse.java rename to plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateDocumentResult.java index d18e16bd56b..33ef7745404 100644 --- a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatedItemResponse.java +++ b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateDocumentResult.java @@ -18,7 +18,7 @@ */ package org.elasticsearch.plugin.ingest.transport.simulate; -import org.elasticsearch.ExceptionsHelper; +import org.elasticsearch.ElasticsearchException; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.Streamable; @@ -26,46 +26,39 @@ import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentBuilderString; import org.elasticsearch.ingest.Data; +import org.elasticsearch.plugin.ingest.transport.TransportData; import java.io.IOException; import java.util.ArrayList; import java.util.List; -import java.util.Map; import java.util.Objects; -public class SimulatedItemResponse implements Streamable, ToXContent { +public class SimulateDocumentResult implements Streamable, ToXContent { - private Data data; - private List processorResultList; + private TransportData data; + private List processorResultList; private Throwable failure; - public SimulatedItemResponse() { + public SimulateDocumentResult() { } - public SimulatedItemResponse(Data data) { - this.data = data; + public SimulateDocumentResult(Data data) { + this.data = new TransportData(data); } - public SimulatedItemResponse(List processorResultList) { + public SimulateDocumentResult(List processorResultList) { this.processorResultList = processorResultList; } - public SimulatedItemResponse(Throwable failure) { + public SimulateDocumentResult(Throwable failure) { this.failure = failure; } public boolean isFailed() { if (failure != null) { return true; - } else if (processorResultList != null) { - for (ProcessorResult result : processorResultList) { - if (result.isFailed()) { - return true; - } - } } - return false; } @@ -74,10 +67,10 @@ public class SimulatedItemResponse implements Streamable, ToXContent { } public Data getData() { - return data; + return data.get(); } - public List getProcessorResultList() { + public List getProcessorResultList() { return processorResultList; } @@ -91,16 +84,13 @@ public class SimulatedItemResponse implements Streamable, ToXContent { int size = in.readVInt(); processorResultList = new ArrayList<>(); for (int i = 0; i < size; i++) { - ProcessorResult processorResult = new ProcessorResult(); + SimulateProcessorResult processorResult = new SimulateProcessorResult(); processorResult.readFrom(in); processorResultList.add(processorResult); } } else { - String index = in.readString(); - String type = in.readString(); - String id = in.readString(); - Map doc = in.readMap(); - this.data = new Data(index, type, id, doc); + this.data = new TransportData(); + this.data.readFrom(in); } } @@ -113,32 +103,27 @@ public class SimulatedItemResponse implements Streamable, ToXContent { out.writeThrowable(failure); } else if (isVerbose()) { out.writeVInt(processorResultList.size()); - for (ProcessorResult p : processorResultList) { + for (SimulateProcessorResult p : processorResultList) { p.writeTo(out); } } else { - out.writeString(data.getIndex()); - out.writeString(data.getType()); - out.writeString(data.getId()); - out.writeMap(data.getDocument()); + data.writeTo(out); } } @Override public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { builder.startObject(); - builder.field(Fields.ERROR, isFailed()); - if (failure != null) { - builder.field(Fields.ERROR_MESSAGE, ExceptionsHelper.detailedMessage(failure)); + if (isFailed()) { + ElasticsearchException.renderThrowable(builder, params, failure); } else if (isVerbose()) { builder.startArray(Fields.PROCESSOR_RESULTS); - for (ProcessorResult processorResult : processorResultList) { - builder.value(processorResult); + for (SimulateProcessorResult processorResult : processorResultList) { + processorResult.toXContent(builder, params); } builder.endArray(); } else { - builder.field(Fields.MODIFIED, data.isModified()); - builder.field(Fields.DOCUMENT, data.asMap()); + data.toXContent(builder, params); } builder.endObject(); return builder; @@ -150,7 +135,7 @@ public class SimulatedItemResponse implements Streamable, ToXContent { if (obj == null || getClass() != obj.getClass()) { return false; } - SimulatedItemResponse other = (SimulatedItemResponse) obj; + SimulateDocumentResult other = (SimulateDocumentResult) obj; return Objects.equals(data, other.data) && Objects.equals(processorResultList, other.processorResultList) && Objects.equals(failure, other.failure); } @@ -160,10 +145,6 @@ public class SimulatedItemResponse implements Streamable, ToXContent { } static final class Fields { - static final XContentBuilderString DOCUMENT = new XContentBuilderString("doc"); - static final XContentBuilderString ERROR = new XContentBuilderString("error"); - static final XContentBuilderString ERROR_MESSAGE = new XContentBuilderString("error_message"); - static final XContentBuilderString MODIFIED = new XContentBuilderString("modified"); static final XContentBuilderString PROCESSOR_RESULTS = new XContentBuilderString("processor_results"); } } diff --git a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateExecutionService.java b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateExecutionService.java index 119bbc91ea8..a4518273467 100644 --- a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateExecutionService.java +++ b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateExecutionService.java @@ -19,6 +19,7 @@ package org.elasticsearch.plugin.ingest.transport.simulate; +import org.elasticsearch.action.ActionListener; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.ingest.Data; import org.elasticsearch.ingest.Pipeline; @@ -30,7 +31,7 @@ import java.util.List; public class SimulateExecutionService { - static final String THREAD_POOL_NAME = ThreadPool.Names.MANAGEMENT; + private static final String THREAD_POOL_NAME = ThreadPool.Names.MANAGEMENT; private final ThreadPool threadPool; @@ -40,18 +41,18 @@ public class SimulateExecutionService { } - SimulatedItemResponse executeItem(Pipeline pipeline, Data data) { + SimulateDocumentResult executeItem(Pipeline pipeline, Data data) { try { pipeline.execute(data); - return new SimulatedItemResponse(data); + return new SimulateDocumentResult(data); } catch (Exception e) { - return new SimulatedItemResponse(e); + return new SimulateDocumentResult(e); } } - SimulatedItemResponse executeVerboseItem(Pipeline pipeline, Data data) { - List processorResultList = new ArrayList<>(); + SimulateDocumentResult executeVerboseItem(Pipeline pipeline, Data data) { + List processorResultList = new ArrayList<>(); Data currentData = new Data(data); for (int i = 0; i < pipeline.getProcessors().size(); i++) { Processor processor = pipeline.getProcessors().get(i); @@ -59,39 +60,30 @@ public class SimulateExecutionService { try { processor.execute(currentData); - processorResultList.add(new ProcessorResult(processorId, currentData)); + processorResultList.add(new SimulateProcessorResult(processorId, currentData)); } catch (Exception e) { - processorResultList.add(new ProcessorResult(processorId, e)); + processorResultList.add(new SimulateProcessorResult(processorId, e)); } currentData = new Data(currentData); } - return new SimulatedItemResponse(processorResultList); + return new SimulateDocumentResult(processorResultList); } - SimulatePipelineResponse execute(ParsedSimulateRequest request) { - List responses = new ArrayList<>(); - for (Data data : request.getDocuments()) { - if (request.isVerbose()) { - responses.add(executeVerboseItem(request.getPipeline(), data)); - } else { - responses.add(executeItem(request.getPipeline(), data)); - } - } - return new SimulatePipelineResponse(request.getPipeline().getId(), responses); - } - - public void execute(ParsedSimulateRequest request, Listener listener) { + public void execute(ParsedSimulateRequest request, ActionListener listener) { threadPool.executor(THREAD_POOL_NAME).execute(new Runnable() { @Override public void run() { - SimulatePipelineResponse response = execute(request); - listener.onResponse(response); + List responses = new ArrayList<>(); + for (Data data : request.getDocuments()) { + if (request.isVerbose()) { + responses.add(executeVerboseItem(request.getPipeline(), data)); + } else { + responses.add(executeItem(request.getPipeline(), data)); + } + } + listener.onResponse(new SimulatePipelineResponse(request.getPipeline().getId(), responses)); } }); } - - public interface Listener { - void onResponse(SimulatePipelineResponse response); - } } diff --git a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineRequest.java b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineRequest.java index 8c5a5f1ef71..4aec025b12d 100644 --- a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineRequest.java +++ b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineRequest.java @@ -44,27 +44,27 @@ public class SimulatePipelineRequest extends ActionRequest { return validationException; } - public String id() { + public String getId() { return id; } - public void id(String id) { + public void setId(String id) { this.id = id; } - public boolean verbose() { + public boolean isVerbose() { return verbose; } - public void verbose(boolean verbose) { + public void setVerbose(boolean verbose) { this.verbose = verbose; } - public BytesReference source() { + public BytesReference getSource() { return source; } - public void source(BytesReference source) { + public void setSource(BytesReference source) { this.source = source; } diff --git a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineRequestBuilder.java b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineRequestBuilder.java index 7b140b345f4..07998291922 100644 --- a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineRequestBuilder.java +++ b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineRequestBuilder.java @@ -30,17 +30,17 @@ public class SimulatePipelineRequestBuilder extends ActionRequestBuilder responses; + private List responses; public SimulatePipelineResponse() { } - public SimulatePipelineResponse(String pipelineId, List responses) { + public SimulatePipelineResponse(String pipelineId, List responses) { this.pipelineId = pipelineId; this.responses = Collections.unmodifiableList(responses); } - public String pipelineId() { + public String getPipelineId() { return pipelineId; } - public void pipelineId(String pipelineId) { + public void setPipelineId(String pipelineId) { this.pipelineId = pipelineId; } - public List responses() { + public List getResponses() { return responses; } - public void responses(List responses) { + public void setResponses(List responses) { this.responses = responses; } @@ -67,7 +67,7 @@ public class SimulatePipelineResponse extends ActionResponse implements StatusTo super.writeTo(out); out.writeString(pipelineId); out.writeVInt(responses.size()); - for (SimulatedItemResponse response : responses) { + for (SimulateDocumentResult response : responses) { response.writeTo(out); } } @@ -79,7 +79,7 @@ public class SimulatePipelineResponse extends ActionResponse implements StatusTo int responsesLength = in.readVInt(); responses = new ArrayList<>(); for (int i = 0; i < responsesLength; i++) { - SimulatedItemResponse response = new SimulatedItemResponse(); + SimulateDocumentResult response = new SimulateDocumentResult(); response.readFrom(in); responses.add(response); } @@ -88,25 +88,15 @@ public class SimulatePipelineResponse extends ActionResponse implements StatusTo @Override public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { - builder.startArray("docs"); - for (SimulatedItemResponse response : responses) { - builder.value(response); + builder.startArray(Fields.DOCUMENTS); + for (SimulateDocumentResult response : responses) { + response.toXContent(builder, params); } builder.endArray(); return builder; } - @Override - public RestStatus status() { - for (SimulatedItemResponse response : responses) { - if (response.isFailed()) { - return RestStatus.BAD_REQUEST; - } - } - return RestStatus.OK; - } - @Override public boolean equals(Object o) { if (this == o) return true; @@ -120,4 +110,8 @@ public class SimulatePipelineResponse extends ActionResponse implements StatusTo public int hashCode() { return Objects.hash(pipelineId, responses); } + + static final class Fields { + static final XContentBuilderString DOCUMENTS = new XContentBuilderString("docs"); + } } diff --git a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineTransportAction.java b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineTransportAction.java index d52a92715a3..305c4122203 100644 --- a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineTransportAction.java +++ b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineTransportAction.java @@ -46,22 +46,21 @@ public class SimulatePipelineTransportAction extends HandledTransportAction listener) { - Map source = XContentHelper.convertToMap(request.source(), false).v2(); + Map source = XContentHelper.convertToMap(request.getSource(), false).v2(); ParsedSimulateRequest simulateRequest; ParsedSimulateRequest.Parser parser = new ParsedSimulateRequest.Parser(); try { - simulateRequest = parser.parse(request.id(), source, request.verbose(), pipelineStore); + if (request.getId() != null) { + simulateRequest = parser.parseWithPipelineId(request.getId(), source, request.isVerbose(), pipelineStore); + } else { + simulateRequest = parser.parse(source, request.isVerbose(), pipelineStore); + } } catch (IOException e) { listener.onFailure(e); return; } - executionService.execute(simulateRequest, new SimulateExecutionService.Listener() { - @Override - public void onResponse(SimulatePipelineResponse response) { - listener.onResponse(response); - } - }); + executionService.execute(simulateRequest, listener); } } diff --git a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/ProcessorResult.java b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateProcessorResult.java similarity index 66% rename from plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/ProcessorResult.java rename to plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateProcessorResult.java index 3f2073e480b..ca97ec5a604 100644 --- a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/ProcessorResult.java +++ b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateProcessorResult.java @@ -18,7 +18,7 @@ */ package org.elasticsearch.plugin.ingest.transport.simulate; -import org.elasticsearch.ExceptionsHelper; +import org.elasticsearch.ElasticsearchException; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.Streamable; @@ -26,27 +26,27 @@ import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentBuilderString; import org.elasticsearch.ingest.Data; +import org.elasticsearch.plugin.ingest.transport.TransportData; import java.io.IOException; -import java.util.Map; import java.util.Objects; -public class ProcessorResult implements Streamable, ToXContent { +public class SimulateProcessorResult implements Streamable, ToXContent { private String processorId; - private Data data; + private TransportData data; private Throwable failure; - public ProcessorResult() { + public SimulateProcessorResult() { } - public ProcessorResult(String processorId, Data data) { + public SimulateProcessorResult(String processorId, Data data) { this.processorId = processorId; - this.data = data; + this.data = new TransportData(data); } - public ProcessorResult(String processorId, Throwable failure) { + public SimulateProcessorResult(String processorId, Throwable failure) { this.processorId = processorId; this.failure = failure; } @@ -56,7 +56,7 @@ public class ProcessorResult implements Streamable, ToXContent { } public Data getData() { - return data; + return data.get(); } public String getProcessorId() { @@ -69,12 +69,8 @@ public class ProcessorResult implements Streamable, ToXContent { if (isFailure) { this.failure = in.readThrowable(); } else { - this.processorId = in.readString(); - String index = in.readString(); - String type = in.readString(); - String id = in.readString(); - Map doc = in.readMap(); - this.data = new Data(index, type, id, doc); + this.data = new TransportData(); + this.data.readFrom(in); } } @@ -85,10 +81,7 @@ public class ProcessorResult implements Streamable, ToXContent { out.writeThrowable(failure); } else { out.writeString(processorId); - out.writeString(data.getIndex()); - out.writeString(data.getType()); - out.writeString(data.getId()); - out.writeMap(data.getDocument()); + data.writeTo(out); } } @@ -96,12 +89,10 @@ public class ProcessorResult implements Streamable, ToXContent { public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { builder.startObject(); builder.field(Fields.PROCESSOR_ID, processorId); - builder.field(Fields.ERROR, isFailed()); if (isFailed()) { - builder.field(Fields.ERROR_MESSAGE, ExceptionsHelper.detailedMessage(failure)); + ElasticsearchException.renderThrowable(builder, params, failure); } else { - builder.field(Fields.MODIFIED, data.isModified()); - builder.field(Fields.DOCUMENT, data.asMap()); + data.toXContent(builder, params); } builder.endObject(); return builder; @@ -109,11 +100,13 @@ public class ProcessorResult implements Streamable, ToXContent { @Override public boolean equals(Object obj) { - if (obj == this) { return true; } + if (obj == this) { + return true; + } if (obj == null || getClass() != obj.getClass()) { return false; } - ProcessorResult other = (ProcessorResult) obj; + SimulateProcessorResult other = (SimulateProcessorResult) obj; return Objects.equals(processorId, other.processorId) && Objects.equals(data, other.data) && Objects.equals(failure, other.failure); } @@ -123,10 +116,6 @@ public class ProcessorResult implements Streamable, ToXContent { } static final class Fields { - static final XContentBuilderString DOCUMENT = new XContentBuilderString("doc"); static final XContentBuilderString PROCESSOR_ID = new XContentBuilderString("processor_id"); - static final XContentBuilderString ERROR = new XContentBuilderString("error"); - static final XContentBuilderString ERROR_MESSAGE = new XContentBuilderString("error_message"); - static final XContentBuilderString MODIFIED = new XContentBuilderString("modified"); } } diff --git a/plugins/ingest/src/test/java/org/elasticsearch/ingest/DataTests.java b/plugins/ingest/src/test/java/org/elasticsearch/ingest/DataTests.java index e6cae34035e..918064a17c7 100644 --- a/plugins/ingest/src/test/java/org/elasticsearch/ingest/DataTests.java +++ b/plugins/ingest/src/test/java/org/elasticsearch/ingest/DataTests.java @@ -22,6 +22,7 @@ package org.elasticsearch.ingest; import org.elasticsearch.test.ESTestCase; import org.junit.Before; +import java.util.Collections; import java.util.HashMap; import java.util.Map; @@ -84,4 +85,29 @@ public class DataTests extends ESTestCase { data.addField("fizz.new", "bar"); assertThat(data.getProperty("fizz.new"), equalTo("bar")); } + + public void testEquals() { + Data otherData = new Data(data); + assertThat(otherData, equalTo(data)); + } + + public void testNotEqualsDiffIndex() { + Data otherData = new Data(data.getIndex() + "foo", data.getType(), data.getId(), data.getDocument()); + assertThat(otherData, not(equalTo(data))); + } + + public void testNotEqualsDiffType() { + Data otherData = new Data(data.getIndex(), data.getType() + "foo", data.getId(), data.getDocument()); + assertThat(otherData, not(equalTo(data))); + } + + public void testNotEqualsDiffId() { + Data otherData = new Data(data.getIndex(), data.getType(), data.getId() + "foo", data.getDocument()); + assertThat(otherData, not(equalTo(data))); + } + + public void testNotEqualsDiffDocument() { + Data otherData = new Data(data.getIndex(), data.getType(), data.getId(), Collections.emptyMap()); + assertThat(otherData, not(equalTo(data))); + } } diff --git a/plugins/ingest/src/test/java/org/elasticsearch/ingest/IngestClientIT.java b/plugins/ingest/src/test/java/org/elasticsearch/ingest/IngestClientIT.java index d0d11ae60a0..97ed3339090 100644 --- a/plugins/ingest/src/test/java/org/elasticsearch/ingest/IngestClientIT.java +++ b/plugins/ingest/src/test/java/org/elasticsearch/ingest/IngestClientIT.java @@ -33,7 +33,7 @@ import org.elasticsearch.plugin.ingest.transport.put.PutPipelineRequestBuilder; import org.elasticsearch.plugin.ingest.transport.simulate.SimulatePipelineAction; import org.elasticsearch.plugin.ingest.transport.simulate.SimulatePipelineRequestBuilder; import org.elasticsearch.plugin.ingest.transport.simulate.SimulatePipelineResponse; -import org.elasticsearch.plugin.ingest.transport.simulate.SimulatedItemResponse; +import org.elasticsearch.plugin.ingest.transport.simulate.SimulateDocumentResult; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.ESIntegTestCase; @@ -104,8 +104,8 @@ public class IngestClientIT extends ESIntegTestCase { Map expectedDoc = new HashMap<>(); expectedDoc.put("foo", "bar"); Data expectedData = new Data("index", "type", "id", expectedDoc); - SimulatedItemResponse expectedResponse = new SimulatedItemResponse(expectedData); - List expectedResponses = Arrays.asList(expectedResponse); + SimulateDocumentResult expectedResponse = new SimulateDocumentResult(expectedData); + List expectedResponses = Arrays.asList(expectedResponse); SimulatePipelineResponse expected = new SimulatePipelineResponse("_id", expectedResponses); assertThat(response, equalTo(expected)); diff --git a/plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/transport/TransportDataTests.java b/plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/transport/TransportDataTests.java new file mode 100644 index 00000000000..08940ee3ce2 --- /dev/null +++ b/plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/transport/TransportDataTests.java @@ -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.plugin.ingest.transport; + +import org.elasticsearch.ingest.Data; +import org.elasticsearch.test.ESTestCase; + +import java.util.Collections; + +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.not; + +public class TransportDataTests extends ESTestCase { + + public void testEquals() throws Exception { + Data data = new Data("_index", "_type", "_id", Collections.singletonMap("foo", "bar")); + Data otherData = new Data("_index", "_type", "_id", Collections.singletonMap("foo", "bar")); + assertThat(data, equalTo(otherData)); + } + + public void testNotEquals() throws Exception { + Data data = new Data("_index", "_type", "_id", Collections.singletonMap("foo", "bar")); + Data otherData = new Data("_index2", "_type", "_id", Collections.emptyMap()); + assertThat(data, not(equalTo(otherData))); + } +} diff --git a/plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/transport/simulate/ParsedSimulateRequestParserTests.java b/plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/transport/simulate/ParsedSimulateRequestParserTests.java index d88bf7a5467..03c4c3af7f8 100644 --- a/plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/transport/simulate/ParsedSimulateRequestParserTests.java +++ b/plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/transport/simulate/ParsedSimulateRequestParserTests.java @@ -35,28 +35,26 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; public class ParsedSimulateRequestParserTests extends ESTestCase { - private static final ParsedSimulateRequest.Parser PARSER = new ParsedSimulateRequest.Parser(); - - private Map processorRegistry; private PipelineStore store; - private Processor processor; + private ParsedSimulateRequest.Parser parser; private Pipeline pipeline; private Data data; @Before public void init() throws IOException { + parser = new ParsedSimulateRequest.Parser(); List uppercase = Collections.unmodifiableList(Collections.singletonList("foo")); - processor = new MutateProcessor(null, null, null, null, null, null, null, null, uppercase, null); + Processor processor = new MutateProcessor(null, null, null, null, null, null, null, null, uppercase, null); pipeline = new Pipeline(ParsedSimulateRequest.Parser.SIMULATED_PIPELINE_ID, null, Collections.unmodifiableList(Arrays.asList(processor))); - data = new Data("_index", "_type", "_id", Collections.emptyMap()); - processorRegistry = new HashMap<>(); + data = new Data("_index", "_type", "_id", Collections.singletonMap("foo", "bar")); + Map processorRegistry = new HashMap<>(); processorRegistry.put("mutate", new MutateProcessor.Factory()); store = mock(PipelineStore.class); when(store.get("_id")).thenReturn(pipeline); when(store.getProcessorFactoryRegistry()).thenReturn(processorRegistry); } - public void testParse_UsingPipelineStore() throws Exception { + public void testParseUsingPipelineStore() throws Exception { ParsedSimulateRequest expectedRequest = new ParsedSimulateRequest(pipeline, Collections.singletonList(data), false); Map raw = new HashMap<>(); @@ -65,14 +63,15 @@ public class ParsedSimulateRequestParserTests extends ESTestCase { doc.put("_index", "_index"); doc.put("_type", "_type"); doc.put("_id", "_id"); + doc.put("_source", data.getDocument()); docs.add(doc); raw.put("docs", docs); - ParsedSimulateRequest actualRequest = PARSER.parse("_id", raw, false, store); + ParsedSimulateRequest actualRequest = parser.parseWithPipelineId("_id", raw, false, store); assertThat(actualRequest, equalTo(expectedRequest)); } - public void testParse_ProvidedPipeline() throws Exception { + public void testParseWithProvidedPipeline() throws Exception { ParsedSimulateRequest expectedRequest = new ParsedSimulateRequest(pipeline, Collections.singletonList(data), false); Map raw = new HashMap<>(); @@ -81,6 +80,7 @@ public class ParsedSimulateRequestParserTests extends ESTestCase { doc.put("_index", "_index"); doc.put("_type", "_type"); doc.put("_id", "_id"); + doc.put("_source", data.getDocument()); docs.add(doc); Map processorConfig = new HashMap<>(); @@ -91,7 +91,7 @@ public class ParsedSimulateRequestParserTests extends ESTestCase { raw.put("docs", docs); raw.put("pipeline", pipelineConfig); - ParsedSimulateRequest actualRequest = PARSER.parse(null, raw, false, store); + ParsedSimulateRequest actualRequest = parser.parse(raw, false, store); assertThat(actualRequest, equalTo(expectedRequest)); } } diff --git a/plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateExecutionServiceTests.java b/plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateExecutionServiceTests.java index 8ed2cc6abec..cf108b0db84 100644 --- a/plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateExecutionServiceTests.java +++ b/plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateExecutionServiceTests.java @@ -19,11 +19,12 @@ package org.elasticsearch.plugin.ingest.transport.simulate; +import org.elasticsearch.action.ActionListener; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.ingest.Data; import org.elasticsearch.ingest.Pipeline; import org.elasticsearch.ingest.processor.Processor; -import org.elasticsearch.plugin.ingest.PipelineStore; +import org.elasticsearch.plugin.ingest.transport.TransportData; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.threadpool.ThreadPool; import org.junit.After; @@ -37,26 +38,28 @@ import static org.mockito.Mockito.*; public class SimulateExecutionServiceTests extends ESTestCase { - private PipelineStore store; private ThreadPool threadPool; private SimulateExecutionService executionService; private Pipeline pipeline; private Processor processor; private Data data; + private TransportData transportData; + private ActionListener listener; @Before public void setup() { - store = mock(PipelineStore.class); threadPool = new ThreadPool( Settings.builder() - .put("name", "_name") + .put("name", getClass().getName()) .build() ); executionService = new SimulateExecutionService(threadPool); processor = mock(Processor.class); when(processor.getType()).thenReturn("mock"); pipeline = new Pipeline("_id", "_description", Arrays.asList(processor, processor)); - data = new Data("_index", "_type", "_id", Collections.emptyMap()); + data = new Data("_index", "_type", "_id", Collections.singletonMap("foo", "bar")); + transportData = new TransportData(data); + listener = mock(ActionListener.class); } @After @@ -65,43 +68,42 @@ public class SimulateExecutionServiceTests extends ESTestCase { } public void testExecuteVerboseItem() throws Exception { - SimulatedItemResponse expectedItemResponse = new SimulatedItemResponse( - Arrays.asList(new ProcessorResult("processor[mock]-0", data), new ProcessorResult("processor[mock]-1", data))); - SimulatedItemResponse actualItemResponse = executionService.executeVerboseItem(pipeline, data); + SimulateDocumentResult expectedItemResponse = new SimulateDocumentResult( + Arrays.asList(new SimulateProcessorResult("processor[mock]-0", data), new SimulateProcessorResult("processor[mock]-1", data))); + SimulateDocumentResult actualItemResponse = executionService.executeVerboseItem(pipeline, data); verify(processor, times(2)).execute(data); assertThat(actualItemResponse, equalTo(expectedItemResponse)); } public void testExecuteItem() throws Exception { - SimulatedItemResponse expectedItemResponse = new SimulatedItemResponse(data); - SimulatedItemResponse actualItemResponse = executionService.executeItem(pipeline, data); + SimulateDocumentResult expectedItemResponse = new SimulateDocumentResult(data); + SimulateDocumentResult actualItemResponse = executionService.executeItem(pipeline, data); verify(processor, times(2)).execute(data); assertThat(actualItemResponse, equalTo(expectedItemResponse)); } - public void testExecuteVerboseItem_Failure() throws Exception { + public void testExecuteVerboseItemWithFailure() throws Exception { Exception e = new RuntimeException("processor failed"); - SimulatedItemResponse expectedItemResponse = new SimulatedItemResponse( - Arrays.asList(new ProcessorResult("processor[mock]-0", e), new ProcessorResult("processor[mock]-1", data)) + SimulateDocumentResult expectedItemResponse = new SimulateDocumentResult( + Arrays.asList(new SimulateProcessorResult("processor[mock]-0", e), new SimulateProcessorResult("processor[mock]-1", data)) ); doThrow(e).doNothing().when(processor).execute(data); - SimulatedItemResponse actualItemResponse = executionService.executeVerboseItem(pipeline, data); + SimulateDocumentResult actualItemResponse = executionService.executeVerboseItem(pipeline, data); verify(processor, times(2)).execute(data); assertThat(actualItemResponse, equalTo(expectedItemResponse)); } - public void testExecuteItem_Failure() throws Exception { + public void testExecuteItemWithFailure() throws Exception { Exception e = new RuntimeException("processor failed"); - SimulatedItemResponse expectedItemResponse = new SimulatedItemResponse(e); + SimulateDocumentResult expectedItemResponse = new SimulateDocumentResult(e); doThrow(e).when(processor).execute(data); - SimulatedItemResponse actualItemResponse = executionService.executeItem(pipeline, data); + SimulateDocumentResult actualItemResponse = executionService.executeItem(pipeline, data); verify(processor, times(1)).execute(data); assertThat(actualItemResponse, equalTo(expectedItemResponse)); } public void testExecute() throws Exception { - SimulateExecutionService.Listener listener = mock(SimulateExecutionService.Listener.class); - SimulatedItemResponse itemResponse = new SimulatedItemResponse(data); + SimulateDocumentResult itemResponse = new SimulateDocumentResult(data); ParsedSimulateRequest request = new ParsedSimulateRequest(pipeline, Collections.singletonList(data), false); executionService.execute(request, listener); SimulatePipelineResponse response = new SimulatePipelineResponse("_id", Collections.singletonList(itemResponse)); @@ -114,11 +116,10 @@ public class SimulateExecutionServiceTests extends ESTestCase { }); } - public void testExecute_Verbose() throws Exception { - SimulateExecutionService.Listener listener = mock(SimulateExecutionService.Listener.class); + public void testExecuteWithVerbose() throws Exception { ParsedSimulateRequest request = new ParsedSimulateRequest(pipeline, Collections.singletonList(data), true); - SimulatedItemResponse itemResponse = new SimulatedItemResponse( - Arrays.asList(new ProcessorResult("processor[mock]-0", data), new ProcessorResult("processor[mock]-1", data))); + SimulateDocumentResult itemResponse = new SimulateDocumentResult( + Arrays.asList(new SimulateProcessorResult("processor[mock]-0", data), new SimulateProcessorResult("processor[mock]-1", data))); executionService.execute(request, listener); SimulatePipelineResponse response = new SimulatePipelineResponse("_id", Collections.singletonList(itemResponse)); assertBusy(new Runnable() { diff --git a/plugins/ingest/src/test/resources/rest-api-spec/test/ingest/80_simulate.yaml b/plugins/ingest/src/test/resources/rest-api-spec/test/ingest/80_simulate.yaml index 7596469b7f2..156ba8cfd67 100644 --- a/plugins/ingest/src/test/resources/rest-api-spec/test/ingest/80_simulate.yaml +++ b/plugins/ingest/src/test/resources/rest-api-spec/test/ingest/80_simulate.yaml @@ -47,8 +47,7 @@ ] } - length: { docs: 1 } - - is_false: docs.0.error - - is_true: docs.0.modified + - is_true: docs.0.doc.modified - match: { docs.0.doc._source.foo: "bar" } - match: { docs.0.doc._source.field2: "_value" } @@ -132,8 +131,7 @@ - length: { docs: 1 } - length: { docs.0.processor_results: 2 } - match: { docs.0.processor_results.0.processor_id: "processor[mutate]-0" } - - is_false: docs.0.processor_results.0.error - - is_true: docs.0.processor_results.0.modified + - is_true: docs.0.processor_results.0.doc.modified - length: { docs.0.processor_results.0.doc._source: 2 } - match: { docs.0.processor_results.0.doc._source.foo: "bar" } - match: { docs.0.processor_results.0.doc._source.field2: "_value" } @@ -149,7 +147,6 @@ wait_for_status: green - do: - catch: request ingest.simulate: body: > { @@ -183,10 +180,8 @@ ] } - length: { docs: 2 } - - is_true: docs.0.error - - match: { docs.0.error_message: "NullPointerException[null]" } - - is_false: docs.1.error - - is_true: docs.1.modified + - match: { docs.0.error.type: "null_pointer_exception" } + - is_true: docs.1.doc.modified - match: { docs.1.doc._source.foo: "BAR" } --- @@ -196,7 +191,6 @@ wait_for_status: green - do: - catch: request ingest.simulate: verbose: true body: > @@ -240,12 +234,11 @@ ] } - length: { docs: 2 } - - is_true: docs.0.error - - is_false: docs.1.error - length: { docs.0.processor_results: 2 } - - is_false: docs.1.processor_results.0.error - - match: { docs.0.processor_results.0.error_message: "NumberFormatException[For input string: \"bar\"]" } - - is_false: docs.1.processor_results.1.error + - match: { docs.0.processor_results.0.error.type: "number_format_exception" } + - match: { docs.0.processor_results.1.doc._index: "index" } + - match: { docs.0.processor_results.1.doc._type: "type" } + - match: { docs.0.processor_results.1.doc._id: "id" } - match: { docs.0.processor_results.1.doc._source.foo: "bar" } - match: { docs.1.processor_results.1.doc._source.bar: "HELLO" } - match: { docs.1.processor_results.0.doc._source.foo: 5 } From 20384aedf09c6bbda424fd50dd1d4d23c19a2d14 Mon Sep 17 00:00:00 2001 From: Tal Levy Date: Thu, 12 Nov 2015 17:31:08 -0800 Subject: [PATCH 10/14] split out SimulateDocumentResult into subclasses, add tests for equalTo and streamable --- .../java/org/elasticsearch/ingest/Data.java | 10 -- .../simulate/ParsedSimulateRequest.java | 11 +- .../simulate/SimulateDocumentResult.java | 122 +----------------- .../simulate/SimulateExecutionService.java | 6 +- .../SimulateFailedDocumentResult.java | 84 ++++++++++++ .../simulate/SimulatePipelineRequest.java | 9 ++ .../simulate/SimulatePipelineResponse.java | 45 ++++--- .../simulate/SimulateProcessorResult.java | 9 +- .../SimulateSimpleDocumentResult.java | 84 ++++++++++++ .../SimulateVerboseDocumentResult.java | 101 +++++++++++++++ .../elasticsearch/ingest/IngestClientIT.java | 7 +- .../elasticsearch/ingest/PipelineTests.java | 67 ++++++++++ .../ParsedSimulateRequestParserTests.java | 28 ++-- .../SimulateExecutionServiceTests.java | 12 +- .../SimulateFailedDocumentResultTests.java | 69 ++++++++++ .../SimulatePipelineResponseTests.java | 80 ++++++++++++ .../SimulateProcessorResultTests.java | 87 +++++++++++++ .../SimulateSimpleDocumentResultTests.java | 67 ++++++++++ .../test/ingest/80_simulate.yaml | 27 ++++ 19 files changed, 744 insertions(+), 181 deletions(-) create mode 100644 plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateFailedDocumentResult.java create mode 100644 plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateSimpleDocumentResult.java create mode 100644 plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateVerboseDocumentResult.java create mode 100644 plugins/ingest/src/test/java/org/elasticsearch/ingest/PipelineTests.java create mode 100644 plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateFailedDocumentResultTests.java create mode 100644 plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineResponseTests.java create mode 100644 plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateProcessorResultTests.java create mode 100644 plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateSimpleDocumentResultTests.java diff --git a/plugins/ingest/src/main/java/org/elasticsearch/ingest/Data.java b/plugins/ingest/src/main/java/org/elasticsearch/ingest/Data.java index 543fd9c1c4b..b360af7d459 100644 --- a/plugins/ingest/src/main/java/org/elasticsearch/ingest/Data.java +++ b/plugins/ingest/src/main/java/org/elasticsearch/ingest/Data.java @@ -130,16 +130,6 @@ public final class Data { return modified; } - public Map asMap() { - Map dataMap = new HashMap<>(); - dataMap.put("_index", index); - dataMap.put("_type", type); - dataMap.put("_id", id); - dataMap.put("_source", document); - - return dataMap; - } - @Override public boolean equals(Object obj) { if (obj == this) { return true; } diff --git a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/ParsedSimulateRequest.java b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/ParsedSimulateRequest.java index 7d02f686469..47af2db583d 100644 --- a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/ParsedSimulateRequest.java +++ b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/ParsedSimulateRequest.java @@ -26,6 +26,8 @@ import org.elasticsearch.plugin.ingest.PipelineStore; import java.io.IOException; import java.util.*; +import static org.elasticsearch.plugin.ingest.transport.simulate.SimulatePipelineRequest.Fields; + public class ParsedSimulateRequest { private final List documents; private final Pipeline pipeline; @@ -98,14 +100,5 @@ public class ParsedSimulateRequest { List dataList = parseDocs(config); return new ParsedSimulateRequest(pipeline, dataList, verbose); } - - static final class Fields { - static final String PIPELINE = "pipeline"; - static final String DOCS = "docs"; - static final String SOURCE = "_source"; - static final String INDEX = "_index"; - static final String TYPE = "_type"; - static final String ID = "_id"; - } } } diff --git a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateDocumentResult.java b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateDocumentResult.java index 33ef7745404..98fdf380bc7 100644 --- a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateDocumentResult.java +++ b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateDocumentResult.java @@ -18,133 +18,23 @@ */ package org.elasticsearch.plugin.ingest.transport.simulate; -import org.elasticsearch.ElasticsearchException; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.Streamable; import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.XContentBuilder; -import org.elasticsearch.common.xcontent.XContentBuilderString; -import org.elasticsearch.ingest.Data; -import org.elasticsearch.plugin.ingest.transport.TransportData; import java.io.IOException; -import java.util.ArrayList; -import java.util.List; -import java.util.Objects; -public class SimulateDocumentResult implements Streamable, ToXContent { - - private TransportData data; - private List processorResultList; - private Throwable failure; - - public SimulateDocumentResult() { +public abstract class SimulateDocumentResult implements Streamable, ToXContent { + public int getStreamId() { + return -1; } - public SimulateDocumentResult(Data data) { - this.data = new TransportData(data); - } + public abstract void readFrom(StreamInput in) throws IOException; - public SimulateDocumentResult(List processorResultList) { - this.processorResultList = processorResultList; - } + public abstract void writeTo(StreamOutput out) throws IOException; - public SimulateDocumentResult(Throwable failure) { - this.failure = failure; - } - - public boolean isFailed() { - if (failure != null) { - return true; - } - return false; - } - - public boolean isVerbose() { - return this.processorResultList != null; - } - - public Data getData() { - return data.get(); - } - - public List getProcessorResultList() { - return processorResultList; - } - - @Override - public void readFrom(StreamInput in) throws IOException { - boolean isFailed = in.readBoolean(); - boolean isVerbose = in.readBoolean(); - if (isFailed) { - this.failure = in.readThrowable(); - } else if (isVerbose) { - int size = in.readVInt(); - processorResultList = new ArrayList<>(); - for (int i = 0; i < size; i++) { - SimulateProcessorResult processorResult = new SimulateProcessorResult(); - processorResult.readFrom(in); - processorResultList.add(processorResult); - } - } else { - this.data = new TransportData(); - this.data.readFrom(in); - } - } - - @Override - public void writeTo(StreamOutput out) throws IOException { - out.writeBoolean(isFailed()); - out.writeBoolean(isVerbose()); - - if (failure != null) { - out.writeThrowable(failure); - } else if (isVerbose()) { - out.writeVInt(processorResultList.size()); - for (SimulateProcessorResult p : processorResultList) { - p.writeTo(out); - } - } else { - data.writeTo(out); - } - } - - @Override - public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { - builder.startObject(); - if (isFailed()) { - ElasticsearchException.renderThrowable(builder, params, failure); - } else if (isVerbose()) { - builder.startArray(Fields.PROCESSOR_RESULTS); - for (SimulateProcessorResult processorResult : processorResultList) { - processorResult.toXContent(builder, params); - } - builder.endArray(); - } else { - data.toXContent(builder, params); - } - builder.endObject(); - return builder; - } - - @Override - public boolean equals(Object obj) { - if (obj == this) { return true; } - if (obj == null || getClass() != obj.getClass()) { - return false; - } - SimulateDocumentResult other = (SimulateDocumentResult) obj; - return Objects.equals(data, other.data) && Objects.equals(processorResultList, other.processorResultList) && Objects.equals(failure, other.failure); - } - - @Override - public int hashCode() { - return Objects.hash(data, processorResultList, failure); - } - - static final class Fields { - static final XContentBuilderString PROCESSOR_RESULTS = new XContentBuilderString("processor_results"); - } + public abstract XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException; } diff --git a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateExecutionService.java b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateExecutionService.java index a4518273467..a8d34b4a150 100644 --- a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateExecutionService.java +++ b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateExecutionService.java @@ -44,9 +44,9 @@ public class SimulateExecutionService { SimulateDocumentResult executeItem(Pipeline pipeline, Data data) { try { pipeline.execute(data); - return new SimulateDocumentResult(data); + return new SimulateSimpleDocumentResult(data); } catch (Exception e) { - return new SimulateDocumentResult(e); + return new SimulateFailedDocumentResult(e); } } @@ -67,7 +67,7 @@ public class SimulateExecutionService { currentData = new Data(currentData); } - return new SimulateDocumentResult(processorResultList); + return new SimulateVerboseDocumentResult(processorResultList); } public void execute(ParsedSimulateRequest request, ActionListener listener) { diff --git a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateFailedDocumentResult.java b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateFailedDocumentResult.java new file mode 100644 index 00000000000..e0bf7555b6b --- /dev/null +++ b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateFailedDocumentResult.java @@ -0,0 +1,84 @@ +/* + * 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.plugin.ingest.transport.simulate; + +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 java.io.IOException; +import java.util.Objects; + +public class SimulateFailedDocumentResult extends SimulateDocumentResult { + public static final int STREAM_ID = 2; + + private Throwable failure; + + public SimulateFailedDocumentResult() { + + } + + public SimulateFailedDocumentResult(Throwable failure) { + this.failure = failure; + } + + @Override + public int getStreamId() { + return STREAM_ID; + } + + @Override + public void readFrom(StreamInput in) throws IOException { + int streamId = in.readVInt(); + if (streamId != STREAM_ID) { + throw new IOException("stream_id [" + streamId + "] does not match " + getClass().getName() + " [stream_id=" + STREAM_ID + "]"); + } + this.failure = in.readThrowable(); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeVInt(STREAM_ID); + out.writeThrowable(failure); + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject(); + ElasticsearchException.renderThrowable(builder, params, failure); + builder.endObject(); + return builder; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + SimulateFailedDocumentResult that = (SimulateFailedDocumentResult) o; + + return Objects.equals((failure == null) ? null : failure.getClass(), + (that.failure == null) ? null : that.failure.getClass()); + } + + @Override + public int hashCode() { + return Objects.hash(failure); + } +} diff --git a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineRequest.java b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineRequest.java index 4aec025b12d..3394de413c8 100644 --- a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineRequest.java +++ b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineRequest.java @@ -83,4 +83,13 @@ public class SimulatePipelineRequest extends ActionRequest { 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 final String INDEX = "_index"; + static final String TYPE = "_type"; + static final String ID = "_id"; + } } diff --git a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineResponse.java b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineResponse.java index 5d5bb57bc1f..5922f9d906e 100644 --- a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineResponse.java +++ b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineResponse.java @@ -33,9 +33,8 @@ import java.util.List; import java.util.Objects; public class SimulatePipelineResponse extends ActionResponse implements ToXContent { - private String pipelineId; - private List responses; + private List results; public SimulatePipelineResponse() { @@ -43,7 +42,7 @@ public class SimulatePipelineResponse extends ActionResponse implements ToXConte public SimulatePipelineResponse(String pipelineId, List responses) { this.pipelineId = pipelineId; - this.responses = Collections.unmodifiableList(responses); + this.results = Collections.unmodifiableList(responses); } public String getPipelineId() { @@ -54,20 +53,21 @@ public class SimulatePipelineResponse extends ActionResponse implements ToXConte this.pipelineId = pipelineId; } - public List getResponses() { - return responses; + public List getResults() { + return results; } - public void setResponses(List responses) { - this.responses = responses; + public void setResults(List results) { + this.results = results; } @Override public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); out.writeString(pipelineId); - out.writeVInt(responses.size()); - for (SimulateDocumentResult response : responses) { + out.writeVInt(results.size()); + for (SimulateDocumentResult response : results) { + out.writeVInt(response.getStreamId()); response.writeTo(out); } } @@ -77,11 +77,24 @@ public class SimulatePipelineResponse extends ActionResponse implements ToXConte super.readFrom(in); this.pipelineId = in.readString(); int responsesLength = in.readVInt(); - responses = new ArrayList<>(); + results = new ArrayList<>(); for (int i = 0; i < responsesLength; i++) { - SimulateDocumentResult response = new SimulateDocumentResult(); - response.readFrom(in); - responses.add(response); + SimulateDocumentResult result; + switch (in.readVInt()) { + case SimulateSimpleDocumentResult.STREAM_ID: + result = new SimulateSimpleDocumentResult(); + break; + case SimulateVerboseDocumentResult.STREAM_ID: + result = new SimulateVerboseDocumentResult(); + break; + case SimulateFailedDocumentResult.STREAM_ID: + result = new SimulateFailedDocumentResult(); + break; + default: + throw new IOException("Cannot read result from stream"); + } + result.readFrom(in); + results.add(result); } } @@ -89,7 +102,7 @@ public class SimulatePipelineResponse extends ActionResponse implements ToXConte @Override public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { builder.startArray(Fields.DOCUMENTS); - for (SimulateDocumentResult response : responses) { + for (SimulateDocumentResult response : results) { response.toXContent(builder, params); } builder.endArray(); @@ -103,12 +116,12 @@ public class SimulatePipelineResponse extends ActionResponse implements ToXConte if (o == null || getClass() != o.getClass()) return false; SimulatePipelineResponse that = (SimulatePipelineResponse) o; return Objects.equals(pipelineId, that.pipelineId) && - Objects.equals(responses, that.responses); + Objects.equals(results, that.results); } @Override public int hashCode() { - return Objects.hash(pipelineId, responses); + return Objects.hash(pipelineId, results); } static final class Fields { diff --git a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateProcessorResult.java b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateProcessorResult.java index ca97ec5a604..7b196f69276 100644 --- a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateProcessorResult.java +++ b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateProcessorResult.java @@ -51,7 +51,7 @@ public class SimulateProcessorResult implements Streamable, ToXContent { this.failure = failure; } - public boolean isFailed() { + private boolean isFailed() { return this.failure != null; } @@ -66,6 +66,7 @@ public class SimulateProcessorResult implements Streamable, ToXContent { @Override public void readFrom(StreamInput in) throws IOException { boolean isFailure = in.readBoolean(); + this.processorId = in.readString(); if (isFailure) { this.failure = in.readThrowable(); } else { @@ -77,10 +78,10 @@ public class SimulateProcessorResult implements Streamable, ToXContent { @Override public void writeTo(StreamOutput out) throws IOException { out.writeBoolean(isFailed()); + out.writeString(processorId); if (isFailed()) { out.writeThrowable(failure); } else { - out.writeString(processorId); data.writeTo(out); } } @@ -107,7 +108,9 @@ public class SimulateProcessorResult implements Streamable, ToXContent { return false; } SimulateProcessorResult other = (SimulateProcessorResult) obj; - return Objects.equals(processorId, other.processorId) && Objects.equals(data, other.data) && Objects.equals(failure, other.failure); + + return Objects.equals(processorId, other.processorId) && Objects.equals(data, other.data) && + Objects.equals((failure == null) ? null : failure.getClass(), (other.failure == null) ? null : other.failure.getClass()); } @Override diff --git a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateSimpleDocumentResult.java b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateSimpleDocumentResult.java new file mode 100644 index 00000000000..9d432bc2855 --- /dev/null +++ b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateSimpleDocumentResult.java @@ -0,0 +1,84 @@ +/* + * 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.plugin.ingest.transport.simulate; + +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.ingest.Data; +import org.elasticsearch.plugin.ingest.transport.TransportData; + +import java.io.IOException; +import java.util.Objects; + +public class SimulateSimpleDocumentResult extends SimulateDocumentResult { + public static final int STREAM_ID = 0; + + private TransportData transportData; + + public SimulateSimpleDocumentResult() { + + } + + public SimulateSimpleDocumentResult(Data data) { + this.transportData = new TransportData(data); + } + + @Override + public int getStreamId() { + return STREAM_ID; + } + + @Override + public void readFrom(StreamInput in) throws IOException { + int streamId = in.readVInt(); + if (streamId != STREAM_ID) { + throw new IOException("stream_id [" + streamId + "] does not match " + getClass().getName() + " [stream_id=" + STREAM_ID + "]"); + } + this.transportData = new TransportData(); + this.transportData.readFrom(in); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeVInt(STREAM_ID); + transportData.writeTo(out); + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject(); + transportData.toXContent(builder, params); + builder.endObject(); + return builder; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + SimulateSimpleDocumentResult that = (SimulateSimpleDocumentResult) o; + return Objects.equals(transportData, that.transportData); + } + + @Override + public int hashCode() { + return Objects.hash(transportData); + } +} diff --git a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateVerboseDocumentResult.java b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateVerboseDocumentResult.java new file mode 100644 index 00000000000..f3da751e78b --- /dev/null +++ b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateVerboseDocumentResult.java @@ -0,0 +1,101 @@ +/* + * 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.plugin.ingest.transport.simulate; + +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentBuilderString; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Objects; + +public class SimulateVerboseDocumentResult extends SimulateDocumentResult { + public static final int STREAM_ID = 1; + + private List processorResults; + + public SimulateVerboseDocumentResult() { + + } + + public SimulateVerboseDocumentResult(List processorResults) { + this.processorResults = processorResults; + } + + @Override + public int getStreamId() { + return STREAM_ID; + } + + @Override + public void readFrom(StreamInput in) throws IOException { + int streamId = in.readVInt(); + if (streamId != STREAM_ID) { + throw new IOException("stream_id [" + streamId + "] does not match " + getClass().getName() + " [stream_id=" + STREAM_ID + "]"); + } + int size = in.readVInt(); + processorResults = new ArrayList<>(); + for (int i = 0; i < size; i++) { + SimulateProcessorResult processorResult = new SimulateProcessorResult(); + processorResult.readFrom(in); + processorResults.add(processorResult); + } + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeVInt(STREAM_ID); + 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(Fields.PROCESSOR_RESULTS); + for (SimulateProcessorResult processorResult : processorResults) { + processorResult.toXContent(builder, params); + } + builder.endArray(); + builder.endObject(); + return builder; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + SimulateVerboseDocumentResult that = (SimulateVerboseDocumentResult) o; + return Objects.equals(processorResults, that.processorResults); + } + + @Override + public int hashCode() { + return Objects.hash(processorResults); + } + + static final class Fields { + static final XContentBuilderString PROCESSOR_RESULTS = new XContentBuilderString("processor_results"); + } +} diff --git a/plugins/ingest/src/test/java/org/elasticsearch/ingest/IngestClientIT.java b/plugins/ingest/src/test/java/org/elasticsearch/ingest/IngestClientIT.java index 97ed3339090..2001b673d06 100644 --- a/plugins/ingest/src/test/java/org/elasticsearch/ingest/IngestClientIT.java +++ b/plugins/ingest/src/test/java/org/elasticsearch/ingest/IngestClientIT.java @@ -30,10 +30,7 @@ import org.elasticsearch.plugin.ingest.transport.get.GetPipelineRequestBuilder; import org.elasticsearch.plugin.ingest.transport.get.GetPipelineResponse; import org.elasticsearch.plugin.ingest.transport.put.PutPipelineAction; import org.elasticsearch.plugin.ingest.transport.put.PutPipelineRequestBuilder; -import org.elasticsearch.plugin.ingest.transport.simulate.SimulatePipelineAction; -import org.elasticsearch.plugin.ingest.transport.simulate.SimulatePipelineRequestBuilder; -import org.elasticsearch.plugin.ingest.transport.simulate.SimulatePipelineResponse; -import org.elasticsearch.plugin.ingest.transport.simulate.SimulateDocumentResult; +import org.elasticsearch.plugin.ingest.transport.simulate.*; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.ESIntegTestCase; @@ -104,7 +101,7 @@ public class IngestClientIT extends ESIntegTestCase { Map expectedDoc = new HashMap<>(); expectedDoc.put("foo", "bar"); Data expectedData = new Data("index", "type", "id", expectedDoc); - SimulateDocumentResult expectedResponse = new SimulateDocumentResult(expectedData); + SimulateDocumentResult expectedResponse = new SimulateSimpleDocumentResult(expectedData); List expectedResponses = Arrays.asList(expectedResponse); SimulatePipelineResponse expected = new SimulatePipelineResponse("_id", expectedResponses); diff --git a/plugins/ingest/src/test/java/org/elasticsearch/ingest/PipelineTests.java b/plugins/ingest/src/test/java/org/elasticsearch/ingest/PipelineTests.java new file mode 100644 index 00000000000..84d0953495b --- /dev/null +++ b/plugins/ingest/src/test/java/org/elasticsearch/ingest/PipelineTests.java @@ -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; + +import org.elasticsearch.ingest.processor.Processor; +import org.elasticsearch.ingest.processor.mutate.MutateProcessor; +import org.elasticsearch.test.ESTestCase; +import org.junit.Before; + +import java.util.*; + +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.instanceOf; +import static org.hamcrest.Matchers.not; +import static org.mockito.Mockito.mock; + +public class PipelineTests extends ESTestCase { + private Processor updateProcessor; + private Processor lowercaseProcessor; + private Pipeline pipeline; + + @Before + public void setup() { + Map update = Collections.singletonMap("foo", 123); + List lowercase = Collections.singletonList("foo"); + updateProcessor = new MutateProcessor(update, null, null, null, null, null, null, null, null, null); + lowercaseProcessor = new MutateProcessor(null, null, null, null, null, null, null, null, null, lowercase); + pipeline = new Pipeline("id", "description", Arrays.asList(updateProcessor, lowercaseProcessor)); + } + + public void testEquals() throws Exception { + Pipeline other = new Pipeline(pipeline.getId(), pipeline.getDescription(), pipeline.getProcessors()); + assertThat(pipeline, equalTo(other)); + } + + public void testNotEqualsDiffId() throws Exception { + Pipeline other = new Pipeline(pipeline.getId() + "foo", pipeline.getDescription(), pipeline.getProcessors()); + assertThat(pipeline, not(equalTo(other))); + } + + public void testNotEqualsDiffDescription() throws Exception { + Pipeline other = new Pipeline(pipeline.getId(), pipeline.getDescription() + "foo", pipeline.getProcessors()); + assertThat(pipeline, not(equalTo(other))); + } + + public void testNotEqualsDiffProcessors() throws Exception { + Pipeline other = new Pipeline(pipeline.getId(), pipeline.getDescription() + "foo", Collections.singletonList(updateProcessor)); + assertThat(pipeline, not(equalTo(other))); + } +} diff --git a/plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/transport/simulate/ParsedSimulateRequestParserTests.java b/plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/transport/simulate/ParsedSimulateRequestParserTests.java index 03c4c3af7f8..09c145a56b6 100644 --- a/plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/transport/simulate/ParsedSimulateRequestParserTests.java +++ b/plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/transport/simulate/ParsedSimulateRequestParserTests.java @@ -34,6 +34,8 @@ import static org.hamcrest.Matchers.equalTo; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; +import static org.elasticsearch.plugin.ingest.transport.simulate.SimulatePipelineRequest.Fields; + public class ParsedSimulateRequestParserTests extends ESTestCase { private PipelineStore store; private ParsedSimulateRequest.Parser parser; @@ -43,9 +45,9 @@ public class ParsedSimulateRequestParserTests extends ESTestCase { @Before public void init() throws IOException { parser = new ParsedSimulateRequest.Parser(); - List uppercase = Collections.unmodifiableList(Collections.singletonList("foo")); + List uppercase = Collections.singletonList("foo"); Processor processor = new MutateProcessor(null, null, null, null, null, null, null, null, uppercase, null); - pipeline = new Pipeline(ParsedSimulateRequest.Parser.SIMULATED_PIPELINE_ID, null, Collections.unmodifiableList(Arrays.asList(processor))); + pipeline = new Pipeline(ParsedSimulateRequest.Parser.SIMULATED_PIPELINE_ID, null, Arrays.asList(processor)); data = new Data("_index", "_type", "_id", Collections.singletonMap("foo", "bar")); Map processorRegistry = new HashMap<>(); processorRegistry.put("mutate", new MutateProcessor.Factory()); @@ -60,12 +62,12 @@ public class ParsedSimulateRequestParserTests extends ESTestCase { Map raw = new HashMap<>(); List> docs = new ArrayList<>(); Map doc = new HashMap<>(); - doc.put("_index", "_index"); - doc.put("_type", "_type"); - doc.put("_id", "_id"); - doc.put("_source", data.getDocument()); + doc.put(Fields.INDEX, "_index"); + doc.put(Fields.TYPE, "_type"); + doc.put(Fields.ID, "_id"); + doc.put(Fields.SOURCE, data.getDocument()); docs.add(doc); - raw.put("docs", docs); + raw.put(Fields.DOCS, docs); ParsedSimulateRequest actualRequest = parser.parseWithPipelineId("_id", raw, false, store); assertThat(actualRequest, equalTo(expectedRequest)); @@ -77,10 +79,10 @@ public class ParsedSimulateRequestParserTests extends ESTestCase { Map raw = new HashMap<>(); List> docs = new ArrayList<>(); Map doc = new HashMap<>(); - doc.put("_index", "_index"); - doc.put("_type", "_type"); - doc.put("_id", "_id"); - doc.put("_source", data.getDocument()); + doc.put(Fields.INDEX, "_index"); + doc.put(Fields.TYPE, "_type"); + doc.put(Fields.ID, "_id"); + doc.put(Fields.SOURCE, data.getDocument()); docs.add(doc); Map processorConfig = new HashMap<>(); @@ -88,8 +90,8 @@ public class ParsedSimulateRequestParserTests extends ESTestCase { Map pipelineConfig = new HashMap<>(); pipelineConfig.put("processors", Collections.singletonList(Collections.singletonMap("mutate", processorConfig))); - raw.put("docs", docs); - raw.put("pipeline", pipelineConfig); + raw.put(Fields.DOCS, docs); + raw.put(Fields.PIPELINE, pipelineConfig); ParsedSimulateRequest actualRequest = parser.parse(raw, false, store); assertThat(actualRequest, equalTo(expectedRequest)); diff --git a/plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateExecutionServiceTests.java b/plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateExecutionServiceTests.java index cf108b0db84..e7754f94380 100644 --- a/plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateExecutionServiceTests.java +++ b/plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateExecutionServiceTests.java @@ -68,7 +68,7 @@ public class SimulateExecutionServiceTests extends ESTestCase { } public void testExecuteVerboseItem() throws Exception { - SimulateDocumentResult expectedItemResponse = new SimulateDocumentResult( + SimulateDocumentResult expectedItemResponse = new SimulateVerboseDocumentResult( Arrays.asList(new SimulateProcessorResult("processor[mock]-0", data), new SimulateProcessorResult("processor[mock]-1", data))); SimulateDocumentResult actualItemResponse = executionService.executeVerboseItem(pipeline, data); verify(processor, times(2)).execute(data); @@ -76,7 +76,7 @@ public class SimulateExecutionServiceTests extends ESTestCase { } public void testExecuteItem() throws Exception { - SimulateDocumentResult expectedItemResponse = new SimulateDocumentResult(data); + SimulateDocumentResult expectedItemResponse = new SimulateSimpleDocumentResult(data); SimulateDocumentResult actualItemResponse = executionService.executeItem(pipeline, data); verify(processor, times(2)).execute(data); assertThat(actualItemResponse, equalTo(expectedItemResponse)); @@ -84,7 +84,7 @@ public class SimulateExecutionServiceTests extends ESTestCase { public void testExecuteVerboseItemWithFailure() throws Exception { Exception e = new RuntimeException("processor failed"); - SimulateDocumentResult expectedItemResponse = new SimulateDocumentResult( + SimulateDocumentResult expectedItemResponse = new SimulateVerboseDocumentResult( Arrays.asList(new SimulateProcessorResult("processor[mock]-0", e), new SimulateProcessorResult("processor[mock]-1", data)) ); doThrow(e).doNothing().when(processor).execute(data); @@ -95,7 +95,7 @@ public class SimulateExecutionServiceTests extends ESTestCase { public void testExecuteItemWithFailure() throws Exception { Exception e = new RuntimeException("processor failed"); - SimulateDocumentResult expectedItemResponse = new SimulateDocumentResult(e); + SimulateDocumentResult expectedItemResponse = new SimulateFailedDocumentResult(e); doThrow(e).when(processor).execute(data); SimulateDocumentResult actualItemResponse = executionService.executeItem(pipeline, data); verify(processor, times(1)).execute(data); @@ -103,7 +103,7 @@ public class SimulateExecutionServiceTests extends ESTestCase { } public void testExecute() throws Exception { - SimulateDocumentResult itemResponse = new SimulateDocumentResult(data); + SimulateDocumentResult itemResponse = new SimulateSimpleDocumentResult(data); ParsedSimulateRequest request = new ParsedSimulateRequest(pipeline, Collections.singletonList(data), false); executionService.execute(request, listener); SimulatePipelineResponse response = new SimulatePipelineResponse("_id", Collections.singletonList(itemResponse)); @@ -118,7 +118,7 @@ public class SimulateExecutionServiceTests extends ESTestCase { public void testExecuteWithVerbose() throws Exception { ParsedSimulateRequest request = new ParsedSimulateRequest(pipeline, Collections.singletonList(data), true); - SimulateDocumentResult itemResponse = new SimulateDocumentResult( + SimulateDocumentResult itemResponse = new SimulateVerboseDocumentResult( Arrays.asList(new SimulateProcessorResult("processor[mock]-0", data), new SimulateProcessorResult("processor[mock]-1", data))); executionService.execute(request, listener); SimulatePipelineResponse response = new SimulatePipelineResponse("_id", Collections.singletonList(itemResponse)); diff --git a/plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateFailedDocumentResultTests.java b/plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateFailedDocumentResultTests.java new file mode 100644 index 00000000000..c766a8979fe --- /dev/null +++ b/plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateFailedDocumentResultTests.java @@ -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.plugin.ingest.transport.simulate; + +import org.elasticsearch.common.io.stream.BytesStreamOutput; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.test.ESTestCase; +import org.junit.Before; + +import java.io.IOException; + +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.not; + +public class SimulateFailedDocumentResultTests extends ESTestCase { + + @Before + public void setup() { + } + + public void testEqualsExact() { + Throwable throwable = new Exception("foo"); + SimulateDocumentResult result = new SimulateFailedDocumentResult(throwable); + SimulateDocumentResult otherResult = new SimulateFailedDocumentResult(throwable); + assertThat(result, equalTo(otherResult)); + } + + public void testEqualsSameExceptionClass() { + SimulateDocumentResult result = new SimulateFailedDocumentResult(new IllegalArgumentException("foo")); + SimulateDocumentResult otherResult = new SimulateFailedDocumentResult(new IllegalArgumentException("bar")); + assertThat(result, equalTo(otherResult)); + } + + public void testNotEqualsDiffExceptionClass() { + SimulateDocumentResult result = new SimulateFailedDocumentResult(new IllegalArgumentException("foo")); + SimulateDocumentResult otherResult = new SimulateFailedDocumentResult(new NullPointerException("foo")); + assertThat(result, not(equalTo(otherResult))); + } + + public void testStreamable() throws IOException { + SimulateDocumentResult result = new SimulateFailedDocumentResult(new IllegalArgumentException("foo")); + + BytesStreamOutput out = new BytesStreamOutput(); + result.writeTo(out); + + StreamInput streamInput = StreamInput.wrap(out.bytes()); + SimulateDocumentResult otherResult = new SimulateFailedDocumentResult(); + otherResult.readFrom(streamInput); + + assertThat(result, equalTo(otherResult)); + } +} diff --git a/plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineResponseTests.java b/plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineResponseTests.java new file mode 100644 index 00000000000..2352f3db0ba --- /dev/null +++ b/plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineResponseTests.java @@ -0,0 +1,80 @@ +/* + * 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.plugin.ingest.transport.simulate; + +import org.elasticsearch.common.io.stream.BytesStreamOutput; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.ingest.Data; +import org.elasticsearch.test.ESTestCase; +import org.junit.Before; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.not; + +public class SimulatePipelineResponseTests extends ESTestCase { + private Data data; + private SimulateDocumentResult documentResult; + private SimulatePipelineResponse response; + + @Before + public void setup() { + data = new Data("_index", "_type", "_id", Collections.singletonMap("foo", "bar")); + documentResult = new SimulateSimpleDocumentResult(data); + response = new SimulatePipelineResponse("_id", Collections.singletonList(documentResult)); + } + + public void testEquals() { + SimulatePipelineResponse otherResponse = new SimulatePipelineResponse("_id", Collections.singletonList(documentResult)); + assertThat(response, equalTo(otherResponse)); + } + + public void testNotEqualsId() { + SimulatePipelineResponse otherResponse = new SimulatePipelineResponse(response.getPipelineId() + "foo", response.getResults()); + assertThat(response, not(equalTo(otherResponse))); + } + + public void testNotEqualsResults() { + SimulatePipelineResponse otherResponse = new SimulatePipelineResponse(response.getPipelineId(), Arrays.asList(documentResult, documentResult)); + assertThat(response, not(equalTo(otherResponse))); + } + + public void testStreamable() throws IOException { + List results = Arrays.asList( + new SimulateSimpleDocumentResult(data), + new SimulateFailedDocumentResult(new IllegalArgumentException("foo")), + new SimulateVerboseDocumentResult(Collections.singletonList(new SimulateProcessorResult("pid", data))) + ); + + response = new SimulatePipelineResponse("_id", results); + BytesStreamOutput out = new BytesStreamOutput(); + response.writeTo(out); + + StreamInput streamInput = StreamInput.wrap(out.bytes()); + SimulatePipelineResponse otherResponse = new SimulatePipelineResponse(); + otherResponse.readFrom(streamInput); + + assertThat(response, equalTo(otherResponse)); + } +} diff --git a/plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateProcessorResultTests.java b/plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateProcessorResultTests.java new file mode 100644 index 00000000000..1a20da74416 --- /dev/null +++ b/plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateProcessorResultTests.java @@ -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.plugin.ingest.transport.simulate; + +import org.elasticsearch.common.io.stream.BytesStreamOutput; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.ingest.Data; +import org.elasticsearch.test.ESTestCase; +import org.junit.Before; + +import java.io.IOException; +import java.util.Collections; + +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.not; + +public class SimulateProcessorResultTests extends ESTestCase { + private Data data; + private SimulateProcessorResult result; + private SimulateProcessorResult failedResult; + private String processorId; + private Throwable throwable; + + @Before + public void setup() { + data = new Data("_index", "_type", "_id", Collections.singletonMap("foo", "bar")); + processorId = "id"; + throwable = new IllegalArgumentException("foo"); + result = new SimulateProcessorResult(processorId, data); + failedResult = new SimulateProcessorResult(processorId, throwable); + } + + public void testEqualsData() { + SimulateProcessorResult otherResult = new SimulateProcessorResult(new String(processorId), new Data(data)); + assertThat(result, equalTo(otherResult)); + } + + public void testEqualsSameClassThrowable() { + SimulateProcessorResult otherFailedResult = new SimulateProcessorResult(new String(processorId), new IllegalArgumentException("foo")); + assertThat(failedResult, equalTo(otherFailedResult)); + } + + public void testNotEqualsThrowable() { + SimulateProcessorResult otherFailedResult = new SimulateProcessorResult(new String(processorId), new NullPointerException("foo")); + assertThat(failedResult, not(equalTo(otherFailedResult))); + } + + public void testStreamableWithThrowable() throws IOException { + BytesStreamOutput out = new BytesStreamOutput(); + failedResult.writeTo(out); + + StreamInput streamInput = StreamInput.wrap(out.bytes()); + SimulateProcessorResult otherFailedResult = new SimulateProcessorResult(); + otherFailedResult.readFrom(streamInput); + + assertThat(failedResult, equalTo(otherFailedResult)); + } + + public void testStreamableWithData() throws IOException { + BytesStreamOutput out = new BytesStreamOutput(); + result.writeTo(out); + + StreamInput streamInput = StreamInput.wrap(out.bytes()); + SimulateProcessorResult otherResult = new SimulateProcessorResult(); + otherResult.readFrom(streamInput); + + assertThat(result, equalTo(otherResult)); + + } +} diff --git a/plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateSimpleDocumentResultTests.java b/plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateSimpleDocumentResultTests.java new file mode 100644 index 00000000000..7c7a497bbcc --- /dev/null +++ b/plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateSimpleDocumentResultTests.java @@ -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.plugin.ingest.transport.simulate; + +import org.elasticsearch.common.io.stream.BytesStreamOutput; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.ingest.Data; +import org.elasticsearch.test.ESTestCase; +import org.junit.Before; + +import java.io.IOException; +import java.util.Collections; + +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.not; + +public class SimulateSimpleDocumentResultTests extends ESTestCase { + private Data data; + + @Before + public void setup() { + data = new Data("_index", "_type", "_id", Collections.singletonMap("foo", "bar")); + } + + public void testEquals() { + SimulateDocumentResult result = new SimulateSimpleDocumentResult(data); + SimulateDocumentResult otherResult = new SimulateSimpleDocumentResult(data); + assertThat(result, equalTo(otherResult)); + } + + public void testNotEqualsDiffData() { + Data otherData = new Data(data.getIndex() + "foo", data.getType(), data.getId(), data.getDocument()); + SimulateDocumentResult result = new SimulateSimpleDocumentResult(data); + SimulateDocumentResult otherResult = new SimulateSimpleDocumentResult(otherData); + assertThat(result, not(equalTo(otherResult))); + } + + public void testStreamable() throws IOException { + SimulateDocumentResult result = new SimulateSimpleDocumentResult(data); + + BytesStreamOutput out = new BytesStreamOutput(); + result.writeTo(out); + + StreamInput streamInput = StreamInput.wrap(out.bytes()); + SimulateDocumentResult otherResult = new SimulateSimpleDocumentResult(); + otherResult.readFrom(streamInput); + + assertThat(result, equalTo(otherResult)); + } +} diff --git a/plugins/ingest/src/test/resources/rest-api-spec/test/ingest/80_simulate.yaml b/plugins/ingest/src/test/resources/rest-api-spec/test/ingest/80_simulate.yaml index 156ba8cfd67..371aeedd3ef 100644 --- a/plugins/ingest/src/test/resources/rest-api-spec/test/ingest/80_simulate.yaml +++ b/plugins/ingest/src/test/resources/rest-api-spec/test/ingest/80_simulate.yaml @@ -87,6 +87,33 @@ } - length: { docs: 1 } +--- +"Test simulate with no provided pipeline or pipeline_id": + - do: + cluster.health: + wait_for_status: green + + - do: + catch: request + ingest.simulate: + body: > + { + "docs": [ + { + "_index": "index", + "_type": "type", + "_id": "id", + "_source": { + "foo": "bar" + } + } + ] + } + - length: { error: 3 } + - match: { status: 400 } + - match: { error.type: "illegal_argument_exception" } + - match: { error.reason: "required property [pipeline] is missing" } + --- "Test simulate with verbose flag": - do: From d093600729e4669a2ee44c020e841f6c9113a721 Mon Sep 17 00:00:00 2001 From: javanna Date: Fri, 13 Nov 2015 16:22:24 +0100 Subject: [PATCH 11/14] simplify serialization for simulate response depending on verbose flag Removed equals and hashcode whenever they wouldn't be reliable because of exception comparison. at the end of the day we use them for testing and we can simplify our tests without requiring equals and hashcode in prod code, which also would require more tests if maintained. Add equals/hashcode test for Data/TransportData and randomize existing serialization tests --- .../ingest/transport/TransportData.java | 17 +-- .../simulate/SimulateDocumentResult.java | 18 +-- .../SimulateDocumentSimpleResult.java | 95 +++++++++++++++ ...ava => SimulateDocumentVerboseResult.java} | 49 +++----- .../simulate/SimulateExecutionService.java | 12 +- .../SimulateFailedDocumentResult.java | 84 ------------- .../simulate/SimulatePipelineResponse.java | 53 ++------ .../simulate/SimulateProcessorResult.java | 83 ++++++------- .../SimulateSimpleDocumentResult.java | 84 ------------- .../elasticsearch/ingest/IngestClientIT.java | 29 +++-- .../ingest/transport/TransportDataTests.java | 80 ++++++++++-- .../SimulateDocumentSimpleResultTests.java | 58 +++++++++ .../SimulateExecutionServiceTests.java | 83 ++++++------- .../SimulateFailedDocumentResultTests.java | 69 ----------- .../SimulatePipelineResponseTests.java | 114 ++++++++++++------ .../SimulateProcessorResultTests.java | 70 ++++------- .../SimulateSimpleDocumentResultTests.java | 67 ---------- 17 files changed, 455 insertions(+), 610 deletions(-) create mode 100644 plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateDocumentSimpleResult.java rename plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/{SimulateVerboseDocumentResult.java => SimulateDocumentVerboseResult.java} (62%) delete mode 100644 plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateFailedDocumentResult.java delete mode 100644 plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateSimpleDocumentResult.java create mode 100644 plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateDocumentSimpleResultTests.java delete mode 100644 plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateFailedDocumentResultTests.java delete mode 100644 plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateSimpleDocumentResultTests.java diff --git a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/TransportData.java b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/TransportData.java index b9fa46fe939..d3753a99306 100644 --- a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/TransportData.java +++ b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/TransportData.java @@ -21,7 +21,7 @@ package org.elasticsearch.plugin.ingest.transport; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; -import org.elasticsearch.common.io.stream.Streamable; +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; @@ -31,12 +31,11 @@ import java.io.IOException; import java.util.Map; import java.util.Objects; -public class TransportData implements Streamable, ToXContent { - private Data data; +public class TransportData implements Writeable, ToXContent { - public TransportData() { + private static final TransportData PROTOTYPE = new TransportData(null); - } + private final Data data; public TransportData(Data data) { this.data = data; @@ -46,13 +45,17 @@ public class TransportData implements Streamable, ToXContent { return data; } + public static TransportData readTransportDataFrom(StreamInput in) throws IOException { + return PROTOTYPE.readFrom(in); + } + @Override - public void readFrom(StreamInput in) throws IOException { + public TransportData readFrom(StreamInput in) throws IOException { String index = in.readString(); String type = in.readString(); String id = in.readString(); Map doc = in.readMap(); - this.data = new Data(index, type, id, doc); + return new TransportData(new Data(index, type, id, doc)); } @Override diff --git a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateDocumentResult.java b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateDocumentResult.java index 98fdf380bc7..ff9ad829aad 100644 --- a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateDocumentResult.java +++ b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateDocumentResult.java @@ -18,23 +18,9 @@ */ package org.elasticsearch.plugin.ingest.transport.simulate; -import org.elasticsearch.common.io.stream.StreamInput; -import org.elasticsearch.common.io.stream.StreamOutput; -import org.elasticsearch.common.io.stream.Streamable; +import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.common.xcontent.ToXContent; -import org.elasticsearch.common.xcontent.XContentBuilder; -import java.io.IOException; +public interface SimulateDocumentResult extends Writeable, ToXContent { -public abstract class SimulateDocumentResult implements Streamable, ToXContent { - - public int getStreamId() { - return -1; - } - - public abstract void readFrom(StreamInput in) throws IOException; - - public abstract void writeTo(StreamOutput out) throws IOException; - - public abstract XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException; } diff --git a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateDocumentSimpleResult.java b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateDocumentSimpleResult.java new file mode 100644 index 00000000000..1783b10f998 --- /dev/null +++ b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateDocumentSimpleResult.java @@ -0,0 +1,95 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.elasticsearch.plugin.ingest.transport.simulate; + +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.Data; +import org.elasticsearch.plugin.ingest.transport.TransportData; + +import java.io.IOException; + +public class SimulateDocumentSimpleResult implements SimulateDocumentResult { + + private static final SimulateDocumentSimpleResult PROTOTYPE = new SimulateDocumentSimpleResult((Data)null); + + private TransportData data; + private Exception failure; + + public SimulateDocumentSimpleResult(Data data) { + this.data = new TransportData(data); + } + + private SimulateDocumentSimpleResult(TransportData data) { + this.data = data; + } + + public SimulateDocumentSimpleResult(Exception failure) { + this.failure = failure; + } + + public Data getData() { + if (data == null) { + return null; + } + return data.get(); + } + + public Exception getFailure() { + return failure; + } + + public static SimulateDocumentSimpleResult readSimulateDocumentSimpleResult(StreamInput in) throws IOException { + return PROTOTYPE.readFrom(in); + } + + @Override + public SimulateDocumentSimpleResult readFrom(StreamInput in) throws IOException { + if (in.readBoolean()) { + Exception exception = in.readThrowable(); + return new SimulateDocumentSimpleResult(exception); + } + return new SimulateDocumentSimpleResult(TransportData.readTransportDataFrom(in)); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + if (failure == null) { + out.writeBoolean(false); + data.writeTo(out); + } else { + out.writeBoolean(true); + out.writeThrowable(failure); + } + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject(); + if (failure == null) { + data.toXContent(builder, params); + } else { + ElasticsearchException.renderThrowable(builder, params, failure); + } + builder.endObject(); + return builder; + } +} diff --git a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateVerboseDocumentResult.java b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateDocumentVerboseResult.java similarity index 62% rename from plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateVerboseDocumentResult.java rename to plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateDocumentVerboseResult.java index f3da751e78b..eac308d9f35 100644 --- a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateVerboseDocumentResult.java +++ b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateDocumentVerboseResult.java @@ -25,45 +25,39 @@ import org.elasticsearch.common.xcontent.XContentBuilderString; import java.io.IOException; import java.util.ArrayList; +import java.util.Collections; import java.util.List; -import java.util.Objects; -public class SimulateVerboseDocumentResult extends SimulateDocumentResult { - public static final int STREAM_ID = 1; +public class SimulateDocumentVerboseResult implements SimulateDocumentResult { - private List processorResults; + private static final SimulateDocumentVerboseResult PROTOTYPE = new SimulateDocumentVerboseResult(Collections.emptyList()); - public SimulateVerboseDocumentResult() { + private final List processorResults; - } - - public SimulateVerboseDocumentResult(List processorResults) { + public SimulateDocumentVerboseResult(List processorResults) { this.processorResults = processorResults; } - @Override - public int getStreamId() { - return STREAM_ID; + public List getProcessorResults() { + return processorResults; + } + + public static SimulateDocumentVerboseResult readSimulateDocumentVerboseResultFrom(StreamInput in) throws IOException { + return PROTOTYPE.readFrom(in); } @Override - public void readFrom(StreamInput in) throws IOException { - int streamId = in.readVInt(); - if (streamId != STREAM_ID) { - throw new IOException("stream_id [" + streamId + "] does not match " + getClass().getName() + " [stream_id=" + STREAM_ID + "]"); - } + public SimulateDocumentVerboseResult readFrom(StreamInput in) throws IOException { int size = in.readVInt(); - processorResults = new ArrayList<>(); + List processorResults = new ArrayList<>(); for (int i = 0; i < size; i++) { - SimulateProcessorResult processorResult = new SimulateProcessorResult(); - processorResult.readFrom(in); - processorResults.add(processorResult); + processorResults.add(SimulateProcessorResult.readSimulateProcessorResultFrom(in)); } + return new SimulateDocumentVerboseResult(processorResults); } @Override public void writeTo(StreamOutput out) throws IOException { - out.writeVInt(STREAM_ID); out.writeVInt(processorResults.size()); for (SimulateProcessorResult result : processorResults) { result.writeTo(out); @@ -82,19 +76,6 @@ public class SimulateVerboseDocumentResult extends SimulateDocumentResult { return builder; } - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - SimulateVerboseDocumentResult that = (SimulateVerboseDocumentResult) o; - return Objects.equals(processorResults, that.processorResults); - } - - @Override - public int hashCode() { - return Objects.hash(processorResults); - } - static final class Fields { static final XContentBuilderString PROCESSOR_RESULTS = new XContentBuilderString("processor_results"); } diff --git a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateExecutionService.java b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateExecutionService.java index a8d34b4a150..e71b20d66b1 100644 --- a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateExecutionService.java +++ b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateExecutionService.java @@ -40,18 +40,16 @@ public class SimulateExecutionService { this.threadPool = threadPool; } - SimulateDocumentResult executeItem(Pipeline pipeline, Data data) { try { pipeline.execute(data); - return new SimulateSimpleDocumentResult(data); + return new SimulateDocumentSimpleResult(data); } catch (Exception e) { - return new SimulateFailedDocumentResult(e); + return new SimulateDocumentSimpleResult(e); } - } - SimulateDocumentResult executeVerboseItem(Pipeline pipeline, Data data) { + SimulateDocumentVerboseResult executeVerboseItem(Pipeline pipeline, Data data) { List processorResultList = new ArrayList<>(); Data currentData = new Data(data); for (int i = 0; i < pipeline.getProcessors().size(); i++) { @@ -67,7 +65,7 @@ public class SimulateExecutionService { currentData = new Data(currentData); } - return new SimulateVerboseDocumentResult(processorResultList); + return new SimulateDocumentVerboseResult(processorResultList); } public void execute(ParsedSimulateRequest request, ActionListener listener) { @@ -82,7 +80,7 @@ public class SimulateExecutionService { responses.add(executeItem(request.getPipeline(), data)); } } - listener.onResponse(new SimulatePipelineResponse(request.getPipeline().getId(), responses)); + listener.onResponse(new SimulatePipelineResponse(request.getPipeline().getId(), request.isVerbose(), responses)); } }); } diff --git a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateFailedDocumentResult.java b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateFailedDocumentResult.java deleted file mode 100644 index e0bf7555b6b..00000000000 --- a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateFailedDocumentResult.java +++ /dev/null @@ -1,84 +0,0 @@ -/* - * 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.plugin.ingest.transport.simulate; - -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 java.io.IOException; -import java.util.Objects; - -public class SimulateFailedDocumentResult extends SimulateDocumentResult { - public static final int STREAM_ID = 2; - - private Throwable failure; - - public SimulateFailedDocumentResult() { - - } - - public SimulateFailedDocumentResult(Throwable failure) { - this.failure = failure; - } - - @Override - public int getStreamId() { - return STREAM_ID; - } - - @Override - public void readFrom(StreamInput in) throws IOException { - int streamId = in.readVInt(); - if (streamId != STREAM_ID) { - throw new IOException("stream_id [" + streamId + "] does not match " + getClass().getName() + " [stream_id=" + STREAM_ID + "]"); - } - this.failure = in.readThrowable(); - } - - @Override - public void writeTo(StreamOutput out) throws IOException { - out.writeVInt(STREAM_ID); - out.writeThrowable(failure); - } - - @Override - public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { - builder.startObject(); - ElasticsearchException.renderThrowable(builder, params, failure); - builder.endObject(); - return builder; - } - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - SimulateFailedDocumentResult that = (SimulateFailedDocumentResult) o; - - return Objects.equals((failure == null) ? null : failure.getClass(), - (that.failure == null) ? null : that.failure.getClass()); - } - - @Override - public int hashCode() { - return Objects.hash(failure); - } -} diff --git a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineResponse.java b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineResponse.java index 5922f9d906e..097595f3a32 100644 --- a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineResponse.java +++ b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineResponse.java @@ -30,18 +30,19 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Collections; import java.util.List; -import java.util.Objects; public class SimulatePipelineResponse extends ActionResponse implements ToXContent { private String pipelineId; + private boolean verbose; private List results; public SimulatePipelineResponse() { } - public SimulatePipelineResponse(String pipelineId, List responses) { + public SimulatePipelineResponse(String pipelineId, boolean verbose, List responses) { this.pipelineId = pipelineId; + this.verbose = verbose; this.results = Collections.unmodifiableList(responses); } @@ -49,25 +50,21 @@ public class SimulatePipelineResponse extends ActionResponse implements ToXConte return pipelineId; } - public void setPipelineId(String pipelineId) { - this.pipelineId = pipelineId; - } - public List getResults() { return results; } - public void setResults(List results) { - this.results = 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) { - out.writeVInt(response.getStreamId()); response.writeTo(out); } } @@ -76,27 +73,18 @@ public class SimulatePipelineResponse extends ActionResponse implements ToXConte 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 result; - switch (in.readVInt()) { - case SimulateSimpleDocumentResult.STREAM_ID: - result = new SimulateSimpleDocumentResult(); - break; - case SimulateVerboseDocumentResult.STREAM_ID: - result = new SimulateVerboseDocumentResult(); - break; - case SimulateFailedDocumentResult.STREAM_ID: - result = new SimulateFailedDocumentResult(); - break; - default: - throw new IOException("Cannot read result from stream"); + SimulateDocumentResult simulateDocumentResult; + if (verbose) { + simulateDocumentResult = SimulateDocumentVerboseResult.readSimulateDocumentVerboseResultFrom(in); + } else { + simulateDocumentResult = SimulateDocumentSimpleResult.readSimulateDocumentSimpleResult(in); } - result.readFrom(in); - results.add(result); + results.add(simulateDocumentResult); } - } @Override @@ -106,24 +94,9 @@ public class SimulatePipelineResponse extends ActionResponse implements ToXConte response.toXContent(builder, params); } builder.endArray(); - return builder; } - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - SimulatePipelineResponse that = (SimulatePipelineResponse) o; - return Objects.equals(pipelineId, that.pipelineId) && - Objects.equals(results, that.results); - } - - @Override - public int hashCode() { - return Objects.hash(pipelineId, results); - } - static final class Fields { static final XContentBuilderString DOCUMENTS = new XContentBuilderString("docs"); } diff --git a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateProcessorResult.java b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateProcessorResult.java index 7b196f69276..138f1ae553c 100644 --- a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateProcessorResult.java +++ b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateProcessorResult.java @@ -21,7 +21,7 @@ package org.elasticsearch.plugin.ingest.transport.simulate; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; -import org.elasticsearch.common.io.stream.Streamable; +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; @@ -29,33 +29,34 @@ import org.elasticsearch.ingest.Data; import org.elasticsearch.plugin.ingest.transport.TransportData; import java.io.IOException; -import java.util.Objects; -public class SimulateProcessorResult implements Streamable, ToXContent { +public class SimulateProcessorResult implements Writeable, ToXContent { + + private static final SimulateProcessorResult PROTOTYPE = new SimulateProcessorResult(null, (Data)null); private String processorId; private TransportData data; - private Throwable failure; - - public SimulateProcessorResult() { - - } + private Exception failure; public SimulateProcessorResult(String processorId, Data data) { this.processorId = processorId; this.data = new TransportData(data); } - public SimulateProcessorResult(String processorId, Throwable failure) { + private SimulateProcessorResult(String processorId, TransportData data) { + this.processorId = processorId; + this.data = data; + } + + public SimulateProcessorResult(String processorId, Exception failure) { this.processorId = processorId; this.failure = failure; } - private boolean isFailed() { - return this.failure != null; - } - public Data getData() { + if (data == null) { + return null; + } return data.get(); } @@ -63,26 +64,33 @@ public class SimulateProcessorResult implements Streamable, ToXContent { return processorId; } + public Exception getFailure() { + return failure; + } + + public static SimulateProcessorResult readSimulateProcessorResultFrom(StreamInput in) throws IOException { + return PROTOTYPE.readFrom(in); + } + @Override - public void readFrom(StreamInput in) throws IOException { - boolean isFailure = in.readBoolean(); - this.processorId = in.readString(); - if (isFailure) { - this.failure = in.readThrowable(); - } else { - this.data = new TransportData(); - this.data.readFrom(in); + public SimulateProcessorResult readFrom(StreamInput in) throws IOException { + String processorId = in.readString(); + if (in.readBoolean()) { + Exception exception = in.readThrowable(); + return new SimulateProcessorResult(processorId, exception); } + return new SimulateProcessorResult(processorId, TransportData.readTransportDataFrom(in)); } @Override public void writeTo(StreamOutput out) throws IOException { - out.writeBoolean(isFailed()); out.writeString(processorId); - if (isFailed()) { - out.writeThrowable(failure); - } else { + if (failure == null) { + out.writeBoolean(false); data.writeTo(out); + } else { + out.writeBoolean(true); + out.writeThrowable(failure); } } @@ -90,34 +98,15 @@ public class SimulateProcessorResult implements Streamable, ToXContent { public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { builder.startObject(); builder.field(Fields.PROCESSOR_ID, processorId); - if (isFailed()) { - ElasticsearchException.renderThrowable(builder, params, failure); - } else { + if (failure == null) { data.toXContent(builder, params); + } else { + ElasticsearchException.renderThrowable(builder, params, failure); } builder.endObject(); return builder; } - @Override - public boolean equals(Object obj) { - if (obj == this) { - return true; - } - if (obj == null || getClass() != obj.getClass()) { - return false; - } - SimulateProcessorResult other = (SimulateProcessorResult) obj; - - return Objects.equals(processorId, other.processorId) && Objects.equals(data, other.data) && - Objects.equals((failure == null) ? null : failure.getClass(), (other.failure == null) ? null : other.failure.getClass()); - } - - @Override - public int hashCode() { - return Objects.hash(processorId, data, failure); - } - static final class Fields { static final XContentBuilderString PROCESSOR_ID = new XContentBuilderString("processor_id"); } diff --git a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateSimpleDocumentResult.java b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateSimpleDocumentResult.java deleted file mode 100644 index 9d432bc2855..00000000000 --- a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateSimpleDocumentResult.java +++ /dev/null @@ -1,84 +0,0 @@ -/* - * 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.plugin.ingest.transport.simulate; - -import org.elasticsearch.common.io.stream.StreamInput; -import org.elasticsearch.common.io.stream.StreamOutput; -import org.elasticsearch.common.xcontent.XContentBuilder; -import org.elasticsearch.ingest.Data; -import org.elasticsearch.plugin.ingest.transport.TransportData; - -import java.io.IOException; -import java.util.Objects; - -public class SimulateSimpleDocumentResult extends SimulateDocumentResult { - public static final int STREAM_ID = 0; - - private TransportData transportData; - - public SimulateSimpleDocumentResult() { - - } - - public SimulateSimpleDocumentResult(Data data) { - this.transportData = new TransportData(data); - } - - @Override - public int getStreamId() { - return STREAM_ID; - } - - @Override - public void readFrom(StreamInput in) throws IOException { - int streamId = in.readVInt(); - if (streamId != STREAM_ID) { - throw new IOException("stream_id [" + streamId + "] does not match " + getClass().getName() + " [stream_id=" + STREAM_ID + "]"); - } - this.transportData = new TransportData(); - this.transportData.readFrom(in); - } - - @Override - public void writeTo(StreamOutput out) throws IOException { - out.writeVInt(STREAM_ID); - transportData.writeTo(out); - } - - @Override - public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { - builder.startObject(); - transportData.toXContent(builder, params); - builder.endObject(); - return builder; - } - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - SimulateSimpleDocumentResult that = (SimulateSimpleDocumentResult) o; - return Objects.equals(transportData, that.transportData); - } - - @Override - public int hashCode() { - return Objects.hash(transportData); - } -} diff --git a/plugins/ingest/src/test/java/org/elasticsearch/ingest/IngestClientIT.java b/plugins/ingest/src/test/java/org/elasticsearch/ingest/IngestClientIT.java index 2001b673d06..9b61ffa4fe3 100644 --- a/plugins/ingest/src/test/java/org/elasticsearch/ingest/IngestClientIT.java +++ b/plugins/ingest/src/test/java/org/elasticsearch/ingest/IngestClientIT.java @@ -30,16 +30,20 @@ import org.elasticsearch.plugin.ingest.transport.get.GetPipelineRequestBuilder; import org.elasticsearch.plugin.ingest.transport.get.GetPipelineResponse; import org.elasticsearch.plugin.ingest.transport.put.PutPipelineAction; import org.elasticsearch.plugin.ingest.transport.put.PutPipelineRequestBuilder; -import org.elasticsearch.plugin.ingest.transport.simulate.*; +import org.elasticsearch.plugin.ingest.transport.simulate.SimulateDocumentSimpleResult; +import org.elasticsearch.plugin.ingest.transport.simulate.SimulatePipelineAction; +import org.elasticsearch.plugin.ingest.transport.simulate.SimulatePipelineRequestBuilder; +import org.elasticsearch.plugin.ingest.transport.simulate.SimulatePipelineResponse; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.ESIntegTestCase; -import java.util.*; +import java.util.Collection; +import java.util.Collections; +import java.util.Map; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; -import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.nullValue; +import static org.hamcrest.Matchers.*; import static org.hamcrest.core.Is.is; import static org.hamcrest.core.IsNull.notNullValue; @@ -53,7 +57,6 @@ public class IngestClientIT extends ESIntegTestCase { @Override protected Collection> transportClientPlugins() { return nodePlugins(); - } public void testSimulate() throws Exception { @@ -98,14 +101,14 @@ public class IngestClientIT extends ESIntegTestCase { .endObject().bytes()) .get(); - Map expectedDoc = new HashMap<>(); - expectedDoc.put("foo", "bar"); - Data expectedData = new Data("index", "type", "id", expectedDoc); - SimulateDocumentResult expectedResponse = new SimulateSimpleDocumentResult(expectedData); - List expectedResponses = Arrays.asList(expectedResponse); - SimulatePipelineResponse expected = new SimulatePipelineResponse("_id", expectedResponses); - - assertThat(response, equalTo(expected)); + assertThat(response.isVerbose(), equalTo(false)); + assertThat(response.getPipelineId(), equalTo("_id")); + assertThat(response.getResults().size(), equalTo(1)); + assertThat(response.getResults().get(0), instanceOf(SimulateDocumentSimpleResult.class)); + SimulateDocumentSimpleResult simulateDocumentSimpleResult = (SimulateDocumentSimpleResult) response.getResults().get(0); + Data expectedData = new Data("index", "type", "id", Collections.singletonMap("foo", "bar")); + assertThat(simulateDocumentSimpleResult.getData(), equalTo(expectedData)); + assertThat(simulateDocumentSimpleResult.getFailure(), nullValue()); } public void test() throws Exception { diff --git a/plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/transport/TransportDataTests.java b/plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/transport/TransportDataTests.java index 08940ee3ce2..89ef7731327 100644 --- a/plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/transport/TransportDataTests.java +++ b/plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/transport/TransportDataTests.java @@ -19,25 +19,89 @@ package org.elasticsearch.plugin.ingest.transport; +import org.elasticsearch.common.io.stream.BytesStreamOutput; +import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.ingest.Data; import org.elasticsearch.test.ESTestCase; +import java.io.IOException; import java.util.Collections; +import java.util.Map; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.not; public class TransportDataTests extends ESTestCase { - public void testEquals() throws Exception { - Data data = new Data("_index", "_type", "_id", Collections.singletonMap("foo", "bar")); - Data otherData = new Data("_index", "_type", "_id", Collections.singletonMap("foo", "bar")); - assertThat(data, equalTo(otherData)); + public void testEqualsAndHashcode() throws Exception { + String index = randomAsciiOfLengthBetween(1, 10); + String type = randomAsciiOfLengthBetween(1, 10); + String id = randomAsciiOfLengthBetween(1, 10); + String fieldName = randomAsciiOfLengthBetween(1, 10); + String fieldValue = randomAsciiOfLengthBetween(1, 10); + Data data = new Data(index, type, id, Collections.singletonMap(fieldName, fieldValue)); + TransportData transportData = new TransportData(data); + + boolean changed = false; + String otherIndex; + if (randomBoolean()) { + otherIndex = randomAsciiOfLengthBetween(1, 10); + changed = true; + } else { + otherIndex = index; + } + String otherType; + if (randomBoolean()) { + otherType = randomAsciiOfLengthBetween(1, 10); + changed = true; + } else { + otherType = type; + } + String otherId; + if (randomBoolean()) { + otherId = randomAsciiOfLengthBetween(1, 10); + changed = true; + } else { + otherId = id; + } + Map document; + if (randomBoolean()) { + document = Collections.singletonMap(randomAsciiOfLengthBetween(1, 10), randomAsciiOfLengthBetween(1, 10)); + changed = true; + } else { + document = Collections.singletonMap(fieldName, fieldValue); + } + + Data otherData = new Data(otherIndex, otherType, otherId, document); + TransportData otherTransportData = new TransportData(otherData); + if (changed) { + assertThat(data, not(equalTo(otherData))); + assertThat(otherData, not(equalTo(data))); + assertThat(transportData, not(equalTo(otherTransportData))); + assertThat(otherTransportData, not(equalTo(transportData))); + } else { + assertThat(data, equalTo(otherData)); + assertThat(otherData, equalTo(data)); + assertThat(transportData, equalTo(otherTransportData)); + assertThat(otherTransportData, equalTo(transportData)); + Data thirdData = new Data(index, type, id, Collections.singletonMap(fieldName, fieldValue)); + TransportData thirdTransportData = new TransportData(thirdData); + assertThat(thirdData, equalTo(data)); + assertThat(data, equalTo(thirdData)); + assertThat(thirdTransportData, equalTo(transportData)); + assertThat(transportData, equalTo(thirdTransportData)); + } } - public void testNotEquals() throws Exception { - Data data = new Data("_index", "_type", "_id", Collections.singletonMap("foo", "bar")); - Data otherData = new Data("_index2", "_type", "_id", Collections.emptyMap()); - assertThat(data, not(equalTo(otherData))); + public void testSerialization() throws IOException { + Data data = new Data(randomAsciiOfLengthBetween(1, 10), randomAsciiOfLengthBetween(1, 10), randomAsciiOfLengthBetween(1, 10), + Collections.singletonMap(randomAsciiOfLengthBetween(1, 10), randomAsciiOfLengthBetween(1, 10))); + TransportData transportData = new TransportData(data); + + BytesStreamOutput out = new BytesStreamOutput(); + transportData.writeTo(out); + StreamInput streamInput = StreamInput.wrap(out.bytes()); + TransportData otherTransportData = TransportData.readTransportDataFrom(streamInput); + assertThat(otherTransportData, equalTo(transportData)); } } diff --git a/plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateDocumentSimpleResultTests.java b/plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateDocumentSimpleResultTests.java new file mode 100644 index 00000000000..e0b1c1c0f88 --- /dev/null +++ b/plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateDocumentSimpleResultTests.java @@ -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.plugin.ingest.transport.simulate; + +import org.elasticsearch.common.io.stream.BytesStreamOutput; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.ingest.Data; +import org.elasticsearch.test.ESTestCase; + +import java.io.IOException; +import java.util.Collections; + +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(); + SimulateDocumentSimpleResult simulateDocumentSimpleResult; + if (isFailure) { + simulateDocumentSimpleResult = new SimulateDocumentSimpleResult(new IllegalArgumentException("test")); + } else { + Data data = new Data(randomAsciiOfLengthBetween(1, 10), randomAsciiOfLengthBetween(1, 10), randomAsciiOfLengthBetween(1, 10), + Collections.singletonMap(randomAsciiOfLengthBetween(1, 10), randomAsciiOfLengthBetween(1, 10))); + simulateDocumentSimpleResult = new SimulateDocumentSimpleResult(data); + } + + BytesStreamOutput out = new BytesStreamOutput(); + simulateDocumentSimpleResult.writeTo(out); + StreamInput streamInput = StreamInput.wrap(out.bytes()); + SimulateDocumentSimpleResult otherSimulateDocumentSimpleResult = SimulateDocumentSimpleResult.readSimulateDocumentSimpleResult(streamInput); + + assertThat(otherSimulateDocumentSimpleResult.getData(), equalTo(simulateDocumentSimpleResult.getData())); + if (isFailure) { + assertThat(otherSimulateDocumentSimpleResult.getFailure(), instanceOf(IllegalArgumentException.class)); + IllegalArgumentException e = (IllegalArgumentException) otherSimulateDocumentSimpleResult.getFailure(); + assertThat(e.getMessage(), equalTo("test")); + } + } +} diff --git a/plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateExecutionServiceTests.java b/plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateExecutionServiceTests.java index e7754f94380..bd2e6397a74 100644 --- a/plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateExecutionServiceTests.java +++ b/plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateExecutionServiceTests.java @@ -19,12 +19,10 @@ package org.elasticsearch.plugin.ingest.transport.simulate; -import org.elasticsearch.action.ActionListener; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.ingest.Data; import org.elasticsearch.ingest.Pipeline; import org.elasticsearch.ingest.processor.Processor; -import org.elasticsearch.plugin.ingest.transport.TransportData; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.threadpool.ThreadPool; import org.junit.After; @@ -33,7 +31,7 @@ import org.junit.Before; import java.util.Arrays; import java.util.Collections; -import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.*; import static org.mockito.Mockito.*; public class SimulateExecutionServiceTests extends ESTestCase { @@ -43,8 +41,6 @@ public class SimulateExecutionServiceTests extends ESTestCase { private Pipeline pipeline; private Processor processor; private Data data; - private TransportData transportData; - private ActionListener listener; @Before public void setup() { @@ -58,8 +54,6 @@ public class SimulateExecutionServiceTests extends ESTestCase { when(processor.getType()).thenReturn("mock"); pipeline = new Pipeline("_id", "_description", Arrays.asList(processor, processor)); data = new Data("_index", "_type", "_id", Collections.singletonMap("foo", "bar")); - transportData = new TransportData(data); - listener = mock(ActionListener.class); } @After @@ -68,66 +62,61 @@ public class SimulateExecutionServiceTests extends ESTestCase { } public void testExecuteVerboseItem() throws Exception { - SimulateDocumentResult expectedItemResponse = new SimulateVerboseDocumentResult( - Arrays.asList(new SimulateProcessorResult("processor[mock]-0", data), new SimulateProcessorResult("processor[mock]-1", data))); SimulateDocumentResult actualItemResponse = executionService.executeVerboseItem(pipeline, data); verify(processor, times(2)).execute(data); - assertThat(actualItemResponse, equalTo(expectedItemResponse)); + assertThat(actualItemResponse, instanceOf(SimulateDocumentVerboseResult.class)); + SimulateDocumentVerboseResult simulateDocumentVerboseResult = (SimulateDocumentVerboseResult) actualItemResponse; + assertThat(simulateDocumentVerboseResult.getProcessorResults().size(), equalTo(2)); + assertThat(simulateDocumentVerboseResult.getProcessorResults().get(0).getProcessorId(), equalTo("processor[mock]-0")); + assertThat(simulateDocumentVerboseResult.getProcessorResults().get(0).getData(), not(sameInstance(data))); + assertThat(simulateDocumentVerboseResult.getProcessorResults().get(0).getData(), equalTo(data)); + assertThat(simulateDocumentVerboseResult.getProcessorResults().get(0).getFailure(), nullValue()); + assertThat(simulateDocumentVerboseResult.getProcessorResults().get(1).getProcessorId(), equalTo("processor[mock]-1")); + assertThat(simulateDocumentVerboseResult.getProcessorResults().get(1).getData(), not(sameInstance(data))); + assertThat(simulateDocumentVerboseResult.getProcessorResults().get(1).getData(), equalTo(data)); + assertThat(simulateDocumentVerboseResult.getProcessorResults().get(1).getFailure(), nullValue()); } public void testExecuteItem() throws Exception { - SimulateDocumentResult expectedItemResponse = new SimulateSimpleDocumentResult(data); SimulateDocumentResult actualItemResponse = executionService.executeItem(pipeline, data); verify(processor, times(2)).execute(data); - assertThat(actualItemResponse, equalTo(expectedItemResponse)); + assertThat(actualItemResponse, instanceOf(SimulateDocumentSimpleResult.class)); + SimulateDocumentSimpleResult simulateDocumentSimpleResult = (SimulateDocumentSimpleResult) actualItemResponse; + assertThat(simulateDocumentSimpleResult.getData(), equalTo(data)); + assertThat(simulateDocumentSimpleResult.getFailure(), nullValue()); } public void testExecuteVerboseItemWithFailure() throws Exception { Exception e = new RuntimeException("processor failed"); - SimulateDocumentResult expectedItemResponse = new SimulateVerboseDocumentResult( - Arrays.asList(new SimulateProcessorResult("processor[mock]-0", e), new SimulateProcessorResult("processor[mock]-1", data)) - ); doThrow(e).doNothing().when(processor).execute(data); SimulateDocumentResult actualItemResponse = executionService.executeVerboseItem(pipeline, data); verify(processor, times(2)).execute(data); - assertThat(actualItemResponse, equalTo(expectedItemResponse)); + assertThat(actualItemResponse, instanceOf(SimulateDocumentVerboseResult.class)); + SimulateDocumentVerboseResult simulateDocumentVerboseResult = (SimulateDocumentVerboseResult) actualItemResponse; + assertThat(simulateDocumentVerboseResult.getProcessorResults().size(), equalTo(2)); + assertThat(simulateDocumentVerboseResult.getProcessorResults().get(0).getProcessorId(), equalTo("processor[mock]-0")); + assertThat(simulateDocumentVerboseResult.getProcessorResults().get(0).getData(), 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).getProcessorId(), equalTo("processor[mock]-1")); + assertThat(simulateDocumentVerboseResult.getProcessorResults().get(1).getData(), not(sameInstance(data))); + assertThat(simulateDocumentVerboseResult.getProcessorResults().get(1).getData(), equalTo(data)); + assertThat(simulateDocumentVerboseResult.getProcessorResults().get(1).getFailure(), nullValue()); + runtimeException = (RuntimeException) simulateDocumentVerboseResult.getProcessorResults().get(0).getFailure(); + assertThat(runtimeException.getMessage(), equalTo("processor failed")); } public void testExecuteItemWithFailure() throws Exception { Exception e = new RuntimeException("processor failed"); - SimulateDocumentResult expectedItemResponse = new SimulateFailedDocumentResult(e); doThrow(e).when(processor).execute(data); SimulateDocumentResult actualItemResponse = executionService.executeItem(pipeline, data); verify(processor, times(1)).execute(data); - assertThat(actualItemResponse, equalTo(expectedItemResponse)); - } - - public void testExecute() throws Exception { - SimulateDocumentResult itemResponse = new SimulateSimpleDocumentResult(data); - ParsedSimulateRequest request = new ParsedSimulateRequest(pipeline, Collections.singletonList(data), false); - executionService.execute(request, listener); - SimulatePipelineResponse response = new SimulatePipelineResponse("_id", Collections.singletonList(itemResponse)); - assertBusy(new Runnable() { - @Override - public void run() { - verify(processor, times(2)).execute(data); - verify(listener).onResponse(response); - } - }); - } - - public void testExecuteWithVerbose() throws Exception { - ParsedSimulateRequest request = new ParsedSimulateRequest(pipeline, Collections.singletonList(data), true); - SimulateDocumentResult itemResponse = new SimulateVerboseDocumentResult( - Arrays.asList(new SimulateProcessorResult("processor[mock]-0", data), new SimulateProcessorResult("processor[mock]-1", data))); - executionService.execute(request, listener); - SimulatePipelineResponse response = new SimulatePipelineResponse("_id", Collections.singletonList(itemResponse)); - assertBusy(new Runnable() { - @Override - public void run() { - verify(processor, times(2)).execute(data); - verify(listener).onResponse(response); - } - }); + assertThat(actualItemResponse, instanceOf(SimulateDocumentSimpleResult.class)); + SimulateDocumentSimpleResult simulateDocumentSimpleResult = (SimulateDocumentSimpleResult) actualItemResponse; + assertThat(simulateDocumentSimpleResult.getData(), nullValue()); + assertThat(simulateDocumentSimpleResult.getFailure(), instanceOf(RuntimeException.class)); + RuntimeException runtimeException = (RuntimeException) simulateDocumentSimpleResult.getFailure(); + assertThat(runtimeException.getMessage(), equalTo("processor failed")); } } diff --git a/plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateFailedDocumentResultTests.java b/plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateFailedDocumentResultTests.java deleted file mode 100644 index c766a8979fe..00000000000 --- a/plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateFailedDocumentResultTests.java +++ /dev/null @@ -1,69 +0,0 @@ -/* - * 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.plugin.ingest.transport.simulate; - -import org.elasticsearch.common.io.stream.BytesStreamOutput; -import org.elasticsearch.common.io.stream.StreamInput; -import org.elasticsearch.test.ESTestCase; -import org.junit.Before; - -import java.io.IOException; - -import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.not; - -public class SimulateFailedDocumentResultTests extends ESTestCase { - - @Before - public void setup() { - } - - public void testEqualsExact() { - Throwable throwable = new Exception("foo"); - SimulateDocumentResult result = new SimulateFailedDocumentResult(throwable); - SimulateDocumentResult otherResult = new SimulateFailedDocumentResult(throwable); - assertThat(result, equalTo(otherResult)); - } - - public void testEqualsSameExceptionClass() { - SimulateDocumentResult result = new SimulateFailedDocumentResult(new IllegalArgumentException("foo")); - SimulateDocumentResult otherResult = new SimulateFailedDocumentResult(new IllegalArgumentException("bar")); - assertThat(result, equalTo(otherResult)); - } - - public void testNotEqualsDiffExceptionClass() { - SimulateDocumentResult result = new SimulateFailedDocumentResult(new IllegalArgumentException("foo")); - SimulateDocumentResult otherResult = new SimulateFailedDocumentResult(new NullPointerException("foo")); - assertThat(result, not(equalTo(otherResult))); - } - - public void testStreamable() throws IOException { - SimulateDocumentResult result = new SimulateFailedDocumentResult(new IllegalArgumentException("foo")); - - BytesStreamOutput out = new BytesStreamOutput(); - result.writeTo(out); - - StreamInput streamInput = StreamInput.wrap(out.bytes()); - SimulateDocumentResult otherResult = new SimulateFailedDocumentResult(); - otherResult.readFrom(streamInput); - - assertThat(result, equalTo(otherResult)); - } -} diff --git a/plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineResponseTests.java b/plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineResponseTests.java index 2352f3db0ba..0a325ca3268 100644 --- a/plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineResponseTests.java +++ b/plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulatePipelineResponseTests.java @@ -23,58 +23,96 @@ import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.ingest.Data; import org.elasticsearch.test.ESTestCase; -import org.junit.Before; import java.io.IOException; -import java.util.Arrays; +import java.util.ArrayList; import java.util.Collections; +import java.util.Iterator; import java.util.List; -import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.not; +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.instanceOf; +import static org.hamcrest.CoreMatchers.nullValue; public class SimulatePipelineResponseTests extends ESTestCase { - private Data data; - private SimulateDocumentResult documentResult; - private SimulatePipelineResponse response; - @Before - public void setup() { - data = new Data("_index", "_type", "_id", Collections.singletonMap("foo", "bar")); - documentResult = new SimulateSimpleDocumentResult(data); - response = new SimulatePipelineResponse("_id", Collections.singletonList(documentResult)); - } + public void testSerialization() throws IOException { + boolean isVerbose = randomBoolean(); + int numResults = randomIntBetween(1, 10); + List results = new ArrayList<>(numResults); + for (int i = 0; i < numResults; i++) { + boolean isFailure = randomBoolean(); + Data data = new Data(randomAsciiOfLengthBetween(1, 10), randomAsciiOfLengthBetween(1, 10), randomAsciiOfLengthBetween(1, 10), + Collections.singletonMap(randomAsciiOfLengthBetween(1, 10), randomAsciiOfLengthBetween(1, 10))); + if (isVerbose) { + int numProcessors = randomIntBetween(1, 10); + List processorResults = new ArrayList<>(numProcessors); + for (int j = 0; j < numProcessors; j++) { + String processorId = randomAsciiOfLengthBetween(1, 10); + SimulateProcessorResult processorResult; + if (isFailure) { + processorResult = new SimulateProcessorResult(processorId, new IllegalArgumentException("test")); + } else { + processorResult = new SimulateProcessorResult(processorId, data); + } + processorResults.add(processorResult); + } + results.add(new SimulateDocumentVerboseResult(processorResults)); + } else { + results.add(new SimulateDocumentSimpleResult(data)); + SimulateDocumentSimpleResult simulateDocumentSimpleResult; + if (isFailure) { + simulateDocumentSimpleResult = new SimulateDocumentSimpleResult(new IllegalArgumentException("test")); + } else { + simulateDocumentSimpleResult = new SimulateDocumentSimpleResult(data); + } + results.add(simulateDocumentSimpleResult); + } + } - public void testEquals() { - SimulatePipelineResponse otherResponse = new SimulatePipelineResponse("_id", Collections.singletonList(documentResult)); - assertThat(response, equalTo(otherResponse)); - } - - public void testNotEqualsId() { - SimulatePipelineResponse otherResponse = new SimulatePipelineResponse(response.getPipelineId() + "foo", response.getResults()); - assertThat(response, not(equalTo(otherResponse))); - } - - public void testNotEqualsResults() { - SimulatePipelineResponse otherResponse = new SimulatePipelineResponse(response.getPipelineId(), Arrays.asList(documentResult, documentResult)); - assertThat(response, not(equalTo(otherResponse))); - } - - public void testStreamable() throws IOException { - List results = Arrays.asList( - new SimulateSimpleDocumentResult(data), - new SimulateFailedDocumentResult(new IllegalArgumentException("foo")), - new SimulateVerboseDocumentResult(Collections.singletonList(new SimulateProcessorResult("pid", data))) - ); - - response = new SimulatePipelineResponse("_id", results); + 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(response, equalTo(otherResponse)); + assertThat(otherResponse.getPipelineId(), equalTo(response.getPipelineId())); + assertThat(otherResponse.getResults().size(), equalTo(response.getResults().size())); + + Iterator 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 expectedProcessorResultIterator = expectedSimulateDocumentVerboseResult.getProcessorResults().iterator(); + for (SimulateProcessorResult simulateProcessorResult : simulateDocumentVerboseResult.getProcessorResults()) { + SimulateProcessorResult expectedProcessorResult = expectedProcessorResultIterator.next(); + assertThat(simulateProcessorResult.getProcessorId(), equalTo(expectedProcessorResult.getProcessorId())); + assertThat(simulateProcessorResult.getData(), equalTo(expectedProcessorResult.getData())); + 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 { + SimulateDocumentSimpleResult expectedSimulateDocumentSimpleResult = (SimulateDocumentSimpleResult) expectedResultIterator.next(); + assertThat(result, instanceOf(SimulateDocumentSimpleResult.class)); + SimulateDocumentSimpleResult simulateDocumentSimpleResult = (SimulateDocumentSimpleResult) result; + assertThat(simulateDocumentSimpleResult.getData(), equalTo(expectedSimulateDocumentSimpleResult.getData())); + if (expectedSimulateDocumentSimpleResult.getFailure() == null) { + assertThat(simulateDocumentSimpleResult.getFailure(), nullValue()); + } else { + assertThat(simulateDocumentSimpleResult.getFailure(), instanceOf(IllegalArgumentException.class)); + IllegalArgumentException e = (IllegalArgumentException) simulateDocumentSimpleResult.getFailure(); + assertThat(e.getMessage(), equalTo("test")); + } + } + } } } diff --git a/plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateProcessorResultTests.java b/plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateProcessorResultTests.java index 1a20da74416..55024219347 100644 --- a/plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateProcessorResultTests.java +++ b/plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateProcessorResultTests.java @@ -23,65 +23,37 @@ import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.ingest.Data; import org.elasticsearch.test.ESTestCase; -import org.junit.Before; import java.io.IOException; import java.util.Collections; import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.not; +import static org.hamcrest.Matchers.instanceOf; public class SimulateProcessorResultTests extends ESTestCase { - private Data data; - private SimulateProcessorResult result; - private SimulateProcessorResult failedResult; - private String processorId; - private Throwable throwable; - @Before - public void setup() { - data = new Data("_index", "_type", "_id", Collections.singletonMap("foo", "bar")); - processorId = "id"; - throwable = new IllegalArgumentException("foo"); - result = new SimulateProcessorResult(processorId, data); - failedResult = new SimulateProcessorResult(processorId, throwable); - } + public void testSerialization() throws IOException { + String processorId = randomAsciiOfLengthBetween(1, 10); + boolean isFailure = randomBoolean(); + SimulateProcessorResult simulateProcessorResult; + if (isFailure) { + simulateProcessorResult = new SimulateProcessorResult(processorId, new IllegalArgumentException("test")); + } else { + Data data = new Data(randomAsciiOfLengthBetween(1, 10), randomAsciiOfLengthBetween(1, 10), randomAsciiOfLengthBetween(1, 10), + Collections.singletonMap(randomAsciiOfLengthBetween(1, 10), randomAsciiOfLengthBetween(1, 10))); + simulateProcessorResult = new SimulateProcessorResult(processorId, data); + } - public void testEqualsData() { - SimulateProcessorResult otherResult = new SimulateProcessorResult(new String(processorId), new Data(data)); - assertThat(result, equalTo(otherResult)); - } - - public void testEqualsSameClassThrowable() { - SimulateProcessorResult otherFailedResult = new SimulateProcessorResult(new String(processorId), new IllegalArgumentException("foo")); - assertThat(failedResult, equalTo(otherFailedResult)); - } - - public void testNotEqualsThrowable() { - SimulateProcessorResult otherFailedResult = new SimulateProcessorResult(new String(processorId), new NullPointerException("foo")); - assertThat(failedResult, not(equalTo(otherFailedResult))); - } - - public void testStreamableWithThrowable() throws IOException { BytesStreamOutput out = new BytesStreamOutput(); - failedResult.writeTo(out); - + simulateProcessorResult.writeTo(out); StreamInput streamInput = StreamInput.wrap(out.bytes()); - SimulateProcessorResult otherFailedResult = new SimulateProcessorResult(); - otherFailedResult.readFrom(streamInput); - - assertThat(failedResult, equalTo(otherFailedResult)); - } - - public void testStreamableWithData() throws IOException { - BytesStreamOutput out = new BytesStreamOutput(); - result.writeTo(out); - - StreamInput streamInput = StreamInput.wrap(out.bytes()); - SimulateProcessorResult otherResult = new SimulateProcessorResult(); - otherResult.readFrom(streamInput); - - assertThat(result, equalTo(otherResult)); - + SimulateProcessorResult otherSimulateProcessorResult = SimulateProcessorResult.readSimulateProcessorResultFrom(streamInput); + assertThat(otherSimulateProcessorResult.getProcessorId(), equalTo(simulateProcessorResult.getProcessorId())); + assertThat(otherSimulateProcessorResult.getData(), equalTo(simulateProcessorResult.getData())); + if (isFailure) { + assertThat(otherSimulateProcessorResult.getFailure(), instanceOf(IllegalArgumentException.class)); + IllegalArgumentException e = (IllegalArgumentException) otherSimulateProcessorResult.getFailure(); + assertThat(e.getMessage(), equalTo("test")); + } } } diff --git a/plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateSimpleDocumentResultTests.java b/plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateSimpleDocumentResultTests.java deleted file mode 100644 index 7c7a497bbcc..00000000000 --- a/plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/transport/simulate/SimulateSimpleDocumentResultTests.java +++ /dev/null @@ -1,67 +0,0 @@ -/* - * 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.plugin.ingest.transport.simulate; - -import org.elasticsearch.common.io.stream.BytesStreamOutput; -import org.elasticsearch.common.io.stream.StreamInput; -import org.elasticsearch.ingest.Data; -import org.elasticsearch.test.ESTestCase; -import org.junit.Before; - -import java.io.IOException; -import java.util.Collections; - -import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.not; - -public class SimulateSimpleDocumentResultTests extends ESTestCase { - private Data data; - - @Before - public void setup() { - data = new Data("_index", "_type", "_id", Collections.singletonMap("foo", "bar")); - } - - public void testEquals() { - SimulateDocumentResult result = new SimulateSimpleDocumentResult(data); - SimulateDocumentResult otherResult = new SimulateSimpleDocumentResult(data); - assertThat(result, equalTo(otherResult)); - } - - public void testNotEqualsDiffData() { - Data otherData = new Data(data.getIndex() + "foo", data.getType(), data.getId(), data.getDocument()); - SimulateDocumentResult result = new SimulateSimpleDocumentResult(data); - SimulateDocumentResult otherResult = new SimulateSimpleDocumentResult(otherData); - assertThat(result, not(equalTo(otherResult))); - } - - public void testStreamable() throws IOException { - SimulateDocumentResult result = new SimulateSimpleDocumentResult(data); - - BytesStreamOutput out = new BytesStreamOutput(); - result.writeTo(out); - - StreamInput streamInput = StreamInput.wrap(out.bytes()); - SimulateDocumentResult otherResult = new SimulateSimpleDocumentResult(); - otherResult.readFrom(streamInput); - - assertThat(result, equalTo(otherResult)); - } -} From 97f4f27b14e6afe3833fecfce79ca39877f2227a Mon Sep 17 00:00:00 2001 From: javanna Date: Fri, 13 Nov 2015 19:35:09 +0100 Subject: [PATCH 12/14] remove equals/hashcode as part of Pipeline and adapt tests Only MutateProcessor implemented equals / hashcode hence we would only use that one in our tests, since they relied on them. Better to not rely on equals/hashcode, drop them and mock processor/pipeline in our tests that need them. That also allow to make MutateProcessor constructor package private as the other processors. --- .../org/elasticsearch/ingest/Pipeline.java | 15 --- .../processor/mutate/MutateProcessor.java | 2 +- .../simulate/ParsedSimulateRequest.java | 15 --- .../org/elasticsearch/ingest/DataTests.java | 71 +++++++--- .../elasticsearch/ingest/PipelineTests.java | 67 --------- .../ingest/transport/TransportDataTests.java | 15 +-- .../ParsedSimulateRequestParserTests.java | 127 ++++++++++++------ 7 files changed, 141 insertions(+), 171 deletions(-) delete mode 100644 plugins/ingest/src/test/java/org/elasticsearch/ingest/PipelineTests.java diff --git a/plugins/ingest/src/main/java/org/elasticsearch/ingest/Pipeline.java b/plugins/ingest/src/main/java/org/elasticsearch/ingest/Pipeline.java index b98a469d3c3..7b44f7d5a7f 100644 --- a/plugins/ingest/src/main/java/org/elasticsearch/ingest/Pipeline.java +++ b/plugins/ingest/src/main/java/org/elasticsearch/ingest/Pipeline.java @@ -71,21 +71,6 @@ public final class Pipeline { return processors; } - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - Pipeline pipeline = (Pipeline) o; - return Objects.equals(id, pipeline.id) && - Objects.equals(description, pipeline.description) && - Objects.equals(processors, pipeline.processors); - } - - @Override - public int hashCode() { - return Objects.hash(id, description, processors); - } - public final static class Factory { public Pipeline create(String id, Map config, Map processorRegistry) throws IOException { diff --git a/plugins/ingest/src/main/java/org/elasticsearch/ingest/processor/mutate/MutateProcessor.java b/plugins/ingest/src/main/java/org/elasticsearch/ingest/processor/mutate/MutateProcessor.java index 4a950bea083..2daadd5fee5 100644 --- a/plugins/ingest/src/main/java/org/elasticsearch/ingest/processor/mutate/MutateProcessor.java +++ b/plugins/ingest/src/main/java/org/elasticsearch/ingest/processor/mutate/MutateProcessor.java @@ -45,7 +45,7 @@ public final class MutateProcessor implements Processor { private final List uppercase; private final List lowercase; - public MutateProcessor(Map update, Map rename, Map convert, + MutateProcessor(Map update, Map rename, Map convert, Map split, List gsub, Map join, List remove, List trim, List uppercase, List lowercase) { this.update = update; diff --git a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/ParsedSimulateRequest.java b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/ParsedSimulateRequest.java index 47af2db583d..50897435217 100644 --- a/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/ParsedSimulateRequest.java +++ b/plugins/ingest/src/main/java/org/elasticsearch/plugin/ingest/transport/simulate/ParsedSimulateRequest.java @@ -51,21 +51,6 @@ public class ParsedSimulateRequest { return verbose; } - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - ParsedSimulateRequest that = (ParsedSimulateRequest) o; - return Objects.equals(verbose, that.verbose) && - Objects.equals(documents, that.documents) && - Objects.equals(pipeline, that.pipeline); - } - - @Override - public int hashCode() { - return Objects.hash(documents, pipeline, verbose); - } - public static class Parser { private static final Pipeline.Factory PIPELINE_FACTORY = new Pipeline.Factory(); public static final String SIMULATED_PIPELINE_ID = "_simulate_pipeline"; diff --git a/plugins/ingest/src/test/java/org/elasticsearch/ingest/DataTests.java b/plugins/ingest/src/test/java/org/elasticsearch/ingest/DataTests.java index 918064a17c7..d17a354b139 100644 --- a/plugins/ingest/src/test/java/org/elasticsearch/ingest/DataTests.java +++ b/plugins/ingest/src/test/java/org/elasticsearch/ingest/DataTests.java @@ -26,7 +26,8 @@ import java.util.Collections; import java.util.HashMap; import java.util.Map; -import static org.hamcrest.Matchers.*; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.not; public class DataTests extends ESTestCase { @@ -86,28 +87,54 @@ public class DataTests extends ESTestCase { assertThat(data.getProperty("fizz.new"), equalTo("bar")); } - public void testEquals() { - Data otherData = new Data(data); - assertThat(otherData, equalTo(data)); - } + public void testEqualsAndHashcode() throws Exception { + String index = randomAsciiOfLengthBetween(1, 10); + String type = randomAsciiOfLengthBetween(1, 10); + String id = randomAsciiOfLengthBetween(1, 10); + String fieldName = randomAsciiOfLengthBetween(1, 10); + String fieldValue = randomAsciiOfLengthBetween(1, 10); + Data data = new Data(index, type, id, Collections.singletonMap(fieldName, fieldValue)); - public void testNotEqualsDiffIndex() { - Data otherData = new Data(data.getIndex() + "foo", data.getType(), data.getId(), data.getDocument()); - assertThat(otherData, not(equalTo(data))); - } + boolean changed = false; + String otherIndex; + if (randomBoolean()) { + otherIndex = randomAsciiOfLengthBetween(1, 10); + changed = true; + } else { + otherIndex = index; + } + String otherType; + if (randomBoolean()) { + otherType = randomAsciiOfLengthBetween(1, 10); + changed = true; + } else { + otherType = type; + } + String otherId; + if (randomBoolean()) { + otherId = randomAsciiOfLengthBetween(1, 10); + changed = true; + } else { + otherId = id; + } + Map document; + if (randomBoolean()) { + document = Collections.singletonMap(randomAsciiOfLengthBetween(1, 10), randomAsciiOfLengthBetween(1, 10)); + changed = true; + } else { + document = Collections.singletonMap(fieldName, fieldValue); + } - public void testNotEqualsDiffType() { - Data otherData = new Data(data.getIndex(), data.getType() + "foo", data.getId(), data.getDocument()); - assertThat(otherData, not(equalTo(data))); - } - - public void testNotEqualsDiffId() { - Data otherData = new Data(data.getIndex(), data.getType(), data.getId() + "foo", data.getDocument()); - assertThat(otherData, not(equalTo(data))); - } - - public void testNotEqualsDiffDocument() { - Data otherData = new Data(data.getIndex(), data.getType(), data.getId(), Collections.emptyMap()); - assertThat(otherData, not(equalTo(data))); + Data otherData = new Data(otherIndex, otherType, otherId, document); + if (changed) { + assertThat(data, not(equalTo(otherData))); + assertThat(otherData, not(equalTo(data))); + } else { + assertThat(data, equalTo(otherData)); + assertThat(otherData, equalTo(data)); + Data thirdData = new Data(index, type, id, Collections.singletonMap(fieldName, fieldValue)); + assertThat(thirdData, equalTo(data)); + assertThat(data, equalTo(thirdData)); + } } } diff --git a/plugins/ingest/src/test/java/org/elasticsearch/ingest/PipelineTests.java b/plugins/ingest/src/test/java/org/elasticsearch/ingest/PipelineTests.java deleted file mode 100644 index 84d0953495b..00000000000 --- a/plugins/ingest/src/test/java/org/elasticsearch/ingest/PipelineTests.java +++ /dev/null @@ -1,67 +0,0 @@ -/* - * 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.processor.Processor; -import org.elasticsearch.ingest.processor.mutate.MutateProcessor; -import org.elasticsearch.test.ESTestCase; -import org.junit.Before; - -import java.util.*; - -import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.instanceOf; -import static org.hamcrest.Matchers.not; -import static org.mockito.Mockito.mock; - -public class PipelineTests extends ESTestCase { - private Processor updateProcessor; - private Processor lowercaseProcessor; - private Pipeline pipeline; - - @Before - public void setup() { - Map update = Collections.singletonMap("foo", 123); - List lowercase = Collections.singletonList("foo"); - updateProcessor = new MutateProcessor(update, null, null, null, null, null, null, null, null, null); - lowercaseProcessor = new MutateProcessor(null, null, null, null, null, null, null, null, null, lowercase); - pipeline = new Pipeline("id", "description", Arrays.asList(updateProcessor, lowercaseProcessor)); - } - - public void testEquals() throws Exception { - Pipeline other = new Pipeline(pipeline.getId(), pipeline.getDescription(), pipeline.getProcessors()); - assertThat(pipeline, equalTo(other)); - } - - public void testNotEqualsDiffId() throws Exception { - Pipeline other = new Pipeline(pipeline.getId() + "foo", pipeline.getDescription(), pipeline.getProcessors()); - assertThat(pipeline, not(equalTo(other))); - } - - public void testNotEqualsDiffDescription() throws Exception { - Pipeline other = new Pipeline(pipeline.getId(), pipeline.getDescription() + "foo", pipeline.getProcessors()); - assertThat(pipeline, not(equalTo(other))); - } - - public void testNotEqualsDiffProcessors() throws Exception { - Pipeline other = new Pipeline(pipeline.getId(), pipeline.getDescription() + "foo", Collections.singletonList(updateProcessor)); - assertThat(pipeline, not(equalTo(other))); - } -} diff --git a/plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/transport/TransportDataTests.java b/plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/transport/TransportDataTests.java index 89ef7731327..1cc3f6baada 100644 --- a/plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/transport/TransportDataTests.java +++ b/plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/transport/TransportDataTests.java @@ -39,8 +39,7 @@ public class TransportDataTests extends ESTestCase { String id = randomAsciiOfLengthBetween(1, 10); String fieldName = randomAsciiOfLengthBetween(1, 10); String fieldValue = randomAsciiOfLengthBetween(1, 10); - Data data = new Data(index, type, id, Collections.singletonMap(fieldName, fieldValue)); - TransportData transportData = new TransportData(data); + TransportData transportData = new TransportData(new Data(index, type, id, Collections.singletonMap(fieldName, fieldValue))); boolean changed = false; String otherIndex; @@ -72,22 +71,14 @@ public class TransportDataTests extends ESTestCase { document = Collections.singletonMap(fieldName, fieldValue); } - Data otherData = new Data(otherIndex, otherType, otherId, document); - TransportData otherTransportData = new TransportData(otherData); + TransportData otherTransportData = new TransportData(new Data(otherIndex, otherType, otherId, document)); if (changed) { - assertThat(data, not(equalTo(otherData))); - assertThat(otherData, not(equalTo(data))); assertThat(transportData, not(equalTo(otherTransportData))); assertThat(otherTransportData, not(equalTo(transportData))); } else { - assertThat(data, equalTo(otherData)); - assertThat(otherData, equalTo(data)); assertThat(transportData, equalTo(otherTransportData)); assertThat(otherTransportData, equalTo(transportData)); - Data thirdData = new Data(index, type, id, Collections.singletonMap(fieldName, fieldValue)); - TransportData thirdTransportData = new TransportData(thirdData); - assertThat(thirdData, equalTo(data)); - assertThat(data, equalTo(thirdData)); + TransportData thirdTransportData = new TransportData(new Data(index, type, id, Collections.singletonMap(fieldName, fieldValue))); assertThat(thirdTransportData, equalTo(transportData)); assertThat(transportData, equalTo(thirdTransportData)); } diff --git a/plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/transport/simulate/ParsedSimulateRequestParserTests.java b/plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/transport/simulate/ParsedSimulateRequestParserTests.java index 09c145a56b6..7f44fc08b9b 100644 --- a/plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/transport/simulate/ParsedSimulateRequestParserTests.java +++ b/plugins/ingest/src/test/java/org/elasticsearch/plugin/ingest/transport/simulate/ParsedSimulateRequestParserTests.java @@ -22,7 +22,6 @@ package org.elasticsearch.plugin.ingest.transport.simulate; import org.elasticsearch.ingest.Data; import org.elasticsearch.ingest.Pipeline; import org.elasticsearch.ingest.processor.Processor; -import org.elasticsearch.ingest.processor.mutate.MutateProcessor; import org.elasticsearch.plugin.ingest.PipelineStore; import org.elasticsearch.test.ESTestCase; import org.junit.Before; @@ -30,70 +29,120 @@ import org.junit.Before; import java.io.IOException; import java.util.*; +import static org.elasticsearch.plugin.ingest.transport.simulate.SimulatePipelineRequest.Fields; import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.nullValue; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; -import static org.elasticsearch.plugin.ingest.transport.simulate.SimulatePipelineRequest.Fields; - public class ParsedSimulateRequestParserTests extends ESTestCase { + private PipelineStore store; - private ParsedSimulateRequest.Parser parser; - private Pipeline pipeline; - private Data data; @Before public void init() throws IOException { - parser = new ParsedSimulateRequest.Parser(); - List uppercase = Collections.singletonList("foo"); - Processor processor = new MutateProcessor(null, null, null, null, null, null, null, null, uppercase, null); - pipeline = new Pipeline(ParsedSimulateRequest.Parser.SIMULATED_PIPELINE_ID, null, Arrays.asList(processor)); - data = new Data("_index", "_type", "_id", Collections.singletonMap("foo", "bar")); + Pipeline pipeline = new Pipeline(ParsedSimulateRequest.Parser.SIMULATED_PIPELINE_ID, null, Collections.singletonList(mock(Processor.class))); Map processorRegistry = new HashMap<>(); - processorRegistry.put("mutate", new MutateProcessor.Factory()); + processorRegistry.put("mock_processor", mock(Processor.Factory.class)); store = mock(PipelineStore.class); - when(store.get("_id")).thenReturn(pipeline); + when(store.get(ParsedSimulateRequest.Parser.SIMULATED_PIPELINE_ID)).thenReturn(pipeline); when(store.getProcessorFactoryRegistry()).thenReturn(processorRegistry); } public void testParseUsingPipelineStore() throws Exception { - ParsedSimulateRequest expectedRequest = new ParsedSimulateRequest(pipeline, Collections.singletonList(data), false); + int numDocs = randomIntBetween(1, 10); - Map raw = new HashMap<>(); + Map requestContent = new HashMap<>(); List> docs = new ArrayList<>(); - Map doc = new HashMap<>(); - doc.put(Fields.INDEX, "_index"); - doc.put(Fields.TYPE, "_type"); - doc.put(Fields.ID, "_id"); - doc.put(Fields.SOURCE, data.getDocument()); - docs.add(doc); - raw.put(Fields.DOCS, docs); + List> expectedDocs = new ArrayList<>(); + requestContent.put(Fields.DOCS, docs); + for (int i = 0; i < numDocs; i++) { + Map doc = new HashMap<>(); + String index = randomAsciiOfLengthBetween(1, 10); + String type = randomAsciiOfLengthBetween(1, 10); + String id = randomAsciiOfLengthBetween(1, 10); + doc.put(Fields.INDEX, index); + doc.put(Fields.TYPE, type); + doc.put(Fields.ID, id); + String fieldName = randomAsciiOfLengthBetween(1, 10); + String fieldValue = randomAsciiOfLengthBetween(1, 10); + doc.put(Fields.SOURCE, Collections.singletonMap(fieldName, fieldValue)); + docs.add(doc); + Map expectedDoc = new HashMap<>(); + expectedDoc.put(Fields.INDEX, index); + expectedDoc.put(Fields.TYPE, type); + expectedDoc.put(Fields.ID, id); + expectedDoc.put(Fields.SOURCE, Collections.singletonMap(fieldName, fieldValue)); + expectedDocs.add(expectedDoc); + } - ParsedSimulateRequest actualRequest = parser.parseWithPipelineId("_id", raw, false, store); - assertThat(actualRequest, equalTo(expectedRequest)); + ParsedSimulateRequest actualRequest = new ParsedSimulateRequest.Parser().parseWithPipelineId(ParsedSimulateRequest.Parser.SIMULATED_PIPELINE_ID, requestContent, false, store); + assertThat(actualRequest.isVerbose(), equalTo(false)); + assertThat(actualRequest.getDocuments().size(), equalTo(numDocs)); + Iterator> expectedDocsIterator = expectedDocs.iterator(); + for (Data data : actualRequest.getDocuments()) { + Map expectedDocument = expectedDocsIterator.next(); + assertThat(data.getDocument(), equalTo(expectedDocument.get(Fields.SOURCE))); + assertThat(data.getIndex(), equalTo(expectedDocument.get(Fields.INDEX))); + assertThat(data.getType(), equalTo(expectedDocument.get(Fields.TYPE))); + assertThat(data.getId(), equalTo(expectedDocument.get(Fields.ID))); + } + + assertThat(actualRequest.getPipeline().getId(), equalTo(ParsedSimulateRequest.Parser.SIMULATED_PIPELINE_ID)); + assertThat(actualRequest.getPipeline().getDescription(), nullValue()); + assertThat(actualRequest.getPipeline().getProcessors().size(), equalTo(1)); } public void testParseWithProvidedPipeline() throws Exception { - ParsedSimulateRequest expectedRequest = new ParsedSimulateRequest(pipeline, Collections.singletonList(data), false); + int numDocs = randomIntBetween(1, 10); - Map raw = new HashMap<>(); + Map requestContent = new HashMap<>(); List> docs = new ArrayList<>(); - Map doc = new HashMap<>(); - doc.put(Fields.INDEX, "_index"); - doc.put(Fields.TYPE, "_type"); - doc.put(Fields.ID, "_id"); - doc.put(Fields.SOURCE, data.getDocument()); - docs.add(doc); + List> expectedDocs = new ArrayList<>(); + requestContent.put(Fields.DOCS, docs); + for (int i = 0; i < numDocs; i++) { + Map doc = new HashMap<>(); + String index = randomAsciiOfLengthBetween(1, 10); + String type = randomAsciiOfLengthBetween(1, 10); + String id = randomAsciiOfLengthBetween(1, 10); + doc.put(Fields.INDEX, index); + doc.put(Fields.TYPE, type); + doc.put(Fields.ID, id); + String fieldName = randomAsciiOfLengthBetween(1, 10); + String fieldValue = randomAsciiOfLengthBetween(1, 10); + doc.put(Fields.SOURCE, Collections.singletonMap(fieldName, fieldValue)); + docs.add(doc); + Map expectedDoc = new HashMap<>(); + expectedDoc.put(Fields.INDEX, index); + expectedDoc.put(Fields.TYPE, type); + expectedDoc.put(Fields.ID, id); + expectedDoc.put(Fields.SOURCE, Collections.singletonMap(fieldName, fieldValue)); + expectedDocs.add(expectedDoc); + } - Map processorConfig = new HashMap<>(); - processorConfig.put("uppercase", Arrays.asList("foo")); Map pipelineConfig = new HashMap<>(); - pipelineConfig.put("processors", Collections.singletonList(Collections.singletonMap("mutate", processorConfig))); + List> processors = new ArrayList<>(); + int numProcessors = randomIntBetween(1, 10); + for (int i = 0; i < numProcessors; i++) { + processors.add(Collections.singletonMap("mock_processor", Collections.emptyMap())); + } + pipelineConfig.put("processors", processors); + requestContent.put(Fields.PIPELINE, pipelineConfig); - raw.put(Fields.DOCS, docs); - raw.put(Fields.PIPELINE, pipelineConfig); + ParsedSimulateRequest actualRequest = new ParsedSimulateRequest.Parser().parse(requestContent, false, store); + assertThat(actualRequest.isVerbose(), equalTo(false)); + assertThat(actualRequest.getDocuments().size(), equalTo(numDocs)); + Iterator> expectedDocsIterator = expectedDocs.iterator(); + for (Data data : actualRequest.getDocuments()) { + Map expectedDocument = expectedDocsIterator.next(); + assertThat(data.getDocument(), equalTo(expectedDocument.get(Fields.SOURCE))); + assertThat(data.getIndex(), equalTo(expectedDocument.get(Fields.INDEX))); + assertThat(data.getType(), equalTo(expectedDocument.get(Fields.TYPE))); + assertThat(data.getId(), equalTo(expectedDocument.get(Fields.ID))); + } - ParsedSimulateRequest actualRequest = parser.parse(raw, false, store); - assertThat(actualRequest, equalTo(expectedRequest)); + assertThat(actualRequest.getPipeline().getId(), equalTo(ParsedSimulateRequest.Parser.SIMULATED_PIPELINE_ID)); + assertThat(actualRequest.getPipeline().getDescription(), nullValue()); + assertThat(actualRequest.getPipeline().getProcessors().size(), equalTo(numProcessors)); } } From 26569045efae8f143e033115afc3736dc8da835e Mon Sep 17 00:00:00 2001 From: javanna Date: Fri, 13 Nov 2015 19:36:18 +0100 Subject: [PATCH 13/14] remove leftover equals/hashcode --- .../processor/mutate/MutateProcessor.java | 22 ------------------- 1 file changed, 22 deletions(-) diff --git a/plugins/ingest/src/main/java/org/elasticsearch/ingest/processor/mutate/MutateProcessor.java b/plugins/ingest/src/main/java/org/elasticsearch/ingest/processor/mutate/MutateProcessor.java index 2daadd5fee5..9b9d98e3fef 100644 --- a/plugins/ingest/src/main/java/org/elasticsearch/ingest/processor/mutate/MutateProcessor.java +++ b/plugins/ingest/src/main/java/org/elasticsearch/ingest/processor/mutate/MutateProcessor.java @@ -277,28 +277,6 @@ public final class MutateProcessor implements Processor { } } - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - MutateProcessor that = (MutateProcessor) o; - return Objects.equals(update, that.update) && - Objects.equals(rename, that.rename) && - Objects.equals(convert, that.convert) && - Objects.equals(split, that.split) && - Objects.equals(gsub, that.gsub) && - Objects.equals(join, that.join) && - Objects.equals(remove, that.remove) && - Objects.equals(trim, that.trim) && - Objects.equals(uppercase, that.uppercase) && - Objects.equals(lowercase, that.lowercase); - } - - @Override - public int hashCode() { - return Objects.hash(update, rename, convert, split, gsub, join, remove, trim, uppercase, lowercase); - } - public static final class Factory implements Processor.Factory { @Override public MutateProcessor create(Map config) throws IOException { From 446fa0c10b6e73358b5a50612ed5af62d7a886b6 Mon Sep 17 00:00:00 2001 From: javanna Date: Fri, 13 Nov 2015 19:37:06 +0100 Subject: [PATCH 14/14] remove unnecessary line breaks --- .../java/org/elasticsearch/ingest/processor/Processor.java | 3 --- 1 file changed, 3 deletions(-) diff --git a/plugins/ingest/src/main/java/org/elasticsearch/ingest/processor/Processor.java b/plugins/ingest/src/main/java/org/elasticsearch/ingest/processor/Processor.java index 46a9d43e280..fc268b2b128 100644 --- a/plugins/ingest/src/main/java/org/elasticsearch/ingest/processor/Processor.java +++ b/plugins/ingest/src/main/java/org/elasticsearch/ingest/processor/Processor.java @@ -59,11 +59,8 @@ public interface Processor { default void setConfigDirectory(Path configDirectory) { } - @Override default void close() throws IOException { } - } - }