diff --git a/core/src/main/java/org/elasticsearch/action/ActionModule.java b/core/src/main/java/org/elasticsearch/action/ActionModule.java index 67f256c6bd1..39aa4b7a2ba 100644 --- a/core/src/main/java/org/elasticsearch/action/ActionModule.java +++ b/core/src/main/java/org/elasticsearch/action/ActionModule.java @@ -149,6 +149,16 @@ import org.elasticsearch.action.indexedscripts.get.GetIndexedScriptAction; import org.elasticsearch.action.indexedscripts.get.TransportGetIndexedScriptAction; import org.elasticsearch.action.indexedscripts.put.PutIndexedScriptAction; import org.elasticsearch.action.indexedscripts.put.TransportPutIndexedScriptAction; +import org.elasticsearch.action.ingest.IngestActionFilter; +import org.elasticsearch.action.ingest.IngestProxyActionFilter; +import org.elasticsearch.action.ingest.DeletePipelineAction; +import org.elasticsearch.action.ingest.DeletePipelineTransportAction; +import org.elasticsearch.action.ingest.GetPipelineAction; +import org.elasticsearch.action.ingest.GetPipelineTransportAction; +import org.elasticsearch.action.ingest.PutPipelineAction; +import org.elasticsearch.action.ingest.PutPipelineTransportAction; +import org.elasticsearch.action.ingest.SimulatePipelineAction; +import org.elasticsearch.action.ingest.SimulatePipelineTransportAction; import org.elasticsearch.action.percolate.MultiPercolateAction; import org.elasticsearch.action.percolate.PercolateAction; import org.elasticsearch.action.percolate.TransportMultiPercolateAction; @@ -186,6 +196,8 @@ import org.elasticsearch.action.update.UpdateAction; import org.elasticsearch.common.inject.AbstractModule; import org.elasticsearch.common.inject.multibindings.MapBinder; import org.elasticsearch.common.inject.multibindings.Multibinder; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.node.NodeModule; import java.util.ArrayList; import java.util.HashMap; @@ -210,13 +222,13 @@ public class ActionModule extends AbstractModule { this.transportAction = transportAction; this.supportTransportActions = supportTransportActions; } - - } + private final boolean ingestEnabled; private final boolean proxy; - public ActionModule(boolean proxy) { + public ActionModule(boolean ingestEnabled, boolean proxy) { + this.ingestEnabled = ingestEnabled; this.proxy = proxy; } @@ -240,6 +252,13 @@ public class ActionModule extends AbstractModule { @Override protected void configure() { + if (proxy == false) { + if (ingestEnabled) { + registerFilter(IngestActionFilter.class); + } else { + registerFilter(IngestProxyActionFilter.class); + } + } Multibinder actionFilterMultibinder = Multibinder.newSetBinder(binder(), ActionFilter.class); for (Class actionFilter : actionFilters) { @@ -340,6 +359,11 @@ public class ActionModule extends AbstractModule { registerAction(FieldStatsAction.INSTANCE, TransportFieldStatsTransportAction.class); + registerAction(PutPipelineAction.INSTANCE, PutPipelineTransportAction.class); + registerAction(GetPipelineAction.INSTANCE, GetPipelineTransportAction.class); + registerAction(DeletePipelineAction.INSTANCE, DeletePipelineTransportAction.class); + registerAction(SimulatePipelineAction.INSTANCE, SimulatePipelineTransportAction.class); + // register Name -> GenericAction Map that can be injected to instances. MapBinder actionsBinder = MapBinder.newMapBinder(binder(), String.class, GenericAction.class); diff --git a/core/src/main/java/org/elasticsearch/action/bulk/BulkProcessor.java b/core/src/main/java/org/elasticsearch/action/bulk/BulkProcessor.java index 9a7299ad179..c54b3588c17 100644 --- a/core/src/main/java/org/elasticsearch/action/bulk/BulkProcessor.java +++ b/core/src/main/java/org/elasticsearch/action/bulk/BulkProcessor.java @@ -289,11 +289,11 @@ public class BulkProcessor implements Closeable { } public BulkProcessor add(BytesReference data, @Nullable String defaultIndex, @Nullable String defaultType) throws Exception { - return add(data, defaultIndex, defaultType, null); + return add(data, defaultIndex, defaultType, null, null); } - public synchronized BulkProcessor add(BytesReference data, @Nullable String defaultIndex, @Nullable String defaultType, @Nullable Object payload) throws Exception { - bulkRequest.add(data, defaultIndex, defaultType, null, null, payload, true); + public synchronized BulkProcessor add(BytesReference data, @Nullable String defaultIndex, @Nullable String defaultType, @Nullable String defaultPipeline, @Nullable Object payload) throws Exception { + bulkRequest.add(data, defaultIndex, defaultType, null, null, defaultPipeline, payload, true); executeIfNeeded(); return this; } diff --git a/core/src/main/java/org/elasticsearch/action/bulk/BulkRequest.java b/core/src/main/java/org/elasticsearch/action/bulk/BulkRequest.java index 00260644892..3bc08d39df0 100644 --- a/core/src/main/java/org/elasticsearch/action/bulk/BulkRequest.java +++ b/core/src/main/java/org/elasticsearch/action/bulk/BulkRequest.java @@ -28,6 +28,7 @@ import org.elasticsearch.action.delete.DeleteRequest; import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.update.UpdateRequest; import org.elasticsearch.common.Nullable; +import org.elasticsearch.common.Strings; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.io.stream.StreamInput; @@ -253,17 +254,17 @@ public class BulkRequest extends ActionRequest implements Composite * Adds a framed data in binary format */ public BulkRequest add(BytesReference data, @Nullable String defaultIndex, @Nullable String defaultType) throws Exception { - return add(data, defaultIndex, defaultType, null, null, null, true); + return add(data, defaultIndex, defaultType, null, null, null, null, true); } /** * Adds a framed data in binary format */ public BulkRequest add(BytesReference data, @Nullable String defaultIndex, @Nullable String defaultType, boolean allowExplicitIndex) throws Exception { - return add(data, defaultIndex, defaultType, null, null, null, allowExplicitIndex); + return add(data, defaultIndex, defaultType, null, null, null, null, allowExplicitIndex); } - public BulkRequest add(BytesReference data, @Nullable String defaultIndex, @Nullable String defaultType, @Nullable String defaultRouting, @Nullable String[] defaultFields, @Nullable Object payload, boolean allowExplicitIndex) throws Exception { + public BulkRequest add(BytesReference data, @Nullable String defaultIndex, @Nullable String defaultType, @Nullable String defaultRouting, @Nullable String[] defaultFields, @Nullable String defaultPipeline, @Nullable Object payload, boolean allowExplicitIndex) throws Exception { XContent xContent = XContentFactory.xContent(data); int line = 0; int from = 0; @@ -304,6 +305,7 @@ public class BulkRequest extends ActionRequest implements Composite long version = Versions.MATCH_ANY; VersionType versionType = VersionType.INTERNAL; int retryOnConflict = 0; + String pipeline = defaultPipeline; // at this stage, next token can either be END_OBJECT (and use default index and type, with auto generated id) // or START_OBJECT which will have another set of parameters @@ -344,6 +346,8 @@ public class BulkRequest extends ActionRequest implements Composite versionType = VersionType.fromString(parser.text()); } else if ("_retry_on_conflict".equals(currentFieldName) || "_retryOnConflict".equals(currentFieldName)) { retryOnConflict = parser.intValue(); + } else if ("pipeline".equals(currentFieldName)) { + pipeline = parser.text(); } else if ("fields".equals(currentFieldName)) { throw new IllegalArgumentException("Action/metadata line [" + line + "] contains a simple value for parameter [fields] while a list is expected"); } else { @@ -380,15 +384,15 @@ public class BulkRequest extends ActionRequest implements Composite if ("index".equals(action)) { if (opType == null) { internalAdd(new IndexRequest(index, type, id).routing(routing).parent(parent).timestamp(timestamp).ttl(ttl).version(version).versionType(versionType) - .source(data.slice(from, nextMarker - from)), payload); + .setPipeline(pipeline).source(data.slice(from, nextMarker - from)), payload); } else { internalAdd(new IndexRequest(index, type, id).routing(routing).parent(parent).timestamp(timestamp).ttl(ttl).version(version).versionType(versionType) - .create("create".equals(opType)) + .create("create".equals(opType)).setPipeline(pipeline) .source(data.slice(from, nextMarker - from)), payload); } } else if ("create".equals(action)) { internalAdd(new IndexRequest(index, type, id).routing(routing).parent(parent).timestamp(timestamp).ttl(ttl).version(version).versionType(versionType) - .create(true) + .create(true).setPipeline(pipeline) .source(data.slice(from, nextMarker - from)), payload); } else if ("update".equals(action)) { UpdateRequest updateRequest = new UpdateRequest(index, type, id).routing(routing).parent(parent).retryOnConflict(retryOnConflict) @@ -479,6 +483,22 @@ public class BulkRequest extends ActionRequest implements Composite return -1; } + /** + * @return Whether this bulk request contains index request with an ingest pipeline enabled. + */ + public boolean hasIndexRequestsWithPipelines() { + for (ActionRequest actionRequest : requests) { + if (actionRequest instanceof IndexRequest) { + IndexRequest indexRequest = (IndexRequest) actionRequest; + if (Strings.hasText(indexRequest.getPipeline())) { + return true; + } + } + } + + return false; + } + @Override public ActionRequestValidationException validate() { ActionRequestValidationException validationException = null; diff --git a/core/src/main/java/org/elasticsearch/action/index/IndexRequest.java b/core/src/main/java/org/elasticsearch/action/index/IndexRequest.java index 9899a5493aa..387f7566e81 100644 --- a/core/src/main/java/org/elasticsearch/action/index/IndexRequest.java +++ b/core/src/main/java/org/elasticsearch/action/index/IndexRequest.java @@ -155,6 +155,8 @@ public class IndexRequest extends ReplicationRequest implements Do private XContentType contentType = Requests.INDEX_CONTENT_TYPE; + private String pipeline; + public IndexRequest() { } @@ -363,6 +365,21 @@ public class IndexRequest extends ReplicationRequest implements Do return this.ttl; } + /** + * Sets the ingest pipeline to be executed before indexing the document + */ + public IndexRequest setPipeline(String pipeline) { + this.pipeline = pipeline; + return this; + } + + /** + * Returns the ingest pipeline to be executed before indexing the document + */ + public String getPipeline() { + return this.pipeline; + } + /** * The source of the document to index, recopied to a new array if it is unsage. */ @@ -658,6 +675,7 @@ public class IndexRequest extends ReplicationRequest implements Do refresh = in.readBoolean(); version = in.readLong(); versionType = VersionType.fromValue(in.readByte()); + pipeline = in.readOptionalString(); } @Override @@ -679,6 +697,7 @@ public class IndexRequest extends ReplicationRequest implements Do out.writeBoolean(refresh); out.writeLong(version); out.writeByte(versionType.getValue()); + out.writeOptionalString(pipeline); } @Override diff --git a/core/src/main/java/org/elasticsearch/action/index/IndexRequestBuilder.java b/core/src/main/java/org/elasticsearch/action/index/IndexRequestBuilder.java index f7134d84843..4116755e4eb 100644 --- a/core/src/main/java/org/elasticsearch/action/index/IndexRequestBuilder.java +++ b/core/src/main/java/org/elasticsearch/action/index/IndexRequestBuilder.java @@ -278,4 +278,12 @@ public class IndexRequestBuilder extends ReplicationRequestBuilder { + + public static final DeletePipelineAction INSTANCE = new DeletePipelineAction(); + public static final String NAME = "cluster:admin/ingest/pipeline/delete"; + + public DeletePipelineAction() { + super(NAME); + } + + @Override + public DeletePipelineRequestBuilder newRequestBuilder(ElasticsearchClient client) { + return new DeletePipelineRequestBuilder(client, this); + } + + @Override + public WritePipelineResponse newResponse() { + return new WritePipelineResponse(); + } +} diff --git a/core/src/main/java/org/elasticsearch/action/ingest/DeletePipelineRequest.java b/core/src/main/java/org/elasticsearch/action/ingest/DeletePipelineRequest.java new file mode 100644 index 00000000000..6e5b9d80c67 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/action/ingest/DeletePipelineRequest.java @@ -0,0 +1,70 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.action.ingest; + +import org.elasticsearch.action.ActionRequestValidationException; +import org.elasticsearch.action.support.master.AcknowledgedRequest; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; + +import java.io.IOException; +import java.util.Objects; + +import static org.elasticsearch.action.ValidateActions.addValidationError; + +public class DeletePipelineRequest extends AcknowledgedRequest { + + private String id; + + public DeletePipelineRequest(String id) { + if (id == null) { + throw new IllegalArgumentException("id is missing"); + } + this.id = id; + } + + DeletePipelineRequest() { + } + + public void setId(String id) { + this.id = Objects.requireNonNull(id); + } + + public String getId() { + return id; + } + + @Override + public ActionRequestValidationException validate() { + return null; + } + + @Override + public void readFrom(StreamInput in) throws IOException { + super.readFrom(in); + id = in.readString(); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + out.writeString(id); + } +} diff --git a/core/src/main/java/org/elasticsearch/action/ingest/DeletePipelineRequestBuilder.java b/core/src/main/java/org/elasticsearch/action/ingest/DeletePipelineRequestBuilder.java new file mode 100644 index 00000000000..fc14e0de2df --- /dev/null +++ b/core/src/main/java/org/elasticsearch/action/ingest/DeletePipelineRequestBuilder.java @@ -0,0 +1,35 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.action.ingest; + +import org.elasticsearch.action.ActionRequestBuilder; +import org.elasticsearch.client.ElasticsearchClient; + +public class DeletePipelineRequestBuilder extends ActionRequestBuilder { + + public DeletePipelineRequestBuilder(ElasticsearchClient client, DeletePipelineAction action) { + super(client, action, new DeletePipelineRequest()); + } + + public DeletePipelineRequestBuilder(ElasticsearchClient client, DeletePipelineAction action, String id) { + super(client, action, new DeletePipelineRequest(id)); + } + +} diff --git a/core/src/main/java/org/elasticsearch/action/ingest/DeletePipelineTransportAction.java b/core/src/main/java/org/elasticsearch/action/ingest/DeletePipelineTransportAction.java new file mode 100644 index 00000000000..6378eb5757b --- /dev/null +++ b/core/src/main/java/org/elasticsearch/action/ingest/DeletePipelineTransportAction.java @@ -0,0 +1,71 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.action.ingest; + +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.support.ActionFilters; +import org.elasticsearch.action.support.master.TransportMasterNodeAction; +import org.elasticsearch.cluster.ClusterService; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.block.ClusterBlockException; +import org.elasticsearch.cluster.block.ClusterBlockLevel; +import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; +import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.ingest.PipelineStore; +import org.elasticsearch.node.service.NodeService; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.TransportService; + +public class DeletePipelineTransportAction extends TransportMasterNodeAction { + + private final PipelineStore pipelineStore; + private final ClusterService clusterService; + + @Inject + public DeletePipelineTransportAction(Settings settings, ThreadPool threadPool, ClusterService clusterService, + TransportService transportService, ActionFilters actionFilters, + IndexNameExpressionResolver indexNameExpressionResolver, NodeService nodeService) { + super(settings, DeletePipelineAction.NAME, transportService, clusterService, threadPool, actionFilters, indexNameExpressionResolver, DeletePipelineRequest::new); + this.clusterService = clusterService; + this.pipelineStore = nodeService.getIngestService().getPipelineStore(); + } + + @Override + protected String executor() { + return ThreadPool.Names.SAME; + } + + @Override + protected WritePipelineResponse newResponse() { + return new WritePipelineResponse(); + } + + @Override + protected void masterOperation(DeletePipelineRequest request, ClusterState state, ActionListener listener) throws Exception { + pipelineStore.delete(clusterService, request, listener); + } + + @Override + protected ClusterBlockException checkBlock(DeletePipelineRequest request, ClusterState state) { + return state.blocks().globalBlockedException(ClusterBlockLevel.METADATA_WRITE); + } + +} diff --git a/core/src/main/java/org/elasticsearch/action/ingest/GetPipelineAction.java b/core/src/main/java/org/elasticsearch/action/ingest/GetPipelineAction.java new file mode 100644 index 00000000000..f6bc3d9a778 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/action/ingest/GetPipelineAction.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.action.ingest; + +import org.elasticsearch.action.Action; +import org.elasticsearch.client.ElasticsearchClient; + +public class GetPipelineAction extends Action { + + public static final GetPipelineAction INSTANCE = new GetPipelineAction(); + public static final String NAME = "cluster:admin/ingest/pipeline/get"; + + public GetPipelineAction() { + super(NAME); + } + + @Override + public GetPipelineRequestBuilder newRequestBuilder(ElasticsearchClient client) { + return new GetPipelineRequestBuilder(client, this); + } + + @Override + public GetPipelineResponse newResponse() { + return new GetPipelineResponse(); + } +} diff --git a/core/src/main/java/org/elasticsearch/action/ingest/GetPipelineRequest.java b/core/src/main/java/org/elasticsearch/action/ingest/GetPipelineRequest.java new file mode 100644 index 00000000000..6525c26c6c6 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/action/ingest/GetPipelineRequest.java @@ -0,0 +1,66 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.action.ingest; + +import org.elasticsearch.action.ActionRequestValidationException; +import org.elasticsearch.action.support.master.MasterNodeReadRequest; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; + +import java.io.IOException; +import java.util.Objects; + +import static org.elasticsearch.action.ValidateActions.addValidationError; + +public class GetPipelineRequest extends MasterNodeReadRequest { + + private String[] ids; + + public GetPipelineRequest(String... ids) { + if (ids == null || ids.length == 0) { + throw new IllegalArgumentException("No ids specified"); + } + this.ids = ids; + } + + GetPipelineRequest() { + } + + public String[] getIds() { + return ids; + } + + @Override + public ActionRequestValidationException validate() { + return null; + } + + @Override + public void readFrom(StreamInput in) throws IOException { + super.readFrom(in); + ids = in.readStringArray(); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + out.writeStringArray(ids); + } +} diff --git a/core/src/main/java/org/elasticsearch/action/ingest/GetPipelineRequestBuilder.java b/core/src/main/java/org/elasticsearch/action/ingest/GetPipelineRequestBuilder.java new file mode 100644 index 00000000000..f96a5ffbe24 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/action/ingest/GetPipelineRequestBuilder.java @@ -0,0 +1,35 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.action.ingest; + +import org.elasticsearch.action.support.master.MasterNodeReadOperationRequestBuilder; +import org.elasticsearch.client.ElasticsearchClient; + +public class GetPipelineRequestBuilder extends MasterNodeReadOperationRequestBuilder { + + public GetPipelineRequestBuilder(ElasticsearchClient client, GetPipelineAction action) { + super(client, action, new GetPipelineRequest()); + } + + public GetPipelineRequestBuilder(ElasticsearchClient client, GetPipelineAction action, String[] ids) { + super(client, action, new GetPipelineRequest(ids)); + } + +} diff --git a/core/src/main/java/org/elasticsearch/action/ingest/GetPipelineResponse.java b/core/src/main/java/org/elasticsearch/action/ingest/GetPipelineResponse.java new file mode 100644 index 00000000000..9f0b229d322 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/action/ingest/GetPipelineResponse.java @@ -0,0 +1,86 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.action.ingest; + +import org.elasticsearch.action.ActionResponse; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.xcontent.StatusToXContent; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.ingest.PipelineConfiguration; +import org.elasticsearch.rest.RestStatus; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +public class GetPipelineResponse extends ActionResponse implements StatusToXContent { + + private List pipelines; + + public GetPipelineResponse() { + } + + public GetPipelineResponse(List pipelines) { + this.pipelines = pipelines; + } + + public List pipelines() { + return pipelines; + } + + @Override + public void readFrom(StreamInput in) throws IOException { + super.readFrom(in); + int size = in.readVInt(); + pipelines = new ArrayList<>(size); + for (int i = 0; i < size; i++) { + pipelines.add(PipelineConfiguration.readPipelineConfiguration(in)); + } + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + out.writeVInt(pipelines.size()); + for (PipelineConfiguration pipeline : pipelines) { + pipeline.writeTo(out); + } + } + + public boolean isFound() { + return !pipelines.isEmpty(); + } + + @Override + public RestStatus status() { + return isFound() ? RestStatus.OK : RestStatus.NOT_FOUND; + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startArray("pipelines"); + for (PipelineConfiguration pipeline : pipelines) { + pipeline.toXContent(builder, params); + } + builder.endArray(); + return builder; + } +} diff --git a/core/src/main/java/org/elasticsearch/action/ingest/GetPipelineTransportAction.java b/core/src/main/java/org/elasticsearch/action/ingest/GetPipelineTransportAction.java new file mode 100644 index 00000000000..e762d0b8d33 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/action/ingest/GetPipelineTransportAction.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.action.ingest; + +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.support.ActionFilters; +import org.elasticsearch.action.support.master.TransportMasterNodeReadAction; +import org.elasticsearch.cluster.ClusterService; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.block.ClusterBlockException; +import org.elasticsearch.cluster.block.ClusterBlockLevel; +import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; +import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.ingest.PipelineStore; +import org.elasticsearch.node.service.NodeService; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.TransportService; + +public class GetPipelineTransportAction extends TransportMasterNodeReadAction { + + private final PipelineStore pipelineStore; + + @Inject + public GetPipelineTransportAction(Settings settings, ThreadPool threadPool, ClusterService clusterService, + TransportService transportService, ActionFilters actionFilters, + IndexNameExpressionResolver indexNameExpressionResolver, NodeService nodeService) { + super(settings, GetPipelineAction.NAME, transportService, clusterService, threadPool, actionFilters, indexNameExpressionResolver, GetPipelineRequest::new); + this.pipelineStore = nodeService.getIngestService().getPipelineStore(); + } + + @Override + protected String executor() { + return ThreadPool.Names.SAME; + } + + @Override + protected GetPipelineResponse newResponse() { + return new GetPipelineResponse(); + } + + @Override + protected void masterOperation(GetPipelineRequest request, ClusterState state, ActionListener listener) throws Exception { + listener.onResponse(new GetPipelineResponse(pipelineStore.getPipelines(state, request.getIds()))); + } + + @Override + protected ClusterBlockException checkBlock(GetPipelineRequest request, ClusterState state) { + return state.blocks().globalBlockedException(ClusterBlockLevel.METADATA_READ); + } + +} diff --git a/core/src/main/java/org/elasticsearch/action/ingest/IngestActionFilter.java b/core/src/main/java/org/elasticsearch/action/ingest/IngestActionFilter.java new file mode 100644 index 00000000000..b35e24c51f0 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/action/ingest/IngestActionFilter.java @@ -0,0 +1,225 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.action.ingest; + +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.ActionRequest; +import org.elasticsearch.action.ActionResponse; +import org.elasticsearch.action.bulk.BulkAction; +import org.elasticsearch.action.bulk.BulkItemResponse; +import org.elasticsearch.action.bulk.BulkRequest; +import org.elasticsearch.action.bulk.BulkResponse; +import org.elasticsearch.action.index.IndexAction; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.support.ActionFilter; +import org.elasticsearch.action.support.ActionFilterChain; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.component.AbstractComponent; +import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.ingest.PipelineExecutionService; +import org.elasticsearch.node.service.NodeService; +import org.elasticsearch.tasks.Task; + +import java.util.ArrayList; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Set; + +public final class IngestActionFilter extends AbstractComponent implements ActionFilter { + + private final PipelineExecutionService executionService; + + @Inject + public IngestActionFilter(Settings settings, NodeService nodeService) { + super(settings); + this.executionService = nodeService.getIngestService().getPipelineExecutionService(); + } + + @Override + public , Response extends ActionResponse> void apply(Task task, String action, Request request, ActionListener listener, ActionFilterChain chain) { + switch (action) { + case IndexAction.NAME: + IndexRequest indexRequest = (IndexRequest) request; + if (Strings.hasText(indexRequest.getPipeline())) { + processIndexRequest(task, action, listener, chain, (IndexRequest) request); + } else { + chain.proceed(task, action, request, listener); + } + break; + case BulkAction.NAME: + BulkRequest bulkRequest = (BulkRequest) request; + if (bulkRequest.hasIndexRequestsWithPipelines()) { + @SuppressWarnings("unchecked") + ActionListener actionListener = (ActionListener) listener; + processBulkIndexRequest(task, bulkRequest, action, chain, actionListener); + } else { + chain.proceed(task, action, request, listener); + } + break; + default: + chain.proceed(task, action, request, listener); + break; + } + } + + @Override + public void apply(String action, Response response, ActionListener listener, ActionFilterChain chain) { + chain.proceed(action, response, listener); + } + + void processIndexRequest(Task task, String action, ActionListener listener, ActionFilterChain chain, IndexRequest indexRequest) { + + executionService.execute(indexRequest, t -> { + logger.error("failed to execute pipeline [{}]", t, indexRequest.getPipeline()); + listener.onFailure(t); + }, success -> { + // TransportIndexAction uses IndexRequest and same action name on the node that receives the request and the node that + // processes the primary action. This could lead to a pipeline being executed twice for the same + // index request, hence we set the pipeline to null once its execution completed. + indexRequest.setPipeline(null); + chain.proceed(task, action, indexRequest, listener); + }); + } + + void processBulkIndexRequest(Task task, BulkRequest original, String action, ActionFilterChain chain, ActionListener listener) { + BulkRequestModifier bulkRequestModifier = new BulkRequestModifier(original); + executionService.execute(() -> bulkRequestModifier, (indexRequest, throwable) -> { + logger.debug("failed to execute pipeline [{}] for document [{}/{}/{}]", indexRequest.getPipeline(), indexRequest.index(), indexRequest.type(), indexRequest.id(), throwable); + bulkRequestModifier.markCurrentItemAsFailed(throwable); + }, (throwable) -> { + if (throwable != null) { + logger.error("failed to execute pipeline for a bulk request", throwable); + listener.onFailure(throwable); + } else { + BulkRequest bulkRequest = bulkRequestModifier.getBulkRequest(); + ActionListener actionListener = bulkRequestModifier.wrapActionListenerIfNeeded(listener); + if (bulkRequest.requests().isEmpty()) { + // at this stage, the transport bulk action can't deal with a bulk request with no requests, + // so we stop and send an empty response back to the client. + // (this will happen if pre-processing all items in the bulk failed) + actionListener.onResponse(new BulkResponse(new BulkItemResponse[0], 0)); + } else { + chain.proceed(task, action, bulkRequest, actionListener); + } + } + }); + } + + @Override + public int order() { + return Integer.MAX_VALUE; + } + + final static class BulkRequestModifier implements Iterator> { + + final BulkRequest bulkRequest; + final Set failedSlots; + final List itemResponses; + + int currentSlot = -1; + int[] originalSlots; + + BulkRequestModifier(BulkRequest bulkRequest) { + this.bulkRequest = bulkRequest; + this.failedSlots = new HashSet<>(); + this.itemResponses = new ArrayList<>(bulkRequest.requests().size()); + } + + @Override + public ActionRequest next() { + return bulkRequest.requests().get(++currentSlot); + } + + @Override + public boolean hasNext() { + return (currentSlot + 1) < bulkRequest.requests().size(); + } + + BulkRequest getBulkRequest() { + if (itemResponses.isEmpty()) { + return bulkRequest; + } else { + BulkRequest modifiedBulkRequest = new BulkRequest(bulkRequest); + modifiedBulkRequest.refresh(bulkRequest.refresh()); + modifiedBulkRequest.consistencyLevel(bulkRequest.consistencyLevel()); + modifiedBulkRequest.timeout(bulkRequest.timeout()); + + int slot = 0; + originalSlots = new int[bulkRequest.requests().size() - failedSlots.size()]; + for (int i = 0; i < bulkRequest.requests().size(); i++) { + ActionRequest request = bulkRequest.requests().get(i); + if (failedSlots.contains(i) == false) { + modifiedBulkRequest.add(request); + originalSlots[slot++] = i; + } + } + return modifiedBulkRequest; + } + } + + ActionListener wrapActionListenerIfNeeded(ActionListener actionListener) { + if (itemResponses.isEmpty()) { + return actionListener; + } else { + return new IngestBulkResponseListener(originalSlots, itemResponses, actionListener); + } + } + + void markCurrentItemAsFailed(Throwable e) { + IndexRequest indexRequest = (IndexRequest) bulkRequest.requests().get(currentSlot); + // We hit a error during preprocessing a request, so we: + // 1) Remember the request item slot from the bulk, so that we're done processing all requests we know what failed + // 2) Add a bulk item failure for this request + // 3) Continue with the next request in the bulk. + failedSlots.add(currentSlot); + BulkItemResponse.Failure failure = new BulkItemResponse.Failure(indexRequest.index(), indexRequest.type(), indexRequest.id(), e); + itemResponses.add(new BulkItemResponse(currentSlot, indexRequest.opType().lowercase(), failure)); + } + + } + + private final static class IngestBulkResponseListener implements ActionListener { + + private final int[] originalSlots; + private final List itemResponses; + private final ActionListener actionListener; + + IngestBulkResponseListener(int[] originalSlots, List itemResponses, ActionListener actionListener) { + this.itemResponses = itemResponses; + this.actionListener = actionListener; + this.originalSlots = originalSlots; + } + + @Override + public void onResponse(BulkResponse bulkItemResponses) { + for (int i = 0; i < bulkItemResponses.getItems().length; i++) { + itemResponses.add(originalSlots[i], bulkItemResponses.getItems()[i]); + } + actionListener.onResponse(new BulkResponse(itemResponses.toArray(new BulkItemResponse[itemResponses.size()]), bulkItemResponses.getTookInMillis())); + } + + @Override + public void onFailure(Throwable e) { + actionListener.onFailure(e); + } + } +} diff --git a/core/src/main/java/org/elasticsearch/action/ingest/IngestProxyActionFilter.java b/core/src/main/java/org/elasticsearch/action/ingest/IngestProxyActionFilter.java new file mode 100644 index 00000000000..39a4b1fa4e8 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/action/ingest/IngestProxyActionFilter.java @@ -0,0 +1,125 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.elasticsearch.action.ingest; + +import org.elasticsearch.action.Action; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.ActionListenerResponseHandler; +import org.elasticsearch.action.ActionRequest; +import org.elasticsearch.action.ActionResponse; +import org.elasticsearch.action.bulk.BulkAction; +import org.elasticsearch.action.bulk.BulkRequest; +import org.elasticsearch.action.index.IndexAction; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.support.ActionFilter; +import org.elasticsearch.action.support.ActionFilterChain; +import org.elasticsearch.cluster.ClusterService; +import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.cluster.node.DiscoveryNodes; +import org.elasticsearch.common.Randomness; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.tasks.Task; +import org.elasticsearch.transport.TransportResponse; +import org.elasticsearch.transport.TransportService; + +import java.util.concurrent.atomic.AtomicInteger; + +public final class IngestProxyActionFilter implements ActionFilter { + + private final ClusterService clusterService; + private final TransportService transportService; + private final AtomicInteger randomNodeGenerator = new AtomicInteger(Randomness.get().nextInt()); + + @Inject + public IngestProxyActionFilter(ClusterService clusterService, TransportService transportService) { + this.clusterService = clusterService; + this.transportService = transportService; + } + + @Override + public , Response extends ActionResponse> void apply(Task task, String action, Request request, ActionListener listener, ActionFilterChain chain) { + Action ingestAction; + switch (action) { + case IndexAction.NAME: + ingestAction = IndexAction.INSTANCE; + IndexRequest indexRequest = (IndexRequest) request; + if (Strings.hasText(indexRequest.getPipeline())) { + forwardIngestRequest(ingestAction, request, listener); + } else { + chain.proceed(task, action, request, listener); + } + break; + case BulkAction.NAME: + ingestAction = BulkAction.INSTANCE; + BulkRequest bulkRequest = (BulkRequest) request; + if (bulkRequest.hasIndexRequestsWithPipelines()) { + forwardIngestRequest(ingestAction, request, listener); + } else { + chain.proceed(task, action, request, listener); + } + break; + default: + chain.proceed(task, action, request, listener); + break; + } + } + + @SuppressWarnings("unchecked") + private void forwardIngestRequest(Action action, ActionRequest request, ActionListener listener) { + transportService.sendRequest(randomIngestNode(), action.name(), request, new ActionListenerResponseHandler(listener) { + @Override + public TransportResponse newInstance() { + return action.newResponse(); + } + + }); + } + + @Override + public void apply(String action, Response response, ActionListener listener, ActionFilterChain chain) { + chain.proceed(action, response, listener); + } + + @Override + public int order() { + return Integer.MAX_VALUE; + } + + private DiscoveryNode randomIngestNode() { + assert clusterService.localNode().isIngestNode() == false; + DiscoveryNodes nodes = clusterService.state().getNodes(); + DiscoveryNode[] ingestNodes = nodes.getIngestNodes().values().toArray(DiscoveryNode.class); + if (ingestNodes.length == 0) { + throw new IllegalStateException("There are no ingest nodes in this cluster, unable to forward request to an ingest node."); + } + + int index = getNodeNumber(); + return ingestNodes[(index) % ingestNodes.length]; + } + + private int getNodeNumber() { + int index = randomNodeGenerator.incrementAndGet(); + if (index < 0) { + index = 0; + randomNodeGenerator.set(0); + } + return index; + } +} diff --git a/core/src/main/java/org/elasticsearch/action/ingest/PutPipelineAction.java b/core/src/main/java/org/elasticsearch/action/ingest/PutPipelineAction.java new file mode 100644 index 00000000000..8f4b4170f51 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/action/ingest/PutPipelineAction.java @@ -0,0 +1,44 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.action.ingest; + +import org.elasticsearch.action.Action; +import org.elasticsearch.action.index.IndexResponse; +import org.elasticsearch.client.ElasticsearchClient; + +public class PutPipelineAction extends Action { + + public static final PutPipelineAction INSTANCE = new PutPipelineAction(); + public static final String NAME = "cluster:admin/ingest/pipeline/put"; + + public PutPipelineAction() { + super(NAME); + } + + @Override + public PutPipelineRequestBuilder newRequestBuilder(ElasticsearchClient client) { + return new PutPipelineRequestBuilder(client, this); + } + + @Override + public WritePipelineResponse newResponse() { + return new WritePipelineResponse(); + } +} diff --git a/core/src/main/java/org/elasticsearch/action/ingest/PutPipelineRequest.java b/core/src/main/java/org/elasticsearch/action/ingest/PutPipelineRequest.java new file mode 100644 index 00000000000..10416146ba8 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/action/ingest/PutPipelineRequest.java @@ -0,0 +1,79 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.action.ingest; + +import org.elasticsearch.action.ActionRequestValidationException; +import org.elasticsearch.action.support.master.AcknowledgedRequest; +import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; + +import java.io.IOException; +import java.util.Objects; + +import static org.elasticsearch.action.ValidateActions.addValidationError; + +public class PutPipelineRequest extends AcknowledgedRequest { + + private String id; + private BytesReference source; + + public PutPipelineRequest(String id, BytesReference source) { + if (id == null) { + throw new IllegalArgumentException("id is missing"); + } + if (source == null) { + throw new IllegalArgumentException("source is missing"); + } + + this.id = id; + this.source = source; + } + + PutPipelineRequest() { + } + + @Override + public ActionRequestValidationException validate() { + return null; + } + + public String getId() { + return id; + } + + public BytesReference getSource() { + return source; + } + + @Override + public void readFrom(StreamInput in) throws IOException { + super.readFrom(in); + id = in.readString(); + source = in.readBytesReference(); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + out.writeString(id); + out.writeBytesReference(source); + } +} diff --git a/core/src/main/java/org/elasticsearch/action/ingest/PutPipelineRequestBuilder.java b/core/src/main/java/org/elasticsearch/action/ingest/PutPipelineRequestBuilder.java new file mode 100644 index 00000000000..bd927115fb5 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/action/ingest/PutPipelineRequestBuilder.java @@ -0,0 +1,36 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.action.ingest; + +import org.elasticsearch.action.ActionRequestBuilder; +import org.elasticsearch.client.ElasticsearchClient; +import org.elasticsearch.common.bytes.BytesReference; + +public class PutPipelineRequestBuilder extends ActionRequestBuilder { + + public PutPipelineRequestBuilder(ElasticsearchClient client, PutPipelineAction action) { + super(client, action, new PutPipelineRequest()); + } + + public PutPipelineRequestBuilder(ElasticsearchClient client, PutPipelineAction action, String id, BytesReference source) { + super(client, action, new PutPipelineRequest(id, source)); + } + +} diff --git a/core/src/main/java/org/elasticsearch/action/ingest/PutPipelineTransportAction.java b/core/src/main/java/org/elasticsearch/action/ingest/PutPipelineTransportAction.java new file mode 100644 index 00000000000..31a911207ab --- /dev/null +++ b/core/src/main/java/org/elasticsearch/action/ingest/PutPipelineTransportAction.java @@ -0,0 +1,71 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.action.ingest; + +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.support.ActionFilters; +import org.elasticsearch.action.support.master.TransportMasterNodeAction; +import org.elasticsearch.cluster.ClusterService; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.block.ClusterBlockException; +import org.elasticsearch.cluster.block.ClusterBlockLevel; +import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; +import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.ingest.PipelineStore; +import org.elasticsearch.node.service.NodeService; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.TransportService; + +public class PutPipelineTransportAction extends TransportMasterNodeAction { + + private final PipelineStore pipelineStore; + private final ClusterService clusterService; + + @Inject + public PutPipelineTransportAction(Settings settings, ThreadPool threadPool, ClusterService clusterService, + TransportService transportService, ActionFilters actionFilters, + IndexNameExpressionResolver indexNameExpressionResolver, NodeService nodeService) { + super(settings, PutPipelineAction.NAME, transportService, clusterService, threadPool, actionFilters, indexNameExpressionResolver, PutPipelineRequest::new); + this.clusterService = clusterService; + this.pipelineStore = nodeService.getIngestService().getPipelineStore(); + } + + @Override + protected String executor() { + return ThreadPool.Names.SAME; + } + + @Override + protected WritePipelineResponse newResponse() { + return new WritePipelineResponse(); + } + + @Override + protected void masterOperation(PutPipelineRequest request, ClusterState state, ActionListener listener) throws Exception { + pipelineStore.put(clusterService, request, listener); + } + + @Override + protected ClusterBlockException checkBlock(PutPipelineRequest request, ClusterState state) { + return state.blocks().globalBlockedException(ClusterBlockLevel.METADATA_WRITE); + } + +} diff --git a/core/src/main/java/org/elasticsearch/action/ingest/SimulateDocumentBaseResult.java b/core/src/main/java/org/elasticsearch/action/ingest/SimulateDocumentBaseResult.java new file mode 100644 index 00000000000..036703e98f1 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/action/ingest/SimulateDocumentBaseResult.java @@ -0,0 +1,98 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.elasticsearch.action.ingest; + +import org.elasticsearch.ElasticsearchException; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.ingest.core.IngestDocument; + +import java.io.IOException; +import java.util.Collections; + +/** + * Holds the end result of what a pipeline did to sample document provided via the simulate api. + */ +public final class SimulateDocumentBaseResult implements SimulateDocumentResult { + + private static final SimulateDocumentBaseResult PROTOTYPE = new SimulateDocumentBaseResult(new WriteableIngestDocument(new IngestDocument(Collections.emptyMap(), Collections.emptyMap()))); + + private WriteableIngestDocument ingestDocument; + private Exception failure; + + public SimulateDocumentBaseResult(IngestDocument ingestDocument) { + this.ingestDocument = new WriteableIngestDocument(ingestDocument); + } + + private SimulateDocumentBaseResult(WriteableIngestDocument ingestDocument) { + this.ingestDocument = ingestDocument; + } + + public SimulateDocumentBaseResult(Exception failure) { + this.failure = failure; + } + + public IngestDocument getIngestDocument() { + if (ingestDocument == null) { + return null; + } + return ingestDocument.getIngestDocument(); + } + + public Exception getFailure() { + return failure; + } + + public static SimulateDocumentBaseResult readSimulateDocumentSimpleResult(StreamInput in) throws IOException { + return PROTOTYPE.readFrom(in); + } + + @Override + public SimulateDocumentBaseResult readFrom(StreamInput in) throws IOException { + if (in.readBoolean()) { + Exception exception = in.readThrowable(); + return new SimulateDocumentBaseResult(exception); + } + return new SimulateDocumentBaseResult(new WriteableIngestDocument(in)); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + if (failure == null) { + out.writeBoolean(false); + ingestDocument.writeTo(out); + } else { + out.writeBoolean(true); + out.writeThrowable(failure); + } + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject(); + if (failure == null) { + ingestDocument.toXContent(builder, params); + } else { + ElasticsearchException.renderThrowable(builder, params, failure); + } + builder.endObject(); + return builder; + } +} diff --git a/core/src/main/java/org/elasticsearch/action/ingest/SimulateDocumentResult.java b/core/src/main/java/org/elasticsearch/action/ingest/SimulateDocumentResult.java new file mode 100644 index 00000000000..7e7682bc250 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/action/ingest/SimulateDocumentResult.java @@ -0,0 +1,26 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.elasticsearch.action.ingest; + +import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.common.xcontent.ToXContent; + +public interface SimulateDocumentResult extends Writeable, ToXContent { + +} diff --git a/core/src/main/java/org/elasticsearch/action/ingest/SimulateDocumentVerboseResult.java b/core/src/main/java/org/elasticsearch/action/ingest/SimulateDocumentVerboseResult.java new file mode 100644 index 00000000000..d9d705f1018 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/action/ingest/SimulateDocumentVerboseResult.java @@ -0,0 +1,81 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.elasticsearch.action.ingest; + +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.xcontent.XContentBuilder; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +/** + * Holds the result of what a pipeline did to a sample document via the simulate api, but instead of {@link SimulateDocumentBaseResult} + * this result class holds the intermediate result each processor did to the sample document. + */ +public final class SimulateDocumentVerboseResult implements SimulateDocumentResult { + + private static final SimulateDocumentVerboseResult PROTOTYPE = new SimulateDocumentVerboseResult(Collections.emptyList()); + + private final List processorResults; + + public SimulateDocumentVerboseResult(List processorResults) { + this.processorResults = processorResults; + } + + public List getProcessorResults() { + return processorResults; + } + + public static SimulateDocumentVerboseResult readSimulateDocumentVerboseResultFrom(StreamInput in) throws IOException { + return PROTOTYPE.readFrom(in); + } + + @Override + public SimulateDocumentVerboseResult readFrom(StreamInput in) throws IOException { + int size = in.readVInt(); + List processorResults = new ArrayList<>(); + for (int i = 0; i < size; i++) { + processorResults.add(new SimulateProcessorResult(in)); + } + return new SimulateDocumentVerboseResult(processorResults); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeVInt(processorResults.size()); + for (SimulateProcessorResult result : processorResults) { + result.writeTo(out); + } + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject(); + builder.startArray("processor_results"); + for (SimulateProcessorResult processorResult : processorResults) { + processorResult.toXContent(builder, params); + } + builder.endArray(); + builder.endObject(); + return builder; + } +} diff --git a/core/src/main/java/org/elasticsearch/action/ingest/SimulateExecutionService.java b/core/src/main/java/org/elasticsearch/action/ingest/SimulateExecutionService.java new file mode 100644 index 00000000000..30efbe1b0fa --- /dev/null +++ b/core/src/main/java/org/elasticsearch/action/ingest/SimulateExecutionService.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.action.ingest; + +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.ActionRunnable; +import org.elasticsearch.ingest.core.IngestDocument; +import org.elasticsearch.ingest.core.Pipeline; +import org.elasticsearch.ingest.core.Processor; +import org.elasticsearch.ingest.core.CompoundProcessor; +import org.elasticsearch.threadpool.ThreadPool; + +import java.util.ArrayList; +import java.util.List; + +class SimulateExecutionService { + + private static final String THREAD_POOL_NAME = ThreadPool.Names.MANAGEMENT; + + private final ThreadPool threadPool; + + SimulateExecutionService(ThreadPool threadPool) { + this.threadPool = threadPool; + } + + void executeVerboseDocument(Processor processor, IngestDocument ingestDocument, List processorResultList) throws Exception { + if (processor instanceof CompoundProcessor) { + CompoundProcessor cp = (CompoundProcessor) processor; + try { + for (Processor p : cp.getProcessors()) { + executeVerboseDocument(p, ingestDocument, processorResultList); + } + } catch (Exception e) { + for (Processor p : cp.getOnFailureProcessors()) { + executeVerboseDocument(p, ingestDocument, processorResultList); + } + } + } else { + try { + processor.execute(ingestDocument); + processorResultList.add(new SimulateProcessorResult(processor.getTag(), new IngestDocument(ingestDocument))); + } catch (Exception e) { + processorResultList.add(new SimulateProcessorResult(processor.getTag(), e)); + throw e; + } + } + } + + SimulateDocumentResult executeDocument(Pipeline pipeline, IngestDocument ingestDocument, boolean verbose) { + if (verbose) { + List processorResultList = new ArrayList<>(); + IngestDocument currentIngestDocument = new IngestDocument(ingestDocument); + CompoundProcessor pipelineProcessor = new CompoundProcessor(pipeline.getProcessors(), pipeline.getOnFailureProcessors()); + try { + executeVerboseDocument(pipelineProcessor, currentIngestDocument, processorResultList); + } catch (Exception e) { + return new SimulateDocumentBaseResult(e); + } + return new SimulateDocumentVerboseResult(processorResultList); + } else { + try { + pipeline.execute(ingestDocument); + return new SimulateDocumentBaseResult(ingestDocument); + } catch (Exception e) { + return new SimulateDocumentBaseResult(e); + } + } + } + + public void execute(SimulatePipelineRequest.Parsed request, ActionListener listener) { + threadPool.executor(THREAD_POOL_NAME).execute(new ActionRunnable(listener) { + @Override + protected void doRun() throws Exception { + List responses = new ArrayList<>(); + for (IngestDocument ingestDocument : request.getDocuments()) { + responses.add(executeDocument(request.getPipeline(), ingestDocument, request.isVerbose())); + } + listener.onResponse(new SimulatePipelineResponse(request.getPipeline().getId(), request.isVerbose(), responses)); + } + }); + } +} diff --git a/core/src/main/java/org/elasticsearch/action/ingest/SimulatePipelineAction.java b/core/src/main/java/org/elasticsearch/action/ingest/SimulatePipelineAction.java new file mode 100644 index 00000000000..c1d219a4190 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/action/ingest/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.action.ingest; + +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/core/src/main/java/org/elasticsearch/action/ingest/SimulatePipelineRequest.java b/core/src/main/java/org/elasticsearch/action/ingest/SimulatePipelineRequest.java new file mode 100644 index 00000000000..af18ac5db46 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/action/ingest/SimulatePipelineRequest.java @@ -0,0 +1,165 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.action.ingest; + +import org.elasticsearch.action.ActionRequest; +import org.elasticsearch.action.ActionRequestValidationException; +import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.ingest.core.ConfigurationUtils; +import org.elasticsearch.ingest.core.IngestDocument; +import org.elasticsearch.ingest.core.Pipeline; +import org.elasticsearch.ingest.PipelineStore; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Objects; + +import static org.elasticsearch.action.ValidateActions.addValidationError; +import static org.elasticsearch.ingest.core.IngestDocument.MetaData; + +public class SimulatePipelineRequest extends ActionRequest { + + private String id; + private boolean verbose; + private BytesReference source; + + public SimulatePipelineRequest(BytesReference source) { + if (source == null) { + throw new IllegalArgumentException("source is missing"); + } + this.source = source; + } + + SimulatePipelineRequest() { + } + + @Override + public ActionRequestValidationException validate() { + return null; + } + + public String getId() { + return id; + } + + public void setId(String id) { + this.id = id; + } + + public boolean isVerbose() { + return verbose; + } + + public void setVerbose(boolean verbose) { + this.verbose = verbose; + } + + public BytesReference getSource() { + return source; + } + + @Override + public void readFrom(StreamInput in) throws IOException { + super.readFrom(in); + id = in.readString(); + verbose = in.readBoolean(); + source = in.readBytesReference(); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + out.writeString(id); + out.writeBoolean(verbose); + out.writeBytesReference(source); + } + + public static final class Fields { + static final String PIPELINE = "pipeline"; + static final String DOCS = "docs"; + static final String SOURCE = "_source"; + } + + static class Parsed { + private final List documents; + private final Pipeline pipeline; + private final boolean verbose; + + Parsed(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; + } + } + + private static final Pipeline.Factory PIPELINE_FACTORY = new Pipeline.Factory(); + static final String SIMULATED_PIPELINE_ID = "_simulate_pipeline"; + + static Parsed 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 ingestDocumentList = parseDocs(config); + return new Parsed(pipeline, ingestDocumentList, verbose); + } + + static Parsed parse(Map config, boolean verbose, PipelineStore pipelineStore) throws Exception { + Map pipelineConfig = ConfigurationUtils.readMap(config, Fields.PIPELINE); + Pipeline pipeline = PIPELINE_FACTORY.create(SIMULATED_PIPELINE_ID, pipelineConfig, pipelineStore.getProcessorFactoryRegistry()); + List ingestDocumentList = parseDocs(config); + return new Parsed(pipeline, ingestDocumentList, verbose); + } + + private static List parseDocs(Map config) { + List> docs = ConfigurationUtils.readList(config, Fields.DOCS); + List ingestDocumentList = new ArrayList<>(); + for (Map dataMap : docs) { + Map document = ConfigurationUtils.readMap(dataMap, Fields.SOURCE); + IngestDocument ingestDocument = new IngestDocument(ConfigurationUtils.readStringProperty(dataMap, MetaData.INDEX.getFieldName(), "_index"), + ConfigurationUtils.readStringProperty(dataMap, MetaData.TYPE.getFieldName(), "_type"), + ConfigurationUtils.readStringProperty(dataMap, MetaData.ID.getFieldName(), "_id"), + ConfigurationUtils.readOptionalStringProperty(dataMap, MetaData.ROUTING.getFieldName()), + ConfigurationUtils.readOptionalStringProperty(dataMap, MetaData.PARENT.getFieldName()), + ConfigurationUtils.readOptionalStringProperty(dataMap, MetaData.TIMESTAMP.getFieldName()), + ConfigurationUtils.readOptionalStringProperty(dataMap, MetaData.TTL.getFieldName()), + document); + ingestDocumentList.add(ingestDocument); + } + return ingestDocumentList; + } +} diff --git a/core/src/main/java/org/elasticsearch/action/ingest/SimulatePipelineRequestBuilder.java b/core/src/main/java/org/elasticsearch/action/ingest/SimulatePipelineRequestBuilder.java new file mode 100644 index 00000000000..4a13fa111e6 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/action/ingest/SimulatePipelineRequestBuilder.java @@ -0,0 +1,46 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.action.ingest; + +import org.elasticsearch.action.ActionRequestBuilder; +import org.elasticsearch.client.ElasticsearchClient; +import org.elasticsearch.common.bytes.BytesReference; + +public class SimulatePipelineRequestBuilder extends ActionRequestBuilder { + + public SimulatePipelineRequestBuilder(ElasticsearchClient client, SimulatePipelineAction action) { + super(client, action, new SimulatePipelineRequest()); + } + + public SimulatePipelineRequestBuilder(ElasticsearchClient client, SimulatePipelineAction action, BytesReference source) { + super(client, action, new SimulatePipelineRequest(source)); + } + + public SimulatePipelineRequestBuilder setId(String id) { + request.setId(id); + return this; + } + + public SimulatePipelineRequestBuilder setVerbose(boolean verbose) { + request.setVerbose(verbose); + return this; + } + +} diff --git a/core/src/main/java/org/elasticsearch/action/ingest/SimulatePipelineResponse.java b/core/src/main/java/org/elasticsearch/action/ingest/SimulatePipelineResponse.java new file mode 100644 index 00000000000..c7c0822f04a --- /dev/null +++ b/core/src/main/java/org/elasticsearch/action/ingest/SimulatePipelineResponse.java @@ -0,0 +1,103 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.action.ingest; + +import org.elasticsearch.action.ActionResponse; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.xcontent.ToXContent; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentBuilderString; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +public class SimulatePipelineResponse extends ActionResponse implements ToXContent { + private String pipelineId; + private boolean verbose; + private List results; + + public SimulatePipelineResponse() { + + } + + public SimulatePipelineResponse(String pipelineId, boolean verbose, List responses) { + this.pipelineId = pipelineId; + this.verbose = verbose; + this.results = Collections.unmodifiableList(responses); + } + + public String getPipelineId() { + return pipelineId; + } + + public List getResults() { + return results; + } + + public boolean isVerbose() { + return verbose; + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + out.writeString(pipelineId); + out.writeBoolean(verbose); + out.writeVInt(results.size()); + for (SimulateDocumentResult response : results) { + response.writeTo(out); + } + } + + @Override + public void readFrom(StreamInput in) throws IOException { + super.readFrom(in); + this.pipelineId = in.readString(); + boolean verbose = in.readBoolean(); + int responsesLength = in.readVInt(); + results = new ArrayList<>(); + for (int i = 0; i < responsesLength; i++) { + SimulateDocumentResult simulateDocumentResult; + if (verbose) { + simulateDocumentResult = SimulateDocumentVerboseResult.readSimulateDocumentVerboseResultFrom(in); + } else { + simulateDocumentResult = SimulateDocumentBaseResult.readSimulateDocumentSimpleResult(in); + } + results.add(simulateDocumentResult); + } + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startArray(Fields.DOCUMENTS); + for (SimulateDocumentResult response : results) { + response.toXContent(builder, params); + } + builder.endArray(); + return builder; + } + + static final class Fields { + static final XContentBuilderString DOCUMENTS = new XContentBuilderString("docs"); + } +} diff --git a/core/src/main/java/org/elasticsearch/action/ingest/SimulatePipelineTransportAction.java b/core/src/main/java/org/elasticsearch/action/ingest/SimulatePipelineTransportAction.java new file mode 100644 index 00000000000..5640d7c1c8c --- /dev/null +++ b/core/src/main/java/org/elasticsearch/action/ingest/SimulatePipelineTransportAction.java @@ -0,0 +1,66 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.action.ingest; + +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.support.ActionFilters; +import org.elasticsearch.action.support.HandledTransportAction; +import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; +import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.xcontent.XContentHelper; +import org.elasticsearch.ingest.PipelineStore; +import org.elasticsearch.node.service.NodeService; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.TransportService; + +import java.util.Map; + +public class SimulatePipelineTransportAction extends HandledTransportAction { + + private final PipelineStore pipelineStore; + private final SimulateExecutionService executionService; + + @Inject + public SimulatePipelineTransportAction(Settings settings, ThreadPool threadPool, TransportService transportService, ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver, NodeService nodeService) { + super(settings, SimulatePipelineAction.NAME, threadPool, transportService, actionFilters, indexNameExpressionResolver, SimulatePipelineRequest::new); + this.pipelineStore = nodeService.getIngestService().getPipelineStore(); + this.executionService = new SimulateExecutionService(threadPool); + } + + @Override + protected void doExecute(SimulatePipelineRequest request, ActionListener listener) { + final Map source = XContentHelper.convertToMap(request.getSource(), false).v2(); + + final SimulatePipelineRequest.Parsed simulateRequest; + try { + if (request.getId() != null) { + simulateRequest = SimulatePipelineRequest.parseWithPipelineId(request.getId(), source, request.isVerbose(), pipelineStore); + } else { + simulateRequest = SimulatePipelineRequest.parse(source, request.isVerbose(), pipelineStore); + } + } catch (Exception e) { + listener.onFailure(e); + return; + } + + executionService.execute(simulateRequest, listener); + } +} diff --git a/core/src/main/java/org/elasticsearch/action/ingest/SimulateProcessorResult.java b/core/src/main/java/org/elasticsearch/action/ingest/SimulateProcessorResult.java new file mode 100644 index 00000000000..6a38434d4c0 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/action/ingest/SimulateProcessorResult.java @@ -0,0 +1,106 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.elasticsearch.action.ingest; + +import org.elasticsearch.ElasticsearchException; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.common.xcontent.ToXContent; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.ingest.core.AbstractProcessorFactory; +import org.elasticsearch.ingest.core.IngestDocument; + +import java.io.IOException; + +public class SimulateProcessorResult implements Writeable, ToXContent { + private final String processorTag; + private final WriteableIngestDocument ingestDocument; + private final Exception failure; + + public SimulateProcessorResult(StreamInput in) throws IOException { + this.processorTag = in.readString(); + if (in.readBoolean()) { + this.failure = in.readThrowable(); + this.ingestDocument = null; + } else { + this.ingestDocument = new WriteableIngestDocument(in); + this.failure = null; + } + } + + public SimulateProcessorResult(String processorTag, IngestDocument ingestDocument) { + this.processorTag = processorTag; + this.ingestDocument = new WriteableIngestDocument(ingestDocument); + this.failure = null; + } + + public SimulateProcessorResult(String processorTag, Exception failure) { + this.processorTag = processorTag; + this.failure = failure; + this.ingestDocument = null; + } + + public IngestDocument getIngestDocument() { + if (ingestDocument == null) { + return null; + } + return ingestDocument.getIngestDocument(); + } + + public String getProcessorTag() { + return processorTag; + } + + public Exception getFailure() { + return failure; + } + + @Override + public SimulateProcessorResult readFrom(StreamInput in) throws IOException { + return new SimulateProcessorResult(in); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeString(processorTag); + if (failure == null) { + out.writeBoolean(false); + ingestDocument.writeTo(out); + } else { + out.writeBoolean(true); + out.writeThrowable(failure); + } + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject(); + if (processorTag != null) { + builder.field(AbstractProcessorFactory.TAG_KEY, processorTag); + } + if (failure == null) { + ingestDocument.toXContent(builder, params); + } else { + ElasticsearchException.renderThrowable(builder, params, failure); + } + builder.endObject(); + return builder; + } +} diff --git a/core/src/main/java/org/elasticsearch/action/ingest/WritePipelineResponse.java b/core/src/main/java/org/elasticsearch/action/ingest/WritePipelineResponse.java new file mode 100644 index 00000000000..885fd9f35d6 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/action/ingest/WritePipelineResponse.java @@ -0,0 +1,48 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.action.ingest; + +import org.elasticsearch.action.support.master.AcknowledgedResponse; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; + +import java.io.IOException; + +public class WritePipelineResponse extends AcknowledgedResponse { + + WritePipelineResponse() { + } + + public WritePipelineResponse(boolean acknowledge) { + super(acknowledge); + } + + @Override + public void readFrom(StreamInput in) throws IOException { + super.readFrom(in); + readAcknowledged(in); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + writeAcknowledged(out); + } +} diff --git a/core/src/main/java/org/elasticsearch/action/ingest/WriteableIngestDocument.java b/core/src/main/java/org/elasticsearch/action/ingest/WriteableIngestDocument.java new file mode 100644 index 00000000000..342e4bd3a30 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/action/ingest/WriteableIngestDocument.java @@ -0,0 +1,105 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.action.ingest; + +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.common.xcontent.ToXContent; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentBuilderString; +import org.elasticsearch.ingest.core.IngestDocument; + +import java.io.IOException; +import java.util.Collections; +import java.util.Map; +import java.util.Objects; + +final class WriteableIngestDocument implements Writeable, ToXContent { + + private final IngestDocument ingestDocument; + + WriteableIngestDocument(IngestDocument ingestDocument) { + assert ingestDocument != null; + this.ingestDocument = ingestDocument; + } + + WriteableIngestDocument(StreamInput in) throws IOException { + Map sourceAndMetadata = in.readMap(); + @SuppressWarnings("unchecked") + Map ingestMetadata = (Map) in.readGenericValue(); + this.ingestDocument = new IngestDocument(sourceAndMetadata, ingestMetadata); + } + + IngestDocument getIngestDocument() { + return ingestDocument; + } + + + @Override + public WriteableIngestDocument readFrom(StreamInput in) throws IOException { + return new WriteableIngestDocument(in); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeMap(ingestDocument.getSourceAndMetadata()); + out.writeGenericValue(ingestDocument.getIngestMetadata()); + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject("doc"); + Map metadataMap = ingestDocument.extractMetadata(); + for (Map.Entry metadata : metadataMap.entrySet()) { + builder.field(metadata.getKey().getFieldName(), metadata.getValue()); + } + builder.field("_source", ingestDocument.getSourceAndMetadata()); + builder.startObject("_ingest"); + for (Map.Entry ingestMetadata : ingestDocument.getIngestMetadata().entrySet()) { + builder.field(ingestMetadata.getKey(), ingestMetadata.getValue()); + } + builder.endObject(); + builder.endObject(); + return builder; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + WriteableIngestDocument that = (WriteableIngestDocument) o; + return Objects.equals(ingestDocument, that.ingestDocument); + } + + @Override + public int hashCode() { + return Objects.hash(ingestDocument); + } + + @Override + public String toString() { + return ingestDocument.toString(); + } +} diff --git a/core/src/main/java/org/elasticsearch/client/Client.java b/core/src/main/java/org/elasticsearch/client/Client.java index e7461dabfe1..dbcd9121c97 100644 --- a/core/src/main/java/org/elasticsearch/client/Client.java +++ b/core/src/main/java/org/elasticsearch/client/Client.java @@ -51,6 +51,17 @@ import org.elasticsearch.action.indexedscripts.get.GetIndexedScriptResponse; import org.elasticsearch.action.indexedscripts.put.PutIndexedScriptRequest; import org.elasticsearch.action.indexedscripts.put.PutIndexedScriptRequestBuilder; import org.elasticsearch.action.indexedscripts.put.PutIndexedScriptResponse; +import org.elasticsearch.action.ingest.DeletePipelineRequest; +import org.elasticsearch.action.ingest.DeletePipelineRequestBuilder; +import org.elasticsearch.action.ingest.GetPipelineRequest; +import org.elasticsearch.action.ingest.GetPipelineRequestBuilder; +import org.elasticsearch.action.ingest.GetPipelineResponse; +import org.elasticsearch.action.ingest.PutPipelineRequest; +import org.elasticsearch.action.ingest.PutPipelineRequestBuilder; +import org.elasticsearch.action.ingest.SimulatePipelineRequest; +import org.elasticsearch.action.ingest.SimulatePipelineRequestBuilder; +import org.elasticsearch.action.ingest.SimulatePipelineResponse; +import org.elasticsearch.action.ingest.WritePipelineResponse; import org.elasticsearch.action.percolate.MultiPercolateRequest; import org.elasticsearch.action.percolate.MultiPercolateRequestBuilder; import org.elasticsearch.action.percolate.MultiPercolateResponse; @@ -82,6 +93,7 @@ import org.elasticsearch.action.update.UpdateRequestBuilder; import org.elasticsearch.action.update.UpdateResponse; import org.elasticsearch.client.support.Headers; import org.elasticsearch.common.Nullable; +import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.lease.Releasable; import org.elasticsearch.common.settings.Settings; @@ -592,6 +604,66 @@ public interface Client extends ElasticsearchClient, Releasable { void fieldStats(FieldStatsRequest request, ActionListener listener); + /** + * Stores an ingest pipeline + */ + void putPipeline(PutPipelineRequest request, ActionListener listener); + + /** + * Stores an ingest pipeline + */ + ActionFuture putPipeline(PutPipelineRequest request); + + /** + * Stores an ingest pipeline + */ + PutPipelineRequestBuilder preparePutPipeline(String id, BytesReference source); + + /** + * Deletes a stored ingest pipeline + */ + void deletePipeline(DeletePipelineRequest request, ActionListener listener); + + /** + * Deletes a stored ingest pipeline + */ + ActionFuture deletePipeline(DeletePipelineRequest request); + + /** + * Deletes a stored ingest pipeline + */ + DeletePipelineRequestBuilder prepareDeletePipeline(); + + /** + * Returns a stored ingest pipeline + */ + void getPipeline(GetPipelineRequest request, ActionListener listener); + + /** + * Returns a stored ingest pipeline + */ + ActionFuture getPipeline(GetPipelineRequest request); + + /** + * Returns a stored ingest pipeline + */ + GetPipelineRequestBuilder prepareGetPipeline(String... ids); + + /** + * Simulates an ingest pipeline + */ + void simulatePipeline(SimulatePipelineRequest request, ActionListener listener); + + /** + * Simulates an ingest pipeline + */ + ActionFuture simulatePipeline(SimulatePipelineRequest request); + + /** + * Simulates an ingest pipeline + */ + SimulatePipelineRequestBuilder prepareSimulatePipeline(BytesReference source); + /** * Returns this clients settings */ diff --git a/core/src/main/java/org/elasticsearch/client/support/AbstractClient.java b/core/src/main/java/org/elasticsearch/client/support/AbstractClient.java index e5e1bea6cb5..182f31a1bee 100644 --- a/core/src/main/java/org/elasticsearch/client/support/AbstractClient.java +++ b/core/src/main/java/org/elasticsearch/client/support/AbstractClient.java @@ -272,6 +272,21 @@ import org.elasticsearch.action.indexedscripts.put.PutIndexedScriptAction; import org.elasticsearch.action.indexedscripts.put.PutIndexedScriptRequest; import org.elasticsearch.action.indexedscripts.put.PutIndexedScriptRequestBuilder; import org.elasticsearch.action.indexedscripts.put.PutIndexedScriptResponse; +import org.elasticsearch.action.ingest.DeletePipelineAction; +import org.elasticsearch.action.ingest.DeletePipelineRequest; +import org.elasticsearch.action.ingest.DeletePipelineRequestBuilder; +import org.elasticsearch.action.ingest.GetPipelineAction; +import org.elasticsearch.action.ingest.GetPipelineRequest; +import org.elasticsearch.action.ingest.GetPipelineRequestBuilder; +import org.elasticsearch.action.ingest.GetPipelineResponse; +import org.elasticsearch.action.ingest.PutPipelineAction; +import org.elasticsearch.action.ingest.PutPipelineRequest; +import org.elasticsearch.action.ingest.PutPipelineRequestBuilder; +import org.elasticsearch.action.ingest.SimulatePipelineAction; +import org.elasticsearch.action.ingest.SimulatePipelineRequest; +import org.elasticsearch.action.ingest.SimulatePipelineRequestBuilder; +import org.elasticsearch.action.ingest.SimulatePipelineResponse; +import org.elasticsearch.action.ingest.WritePipelineResponse; import org.elasticsearch.action.percolate.MultiPercolateAction; import org.elasticsearch.action.percolate.MultiPercolateRequest; import org.elasticsearch.action.percolate.MultiPercolateRequestBuilder; @@ -319,6 +334,7 @@ import org.elasticsearch.client.ClusterAdminClient; import org.elasticsearch.client.ElasticsearchClient; import org.elasticsearch.client.IndicesAdminClient; import org.elasticsearch.common.Nullable; +import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.component.AbstractComponent; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.threadpool.ThreadPool; @@ -794,6 +810,66 @@ public abstract class AbstractClient extends AbstractComponent implements Client return new FieldStatsRequestBuilder(this, FieldStatsAction.INSTANCE); } + @Override + public void putPipeline(PutPipelineRequest request, ActionListener listener) { + execute(PutPipelineAction.INSTANCE, request, listener); + } + + @Override + public ActionFuture putPipeline(PutPipelineRequest request) { + return execute(PutPipelineAction.INSTANCE, request); + } + + @Override + public PutPipelineRequestBuilder preparePutPipeline(String id, BytesReference source) { + return new PutPipelineRequestBuilder(this, PutPipelineAction.INSTANCE, id, source); + } + + @Override + public void deletePipeline(DeletePipelineRequest request, ActionListener listener) { + execute(DeletePipelineAction.INSTANCE, request, listener); + } + + @Override + public ActionFuture deletePipeline(DeletePipelineRequest request) { + return execute(DeletePipelineAction.INSTANCE, request); + } + + @Override + public DeletePipelineRequestBuilder prepareDeletePipeline() { + return new DeletePipelineRequestBuilder(this, DeletePipelineAction.INSTANCE); + } + + @Override + public void getPipeline(GetPipelineRequest request, ActionListener listener) { + execute(GetPipelineAction.INSTANCE, request, listener); + } + + @Override + public ActionFuture getPipeline(GetPipelineRequest request) { + return execute(GetPipelineAction.INSTANCE, request); + } + + @Override + public GetPipelineRequestBuilder prepareGetPipeline(String... ids) { + return new GetPipelineRequestBuilder(this, GetPipelineAction.INSTANCE, ids); + } + + @Override + public void simulatePipeline(SimulatePipelineRequest request, ActionListener listener) { + execute(SimulatePipelineAction.INSTANCE, request, listener); + } + + @Override + public ActionFuture simulatePipeline(SimulatePipelineRequest request) { + return execute(SimulatePipelineAction.INSTANCE, request); + } + + @Override + public SimulatePipelineRequestBuilder prepareSimulatePipeline(BytesReference source) { + return new SimulatePipelineRequestBuilder(this, SimulatePipelineAction.INSTANCE, source); + } + static class Admin implements AdminClient { private final ClusterAdmin clusterAdmin; diff --git a/core/src/main/java/org/elasticsearch/client/transport/TransportClient.java b/core/src/main/java/org/elasticsearch/client/transport/TransportClient.java index 3d68e642c46..9930a9d1539 100644 --- a/core/src/main/java/org/elasticsearch/client/transport/TransportClient.java +++ b/core/src/main/java/org/elasticsearch/client/transport/TransportClient.java @@ -19,10 +19,6 @@ package org.elasticsearch.client.transport; -import java.util.ArrayList; -import java.util.List; -import java.util.concurrent.TimeUnit; - import org.elasticsearch.Version; import org.elasticsearch.action.Action; import org.elasticsearch.action.ActionListener; @@ -59,6 +55,10 @@ import org.elasticsearch.threadpool.ThreadPoolModule; import org.elasticsearch.transport.TransportService; import org.elasticsearch.transport.netty.NettyTransport; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.TimeUnit; + import static org.elasticsearch.common.settings.Settings.settingsBuilder; /** @@ -116,7 +116,7 @@ public class TransportClient extends AbstractClient { .put("node.client", true) .put(CLIENT_TYPE_SETTING, CLIENT_TYPE); return new PluginsService(settingsBuilder.build(), null, null, pluginClasses); - }; + } /** * Builds a new instance of the transport client. @@ -150,7 +150,7 @@ public class TransportClient extends AbstractClient { // noop } }); - modules.add(new ActionModule(true)); + modules.add(new ActionModule(false, true)); modules.add(new CircuitBreakerModule(settings)); pluginsService.processModules(modules); diff --git a/core/src/main/java/org/elasticsearch/cluster/metadata/MetaData.java b/core/src/main/java/org/elasticsearch/cluster/metadata/MetaData.java index 002d1a51107..0e41dda1888 100644 --- a/core/src/main/java/org/elasticsearch/cluster/metadata/MetaData.java +++ b/core/src/main/java/org/elasticsearch/cluster/metadata/MetaData.java @@ -54,6 +54,7 @@ import org.elasticsearch.index.IndexNotFoundException; import org.elasticsearch.index.store.IndexStoreConfig; import org.elasticsearch.indices.recovery.RecoverySettings; import org.elasticsearch.indices.ttl.IndicesTTLService; +import org.elasticsearch.ingest.IngestMetadata; import org.elasticsearch.rest.RestStatus; import java.io.IOException; @@ -111,6 +112,7 @@ public class MetaData implements Iterable, Diffable, Fr static { // register non plugin custom metadata registerPrototype(RepositoriesMetaData.TYPE, RepositoriesMetaData.PROTO); + registerPrototype(IngestMetadata.TYPE, IngestMetadata.PROTO); } /** diff --git a/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNode.java b/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNode.java index 7dce2172879..e05bab6d4a4 100644 --- a/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNode.java +++ b/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNode.java @@ -32,6 +32,7 @@ import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.common.transport.TransportAddressSerializers; import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.node.Node; import java.io.IOException; import java.util.Collections; @@ -87,6 +88,10 @@ public class DiscoveryNode implements Streamable, ToXContent { return Booleans.isExplicitTrue(data); } + public static boolean ingestNode(Settings settings) { + return Node.NODE_INGEST_SETTING.get(settings); + } + public static final List EMPTY_LIST = Collections.emptyList(); private String nodeName = ""; @@ -316,6 +321,14 @@ public class DiscoveryNode implements Streamable, ToXContent { return masterNode(); } + /** + * Returns a boolean that tells whether this an ingest node or not + */ + public boolean isIngestNode() { + String ingest = attributes.get("ingest"); + return ingest == null ? true : Booleans.parseBooleanExact(ingest); + } + public Version version() { return this.version; } diff --git a/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodes.java b/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodes.java index d07d3c334ac..e24c25dacbb 100644 --- a/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodes.java +++ b/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodes.java @@ -52,16 +52,20 @@ public class DiscoveryNodes extends AbstractDiffable implements private final ImmutableOpenMap nodes; private final ImmutableOpenMap dataNodes; private final ImmutableOpenMap masterNodes; + private final ImmutableOpenMap ingestNodes; private final String masterNodeId; private final String localNodeId; private final Version minNodeVersion; private final Version minNonClientNodeVersion; - private DiscoveryNodes(ImmutableOpenMap nodes, ImmutableOpenMap dataNodes, ImmutableOpenMap masterNodes, String masterNodeId, String localNodeId, Version minNodeVersion, Version minNonClientNodeVersion) { + private DiscoveryNodes(ImmutableOpenMap nodes, ImmutableOpenMap dataNodes, + ImmutableOpenMap masterNodes, ImmutableOpenMap ingestNodes, + String masterNodeId, String localNodeId, Version minNodeVersion, Version minNonClientNodeVersion) { this.nodes = nodes; this.dataNodes = dataNodes; this.masterNodes = masterNodes; + this.ingestNodes = ingestNodes; this.masterNodeId = masterNodeId; this.localNodeId = localNodeId; this.minNodeVersion = minNodeVersion; @@ -164,6 +168,13 @@ public class DiscoveryNodes extends AbstractDiffable implements return masterNodes(); } + /** + * @return All the ingest nodes arranged by their ids + */ + public ImmutableOpenMap getIngestNodes() { + return ingestNodes; + } + /** * Get a {@link Map} of the discovered master and data nodes arranged by their ids * @@ -654,6 +665,7 @@ public class DiscoveryNodes extends AbstractDiffable implements public DiscoveryNodes build() { ImmutableOpenMap.Builder dataNodesBuilder = ImmutableOpenMap.builder(); ImmutableOpenMap.Builder masterNodesBuilder = ImmutableOpenMap.builder(); + ImmutableOpenMap.Builder ingestNodesBuilder = ImmutableOpenMap.builder(); Version minNodeVersion = Version.CURRENT; Version minNonClientNodeVersion = Version.CURRENT; for (ObjectObjectCursor nodeEntry : nodes) { @@ -665,10 +677,16 @@ public class DiscoveryNodes extends AbstractDiffable implements masterNodesBuilder.put(nodeEntry.key, nodeEntry.value); minNonClientNodeVersion = Version.smallest(minNonClientNodeVersion, nodeEntry.value.version()); } + if (nodeEntry.value.isIngestNode()) { + ingestNodesBuilder.put(nodeEntry.key, nodeEntry.value); + } minNodeVersion = Version.smallest(minNodeVersion, nodeEntry.value.version()); } - return new DiscoveryNodes(nodes.build(), dataNodesBuilder.build(), masterNodesBuilder.build(), masterNodeId, localNodeId, minNodeVersion, minNonClientNodeVersion); + return new DiscoveryNodes( + nodes.build(), dataNodesBuilder.build(), masterNodesBuilder.build(), ingestNodesBuilder.build(), + masterNodeId, localNodeId, minNodeVersion, minNonClientNodeVersion + ); } public static DiscoveryNodes readFrom(StreamInput in, @Nullable DiscoveryNode localNode) throws IOException { diff --git a/core/src/main/java/org/elasticsearch/common/network/NetworkModule.java b/core/src/main/java/org/elasticsearch/common/network/NetworkModule.java index 8be907e3072..fab02b66ff6 100644 --- a/core/src/main/java/org/elasticsearch/common/network/NetworkModule.java +++ b/core/src/main/java/org/elasticsearch/common/network/NetworkModule.java @@ -118,6 +118,10 @@ import org.elasticsearch.rest.action.get.RestGetSourceAction; import org.elasticsearch.rest.action.get.RestHeadAction; import org.elasticsearch.rest.action.get.RestMultiGetAction; import org.elasticsearch.rest.action.index.RestIndexAction; +import org.elasticsearch.rest.action.ingest.RestDeletePipelineAction; +import org.elasticsearch.rest.action.ingest.RestGetPipelineAction; +import org.elasticsearch.rest.action.ingest.RestPutPipelineAction; +import org.elasticsearch.rest.action.ingest.RestSimulatePipelineAction; import org.elasticsearch.rest.action.main.RestMainAction; import org.elasticsearch.rest.action.percolate.RestMultiPercolateAction; import org.elasticsearch.rest.action.percolate.RestPercolateAction; @@ -258,7 +262,13 @@ public class NetworkModule extends AbstractModule { RestCatAction.class, // Tasks API - RestListTasksAction.class + RestListTasksAction.class, + + // Ingest API + RestPutPipelineAction.class, + RestGetPipelineAction.class, + RestDeletePipelineAction.class, + RestSimulatePipelineAction.class ); private static final List> builtinCatHandlers = Arrays.asList( diff --git a/core/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java b/core/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java index 4680146dd88..05978a13053 100644 --- a/core/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java +++ b/core/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java @@ -175,7 +175,7 @@ public final class ClusterSettings extends AbstractScopedSettings { NettyHttpServerTransport.SETTING_CORS_ENABLED, NettyHttpServerTransport.SETTING_CORS_MAX_AGE, NettyHttpServerTransport.SETTING_HTTP_DETAILED_ERRORS_ENABLED, - NettyHttpServerTransport.SETTING_PIPELINING, + NettyHttpServerTransport.SETTING_PIPELINING, HierarchyCircuitBreakerService.TOTAL_CIRCUIT_BREAKER_LIMIT_SETTING, HierarchyCircuitBreakerService.FIELDDATA_CIRCUIT_BREAKER_LIMIT_SETTING, HierarchyCircuitBreakerService.FIELDDATA_CIRCUIT_BREAKER_OVERHEAD_SETTING, @@ -252,5 +252,6 @@ public final class ClusterSettings extends AbstractScopedSettings { URLRepository.ALLOWED_URLS_SETTING, URLRepository.REPOSITORIES_LIST_DIRECTORIES_SETTING, URLRepository.REPOSITORIES_URL_SETTING, - URLRepository.SUPPORTED_PROTOCOLS_SETTING))); + URLRepository.SUPPORTED_PROTOCOLS_SETTING, + Node.NODE_INGEST_SETTING))); } diff --git a/core/src/main/java/org/elasticsearch/common/xcontent/ObjectParser.java b/core/src/main/java/org/elasticsearch/common/xcontent/ObjectParser.java index 979a1f2522c..395dcad8221 100644 --- a/core/src/main/java/org/elasticsearch/common/xcontent/ObjectParser.java +++ b/core/src/main/java/org/elasticsearch/common/xcontent/ObjectParser.java @@ -223,7 +223,7 @@ public final class ObjectParser implements BiFunction implements BiFunction consumer.accept(v, objectParser.apply(p, c)), field, ValueType.OBJECT); } + public void declareObjectArray(BiConsumer> consumer, BiFunction objectParser, ParseField field) { + declareField((p, v, c) -> consumer.accept(v, parseArray(p, () -> objectParser.apply(p, c))), field, ValueType.OBJECT_ARRAY); + } + + public void declareObjectOrDefault(BiConsumer consumer, BiFunction objectParser, Supplier defaultValue, ParseField field) { declareField((p, v, c) -> { if (p.currentToken() == XContentParser.Token.VALUE_BOOLEAN) { @@ -333,6 +338,7 @@ public final class ObjectParser implements BiFunction implements MetaData.Custom { + + public final static String TYPE = "ingest"; + public final static IngestMetadata PROTO = new IngestMetadata(); + private static final ParseField PIPELINES_FIELD = new ParseField("pipeline"); + private static final ObjectParser, Void> INGEST_METADATA_PARSER = new ObjectParser<>("ingest_metadata", ArrayList::new); + + static { + INGEST_METADATA_PARSER.declareObjectArray(List::addAll , PipelineConfiguration.getParser(), PIPELINES_FIELD); + } + + + // We can't use Pipeline class directly in cluster state, because we don't have the processor factories around when + // IngestMetadata is registered as custom metadata. + private final Map pipelines; + + private IngestMetadata() { + this.pipelines = Collections.emptyMap(); + } + + public IngestMetadata(Map pipelines) { + this.pipelines = Collections.unmodifiableMap(pipelines); + } + + @Override + public String type() { + return TYPE; + } + + public Map getPipelines() { + return pipelines; + } + + @Override + public MetaData.Custom readFrom(StreamInput in) throws IOException { + int size = in.readVInt(); + Map pipelines = new HashMap<>(size); + for (int i = 0; i < size; i++) { + PipelineConfiguration pipeline = PipelineConfiguration.readPipelineConfiguration(in); + pipelines.put(pipeline.getId(), pipeline); + } + return new IngestMetadata(pipelines); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeVInt(pipelines.size()); + for (PipelineConfiguration pipeline : pipelines.values()) { + pipeline.writeTo(out); + } + } + + @Override + public MetaData.Custom fromXContent(XContentParser parser) throws IOException { + Map pipelines = new HashMap<>(); + List configs = INGEST_METADATA_PARSER.parse(parser); + for (PipelineConfiguration pipeline : configs) { + pipelines.put(pipeline.getId(), pipeline); + } + return new IngestMetadata(pipelines); + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startArray(PIPELINES_FIELD.getPreferredName()); + for (PipelineConfiguration pipeline : pipelines.values()) { + pipeline.toXContent(builder, params); + } + builder.endArray(); + return builder; + } + + @Override + public EnumSet context() { + return MetaData.API_AND_GATEWAY; + } + +} diff --git a/core/src/main/java/org/elasticsearch/ingest/IngestService.java b/core/src/main/java/org/elasticsearch/ingest/IngestService.java new file mode 100644 index 00000000000..bc7cd75070c --- /dev/null +++ b/core/src/main/java/org/elasticsearch/ingest/IngestService.java @@ -0,0 +1,63 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.ingest; + +import org.elasticsearch.cluster.ClusterService; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.script.ScriptService; +import org.elasticsearch.threadpool.ThreadPool; + +import java.io.Closeable; +import java.io.IOException; + +/** + * Instantiates and wires all the services that the ingest plugin will be needing. + * Also the bootstrapper is in charge of starting and stopping the ingest plugin based on the cluster state. + */ +public class IngestService implements Closeable { + + private final PipelineStore pipelineStore; + private final PipelineExecutionService pipelineExecutionService; + private final ProcessorsRegistry processorsRegistry; + + public IngestService(Settings settings, ThreadPool threadPool, ProcessorsRegistry processorsRegistry) { + this.processorsRegistry = processorsRegistry; + this.pipelineStore = new PipelineStore(settings); + this.pipelineExecutionService = new PipelineExecutionService(pipelineStore, threadPool); + } + + public PipelineStore getPipelineStore() { + return pipelineStore; + } + + public PipelineExecutionService getPipelineExecutionService() { + return pipelineExecutionService; + } + + public void setScriptService(ScriptService scriptService) { + pipelineStore.buildProcessorFactoryRegistry(processorsRegistry, scriptService); + } + + @Override + public void close() throws IOException { + pipelineStore.close(); + } + +} diff --git a/core/src/main/java/org/elasticsearch/ingest/InternalTemplateService.java b/core/src/main/java/org/elasticsearch/ingest/InternalTemplateService.java new file mode 100644 index 00000000000..b4b5ce88fcb --- /dev/null +++ b/core/src/main/java/org/elasticsearch/ingest/InternalTemplateService.java @@ -0,0 +1,92 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.ingest; + +import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.ingest.core.TemplateService; +import org.elasticsearch.script.CompiledScript; +import org.elasticsearch.script.ExecutableScript; +import org.elasticsearch.script.Script; +import org.elasticsearch.script.ScriptContext; +import org.elasticsearch.script.ScriptService; + +import java.util.Collections; +import java.util.Map; + +public class InternalTemplateService implements TemplateService { + + private final ScriptService scriptService; + + InternalTemplateService(ScriptService scriptService) { + this.scriptService = scriptService; + } + + @Override + public Template compile(String template) { + int mustacheStart = template.indexOf("{{"); + int mustacheEnd = template.indexOf("}}"); + if (mustacheStart != -1 && mustacheEnd != -1 && mustacheStart < mustacheEnd) { + Script script = new Script(template, ScriptService.ScriptType.INLINE, "mustache", Collections.emptyMap()); + CompiledScript compiledScript = scriptService.compile( + script, + ScriptContext.Standard.INGEST, + null /* we can supply null here, because ingest doesn't use indexed scripts */, + Collections.emptyMap() + ); + return new Template() { + @Override + public String execute(Map model) { + ExecutableScript executableScript = scriptService.executable(compiledScript, model); + Object result = executableScript.run(); + if (result instanceof BytesReference) { + return ((BytesReference) result).toUtf8(); + } + return String.valueOf(result); + } + + @Override + public String getKey() { + return template; + } + }; + } else { + return new StringTemplate(template); + } + } + + class StringTemplate implements Template { + + private final String value; + + public StringTemplate(String value) { + this.value = value; + } + + @Override + public String execute(Map model) { + return value; + } + + @Override + public String getKey() { + return value; + } + } +} diff --git a/core/src/main/java/org/elasticsearch/ingest/PipelineConfiguration.java b/core/src/main/java/org/elasticsearch/ingest/PipelineConfiguration.java new file mode 100644 index 00000000000..90ab2a76c2e --- /dev/null +++ b/core/src/main/java/org/elasticsearch/ingest/PipelineConfiguration.java @@ -0,0 +1,119 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.ingest; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.Build; +import org.elasticsearch.common.ParseField; +import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.common.xcontent.ObjectParser; +import org.elasticsearch.common.xcontent.ToXContent; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentHelper; +import org.elasticsearch.common.xcontent.XContentParser; + +import java.io.IOException; +import java.util.Map; +import java.util.Objects; +import java.util.function.BiFunction; + +/** + * Encapsulates a pipeline's id and configuration as a blob + */ +public final class PipelineConfiguration implements Writeable, ToXContent { + + private final static PipelineConfiguration PROTOTYPE = new PipelineConfiguration(null, null); + + public static PipelineConfiguration readPipelineConfiguration(StreamInput in) throws IOException { + return PROTOTYPE.readFrom(in); + } + private final static ObjectParser PARSER = new ObjectParser<>("pipeline_config", Builder::new); + static { + PARSER.declareString(Builder::setId, new ParseField("id")); + PARSER.declareField((parser, builder, aVoid) -> { + XContentBuilder contentBuilder = XContentBuilder.builder(parser.contentType().xContent()); + XContentHelper.copyCurrentStructure(contentBuilder.generator(), parser); + builder.setConfig(contentBuilder.bytes()); + }, new ParseField("config"), ObjectParser.ValueType.OBJECT); + } + + public static BiFunction getParser() { + return (p, c) -> PARSER.apply(p ,c).build(); + } + private static class Builder { + + private String id; + private BytesReference config; + + void setId(String id) { + this.id = id; + } + + void setConfig(BytesReference config) { + this.config = config; + } + + PipelineConfiguration build() { + return new PipelineConfiguration(id, config); + } + } + + private final String id; + // Store config as bytes reference, because the config is only used when the pipeline store reads the cluster state + // and the way the map of maps config is read requires a deep copy (it removes instead of gets entries to check for unused options) + // also the get pipeline api just directly returns this to the caller + private final BytesReference config; + + public PipelineConfiguration(String id, BytesReference config) { + this.id = id; + this.config = config; + } + + public String getId() { + return id; + } + + public Map getConfigAsMap() { + return XContentHelper.convertToMap(config, true).v2(); + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject(); + builder.field("id", id); + builder.field("config", getConfigAsMap()); + builder.endObject(); + return builder; + } + + @Override + public PipelineConfiguration readFrom(StreamInput in) throws IOException { + return new PipelineConfiguration(in.readString(), in.readBytesReference()); + } + + public void writeTo(StreamOutput out) throws IOException { + out.writeString(id); + out.writeBytesReference(config); + } + +} diff --git a/core/src/main/java/org/elasticsearch/ingest/PipelineExecutionService.java b/core/src/main/java/org/elasticsearch/ingest/PipelineExecutionService.java new file mode 100644 index 00000000000..c6a3b4b843d --- /dev/null +++ b/core/src/main/java/org/elasticsearch/ingest/PipelineExecutionService.java @@ -0,0 +1,124 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.ingest; + +import org.elasticsearch.action.ActionRequest; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.util.concurrent.AbstractRunnable; +import org.elasticsearch.ingest.core.IngestDocument; +import org.elasticsearch.ingest.core.Pipeline; +import org.elasticsearch.threadpool.ThreadPool; + +import java.util.Map; +import java.util.function.BiConsumer; +import java.util.function.Consumer; + +public class PipelineExecutionService { + + private final PipelineStore store; + private final ThreadPool threadPool; + + public PipelineExecutionService(PipelineStore store, ThreadPool threadPool) { + this.store = store; + this.threadPool = threadPool; + } + + public void execute(IndexRequest request, Consumer failureHandler, Consumer completionHandler) { + Pipeline pipeline = getPipeline(request.getPipeline()); + threadPool.executor(ThreadPool.Names.INDEX).execute(new AbstractRunnable() { + + @Override + public void onFailure(Throwable t) { + failureHandler.accept(t); + } + + @Override + protected void doRun() throws Exception { + innerExecute(request, pipeline); + completionHandler.accept(true); + } + }); + } + + public void execute(Iterable> actionRequests, + BiConsumer itemFailureHandler, + Consumer completionHandler) { + threadPool.executor(ThreadPool.Names.INDEX).execute(new AbstractRunnable() { + + @Override + public void onFailure(Throwable t) { + completionHandler.accept(t); + } + + @Override + protected void doRun() throws Exception { + for (ActionRequest actionRequest : actionRequests) { + if ((actionRequest instanceof IndexRequest)) { + IndexRequest indexRequest = (IndexRequest) actionRequest; + if (Strings.hasText(indexRequest.getPipeline())) { + try { + innerExecute(indexRequest, getPipeline(indexRequest.getPipeline())); + //this shouldn't be needed here but we do it for consistency with index api which requires it to prevent double execution + indexRequest.setPipeline(null); + } catch (Throwable e) { + itemFailureHandler.accept(indexRequest, e); + } + } + } + } + completionHandler.accept(null); + } + }); + } + + private void innerExecute(IndexRequest indexRequest, Pipeline pipeline) throws Exception { + String index = indexRequest.index(); + String type = indexRequest.type(); + String id = indexRequest.id(); + String routing = indexRequest.routing(); + String parent = indexRequest.parent(); + String timestamp = indexRequest.timestamp(); + String ttl = indexRequest.ttl() == null ? null : indexRequest.ttl().toString(); + Map sourceAsMap = indexRequest.sourceAsMap(); + IngestDocument ingestDocument = new IngestDocument(index, type, id, routing, parent, timestamp, ttl, sourceAsMap); + pipeline.execute(ingestDocument); + + Map metadataMap = ingestDocument.extractMetadata(); + //it's fine to set all metadata fields all the time, as ingest document holds their starting values + //before ingestion, which might also get modified during ingestion. + indexRequest.index(metadataMap.get(IngestDocument.MetaData.INDEX)); + indexRequest.type(metadataMap.get(IngestDocument.MetaData.TYPE)); + indexRequest.id(metadataMap.get(IngestDocument.MetaData.ID)); + indexRequest.routing(metadataMap.get(IngestDocument.MetaData.ROUTING)); + indexRequest.parent(metadataMap.get(IngestDocument.MetaData.PARENT)); + indexRequest.timestamp(metadataMap.get(IngestDocument.MetaData.TIMESTAMP)); + indexRequest.ttl(metadataMap.get(IngestDocument.MetaData.TTL)); + indexRequest.source(ingestDocument.getSourceAndMetadata()); + } + + private Pipeline getPipeline(String pipelineId) { + Pipeline pipeline = store.get(pipelineId); + if (pipeline == null) { + throw new IllegalArgumentException("pipeline with id [" + pipelineId + "] does not exist"); + } + return pipeline; + } +} diff --git a/core/src/main/java/org/elasticsearch/ingest/PipelineStore.java b/core/src/main/java/org/elasticsearch/ingest/PipelineStore.java new file mode 100644 index 00000000000..805f1e417ec --- /dev/null +++ b/core/src/main/java/org/elasticsearch/ingest/PipelineStore.java @@ -0,0 +1,242 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.ingest; + +import org.apache.lucene.util.IOUtils; +import org.elasticsearch.ResourceNotFoundException; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.ingest.DeletePipelineRequest; +import org.elasticsearch.action.ingest.PutPipelineRequest; +import org.elasticsearch.action.ingest.WritePipelineResponse; +import org.elasticsearch.cluster.AckedClusterStateUpdateTask; +import org.elasticsearch.cluster.ClusterChangedEvent; +import org.elasticsearch.cluster.ClusterService; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.ClusterStateListener; +import org.elasticsearch.cluster.metadata.MetaData; +import org.elasticsearch.common.component.AbstractComponent; +import org.elasticsearch.common.regex.Regex; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.xcontent.XContentHelper; +import org.elasticsearch.ingest.core.Pipeline; +import org.elasticsearch.ingest.core.Processor; +import org.elasticsearch.ingest.core.TemplateService; +import org.elasticsearch.script.ScriptService; + +import java.io.Closeable; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.function.Function; + +public class PipelineStore extends AbstractComponent implements Closeable, ClusterStateListener { + + private final Pipeline.Factory factory = new Pipeline.Factory(); + private Map processorFactoryRegistry; + + // Ideally this should be in IngestMetadata class, but we don't have the processor factories around there. + // We know of all the processor factories when a node with all its plugin have been initialized. Also some + // processor factories rely on other node services. Custom metadata is statically registered when classes + // are loaded, so in the cluster state we just save the pipeline config and here we keep the actual pipelines around. + volatile Map pipelines = new HashMap<>(); + + public PipelineStore(Settings settings) { + super(settings); + } + + public void buildProcessorFactoryRegistry(ProcessorsRegistry processorsRegistry, ScriptService scriptService) { + Map processorFactories = new HashMap<>(); + TemplateService templateService = new InternalTemplateService(scriptService); + for (Map.Entry>> entry : processorsRegistry.entrySet()) { + Processor.Factory processorFactory = entry.getValue().apply(templateService); + processorFactories.put(entry.getKey(), processorFactory); + } + this.processorFactoryRegistry = Collections.unmodifiableMap(processorFactories); + } + + @Override + public void close() throws IOException { + // TODO: When org.elasticsearch.node.Node can close Closable instances we should try to remove this code, + // since any wired closable should be able to close itself + List closeables = new ArrayList<>(); + for (Processor.Factory factory : processorFactoryRegistry.values()) { + if (factory instanceof Closeable) { + closeables.add((Closeable) factory); + } + } + IOUtils.close(closeables); + } + + @Override + public void clusterChanged(ClusterChangedEvent event) { + innerUpdatePipelines(event.state()); + } + + void innerUpdatePipelines(ClusterState state) { + IngestMetadata ingestMetadata = state.getMetaData().custom(IngestMetadata.TYPE); + if (ingestMetadata == null) { + return; + } + + Map pipelines = new HashMap<>(); + for (PipelineConfiguration pipeline : ingestMetadata.getPipelines().values()) { + try { + pipelines.put(pipeline.getId(), constructPipeline(pipeline.getId(), pipeline.getConfigAsMap())); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + this.pipelines = Collections.unmodifiableMap(pipelines); + } + + /** + * Deletes the pipeline specified by id in the request. + */ + public void delete(ClusterService clusterService, DeletePipelineRequest request, ActionListener listener) { + clusterService.submitStateUpdateTask("delete-pipeline-" + request.getId(), new AckedClusterStateUpdateTask(request, listener) { + + @Override + protected WritePipelineResponse newResponse(boolean acknowledged) { + return new WritePipelineResponse(acknowledged); + } + + @Override + public ClusterState execute(ClusterState currentState) throws Exception { + return innerDelete(request, currentState); + } + }); + } + + ClusterState innerDelete(DeletePipelineRequest request, ClusterState currentState) { + IngestMetadata currentIngestMetadata = currentState.metaData().custom(IngestMetadata.TYPE); + if (currentIngestMetadata == null) { + return currentState; + } + Map pipelines = currentIngestMetadata.getPipelines(); + if (pipelines.containsKey(request.getId()) == false) { + throw new ResourceNotFoundException("pipeline [{}] is missing", request.getId()); + } else { + pipelines = new HashMap<>(pipelines); + pipelines.remove(request.getId()); + ClusterState.Builder newState = ClusterState.builder(currentState); + newState.metaData(MetaData.builder(currentState.getMetaData()) + .putCustom(IngestMetadata.TYPE, new IngestMetadata(pipelines)) + .build()); + return newState.build(); + } + } + + /** + * Stores the specified pipeline definition in the request. + * + * @throws IllegalArgumentException If the pipeline holds incorrect configuration + */ + public void put(ClusterService clusterService, PutPipelineRequest request, ActionListener listener) throws IllegalArgumentException { + try { + // validates the pipeline and processor configuration before submitting a cluster update task: + Map pipelineConfig = XContentHelper.convertToMap(request.getSource(), false).v2(); + constructPipeline(request.getId(), pipelineConfig); + } catch (Exception e) { + throw new IllegalArgumentException("Invalid pipeline configuration", e); + } + clusterService.submitStateUpdateTask("put-pipeline-" + request.getId(), new AckedClusterStateUpdateTask(request, listener) { + + @Override + protected WritePipelineResponse newResponse(boolean acknowledged) { + return new WritePipelineResponse(acknowledged); + } + + @Override + public ClusterState execute(ClusterState currentState) throws Exception { + return innerPut(request, currentState); + } + }); + } + + ClusterState innerPut(PutPipelineRequest request, ClusterState currentState) { + IngestMetadata currentIngestMetadata = currentState.metaData().custom(IngestMetadata.TYPE); + Map pipelines; + if (currentIngestMetadata != null) { + pipelines = new HashMap<>(currentIngestMetadata.getPipelines()); + } else { + pipelines = new HashMap<>(); + } + + pipelines.put(request.getId(), new PipelineConfiguration(request.getId(), request.getSource())); + ClusterState.Builder newState = ClusterState.builder(currentState); + newState.metaData(MetaData.builder(currentState.getMetaData()) + .putCustom(IngestMetadata.TYPE, new IngestMetadata(pipelines)) + .build()); + return newState.build(); + } + + /** + * Returns the pipeline by the specified id + */ + public Pipeline get(String id) { + return pipelines.get(id); + } + + public Map getProcessorFactoryRegistry() { + return processorFactoryRegistry; + } + + /** + * @return pipeline configuration specified by id. If multiple ids or wildcards are specified multiple pipelines + * may be returned + */ + // Returning PipelineConfiguration instead of Pipeline, because Pipeline and Processor interface don't + // know how to serialize themselves. + public List getPipelines(ClusterState clusterState, String... ids) { + IngestMetadata ingestMetadata = clusterState.getMetaData().custom(IngestMetadata.TYPE); + return innerGetPipelines(ingestMetadata, ids); + } + + List innerGetPipelines(IngestMetadata ingestMetadata, String... ids) { + if (ingestMetadata == null) { + return Collections.emptyList(); + } + + List result = new ArrayList<>(ids.length); + for (String id : ids) { + if (Regex.isSimpleMatchPattern(id)) { + for (Map.Entry entry : ingestMetadata.getPipelines().entrySet()) { + if (Regex.simpleMatch(id, entry.getKey())) { + result.add(entry.getValue()); + } + } + } else { + PipelineConfiguration pipeline = ingestMetadata.getPipelines().get(id); + if (pipeline != null) { + result.add(pipeline); + } + } + } + return result; + } + + private Pipeline constructPipeline(String id, Map config) throws Exception { + return factory.create(id, config, processorFactoryRegistry); + } + +} diff --git a/core/src/main/java/org/elasticsearch/ingest/ProcessorsRegistry.java b/core/src/main/java/org/elasticsearch/ingest/ProcessorsRegistry.java new file mode 100644 index 00000000000..766ba772932 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/ingest/ProcessorsRegistry.java @@ -0,0 +1,47 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.ingest; + +import org.elasticsearch.ingest.core.Processor; +import org.elasticsearch.ingest.core.TemplateService; + +import java.util.HashMap; +import java.util.Map; +import java.util.Set; +import java.util.function.Function; + +public class ProcessorsRegistry { + + private final Map>> processorFactoryProviders = new HashMap<>(); + + /** + * Adds a processor factory under a specific name. + */ + public void registerProcessor(String name, Function> processorFactoryProvider) { + Function> provider = processorFactoryProviders.putIfAbsent(name, processorFactoryProvider); + if (provider != null) { + throw new IllegalArgumentException("Processor factory already registered for name [" + name + "]"); + } + } + + public Set>>> entrySet() { + return processorFactoryProviders.entrySet(); + } +} diff --git a/core/src/main/java/org/elasticsearch/ingest/core/AbstractProcessor.java b/core/src/main/java/org/elasticsearch/ingest/core/AbstractProcessor.java new file mode 100644 index 00000000000..e709ae395cb --- /dev/null +++ b/core/src/main/java/org/elasticsearch/ingest/core/AbstractProcessor.java @@ -0,0 +1,38 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + + +package org.elasticsearch.ingest.core; + +/** + * An Abstract Processor that holds a processorTag field to be used + * by other processors. + */ +public abstract class AbstractProcessor implements Processor { + protected final String tag; + + protected AbstractProcessor(String tag) { + this.tag = tag; + } + + @Override + public String getTag() { + return tag; + } +} diff --git a/core/src/main/java/org/elasticsearch/ingest/core/AbstractProcessorFactory.java b/core/src/main/java/org/elasticsearch/ingest/core/AbstractProcessorFactory.java new file mode 100644 index 00000000000..1082461845e --- /dev/null +++ b/core/src/main/java/org/elasticsearch/ingest/core/AbstractProcessorFactory.java @@ -0,0 +1,39 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + + +package org.elasticsearch.ingest.core; + +import java.util.Map; + +/** + * A processor implementation may modify the data belonging to a document. + * Whether changes are made and what exactly is modified is up to the implementation. + */ +public abstract class AbstractProcessorFactory

implements Processor.Factory

{ + public static final String TAG_KEY = "tag"; + + @Override + public P create(Map config) throws Exception { + String tag = ConfigurationUtils.readOptionalStringProperty(config, TAG_KEY); + return doCreate(tag, config); + } + + protected abstract P doCreate(String tag, Map config) throws Exception; +} diff --git a/core/src/main/java/org/elasticsearch/ingest/core/CompoundProcessor.java b/core/src/main/java/org/elasticsearch/ingest/core/CompoundProcessor.java new file mode 100644 index 00000000000..bc5fd19aac7 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/ingest/core/CompoundProcessor.java @@ -0,0 +1,98 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +//TODO(simonw): can all these classes go into org.elasticsearch.ingest? + +package org.elasticsearch.ingest.core; + +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.stream.Collectors; + +/** + * A Processor that executes a list of other "processors". It executes a separate list of + * "onFailureProcessors" when any of the processors throw an {@link Exception}. + */ +public class CompoundProcessor implements Processor { + static final String ON_FAILURE_MESSAGE_FIELD = "on_failure_message"; + static final String ON_FAILURE_PROCESSOR_FIELD = "on_failure_processor"; + + private final List processors; + private final List onFailureProcessors; + + public CompoundProcessor(Processor... processor) { + this(Arrays.asList(processor), Collections.emptyList()); + } + + public CompoundProcessor(List processors, List onFailureProcessors) { + super(); + this.processors = processors; + this.onFailureProcessors = onFailureProcessors; + } + + public List getOnFailureProcessors() { + return onFailureProcessors; + } + + public List getProcessors() { + return processors; + } + + @Override + public String getType() { + return "compound"; + } + + @Override + public String getTag() { + return "compound-processor-" + Objects.hash(processors, onFailureProcessors); + } + + @Override + public void execute(IngestDocument ingestDocument) throws Exception { + for (Processor processor : processors) { + try { + processor.execute(ingestDocument); + } catch (Exception e) { + if (onFailureProcessors.isEmpty()) { + throw e; + } else { + executeOnFailure(ingestDocument, e, processor.getType()); + } + break; + } + } + } + + void executeOnFailure(IngestDocument ingestDocument, Exception cause, String failedProcessorType) throws Exception { + Map ingestMetadata = ingestDocument.getIngestMetadata(); + try { + ingestMetadata.put(ON_FAILURE_MESSAGE_FIELD, cause.getMessage()); + ingestMetadata.put(ON_FAILURE_PROCESSOR_FIELD, failedProcessorType); + for (Processor processor : onFailureProcessors) { + processor.execute(ingestDocument); + } + } finally { + ingestMetadata.remove(ON_FAILURE_MESSAGE_FIELD); + ingestMetadata.remove(ON_FAILURE_PROCESSOR_FIELD); + } + } +} diff --git a/core/src/main/java/org/elasticsearch/ingest/core/ConfigurationUtils.java b/core/src/main/java/org/elasticsearch/ingest/core/ConfigurationUtils.java new file mode 100644 index 00000000000..c6204166908 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/ingest/core/ConfigurationUtils.java @@ -0,0 +1,163 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.ingest.core; + +import java.util.List; +import java.util.Map; + +public final class ConfigurationUtils { + + private ConfigurationUtils() { + } + + /** + * Returns and removes the specified optional property from the specified configuration map. + * + * If the property value isn't of type string a {@link IllegalArgumentException} is thrown. + */ + public static String readOptionalStringProperty(Map configuration, String propertyName) { + Object value = configuration.remove(propertyName); + return readString(propertyName, value); + } + + /** + * Returns and removes the specified property from the specified configuration map. + * + * If the property value isn't of type string an {@link IllegalArgumentException} is thrown. + * If the property is missing an {@link IllegalArgumentException} is thrown + */ + public static String readStringProperty(Map configuration, String propertyName) { + return readStringProperty(configuration, propertyName, null); + } + + /** + * Returns and removes the specified property from the specified configuration map. + * + * If the property value isn't of type string a {@link IllegalArgumentException} is thrown. + * If the property is missing and no default value has been specified a {@link IllegalArgumentException} is thrown + */ + public static String readStringProperty(Map configuration, String propertyName, String defaultValue) { + Object value = configuration.remove(propertyName); + if (value == null && defaultValue != null) { + return defaultValue; + } else if (value == null) { + throw new IllegalArgumentException("required property [" + propertyName + "] is missing"); + } + return readString(propertyName, value); + } + + private static String readString(String propertyName, Object value) { + if (value == null) { + return null; + } + if (value instanceof String) { + return (String) value; + } + throw new IllegalArgumentException("property [" + propertyName + "] isn't a string, but of type [" + value.getClass().getName() + "]"); + } + + /** + * Returns and removes the specified property of type list from the specified configuration map. + * + * If the property value isn't of type list an {@link IllegalArgumentException} is thrown. + */ + public static List readOptionalList(Map configuration, String propertyName) { + Object value = configuration.remove(propertyName); + if (value == null) { + return null; + } + return readList(propertyName, value); + } + + /** + * Returns and removes the specified property of type list from the specified configuration map. + * + * If the property value isn't of type list an {@link IllegalArgumentException} is thrown. + * If the property is missing an {@link IllegalArgumentException} is thrown + */ + public static List readList(Map configuration, String propertyName) { + Object value = configuration.remove(propertyName); + if (value == null) { + throw new IllegalArgumentException("required property [" + propertyName + "] is missing"); + } + + return readList(propertyName, value); + } + + private static List readList(String propertyName, Object value) { + if (value instanceof List) { + @SuppressWarnings("unchecked") + 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. + * + * 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. + * + * If the property value isn't of type map an {@link IllegalArgumentException} is thrown. + */ + public static Map readOptionalMap(Map configuration, String propertyName) { + Object value = configuration.remove(propertyName); + 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; + return map; + } else { + throw new IllegalArgumentException("property [" + propertyName + "] isn't a map, but of type [" + value.getClass().getName() + "]"); + } + } + + /** + * Returns and removes the specified property as an {@link Object} from the specified configuration map. + */ + public static Object readObject(Map configuration, String propertyName) { + Object value = configuration.remove(propertyName); + if (value == null) { + throw new IllegalArgumentException("required property [" + propertyName + "] is missing"); + } + return value; + } +} diff --git a/core/src/main/java/org/elasticsearch/ingest/core/IngestDocument.java b/core/src/main/java/org/elasticsearch/ingest/core/IngestDocument.java new file mode 100644 index 00000000000..c8f87faa53e --- /dev/null +++ b/core/src/main/java/org/elasticsearch/ingest/core/IngestDocument.java @@ -0,0 +1,544 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.ingest.core; + +import org.elasticsearch.common.Strings; +import org.elasticsearch.index.mapper.internal.IdFieldMapper; +import org.elasticsearch.index.mapper.internal.IndexFieldMapper; +import org.elasticsearch.index.mapper.internal.ParentFieldMapper; +import org.elasticsearch.index.mapper.internal.RoutingFieldMapper; +import org.elasticsearch.index.mapper.internal.SourceFieldMapper; +import org.elasticsearch.index.mapper.internal.TTLFieldMapper; +import org.elasticsearch.index.mapper.internal.TimestampFieldMapper; +import org.elasticsearch.index.mapper.internal.TypeFieldMapper; + +import java.text.DateFormat; +import java.text.SimpleDateFormat; +import java.util.ArrayList; +import java.util.Date; +import java.util.HashMap; +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.Objects; +import java.util.TimeZone; + +/** + * Represents a single document being captured before indexing and holds the source and metadata (like id, type and index). + */ +public final class IngestDocument { + + public final static String INGEST_KEY = "_ingest"; + + static final String TIMESTAMP = "timestamp"; + + private final Map sourceAndMetadata; + private final Map ingestMetadata; + + public IngestDocument(String index, String type, String id, String routing, String parent, String timestamp, String ttl, Map source) { + this.sourceAndMetadata = new HashMap<>(); + this.sourceAndMetadata.putAll(source); + this.sourceAndMetadata.put(MetaData.INDEX.getFieldName(), index); + this.sourceAndMetadata.put(MetaData.TYPE.getFieldName(), type); + this.sourceAndMetadata.put(MetaData.ID.getFieldName(), id); + if (routing != null) { + this.sourceAndMetadata.put(MetaData.ROUTING.getFieldName(), routing); + } + if (parent != null) { + this.sourceAndMetadata.put(MetaData.PARENT.getFieldName(), parent); + } + if (timestamp != null) { + this.sourceAndMetadata.put(MetaData.TIMESTAMP.getFieldName(), timestamp); + } + if (ttl != null) { + this.sourceAndMetadata.put(MetaData.TTL.getFieldName(), ttl); + } + + this.ingestMetadata = new HashMap<>(); + DateFormat df = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSSZZ", Locale.ROOT); + df.setTimeZone(TimeZone.getTimeZone("UTC")); + this.ingestMetadata.put(TIMESTAMP, df.format(new Date())); + } + + /** + * Copy constructor that creates a new {@link IngestDocument} which has exactly the same properties as the one provided as argument + */ + public IngestDocument(IngestDocument other) { + this(new HashMap<>(other.sourceAndMetadata), new HashMap<>(other.ingestMetadata)); + } + + /** + * Constructor needed for testing that allows to create a new {@link IngestDocument} given the provided elasticsearch metadata, + * source and ingest metadata. This is needed because the ingest metadata will be initialized with the current timestamp at + * init time, which makes equality comparisons impossible in tests. + */ + public IngestDocument(Map sourceAndMetadata, Map ingestMetadata) { + this.sourceAndMetadata = sourceAndMetadata; + this.ingestMetadata = ingestMetadata; + } + + /** + * Returns the value contained in the document for the provided path + * @param path The path within the document in dot-notation + * @param clazz The expected class of the field value + * @return the value for the provided path if existing, null otherwise + * @throws IllegalArgumentException if the path is null, empty, invalid, if the field doesn't exist + * or if the field that is found at the provided path is not of the expected type. + */ + public T getFieldValue(String path, Class clazz) { + FieldPath fieldPath = new FieldPath(path); + Object context = fieldPath.initialContext; + for (String pathElement : fieldPath.pathElements) { + context = resolve(pathElement, path, context); + } + return cast(path, context, clazz); + } + + /** + * Checks whether the document contains a value for the provided path + * @param path The path within the document in dot-notation + * @return true if the document contains a value for the field, false otherwise + * @throws IllegalArgumentException if the path is null, empty or invalid. + */ + public boolean hasField(String path) { + FieldPath fieldPath = new FieldPath(path); + Object context = fieldPath.initialContext; + for (int i = 0; i < fieldPath.pathElements.length - 1; i++) { + String pathElement = fieldPath.pathElements[i]; + if (context == null) { + return false; + } + if (context instanceof Map) { + @SuppressWarnings("unchecked") + Map map = (Map) context; + context = map.get(pathElement); + } else if (context instanceof List) { + @SuppressWarnings("unchecked") + List list = (List) context; + try { + int index = Integer.parseInt(pathElement); + if (index < 0 || index >= list.size()) { + return false; + } + context = list.get(index); + } catch (NumberFormatException e) { + return false; + } + + } else { + return false; + } + } + + String leafKey = fieldPath.pathElements[fieldPath.pathElements.length - 1]; + if (context instanceof Map) { + @SuppressWarnings("unchecked") + Map map = (Map) context; + return map.containsKey(leafKey); + } + if (context instanceof List) { + @SuppressWarnings("unchecked") + List list = (List) context; + try { + int index = Integer.parseInt(leafKey); + return index >= 0 && index < list.size(); + } catch (NumberFormatException e) { + return false; + } + } + return false; + } + + /** + * Removes the field identified by the provided path. + * @param fieldPathTemplate Resolves to the path with dot-notation within the document + * @throws IllegalArgumentException if the path is null, empty, invalid or if the field doesn't exist. + */ + public void removeField(TemplateService.Template fieldPathTemplate) { + removeField(renderTemplate(fieldPathTemplate)); + } + + /** + * Removes the field identified by the provided path. + * @param path the path of the field to be removed + * @throws IllegalArgumentException if the path is null, empty, invalid or if the field doesn't exist. + */ + public void removeField(String path) { + FieldPath fieldPath = new FieldPath(path); + Object context = fieldPath.initialContext; + for (int i = 0; i < fieldPath.pathElements.length - 1; i++) { + context = resolve(fieldPath.pathElements[i], path, context); + } + + String leafKey = fieldPath.pathElements[fieldPath.pathElements.length - 1]; + if (context instanceof Map) { + @SuppressWarnings("unchecked") + Map map = (Map) context; + if (map.containsKey(leafKey)) { + map.remove(leafKey); + return; + } + throw new IllegalArgumentException("field [" + leafKey + "] not present as part of path [" + path + "]"); + } + if (context instanceof List) { + @SuppressWarnings("unchecked") + List list = (List) context; + int index; + try { + index = Integer.parseInt(leafKey); + } catch (NumberFormatException e) { + throw new IllegalArgumentException("[" + leafKey + "] is not an integer, cannot be used as an index as part of path [" + path + "]", e); + } + if (index < 0 || index >= list.size()) { + throw new IllegalArgumentException("[" + index + "] is out of bounds for array with length [" + list.size() + "] as part of path [" + path + "]"); + } + list.remove(index); + return; + } + + if (context == null) { + throw new IllegalArgumentException("cannot remove [" + leafKey + "] from null as part of path [" + path + "]"); + } + throw new IllegalArgumentException("cannot remove [" + leafKey + "] from object of type [" + context.getClass().getName() + "] as part of path [" + path + "]"); + } + + private static Object resolve(String pathElement, String fullPath, Object context) { + if (context == null) { + throw new IllegalArgumentException("cannot resolve [" + pathElement + "] from null as part of path [" + fullPath + "]"); + } + if (context instanceof Map) { + @SuppressWarnings("unchecked") + Map map = (Map) context; + if (map.containsKey(pathElement)) { + return map.get(pathElement); + } + throw new IllegalArgumentException("field [" + pathElement + "] not present as part of path [" + fullPath + "]"); + } + if (context instanceof List) { + @SuppressWarnings("unchecked") + List list = (List) context; + int index; + try { + index = Integer.parseInt(pathElement); + } catch (NumberFormatException e) { + throw new IllegalArgumentException("[" + pathElement + "] is not an integer, cannot be used as an index as part of path [" + fullPath + "]", e); + } + if (index < 0 || index >= list.size()) { + throw new IllegalArgumentException("[" + index + "] is out of bounds for array with length [" + list.size() + "] as part of path [" + fullPath + "]"); + } + return list.get(index); + } + throw new IllegalArgumentException("cannot resolve [" + pathElement + "] from object of type [" + context.getClass().getName() + "] as part of path [" + fullPath + "]"); + } + + /** + * Appends the provided value to the provided path in the document. + * Any non existing path element will be created. + * If the path identifies a list, the value will be appended to the existing list. + * If the path identifies a scalar, the scalar will be converted to a list and + * the provided value will be added to the newly created list. + * Supports multiple values too provided in forms of list, in that case all the values will be appeneded to the + * existing (or newly created) list. + * @param path The path within the document in dot-notation + * @param value The value or values to append to the existing ones + * @throws IllegalArgumentException if the path is null, empty or invalid. + */ + public void appendFieldValue(String path, Object value) { + setFieldValue(path, value, true); + } + + /** + * Appends the provided value to the provided path in the document. + * Any non existing path element will be created. + * If the path identifies a list, the value will be appended to the existing list. + * If the path identifies a scalar, the scalar will be converted to a list and + * the provided value will be added to the newly created list. + * Supports multiple values too provided in forms of list, in that case all the values will be appeneded to the + * existing (or newly created) list. + * @param fieldPathTemplate Resolves to the path with dot-notation within the document + * @param valueSource The value source that will produce the value or values to append to the existing ones + * @throws IllegalArgumentException if the path is null, empty or invalid. + */ + public void appendFieldValue(TemplateService.Template fieldPathTemplate, ValueSource valueSource) { + Map model = createTemplateModel(); + appendFieldValue(fieldPathTemplate.execute(model), valueSource.copyAndResolve(model)); + } + + /** + * Sets the provided value to the provided path in the document. + * Any non existing path element will be created. + * If the last item in the path is a list, the value will replace the existing list as a whole. + * Use {@link #appendFieldValue(String, Object)} to append values to lists instead. + * @param path The path within the document in dot-notation + * @param value The value to put in for the path key + * @throws IllegalArgumentException if the path is null, empty, invalid or if the value cannot be set to the + * item identified by the provided path. + */ + public void setFieldValue(String path, Object value) { + setFieldValue(path, value, false); + } + + /** + * Sets the provided value to the provided path in the document. + * Any non existing path element will be created. If the last element is a list, + * the value will replace the existing list. + * @param fieldPathTemplate Resolves to the path with dot-notation within the document + * @param valueSource The value source that will produce the value to put in for the path key + * @throws IllegalArgumentException if the path is null, empty, invalid or if the value cannot be set to the + * item identified by the provided path. + */ + public void setFieldValue(TemplateService.Template fieldPathTemplate, ValueSource valueSource) { + Map model = createTemplateModel(); + setFieldValue(fieldPathTemplate.execute(model), valueSource.copyAndResolve(model), false); + } + + private void setFieldValue(String path, Object value, boolean append) { + FieldPath fieldPath = new FieldPath(path); + Object context = fieldPath.initialContext; + for (int i = 0; i < fieldPath.pathElements.length - 1; i++) { + String pathElement = fieldPath.pathElements[i]; + if (context == null) { + throw new IllegalArgumentException("cannot resolve [" + pathElement + "] from null as part of path [" + path + "]"); + } + if (context instanceof Map) { + @SuppressWarnings("unchecked") + Map map = (Map) context; + if (map.containsKey(pathElement)) { + context = map.get(pathElement); + } else { + HashMap newMap = new HashMap<>(); + map.put(pathElement, newMap); + context = newMap; + } + } else if (context instanceof List) { + @SuppressWarnings("unchecked") + List list = (List) context; + int index; + try { + index = Integer.parseInt(pathElement); + } catch (NumberFormatException e) { + throw new IllegalArgumentException("[" + pathElement + "] is not an integer, cannot be used as an index as part of path [" + path + "]", e); + } + if (index < 0 || index >= list.size()) { + throw new IllegalArgumentException("[" + index + "] is out of bounds for array with length [" + list.size() + "] as part of path [" + path + "]"); + } + context = list.get(index); + } else { + throw new IllegalArgumentException("cannot resolve [" + pathElement + "] from object of type [" + context.getClass().getName() + "] as part of path [" + path + "]"); + } + } + + String leafKey = fieldPath.pathElements[fieldPath.pathElements.length - 1]; + if (context == null) { + throw new IllegalArgumentException("cannot set [" + leafKey + "] with null parent as part of path [" + path + "]"); + } + if (context instanceof Map) { + @SuppressWarnings("unchecked") + Map map = (Map) context; + if (append) { + if (map.containsKey(leafKey)) { + Object object = map.get(leafKey); + List list = appendValues(object, value); + if (list != object) { + map.put(leafKey, list); + } + } else { + List list = new ArrayList<>(); + appendValues(list, value); + map.put(leafKey, list); + } + return; + } + map.put(leafKey, value); + } else if (context instanceof List) { + @SuppressWarnings("unchecked") + List list = (List) context; + int index; + try { + index = Integer.parseInt(leafKey); + } catch (NumberFormatException e) { + throw new IllegalArgumentException("[" + leafKey + "] is not an integer, cannot be used as an index as part of path [" + path + "]", e); + } + if (index < 0 || index >= list.size()) { + throw new IllegalArgumentException("[" + index + "] is out of bounds for array with length [" + list.size() + "] as part of path [" + path + "]"); + } + if (append) { + Object object = list.get(index); + List newList = appendValues(object, value); + if (newList != object) { + list.set(index, newList); + } + return; + } + list.set(index, value); + } else { + throw new IllegalArgumentException("cannot set [" + leafKey + "] with parent object of type [" + context.getClass().getName() + "] as part of path [" + path + "]"); + } + } + + @SuppressWarnings("unchecked") + private static List appendValues(Object maybeList, Object value) { + List list; + if (maybeList instanceof List) { + //maybeList is already a list, we append the provided values to it + list = (List) maybeList; + } else { + //maybeList is a scalar, we convert it to a list and append the provided values to it + list = new ArrayList<>(); + list.add(maybeList); + } + appendValues(list, value); + return list; + } + + private static void appendValues(List list, Object value) { + if (value instanceof List) { + @SuppressWarnings("unchecked") + List valueList = (List) value; + valueList.stream().forEach(list::add); + } else { + list.add(value); + } + } + + private static T cast(String path, Object object, Class clazz) { + if (object == null) { + return null; + } + if (clazz.isInstance(object)) { + return clazz.cast(object); + } + throw new IllegalArgumentException("field [" + path + "] of type [" + object.getClass().getName() + "] cannot be cast to [" + clazz.getName() + "]"); + } + + public String renderTemplate(TemplateService.Template template) { + return template.execute(createTemplateModel()); + } + + private Map createTemplateModel() { + Map model = new HashMap<>(sourceAndMetadata); + model.put(SourceFieldMapper.NAME, sourceAndMetadata); + // If there is a field in the source with the name '_ingest' it gets overwritten here, + // if access to that field is required then it get accessed via '_source._ingest' + model.put(INGEST_KEY, ingestMetadata); + return model; + } + + /** + * one time operation that extracts the metadata fields from the ingest document and returns them. + * Metadata fields that used to be accessible as ordinary top level fields will be removed as part of this call. + */ + public Map extractMetadata() { + Map metadataMap = new HashMap<>(); + for (MetaData metaData : MetaData.values()) { + metadataMap.put(metaData, cast(metaData.getFieldName(), sourceAndMetadata.remove(metaData.getFieldName()), String.class)); + } + return metadataMap; + } + + /** + * Returns the available ingest metadata fields, by default only timestamp, but it is possible to set additional ones. + * Use only for reading values, modify them instead using {@link #setFieldValue(String, Object)} and {@link #removeField(String)} + */ + public Map getIngestMetadata() { + return this.ingestMetadata; + } + + /** + * Returns the document including its metadata fields, unless {@link #extractMetadata()} has been called, in which case the + * metadata fields will not be present anymore. + * Modify the document instead using {@link #setFieldValue(String, Object)} and {@link #removeField(String)} + */ + public Map getSourceAndMetadata() { + return this.sourceAndMetadata; + } + + @Override + public boolean equals(Object obj) { + if (obj == this) { return true; } + if (obj == null || getClass() != obj.getClass()) { + return false; + } + + IngestDocument other = (IngestDocument) obj; + return Objects.equals(sourceAndMetadata, other.sourceAndMetadata) && + Objects.equals(ingestMetadata, other.ingestMetadata); + } + + @Override + public int hashCode() { + return Objects.hash(sourceAndMetadata, ingestMetadata); + } + + @Override + public String toString() { + return "IngestDocument{" + + " sourceAndMetadata=" + sourceAndMetadata + + ", ingestMetadata=" + ingestMetadata + + '}'; + } + + public enum MetaData { + INDEX(IndexFieldMapper.NAME), + TYPE(TypeFieldMapper.NAME), + ID(IdFieldMapper.NAME), + ROUTING(RoutingFieldMapper.NAME), + PARENT(ParentFieldMapper.NAME), + TIMESTAMP(TimestampFieldMapper.NAME), + TTL(TTLFieldMapper.NAME); + + private final String fieldName; + + MetaData(String fieldName) { + this.fieldName = fieldName; + } + + public String getFieldName() { + return fieldName; + } + } + + private class FieldPath { + private final String[] pathElements; + private final Object initialContext; + + private FieldPath(String path) { + if (Strings.isEmpty(path)) { + throw new IllegalArgumentException("path cannot be null nor empty"); + } + String newPath; + if (path.startsWith(INGEST_KEY + ".")) { + initialContext = ingestMetadata; + newPath = path.substring(8, path.length()); + } else { + initialContext = sourceAndMetadata; + if (path.startsWith(SourceFieldMapper.NAME + ".")) { + newPath = path.substring(8, path.length()); + } else { + newPath = path; + } + } + this.pathElements = Strings.splitStringToArray(newPath, '.'); + if (pathElements.length == 0) { + throw new IllegalArgumentException("path [" + path + "] is not valid"); + } + } + } +} diff --git a/core/src/main/java/org/elasticsearch/ingest/core/Pipeline.java b/core/src/main/java/org/elasticsearch/ingest/core/Pipeline.java new file mode 100644 index 00000000000..68ba8da4855 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/ingest/core/Pipeline.java @@ -0,0 +1,126 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.ingest.core; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Map; + +/** + * A pipeline is a list of {@link Processor} instances grouped under a unique id. + */ +public final class Pipeline { + + final static String DESCRIPTION_KEY = "description"; + final static String PROCESSORS_KEY = "processors"; + final static String ON_FAILURE_KEY = "on_failure"; + + private final String id; + private final String description; + private final CompoundProcessor compoundProcessor; + + public Pipeline(String id, String description, CompoundProcessor compoundProcessor) { + this.id = id; + this.description = description; + this.compoundProcessor = compoundProcessor; + } + + /** + * Modifies the data of a document to be indexed based on the processor this pipeline holds + */ + public void execute(IngestDocument ingestDocument) throws Exception { + compoundProcessor.execute(ingestDocument); + } + + /** + * The unique id of this pipeline + */ + public String getId() { + return id; + } + + /** + * An optional description of what this pipeline is doing to the data gets processed by this pipeline. + */ + public String getDescription() { + return description; + } + + /** + * Unmodifiable list containing each processor that operates on the data. + */ + public List getProcessors() { + return compoundProcessor.getProcessors(); + } + + /** + * Unmodifiable list containing each on_failure processor that operates on the data in case of + * exception thrown in pipeline processors + */ + public List getOnFailureProcessors() { + return compoundProcessor.getOnFailureProcessors(); + } + + public final static class Factory { + + public Pipeline create(String id, Map config, Map processorRegistry) throws Exception { + String description = ConfigurationUtils.readOptionalStringProperty(config, DESCRIPTION_KEY); + List processors = readProcessors(PROCESSORS_KEY, processorRegistry, config); + List onFailureProcessors = readProcessors(ON_FAILURE_KEY, processorRegistry, config); + if (config.isEmpty() == false) { + throw new IllegalArgumentException("pipeline [" + id + "] doesn't support one or more provided configuration parameters " + Arrays.toString(config.keySet().toArray())); + } + CompoundProcessor compoundProcessor = new CompoundProcessor(Collections.unmodifiableList(processors), Collections.unmodifiableList(onFailureProcessors)); + return new Pipeline(id, description, compoundProcessor); + } + + private List readProcessors(String fieldName, Map processorRegistry, Map config) throws Exception { + List>> processorConfigs = ConfigurationUtils.readOptionalList(config, fieldName); + List processors = new ArrayList<>(); + if (processorConfigs != null) { + for (Map> processorConfigWithKey : processorConfigs) { + for (Map.Entry> entry : processorConfigWithKey.entrySet()) { + processors.add(readProcessor(processorRegistry, entry.getKey(), entry.getValue())); + } + } + } + + return processors; + } + + private Processor readProcessor(Map processorRegistry, String type, Map config) throws Exception { + Processor.Factory factory = processorRegistry.get(type); + if (factory != null) { + List onFailureProcessors = readProcessors(ON_FAILURE_KEY, processorRegistry, config); + Processor processor = factory.create(config); + if (config.isEmpty() == false) { + throw new IllegalArgumentException("processor [" + type + "] doesn't support one or more provided configuration parameters " + Arrays.toString(config.keySet().toArray())); + } + if (onFailureProcessors.isEmpty()) { + return processor; + } + return new CompoundProcessor(Collections.singletonList(processor), onFailureProcessors); + } + throw new IllegalArgumentException("No processor type exists with name [" + type + "]"); + } + } +} diff --git a/core/src/main/java/org/elasticsearch/ingest/core/Processor.java b/core/src/main/java/org/elasticsearch/ingest/core/Processor.java new file mode 100644 index 00000000000..f178051b751 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/ingest/core/Processor.java @@ -0,0 +1,59 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + + +package org.elasticsearch.ingest.core; + +import java.util.Map; + +/** + * A processor implementation may modify the data belonging to a document. + * Whether changes are made and what exactly is modified is up to the implementation. + */ +public interface Processor { + + /** + * Introspect and potentially modify the incoming data. + */ + void execute(IngestDocument ingestDocument) throws Exception; + + /** + * Gets the type of a processor + */ + String getType(); + + /** + * Gets the tag of a processor. + */ + String getTag(); + + /** + * A factory that knows how to construct a processor based on a map of maps. + */ + interface Factory

{ + + /** + * Creates a processor based on the specified map of maps config. + * + * Implementations are responsible for removing the used keys, so that after creating a pipeline ingest can + * verify if all configurations settings have been used. + */ + P create(Map config) throws Exception; + } +} diff --git a/core/src/main/java/org/elasticsearch/ingest/core/TemplateService.java b/core/src/main/java/org/elasticsearch/ingest/core/TemplateService.java new file mode 100644 index 00000000000..8988c924c35 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/ingest/core/TemplateService.java @@ -0,0 +1,38 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.elasticsearch.ingest.core; + +import java.util.Map; + +/** + * Abstraction for the ingest template engine used to decouple {@link IngestDocument} from {@link org.elasticsearch.script.ScriptService}. + * Allows to compile a template into an ingest {@link Template} object. + * A compiled template can be executed by calling its {@link Template#execute(Map)} method. + */ +public interface TemplateService { + + Template compile(String template); + + interface Template { + + String execute(Map model); + + String getKey(); + } +} diff --git a/core/src/main/java/org/elasticsearch/ingest/core/ValueSource.java b/core/src/main/java/org/elasticsearch/ingest/core/ValueSource.java new file mode 100644 index 00000000000..e9f09a1a9f8 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/ingest/core/ValueSource.java @@ -0,0 +1,191 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.ingest.core; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; + +/** + * Holds a value. If the value is requested a copy is made and optionally template snippets are resolved too. + */ +public interface ValueSource { + + /** + * Returns a copy of the value this ValueSource holds and resolves templates if there're any. + * + * For immutable values only a copy of the reference to the value is made. + * + * @param model The model to be used when resolving any templates + * @return copy of the wrapped value + */ + Object copyAndResolve(Map model); + + static ValueSource wrap(Object value, TemplateService templateService) { + if (value instanceof Map) { + @SuppressWarnings("unchecked") + Map mapValue = (Map) value; + Map valueTypeMap = new HashMap<>(mapValue.size()); + for (Map.Entry entry : mapValue.entrySet()) { + valueTypeMap.put(wrap(entry.getKey(), templateService), wrap(entry.getValue(), templateService)); + } + return new MapValue(valueTypeMap); + } else if (value instanceof List) { + @SuppressWarnings("unchecked") + List listValue = (List) value; + List valueSourceList = new ArrayList<>(listValue.size()); + for (Object item : listValue) { + valueSourceList.add(wrap(item, templateService)); + } + return new ListValue(valueSourceList); + } else if (value == null || value instanceof Number || value instanceof Boolean) { + return new ObjectValue(value); + } else if (value instanceof String) { + return new TemplatedValue(templateService.compile((String) value)); + } else { + throw new IllegalArgumentException("unexpected value type [" + value.getClass() + "]"); + } + } + + final class MapValue implements ValueSource { + + private final Map map; + + MapValue(Map map) { + this.map = map; + } + + @Override + public Object copyAndResolve(Map model) { + Map copy = new HashMap<>(); + for (Map.Entry entry : this.map.entrySet()) { + copy.put(entry.getKey().copyAndResolve(model), entry.getValue().copyAndResolve(model)); + } + return copy; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + MapValue mapValue = (MapValue) o; + return map.equals(mapValue.map); + + } + + @Override + public int hashCode() { + return map.hashCode(); + } + } + + final class ListValue implements ValueSource { + + private final List values; + + ListValue(List values) { + this.values = values; + } + + @Override + public Object copyAndResolve(Map model) { + List copy = new ArrayList<>(values.size()); + for (ValueSource value : values) { + copy.add(value.copyAndResolve(model)); + } + return copy; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + ListValue listValue = (ListValue) o; + return values.equals(listValue.values); + + } + + @Override + public int hashCode() { + return values.hashCode(); + } + } + + final class ObjectValue implements ValueSource { + + private final Object value; + + ObjectValue(Object value) { + this.value = value; + } + + @Override + public Object copyAndResolve(Map model) { + return value; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + ObjectValue objectValue = (ObjectValue) o; + return Objects.equals(value, objectValue.value); + } + + @Override + public int hashCode() { + return Objects.hashCode(value); + } + } + + final class TemplatedValue implements ValueSource { + + private final TemplateService.Template template; + + TemplatedValue(TemplateService.Template template) { + this.template = template; + } + + @Override + public Object copyAndResolve(Map model) { + return template.execute(model); + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + TemplatedValue templatedValue = (TemplatedValue) o; + return Objects.equals(template.getKey(), templatedValue.template.getKey()); + } + + @Override + public int hashCode() { + return Objects.hashCode(template.getKey()); + } + } + +} diff --git a/core/src/main/java/org/elasticsearch/ingest/processor/AbstractStringProcessor.java b/core/src/main/java/org/elasticsearch/ingest/processor/AbstractStringProcessor.java new file mode 100644 index 00000000000..32e54765b18 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/ingest/processor/AbstractStringProcessor.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.processor; + +import org.elasticsearch.ingest.core.AbstractProcessor; +import org.elasticsearch.ingest.core.AbstractProcessorFactory; +import org.elasticsearch.ingest.core.ConfigurationUtils; +import org.elasticsearch.ingest.core.IngestDocument; +import org.elasticsearch.ingest.core.Processor; + +import java.util.Map; + +/** + * Base class for processors that manipulate strings and require a single "fields" array config value, which + * holds a list of field names in string format. + */ +public abstract class AbstractStringProcessor extends AbstractProcessor { + private final String field; + + protected AbstractStringProcessor(String tag, String field) { + super(tag); + this.field = field; + } + + public String getField() { + return field; + } + + @Override + public final void execute(IngestDocument document) { + String val = document.getFieldValue(field, String.class); + if (val == null) { + throw new IllegalArgumentException("field [" + field + "] is null, cannot process it."); + } + document.setFieldValue(field, process(val)); + } + + protected abstract String process(String value); + + public static abstract class Factory extends AbstractProcessorFactory { + + @Override + public T doCreate(String processorTag, Map config) throws Exception { + String field = ConfigurationUtils.readStringProperty(config, "field"); + return newProcessor(processorTag, field); + } + + protected abstract T newProcessor(String processorTag, String field); + } +} diff --git a/core/src/main/java/org/elasticsearch/ingest/processor/AppendProcessor.java b/core/src/main/java/org/elasticsearch/ingest/processor/AppendProcessor.java new file mode 100644 index 00000000000..deff384cf92 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/ingest/processor/AppendProcessor.java @@ -0,0 +1,82 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.ingest.processor; + +import org.elasticsearch.ingest.core.AbstractProcessor; +import org.elasticsearch.ingest.core.AbstractProcessorFactory; +import org.elasticsearch.ingest.core.IngestDocument; +import org.elasticsearch.ingest.core.TemplateService; +import org.elasticsearch.ingest.core.ValueSource; +import org.elasticsearch.ingest.core.ConfigurationUtils; + +import java.util.Map; + +/** + * Processor that appends value or values to existing lists. If the field is not present a new list holding the + * provided values will be added. If the field is a scalar it will be converted to a single item list and the provided + * values will be added to the newly created list. + */ +public class AppendProcessor extends AbstractProcessor { + + public static final String TYPE = "append"; + + private final TemplateService.Template field; + private final ValueSource value; + + AppendProcessor(String tag, TemplateService.Template field, ValueSource value) { + super(tag); + this.field = field; + this.value = value; + } + + public TemplateService.Template getField() { + return field; + } + + public ValueSource getValue() { + return value; + } + + @Override + public void execute(IngestDocument ingestDocument) throws Exception { + ingestDocument.appendFieldValue(field, value); + } + + @Override + public String getType() { + return TYPE; + } + + public static final class Factory extends AbstractProcessorFactory { + + private final TemplateService templateService; + + public Factory(TemplateService templateService) { + this.templateService = templateService; + } + + @Override + public AppendProcessor doCreate(String processorTag, Map config) throws Exception { + String field = ConfigurationUtils.readStringProperty(config, "field"); + Object value = ConfigurationUtils.readObject(config, "value"); + return new AppendProcessor(processorTag, templateService.compile(field), ValueSource.wrap(value, templateService)); + } + } +} diff --git a/core/src/main/java/org/elasticsearch/ingest/processor/ConvertProcessor.java b/core/src/main/java/org/elasticsearch/ingest/processor/ConvertProcessor.java new file mode 100644 index 00000000000..5b6bacf2ed1 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/ingest/processor/ConvertProcessor.java @@ -0,0 +1,145 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.ingest.processor; + +import org.elasticsearch.ingest.core.AbstractProcessor; +import org.elasticsearch.ingest.core.AbstractProcessorFactory; +import org.elasticsearch.ingest.core.IngestDocument; +import org.elasticsearch.ingest.core.ConfigurationUtils; + +import java.util.ArrayList; +import java.util.List; +import java.util.Locale; +import java.util.Map; + +/** + * Processor that converts fields content to a different type. Supported types are: integer, float, boolean and string. + * Throws exception if the field is not there or the conversion fails. + */ +public class ConvertProcessor extends AbstractProcessor { + + enum Type { + INTEGER { + @Override + public Object convert(Object value) { + try { + return Integer.parseInt(value.toString()); + } catch(NumberFormatException e) { + throw new IllegalArgumentException("unable to convert [" + value + "] to integer", e); + } + + } + }, FLOAT { + @Override + public Object convert(Object value) { + try { + return Float.parseFloat(value.toString()); + } catch(NumberFormatException e) { + throw new IllegalArgumentException("unable to convert [" + value + "] to float", e); + } + } + }, BOOLEAN { + @Override + public Object convert(Object value) { + if (value.toString().equalsIgnoreCase("true")) { + return true; + } else if (value.toString().equalsIgnoreCase("false")) { + return false; + } else { + throw new IllegalArgumentException("[" + value + "] is not a boolean value, cannot convert to boolean"); + } + } + }, STRING { + @Override + public Object convert(Object value) { + return value.toString(); + } + }; + + @Override + public final String toString() { + return name().toLowerCase(Locale.ROOT); + } + + public abstract Object convert(Object value); + + public static Type fromString(String type) { + try { + return Type.valueOf(type.toUpperCase(Locale.ROOT)); + } catch(IllegalArgumentException e) { + throw new IllegalArgumentException("type [" + type + "] not supported, cannot convert field.", e); + } + } + } + + public static final String TYPE = "convert"; + + private final String field; + private final Type convertType; + + ConvertProcessor(String tag, String field, Type convertType) { + super(tag); + this.field = field; + this.convertType = convertType; + } + + String getField() { + return field; + } + + Type getConvertType() { + return convertType; + } + + @Override + public void execute(IngestDocument document) { + Object oldValue = document.getFieldValue(field, Object.class); + Object newValue; + if (oldValue == null) { + throw new IllegalArgumentException("Field [" + field + "] is null, cannot be converted to type [" + convertType + "]"); + } + + if (oldValue instanceof List) { + List list = (List) oldValue; + List newList = new ArrayList<>(); + for (Object value : list) { + newList.add(convertType.convert(value)); + } + newValue = newList; + } else { + newValue = convertType.convert(oldValue); + } + document.setFieldValue(field, newValue); + } + + @Override + public String getType() { + return TYPE; + } + + public static class Factory extends AbstractProcessorFactory { + @Override + public ConvertProcessor doCreate(String processorTag, Map config) throws Exception { + String field = ConfigurationUtils.readStringProperty(config, "field"); + Type convertType = Type.fromString(ConfigurationUtils.readStringProperty(config, "type")); + return new ConvertProcessor(processorTag, field, convertType); + } + } +} diff --git a/core/src/main/java/org/elasticsearch/ingest/processor/DateFormat.java b/core/src/main/java/org/elasticsearch/ingest/processor/DateFormat.java new file mode 100644 index 00000000000..282b29176bf --- /dev/null +++ b/core/src/main/java/org/elasticsearch/ingest/processor/DateFormat.java @@ -0,0 +1,90 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.ingest.processor; + +import org.joda.time.DateTime; +import org.joda.time.DateTimeZone; +import org.joda.time.format.DateTimeFormat; +import org.joda.time.format.ISODateTimeFormat; + +import java.util.Locale; +import java.util.function.Function; + +enum DateFormat { + Iso8601 { + @Override + Function getFunction(String format, DateTimeZone timezone, Locale locale) { + return ISODateTimeFormat.dateTimeParser().withZone(timezone)::parseDateTime; + } + }, + Unix { + @Override + Function getFunction(String format, DateTimeZone timezone, Locale locale) { + return (date) -> new DateTime((long)(Float.parseFloat(date) * 1000), timezone); + } + }, + UnixMs { + @Override + Function getFunction(String format, DateTimeZone timezone, Locale locale) { + return (date) -> new DateTime(Long.parseLong(date), timezone); + } + }, + Tai64n { + @Override + Function getFunction(String format, DateTimeZone timezone, Locale locale) { + return (date) -> new DateTime(parseMillis(date), timezone); + } + + private long parseMillis(String date) { + if (date.startsWith("@")) { + date = date.substring(1); + } + long base = Long.parseLong(date.substring(1, 16), 16); + // 1356138046000 + long rest = Long.parseLong(date.substring(16, 24), 16); + return ((base * 1000) - 10000) + (rest/1000000); + } + }, + Joda { + @Override + Function getFunction(String format, DateTimeZone timezone, Locale locale) { + return DateTimeFormat.forPattern(format) + .withDefaultYear((new DateTime(DateTimeZone.UTC)).getYear()) + .withZone(timezone).withLocale(locale)::parseDateTime; + } + }; + + abstract Function getFunction(String format, DateTimeZone timezone, Locale locale); + + static DateFormat fromString(String format) { + switch (format) { + case "ISO8601": + return Iso8601; + case "UNIX": + return Unix; + case "UNIX_MS": + return UnixMs; + case "TAI64N": + return Tai64n; + default: + return Joda; + } + } +} diff --git a/core/src/main/java/org/elasticsearch/ingest/processor/DateProcessor.java b/core/src/main/java/org/elasticsearch/ingest/processor/DateProcessor.java new file mode 100644 index 00000000000..9fc0378d774 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/ingest/processor/DateProcessor.java @@ -0,0 +1,132 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.ingest.processor; + +import org.elasticsearch.ExceptionsHelper; +import org.elasticsearch.ingest.core.AbstractProcessor; +import org.elasticsearch.ingest.core.AbstractProcessorFactory; +import org.elasticsearch.ingest.core.ConfigurationUtils; +import org.elasticsearch.ingest.core.IngestDocument; +import org.joda.time.DateTime; +import org.joda.time.DateTimeZone; +import org.joda.time.format.ISODateTimeFormat; + +import java.util.ArrayList; +import java.util.IllformedLocaleException; +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.function.Function; + +public final class DateProcessor extends AbstractProcessor { + + public static final String TYPE = "date"; + static final String DEFAULT_TARGET_FIELD = "@timestamp"; + + private final DateTimeZone timezone; + private final Locale locale; + private final String matchField; + private final String targetField; + private final List matchFormats; + private final List> dateParsers; + + DateProcessor(String tag, DateTimeZone timezone, Locale locale, String matchField, List matchFormats, String targetField) { + super(tag); + this.timezone = timezone; + this.locale = locale; + this.matchField = matchField; + this.targetField = targetField; + this.matchFormats = matchFormats; + this.dateParsers = new ArrayList<>(); + for (String matchFormat : matchFormats) { + DateFormat dateFormat = DateFormat.fromString(matchFormat); + dateParsers.add(dateFormat.getFunction(matchFormat, timezone, locale)); + } + } + + @Override + public void execute(IngestDocument ingestDocument) { + String value = ingestDocument.getFieldValue(matchField, String.class); + + DateTime dateTime = null; + Exception lastException = null; + for (Function dateParser : dateParsers) { + try { + dateTime = dateParser.apply(value); + } catch (Exception e) { + //try the next parser and keep track of the exceptions + lastException = ExceptionsHelper.useOrSuppress(lastException, e); + } + } + + if (dateTime == null) { + throw new IllegalArgumentException("unable to parse date [" + value + "]", lastException); + } + + ingestDocument.setFieldValue(targetField, ISODateTimeFormat.dateTime().print(dateTime)); + } + + @Override + public String getType() { + return TYPE; + } + + DateTimeZone getTimezone() { + return timezone; + } + + Locale getLocale() { + return locale; + } + + String getMatchField() { + return matchField; + } + + String getTargetField() { + return targetField; + } + + List getMatchFormats() { + return matchFormats; + } + + public static class Factory extends AbstractProcessorFactory { + + @SuppressWarnings("unchecked") + public DateProcessor doCreate(String processorTag, Map config) throws Exception { + String matchField = ConfigurationUtils.readStringProperty(config, "match_field"); + String targetField = ConfigurationUtils.readStringProperty(config, "target_field", DEFAULT_TARGET_FIELD); + String timezoneString = ConfigurationUtils.readOptionalStringProperty(config, "timezone"); + DateTimeZone timezone = timezoneString == null ? DateTimeZone.UTC : DateTimeZone.forID(timezoneString); + String localeString = ConfigurationUtils.readOptionalStringProperty(config, "locale"); + Locale locale = Locale.ENGLISH; + if (localeString != null) { + try { + locale = (new Locale.Builder()).setLanguageTag(localeString).build(); + } catch (IllformedLocaleException e) { + throw new IllegalArgumentException("Invalid language tag specified: " + localeString); + } + } + List matchFormats = ConfigurationUtils.readList(config, "match_formats"); + return new DateProcessor(processorTag, timezone, locale, matchField, matchFormats, targetField); + } + } +} diff --git a/core/src/main/java/org/elasticsearch/ingest/processor/DeDotProcessor.java b/core/src/main/java/org/elasticsearch/ingest/processor/DeDotProcessor.java new file mode 100644 index 00000000000..295a9884997 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/ingest/processor/DeDotProcessor.java @@ -0,0 +1,107 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.ingest.processor; + +import org.elasticsearch.ingest.core.AbstractProcessor; +import org.elasticsearch.ingest.core.AbstractProcessorFactory; +import org.elasticsearch.ingest.core.ConfigurationUtils; +import org.elasticsearch.ingest.core.IngestDocument; +import org.elasticsearch.ingest.core.Processor; + +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; + +/** + * Processor that replaces dots in document field names with a + * specified separator. + */ +public class DeDotProcessor extends AbstractProcessor { + + public static final String TYPE = "dedot"; + static final String DEFAULT_SEPARATOR = "_"; + + private final String separator; + + DeDotProcessor(String tag, String separator) { + super(tag); + this.separator = separator; + } + + public String getSeparator() { + return separator; + } + + @Override + public void execute(IngestDocument document) { + deDot(document.getSourceAndMetadata()); + } + + @Override + public String getType() { + return TYPE; + } + + /** + * Recursively iterates through Maps and Lists in search of map entries with + * keys containing dots. The dots in these fields are replaced with {@link #separator}. + * + * @param obj The current object in context to be checked for dots in its fields. + */ + private void deDot(Object obj) { + if (obj instanceof Map) { + @SuppressWarnings("unchecked") + Map doc = (Map) obj; + Iterator> it = doc.entrySet().iterator(); + Map deDottedFields = new HashMap<>(); + while (it.hasNext()) { + Map.Entry entry = it.next(); + deDot(entry.getValue()); + String fieldName = entry.getKey(); + if (fieldName.contains(".")) { + String deDottedFieldName = fieldName.replaceAll("\\.", separator); + deDottedFields.put(deDottedFieldName, entry.getValue()); + it.remove(); + } + } + doc.putAll(deDottedFields); + } else if (obj instanceof List) { + @SuppressWarnings("unchecked") + List list = (List) obj; + for (Object value : list) { + deDot(value); + } + } + } + + public static class Factory extends AbstractProcessorFactory { + + @Override + public DeDotProcessor doCreate(String processorTag, Map config) throws Exception { + String separator = ConfigurationUtils.readOptionalStringProperty(config, "separator"); + if (separator == null) { + separator = DEFAULT_SEPARATOR; + } + return new DeDotProcessor(processorTag, separator); + } + } +} + diff --git a/core/src/main/java/org/elasticsearch/ingest/processor/FailProcessor.java b/core/src/main/java/org/elasticsearch/ingest/processor/FailProcessor.java new file mode 100644 index 00000000000..65b4b602bd0 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/ingest/processor/FailProcessor.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.ingest.processor; + +import org.elasticsearch.ingest.core.AbstractProcessor; +import org.elasticsearch.ingest.core.AbstractProcessorFactory; +import org.elasticsearch.ingest.core.ConfigurationUtils; +import org.elasticsearch.ingest.core.IngestDocument; +import org.elasticsearch.ingest.core.Processor; +import org.elasticsearch.ingest.core.TemplateService; + +import java.util.Map; + +/** + * Processor that raises a runtime exception with a provided + * error message. + */ +public class FailProcessor extends AbstractProcessor { + + public static final String TYPE = "fail"; + + private final TemplateService.Template message; + + FailProcessor(String tag, TemplateService.Template message) { + super(tag); + this.message = message; + } + + public TemplateService.Template getMessage() { + return message; + } + + @Override + public void execute(IngestDocument document) { + throw new FailProcessorException(document.renderTemplate(message)); + } + + @Override + public String getType() { + return TYPE; + } + + public static class Factory extends AbstractProcessorFactory { + + private final TemplateService templateService; + + public Factory(TemplateService templateService) { + this.templateService = templateService; + } + + @Override + public FailProcessor doCreate(String processorTag, Map config) throws Exception { + String message = ConfigurationUtils.readStringProperty(config, "message"); + return new FailProcessor(processorTag, templateService.compile(message)); + } + } +} + diff --git a/core/src/main/java/org/elasticsearch/ingest/processor/FailProcessorException.java b/core/src/main/java/org/elasticsearch/ingest/processor/FailProcessorException.java new file mode 100644 index 00000000000..bfdfe11178e --- /dev/null +++ b/core/src/main/java/org/elasticsearch/ingest/processor/FailProcessorException.java @@ -0,0 +1,35 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.ingest.processor; + +/** + * Exception class thrown by {@link FailProcessor}. + * + * This exception is caught in the {@link org.elasticsearch.ingest.core.CompoundProcessor} and + * then changes the state of {@link org.elasticsearch.ingest.core.IngestDocument}. This + * exception should get serialized. + */ +public class FailProcessorException extends RuntimeException { + + public FailProcessorException(String message) { + super(message); + } +} + diff --git a/core/src/main/java/org/elasticsearch/ingest/processor/GsubProcessor.java b/core/src/main/java/org/elasticsearch/ingest/processor/GsubProcessor.java new file mode 100644 index 00000000000..3dc4b3f0cad --- /dev/null +++ b/core/src/main/java/org/elasticsearch/ingest/processor/GsubProcessor.java @@ -0,0 +1,90 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.ingest.processor; + +import org.elasticsearch.ingest.core.AbstractProcessor; +import org.elasticsearch.ingest.core.AbstractProcessorFactory; +import org.elasticsearch.ingest.core.IngestDocument; +import org.elasticsearch.ingest.core.ConfigurationUtils; +import org.elasticsearch.ingest.core.Processor; + +import java.util.Map; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +/** + * Processor that allows to search for patterns in field content and replace them with corresponding string replacement. + * Support fields of string type only, throws exception if a field is of a different type. + */ +public class GsubProcessor extends AbstractProcessor { + + public static final String TYPE = "gsub"; + + private final String field; + private final Pattern pattern; + private final String replacement; + + GsubProcessor(String tag, String field, Pattern pattern, String replacement) { + super(tag); + this.field = field; + this.pattern = pattern; + this.replacement = replacement; + } + + String getField() { + return field; + } + + Pattern getPattern() { + return pattern; + } + + String getReplacement() { + return replacement; + } + + + @Override + public void execute(IngestDocument document) { + String oldVal = document.getFieldValue(field, String.class); + if (oldVal == null) { + throw new IllegalArgumentException("field [" + field + "] is null, cannot match pattern."); + } + Matcher matcher = pattern.matcher(oldVal); + String newVal = matcher.replaceAll(replacement); + document.setFieldValue(field, newVal); + } + + @Override + public String getType() { + return TYPE; + } + + public static class Factory extends AbstractProcessorFactory { + @Override + public GsubProcessor doCreate(String processorTag, Map config) throws Exception { + String field = ConfigurationUtils.readStringProperty(config, "field"); + String pattern = ConfigurationUtils.readStringProperty(config, "pattern"); + String replacement = ConfigurationUtils.readStringProperty(config, "replacement"); + Pattern searchPattern = Pattern.compile(pattern); + return new GsubProcessor(processorTag, field, searchPattern, replacement); + } + } +} diff --git a/core/src/main/java/org/elasticsearch/ingest/processor/JoinProcessor.java b/core/src/main/java/org/elasticsearch/ingest/processor/JoinProcessor.java new file mode 100644 index 00000000000..3516929e26f --- /dev/null +++ b/core/src/main/java/org/elasticsearch/ingest/processor/JoinProcessor.java @@ -0,0 +1,83 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.ingest.processor; + +import org.elasticsearch.ingest.core.AbstractProcessor; +import org.elasticsearch.ingest.core.AbstractProcessorFactory; +import org.elasticsearch.ingest.core.IngestDocument; +import org.elasticsearch.ingest.core.ConfigurationUtils; +import org.elasticsearch.ingest.core.Processor; + +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +/** + * Processor that joins the different items of an array into a single string value using a separator between each item. + * Throws exception is the specified field is not an array. + */ +public class JoinProcessor extends AbstractProcessor { + + public static final String TYPE = "join"; + + private final String field; + private final String separator; + + JoinProcessor(String tag, String field, String separator) { + super(tag); + this.field = field; + this.separator = separator; + } + + String getField() { + return field; + } + + String getSeparator() { + return separator; + } + + @Override + public void execute(IngestDocument document) { + List list = document.getFieldValue(field, List.class); + if (list == null) { + throw new IllegalArgumentException("field [" + field + "] is null, cannot join."); + } + String joined = list.stream() + .map(Object::toString) + .collect(Collectors.joining(separator)); + document.setFieldValue(field, joined); + } + + @Override + public String getType() { + return TYPE; + } + + public static class Factory extends AbstractProcessorFactory { + @Override + public JoinProcessor doCreate(String processorTag, Map config) throws Exception { + String field = ConfigurationUtils.readStringProperty(config, "field"); + String separator = ConfigurationUtils.readStringProperty(config, "separator"); + return new JoinProcessor(processorTag, field, separator); + } + } +} + diff --git a/core/src/main/java/org/elasticsearch/ingest/processor/LowercaseProcessor.java b/core/src/main/java/org/elasticsearch/ingest/processor/LowercaseProcessor.java new file mode 100644 index 00000000000..617efd9b480 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/ingest/processor/LowercaseProcessor.java @@ -0,0 +1,53 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.ingest.processor; + +import java.util.Locale; + +/** + * Processor that converts the content of string fields to lowercase. + * Throws exception is the field is not of type string. + */ + +public class LowercaseProcessor extends AbstractStringProcessor { + + public static final String TYPE = "lowercase"; + + LowercaseProcessor(String processorTag, String field) { + super(processorTag, field); + } + + @Override + protected String process(String value) { + return value.toLowerCase(Locale.ROOT); + } + + @Override + public String getType() { + return TYPE; + } + + public static class Factory extends AbstractStringProcessor.Factory { + @Override + protected LowercaseProcessor newProcessor(String tag, String field) { + return new LowercaseProcessor(tag, field); + } + } +} diff --git a/core/src/main/java/org/elasticsearch/ingest/processor/RemoveProcessor.java b/core/src/main/java/org/elasticsearch/ingest/processor/RemoveProcessor.java new file mode 100644 index 00000000000..e994954a034 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/ingest/processor/RemoveProcessor.java @@ -0,0 +1,74 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.ingest.processor; + +import org.elasticsearch.ingest.core.AbstractProcessor; +import org.elasticsearch.ingest.core.AbstractProcessorFactory; +import org.elasticsearch.ingest.core.IngestDocument; +import org.elasticsearch.ingest.core.TemplateService; +import org.elasticsearch.ingest.core.ConfigurationUtils; +import org.elasticsearch.ingest.core.Processor; + +import java.util.Map; + +/** + * Processor that removes existing fields. Nothing happens if the field is not present. + */ +public class RemoveProcessor extends AbstractProcessor { + + public static final String TYPE = "remove"; + + private final TemplateService.Template field; + + RemoveProcessor(String tag, TemplateService.Template field) { + super(tag); + this.field = field; + } + + public TemplateService.Template getField() { + return field; + } + + @Override + public void execute(IngestDocument document) { + document.removeField(field); + } + + @Override + public String getType() { + return TYPE; + } + + public static class Factory extends AbstractProcessorFactory { + + private final TemplateService templateService; + + public Factory(TemplateService templateService) { + this.templateService = templateService; + } + + @Override + public RemoveProcessor doCreate(String processorTag, Map config) throws Exception { + String field = ConfigurationUtils.readStringProperty(config, "field"); + return new RemoveProcessor(processorTag, templateService.compile(field)); + } + } +} + diff --git a/core/src/main/java/org/elasticsearch/ingest/processor/RenameProcessor.java b/core/src/main/java/org/elasticsearch/ingest/processor/RenameProcessor.java new file mode 100644 index 00000000000..7726a720b49 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/ingest/processor/RenameProcessor.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.ingest.processor; + +import org.elasticsearch.ingest.core.AbstractProcessor; +import org.elasticsearch.ingest.core.AbstractProcessorFactory; +import org.elasticsearch.ingest.core.IngestDocument; +import org.elasticsearch.ingest.core.ConfigurationUtils; +import org.elasticsearch.ingest.core.Processor; + +import java.util.Map; + +/** + * Processor that allows to rename existing fields. Will throw exception if the field is not present. + */ +public class RenameProcessor extends AbstractProcessor { + + public static final String TYPE = "rename"; + + private final String oldFieldName; + private final String newFieldName; + + RenameProcessor(String tag, String oldFieldName, String newFieldName) { + super(tag); + this.oldFieldName = oldFieldName; + this.newFieldName = newFieldName; + } + + String getOldFieldName() { + return oldFieldName; + } + + String getNewFieldName() { + return newFieldName; + } + + @Override + public void execute(IngestDocument document) { + if (document.hasField(oldFieldName) == false) { + throw new IllegalArgumentException("field [" + oldFieldName + "] doesn't exist"); + } + if (document.hasField(newFieldName)) { + throw new IllegalArgumentException("field [" + newFieldName + "] already exists"); + } + + Object oldValue = document.getFieldValue(oldFieldName, Object.class); + document.setFieldValue(newFieldName, oldValue); + try { + document.removeField(oldFieldName); + } catch (Exception e) { + //remove the new field if the removal of the old one failed + document.removeField(newFieldName); + throw e; + } + } + + @Override + public String getType() { + return TYPE; + } + + public static class Factory extends AbstractProcessorFactory { + @Override + public RenameProcessor doCreate(String processorTag, Map config) throws Exception { + String field = ConfigurationUtils.readStringProperty(config, "field"); + String newField = ConfigurationUtils.readStringProperty(config, "to"); + return new RenameProcessor(processorTag, field, newField); + } + } +} diff --git a/core/src/main/java/org/elasticsearch/ingest/processor/SetProcessor.java b/core/src/main/java/org/elasticsearch/ingest/processor/SetProcessor.java new file mode 100644 index 00000000000..e046a5f3bdb --- /dev/null +++ b/core/src/main/java/org/elasticsearch/ingest/processor/SetProcessor.java @@ -0,0 +1,81 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.ingest.processor; + +import org.elasticsearch.ingest.core.AbstractProcessor; +import org.elasticsearch.ingest.core.AbstractProcessorFactory; +import org.elasticsearch.ingest.core.IngestDocument; +import org.elasticsearch.ingest.core.TemplateService; +import org.elasticsearch.ingest.core.ValueSource; +import org.elasticsearch.ingest.core.ConfigurationUtils; + +import java.util.Map; + +/** + * Processor that adds new fields with their corresponding values. If the field is already present, its value + * will be replaced with the provided one. + */ +public class SetProcessor extends AbstractProcessor { + + public static final String TYPE = "set"; + + private final TemplateService.Template field; + private final ValueSource value; + + SetProcessor(String tag, TemplateService.Template field, ValueSource value) { + super(tag); + this.field = field; + this.value = value; + } + + public TemplateService.Template getField() { + return field; + } + + public ValueSource getValue() { + return value; + } + + @Override + public void execute(IngestDocument document) { + document.setFieldValue(field, value); + } + + @Override + public String getType() { + return TYPE; + } + + public static final class Factory extends AbstractProcessorFactory { + + private final TemplateService templateService; + + public Factory(TemplateService templateService) { + this.templateService = templateService; + } + + @Override + public SetProcessor doCreate(String processorTag, Map config) throws Exception { + String field = ConfigurationUtils.readStringProperty(config, "field"); + Object value = ConfigurationUtils.readObject(config, "value"); + return new SetProcessor(processorTag, templateService.compile(field), ValueSource.wrap(value, templateService)); + } + } +} diff --git a/core/src/main/java/org/elasticsearch/ingest/processor/SplitProcessor.java b/core/src/main/java/org/elasticsearch/ingest/processor/SplitProcessor.java new file mode 100644 index 00000000000..ad0bffb061a --- /dev/null +++ b/core/src/main/java/org/elasticsearch/ingest/processor/SplitProcessor.java @@ -0,0 +1,82 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.ingest.processor; + +import org.elasticsearch.ingest.core.AbstractProcessor; +import org.elasticsearch.ingest.core.AbstractProcessorFactory; +import org.elasticsearch.ingest.core.ConfigurationUtils; +import org.elasticsearch.ingest.core.IngestDocument; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; + +/** + * Processor that splits fields content into different items based on the occurrence of a specified separator. + * New field value will be an array containing all of the different extracted items. + * Throws exception if the field is null or a type other than string. + */ +public class SplitProcessor extends AbstractProcessor { + + public static final String TYPE = "split"; + + private final String field; + private final String separator; + + SplitProcessor(String tag, String field, String separator) { + super(tag); + this.field = field; + this.separator = separator; + } + + String getField() { + return field; + } + + String getSeparator() { + return separator; + } + + @Override + public void execute(IngestDocument document) { + String oldVal = document.getFieldValue(field, String.class); + if (oldVal == null) { + throw new IllegalArgumentException("field [" + field + "] is null, cannot split."); + } + String[] strings = oldVal.split(separator); + List splitList = new ArrayList<>(strings.length); + Collections.addAll(splitList, strings); + document.setFieldValue(field, splitList); + } + + @Override + public String getType() { + return TYPE; + } + + public static class Factory extends AbstractProcessorFactory { + @Override + public SplitProcessor doCreate(String processorTag, Map config) throws Exception { + String field = ConfigurationUtils.readStringProperty(config, "field"); + return new SplitProcessor(processorTag, field, ConfigurationUtils.readStringProperty(config, "separator")); + } + } +} diff --git a/core/src/main/java/org/elasticsearch/ingest/processor/TrimProcessor.java b/core/src/main/java/org/elasticsearch/ingest/processor/TrimProcessor.java new file mode 100644 index 00000000000..c66cc848933 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/ingest/processor/TrimProcessor.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.ingest.processor; + +/** + * Processor that trims the content of string fields. + * Throws exception is the field is not of type string. + */ +public class TrimProcessor extends AbstractStringProcessor { + + public static final String TYPE = "trim"; + + TrimProcessor(String processorTag, String field) { + super(processorTag, field); + } + + @Override + protected String process(String value) { + return value.trim(); + } + + @Override + public String getType() { + return TYPE; + } + + public static class Factory extends AbstractStringProcessor.Factory { + @Override + protected TrimProcessor newProcessor(String tag, String field) { + return new TrimProcessor(tag, field); + } + } +} + diff --git a/core/src/main/java/org/elasticsearch/ingest/processor/UppercaseProcessor.java b/core/src/main/java/org/elasticsearch/ingest/processor/UppercaseProcessor.java new file mode 100644 index 00000000000..e6a1f77cb86 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/ingest/processor/UppercaseProcessor.java @@ -0,0 +1,53 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.ingest.processor; + +import java.util.Locale; + +/** + * Processor that converts the content of string fields to uppercase. + * Throws exception is the field is not of type string. + */ +public class UppercaseProcessor extends AbstractStringProcessor { + + public static final String TYPE = "uppercase"; + + UppercaseProcessor(String processorTag, String field) { + super(processorTag, field); + } + + @Override + protected String process(String value) { + return value.toUpperCase(Locale.ROOT); + } + + @Override + public String getType() { + return TYPE; + } + + public static class Factory extends AbstractStringProcessor.Factory { + @Override + protected UppercaseProcessor newProcessor(String tag, String field) { + return new UppercaseProcessor(tag, field); + } + } +} + diff --git a/core/src/main/java/org/elasticsearch/node/Node.java b/core/src/main/java/org/elasticsearch/node/Node.java index 724efe94e41..f51c7e27dfc 100644 --- a/core/src/main/java/org/elasticsearch/node/Node.java +++ b/core/src/main/java/org/elasticsearch/node/Node.java @@ -30,6 +30,7 @@ import org.elasticsearch.cluster.ClusterModule; import org.elasticsearch.cluster.ClusterNameModule; import org.elasticsearch.cluster.ClusterService; import org.elasticsearch.cluster.action.index.MappingUpdatedAction; +import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.routing.RoutingService; import org.elasticsearch.common.StopWatch; import org.elasticsearch.common.component.Lifecycle; @@ -76,6 +77,7 @@ import org.elasticsearch.indices.ttl.IndicesTTLService; import org.elasticsearch.monitor.MonitorService; import org.elasticsearch.monitor.jvm.JvmInfo; import org.elasticsearch.node.internal.InternalSettingsPreparer; +import org.elasticsearch.node.service.NodeService; import org.elasticsearch.percolator.PercolatorModule; import org.elasticsearch.percolator.PercolatorService; import org.elasticsearch.plugins.Plugin; @@ -119,6 +121,7 @@ import static org.elasticsearch.common.settings.Settings.settingsBuilder; */ public class Node implements Releasable { + public static final Setting NODE_INGEST_SETTING = Setting.boolSetting("node.ingest", true, false, Setting.Scope.CLUSTER); private static final String CLIENT_TYPE = "node"; public static final Setting WRITE_PORTS_FIELD_SETTING = Setting.boolSetting("node.portsfile", false, false, Setting.Scope.CLUSTER); private final Lifecycle lifecycle = new Lifecycle(); @@ -190,7 +193,7 @@ public class Node implements Releasable { modules.add(new ClusterModule(this.settings)); modules.add(new IndicesModule()); modules.add(new SearchModule(settings, namedWriteableRegistry)); - modules.add(new ActionModule(false)); + modules.add(new ActionModule(DiscoveryNode.ingestNode(settings), false)); modules.add(new GatewayModule(settings)); modules.add(new NodeClientModule()); modules.add(new PercolatorModule()); @@ -232,6 +235,13 @@ public class Node implements Releasable { return client; } + /** + * Returns the environment of the node + */ + public Environment getEnvironment() { + return environment; + } + /** * Start the node. If the node is already started, this method is no-op. */ @@ -347,6 +357,12 @@ public class Node implements Releasable { StopWatch stopWatch = new StopWatch("node_close"); stopWatch.start("tribe"); injector.getInstance(TribeService.class).close(); + stopWatch.stop().start("node_service"); + try { + injector.getInstance(NodeService.class).close(); + } catch (IOException e) { + logger.warn("NodeService close failed", e); + } stopWatch.stop().start("http"); if (settings.getAsBoolean("http.enabled", true)) { injector.getInstance(HttpServer.class).close(); diff --git a/core/src/main/java/org/elasticsearch/node/NodeModule.java b/core/src/main/java/org/elasticsearch/node/NodeModule.java index aa52d389340..442dc727007 100644 --- a/core/src/main/java/org/elasticsearch/node/NodeModule.java +++ b/core/src/main/java/org/elasticsearch/node/NodeModule.java @@ -22,9 +22,28 @@ package org.elasticsearch.node; import org.elasticsearch.cache.recycler.PageCacheRecycler; import org.elasticsearch.common.inject.AbstractModule; import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.ingest.ProcessorsRegistry; +import org.elasticsearch.ingest.core.Processor; +import org.elasticsearch.ingest.core.TemplateService; +import org.elasticsearch.ingest.processor.AppendProcessor; +import org.elasticsearch.ingest.processor.ConvertProcessor; +import org.elasticsearch.ingest.processor.DateProcessor; +import org.elasticsearch.ingest.processor.DeDotProcessor; +import org.elasticsearch.ingest.processor.FailProcessor; +import org.elasticsearch.ingest.processor.GsubProcessor; +import org.elasticsearch.ingest.processor.JoinProcessor; +import org.elasticsearch.ingest.processor.LowercaseProcessor; +import org.elasticsearch.ingest.processor.RemoveProcessor; +import org.elasticsearch.ingest.processor.RenameProcessor; +import org.elasticsearch.ingest.processor.SetProcessor; +import org.elasticsearch.ingest.processor.SplitProcessor; +import org.elasticsearch.ingest.processor.TrimProcessor; +import org.elasticsearch.ingest.processor.UppercaseProcessor; import org.elasticsearch.monitor.MonitorService; import org.elasticsearch.node.service.NodeService; +import java.util.function.Function; + /** * */ @@ -32,6 +51,7 @@ public class NodeModule extends AbstractModule { private final Node node; private final MonitorService monitorService; + private final ProcessorsRegistry processorsRegistry; // pkg private so tests can mock Class pageCacheRecyclerImpl = PageCacheRecycler.class; @@ -40,6 +60,22 @@ public class NodeModule extends AbstractModule { public NodeModule(Node node, MonitorService monitorService) { this.node = node; this.monitorService = monitorService; + this.processorsRegistry = new ProcessorsRegistry(); + + registerProcessor(DateProcessor.TYPE, (templateService) -> new DateProcessor.Factory()); + registerProcessor(SetProcessor.TYPE, SetProcessor.Factory::new); + registerProcessor(AppendProcessor.TYPE, AppendProcessor.Factory::new); + registerProcessor(RenameProcessor.TYPE, (templateService) -> new RenameProcessor.Factory()); + registerProcessor(RemoveProcessor.TYPE, RemoveProcessor.Factory::new); + registerProcessor(SplitProcessor.TYPE, (templateService) -> new SplitProcessor.Factory()); + registerProcessor(JoinProcessor.TYPE, (templateService) -> new JoinProcessor.Factory()); + registerProcessor(UppercaseProcessor.TYPE, (templateService) -> new UppercaseProcessor.Factory()); + registerProcessor(LowercaseProcessor.TYPE, (templateService) -> new LowercaseProcessor.Factory()); + registerProcessor(TrimProcessor.TYPE, (templateService) -> new TrimProcessor.Factory()); + registerProcessor(ConvertProcessor.TYPE, (templateService) -> new ConvertProcessor.Factory()); + registerProcessor(GsubProcessor.TYPE, (templateService) -> new GsubProcessor.Factory()); + registerProcessor(FailProcessor.TYPE, FailProcessor.Factory::new); + registerProcessor(DeDotProcessor.TYPE, (templateService) -> new DeDotProcessor.Factory()); } @Override @@ -58,5 +94,20 @@ public class NodeModule extends AbstractModule { bind(Node.class).toInstance(node); bind(MonitorService.class).toInstance(monitorService); bind(NodeService.class).asEagerSingleton(); + bind(ProcessorsRegistry.class).toInstance(processorsRegistry); + } + + /** + * Returns the node + */ + public Node getNode() { + return node; + } + + /** + * Adds a processor factory under a specific type name. + */ + public void registerProcessor(String type, Function> processorFactoryProvider) { + processorsRegistry.registerProcessor(type, processorFactoryProvider); } } diff --git a/core/src/main/java/org/elasticsearch/node/service/NodeService.java b/core/src/main/java/org/elasticsearch/node/service/NodeService.java index b4fe59e3473..7c385b5b39a 100644 --- a/core/src/main/java/org/elasticsearch/node/service/NodeService.java +++ b/core/src/main/java/org/elasticsearch/node/service/NodeService.java @@ -24,20 +24,25 @@ import org.elasticsearch.Version; import org.elasticsearch.action.admin.cluster.node.info.NodeInfo; import org.elasticsearch.action.admin.cluster.node.stats.NodeStats; import org.elasticsearch.action.admin.indices.stats.CommonStatsFlags; +import org.elasticsearch.cluster.ClusterService; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.component.AbstractComponent; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.discovery.Discovery; +import org.elasticsearch.env.Environment; import org.elasticsearch.http.HttpServer; import org.elasticsearch.indices.IndicesService; import org.elasticsearch.indices.breaker.CircuitBreakerService; +import org.elasticsearch.ingest.IngestService; +import org.elasticsearch.ingest.ProcessorsRegistry; import org.elasticsearch.monitor.MonitorService; import org.elasticsearch.plugins.PluginsService; import org.elasticsearch.script.ScriptService; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; +import java.io.Closeable; import java.io.IOException; import java.util.HashMap; import java.util.Map; @@ -47,7 +52,7 @@ import static java.util.Collections.unmodifiableMap; /** */ -public class NodeService extends AbstractComponent { +public class NodeService extends AbstractComponent implements Closeable { private final ThreadPool threadPool; private final MonitorService monitorService; @@ -55,6 +60,7 @@ public class NodeService extends AbstractComponent { private final IndicesService indicesService; private final PluginsService pluginService; private final CircuitBreakerService circuitBreakerService; + private final IngestService ingestService; private ScriptService scriptService; @Nullable @@ -67,10 +73,10 @@ public class NodeService extends AbstractComponent { private final Discovery discovery; @Inject - public NodeService(Settings settings, ThreadPool threadPool, MonitorService monitorService, Discovery discovery, - TransportService transportService, IndicesService indicesService, - PluginsService pluginService, CircuitBreakerService circuitBreakerService, - Version version) { + public NodeService(Settings settings, Environment environment, ThreadPool threadPool, MonitorService monitorService, + Discovery discovery, TransportService transportService, IndicesService indicesService, + PluginsService pluginService, CircuitBreakerService circuitBreakerService, Version version, + ProcessorsRegistry processorsRegistry, ClusterService clusterService) { super(settings); this.threadPool = threadPool; this.monitorService = monitorService; @@ -81,12 +87,15 @@ public class NodeService extends AbstractComponent { this.version = version; this.pluginService = pluginService; this.circuitBreakerService = circuitBreakerService; + this.ingestService = new IngestService(settings, threadPool, processorsRegistry); + clusterService.add(ingestService.getPipelineStore()); } // can not use constructor injection or there will be a circular dependency @Inject(optional = true) public void setScriptService(ScriptService scriptService) { this.scriptService = scriptService; + this.ingestService.setScriptService(scriptService); } public void setHttpServer(@Nullable HttpServer httpServer) { @@ -176,4 +185,13 @@ public class NodeService extends AbstractComponent { discoveryStats ? discovery.stats() : null ); } + + public IngestService getIngestService() { + return ingestService; + } + + @Override + public void close() throws IOException { + indicesService.close(); + } } diff --git a/core/src/main/java/org/elasticsearch/percolator/MultiDocumentPercolatorIndex.java b/core/src/main/java/org/elasticsearch/percolator/MultiDocumentPercolatorIndex.java index 10eeec75cbc..9d091a4c0bd 100644 --- a/core/src/main/java/org/elasticsearch/percolator/MultiDocumentPercolatorIndex.java +++ b/core/src/main/java/org/elasticsearch/percolator/MultiDocumentPercolatorIndex.java @@ -39,6 +39,8 @@ import org.apache.lucene.util.CloseableThreadLocal; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.common.lucene.search.Queries; import org.elasticsearch.index.engine.Engine; +import org.elasticsearch.index.mapper.DocumentMapper; +import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.ParseContext; import org.elasticsearch.index.mapper.ParsedDocument; import org.elasticsearch.index.mapper.internal.UidFieldMapper; @@ -76,8 +78,7 @@ class MultiDocumentPercolatorIndex implements PercolatorIndex { } else { memoryIndex = new MemoryIndex(true); } - Analyzer analyzer = context.mapperService().documentMapper(parsedDocument.type()).mappers().indexAnalyzer(); - memoryIndices[i] = indexDoc(d, analyzer, memoryIndex).createSearcher().getIndexReader(); + memoryIndices[i] = indexDoc(d, memoryIndex, context, parsedDocument).createSearcher().getIndexReader(); } try { MultiReader mReader = new MultiReader(memoryIndices, true); @@ -101,8 +102,13 @@ class MultiDocumentPercolatorIndex implements PercolatorIndex { } } - MemoryIndex indexDoc(ParseContext.Document d, Analyzer analyzer, MemoryIndex memoryIndex) { + MemoryIndex indexDoc(ParseContext.Document d, MemoryIndex memoryIndex, PercolateContext context, ParsedDocument parsedDocument) { for (IndexableField field : d.getFields()) { + Analyzer analyzer = context.analysisService().defaultIndexAnalyzer(); + DocumentMapper documentMapper = context.mapperService().documentMapper(parsedDocument.type()); + if (documentMapper != null && documentMapper.mappers().getMapper(field.name()) != null) { + analyzer = documentMapper.mappers().indexAnalyzer(); + } if (field.fieldType().indexOptions() == IndexOptions.NONE && field.name().equals(UidFieldMapper.NAME)) { continue; } diff --git a/core/src/main/java/org/elasticsearch/percolator/PercolateDocumentParser.java b/core/src/main/java/org/elasticsearch/percolator/PercolateDocumentParser.java index 6733ebd0b3a..8edc5212523 100644 --- a/core/src/main/java/org/elasticsearch/percolator/PercolateDocumentParser.java +++ b/core/src/main/java/org/elasticsearch/percolator/PercolateDocumentParser.java @@ -49,14 +49,13 @@ public class PercolateDocumentParser { private final HighlightPhase highlightPhase; private final SortParseElement sortParseElement; private final AggregationPhase aggregationPhase; - private final MappingUpdatedAction mappingUpdatedAction; @Inject - public PercolateDocumentParser(HighlightPhase highlightPhase, SortParseElement sortParseElement, AggregationPhase aggregationPhase, MappingUpdatedAction mappingUpdatedAction) { + public PercolateDocumentParser(HighlightPhase highlightPhase, SortParseElement sortParseElement, + AggregationPhase aggregationPhase) { this.highlightPhase = highlightPhase; this.sortParseElement = sortParseElement; this.aggregationPhase = aggregationPhase; - this.mappingUpdatedAction = mappingUpdatedAction; } public ParsedDocument parse(PercolateShardRequest request, PercolateContext context, MapperService mapperService, QueryShardContext queryShardContext) { @@ -98,9 +97,6 @@ public class PercolateDocumentParser { if (docMapper.getMapping() != null) { doc.addDynamicMappingsUpdate(docMapper.getMapping()); } - if (doc.dynamicMappingsUpdate() != null) { - mappingUpdatedAction.updateMappingOnMasterSynchronously(request.shardId().getIndex(), request.documentType(), doc.dynamicMappingsUpdate()); - } // the document parsing exists the "doc" object, so we need to set the new current field. currentFieldName = parser.currentName(); } diff --git a/core/src/main/java/org/elasticsearch/percolator/PercolatorService.java b/core/src/main/java/org/elasticsearch/percolator/PercolatorService.java index b7d813f6aca..bc6dca139e5 100644 --- a/core/src/main/java/org/elasticsearch/percolator/PercolatorService.java +++ b/core/src/main/java/org/elasticsearch/percolator/PercolatorService.java @@ -53,6 +53,7 @@ import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.index.IndexService; import org.elasticsearch.index.fieldvisitor.SingleFieldsVisitor; +import org.elasticsearch.index.mapper.DocumentMapper; import org.elasticsearch.index.mapper.ParsedDocument; import org.elasticsearch.index.mapper.internal.UidFieldMapper; import org.elasticsearch.index.percolator.PercolatorFieldMapper; @@ -205,7 +206,8 @@ public class PercolatorService extends AbstractComponent { // parse the source either into one MemoryIndex, if it is a single document or index multiple docs if nested PercolatorIndex percolatorIndex; - boolean isNested = indexShard.mapperService().documentMapper(request.documentType()).hasNestedObjects(); + DocumentMapper documentMapper = indexShard.mapperService().documentMapper(request.documentType()); + boolean isNested = documentMapper != null && documentMapper.hasNestedObjects(); if (parsedDocument.docs().size() > 1) { assert isNested; percolatorIndex = multi; diff --git a/core/src/main/java/org/elasticsearch/percolator/SingleDocumentPercolatorIndex.java b/core/src/main/java/org/elasticsearch/percolator/SingleDocumentPercolatorIndex.java index 1271872cab6..1d5268e3794 100644 --- a/core/src/main/java/org/elasticsearch/percolator/SingleDocumentPercolatorIndex.java +++ b/core/src/main/java/org/elasticsearch/percolator/SingleDocumentPercolatorIndex.java @@ -28,6 +28,7 @@ import org.apache.lucene.index.memory.MemoryIndex; import org.apache.lucene.util.CloseableThreadLocal; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.index.engine.Engine; +import org.elasticsearch.index.mapper.DocumentMapper; import org.elasticsearch.index.mapper.ParsedDocument; import org.elasticsearch.index.mapper.internal.UidFieldMapper; @@ -49,11 +50,15 @@ class SingleDocumentPercolatorIndex implements PercolatorIndex { public void prepare(PercolateContext context, ParsedDocument parsedDocument) { MemoryIndex memoryIndex = cache.get(); for (IndexableField field : parsedDocument.rootDoc().getFields()) { + Analyzer analyzer = context.analysisService().defaultIndexAnalyzer(); + DocumentMapper documentMapper = context.mapperService().documentMapper(parsedDocument.type()); + if (documentMapper != null && documentMapper.mappers().getMapper(field.name()) != null) { + analyzer = documentMapper.mappers().indexAnalyzer(); + } if (field.fieldType().indexOptions() == IndexOptions.NONE && field.name().equals(UidFieldMapper.NAME)) { continue; } try { - Analyzer analyzer = context.mapperService().documentMapper(parsedDocument.type()).mappers().indexAnalyzer(); // TODO: instead of passing null here, we can have a CTL> and pass previous, // like the indexer does try (TokenStream tokenStream = field.tokenStream(analyzer, null)) { diff --git a/core/src/main/java/org/elasticsearch/plugins/PluginManager.java b/core/src/main/java/org/elasticsearch/plugins/PluginManager.java index 8e6391ee0c6..29da911f07f 100644 --- a/core/src/main/java/org/elasticsearch/plugins/PluginManager.java +++ b/core/src/main/java/org/elasticsearch/plugins/PluginManager.java @@ -101,6 +101,7 @@ public class PluginManager { "discovery-ec2", "discovery-gce", "discovery-multicast", + "ingest-geoip", "lang-javascript", "lang-plan-a", "lang-python", diff --git a/core/src/main/java/org/elasticsearch/rest/action/bulk/RestBulkAction.java b/core/src/main/java/org/elasticsearch/rest/action/bulk/RestBulkAction.java index 37ce03bac70..df20438fa97 100644 --- a/core/src/main/java/org/elasticsearch/rest/action/bulk/RestBulkAction.java +++ b/core/src/main/java/org/elasticsearch/rest/action/bulk/RestBulkAction.java @@ -77,6 +77,7 @@ public class RestBulkAction extends BaseRestHandler { String defaultType = request.param("type"); String defaultRouting = request.param("routing"); String fieldsParam = request.param("fields"); + String defaultPipeline = request.param("pipeline"); String[] defaultFields = fieldsParam != null ? Strings.commaDelimitedListToStringArray(fieldsParam) : null; String consistencyLevel = request.param("consistency"); @@ -85,7 +86,7 @@ public class RestBulkAction extends BaseRestHandler { } bulkRequest.timeout(request.paramAsTime("timeout", BulkShardRequest.DEFAULT_TIMEOUT)); bulkRequest.refresh(request.paramAsBoolean("refresh", bulkRequest.refresh())); - bulkRequest.add(request.content(), defaultIndex, defaultType, defaultRouting, defaultFields, null, allowExplicitIndex); + bulkRequest.add(request.content(), defaultIndex, defaultType, defaultRouting, defaultFields, defaultPipeline, null, allowExplicitIndex); client.bulk(bulkRequest, new RestBuilderListener(channel) { @Override diff --git a/core/src/main/java/org/elasticsearch/rest/action/index/RestIndexAction.java b/core/src/main/java/org/elasticsearch/rest/action/index/RestIndexAction.java index 13a93299187..0fc15454ecb 100644 --- a/core/src/main/java/org/elasticsearch/rest/action/index/RestIndexAction.java +++ b/core/src/main/java/org/elasticsearch/rest/action/index/RestIndexAction.java @@ -77,6 +77,7 @@ public class RestIndexAction extends BaseRestHandler { if (request.hasParam("ttl")) { indexRequest.ttl(request.param("ttl")); } + indexRequest.setPipeline(request.param("pipeline")); indexRequest.source(request.content()); indexRequest.timeout(request.paramAsTime("timeout", IndexRequest.DEFAULT_TIMEOUT)); indexRequest.refresh(request.paramAsBoolean("refresh", indexRequest.refresh())); diff --git a/core/src/main/java/org/elasticsearch/rest/action/ingest/RestDeletePipelineAction.java b/core/src/main/java/org/elasticsearch/rest/action/ingest/RestDeletePipelineAction.java new file mode 100644 index 00000000000..c4526d49368 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/rest/action/ingest/RestDeletePipelineAction.java @@ -0,0 +1,47 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.rest.action.ingest; + +import org.elasticsearch.action.ingest.DeletePipelineRequest; +import org.elasticsearch.client.Client; +import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.rest.BaseRestHandler; +import org.elasticsearch.rest.RestChannel; +import org.elasticsearch.rest.RestController; +import org.elasticsearch.rest.RestRequest; +import org.elasticsearch.rest.action.support.AcknowledgedRestListener; + +public class RestDeletePipelineAction extends BaseRestHandler { + + @Inject + public RestDeletePipelineAction(Settings settings, RestController controller, Client client) { + super(settings, controller, client); + controller.registerHandler(RestRequest.Method.DELETE, "/_ingest/pipeline/{id}", this); + } + + @Override + protected void handleRequest(RestRequest restRequest, RestChannel channel, Client client) throws Exception { + DeletePipelineRequest request = new DeletePipelineRequest(restRequest.param("id")); + request.masterNodeTimeout(restRequest.paramAsTime("master_timeout", request.masterNodeTimeout())); + request.timeout(restRequest.paramAsTime("timeout", request.timeout())); + client.deletePipeline(request, new AcknowledgedRestListener<>(channel)); + } +} diff --git a/core/src/main/java/org/elasticsearch/rest/action/ingest/RestGetPipelineAction.java b/core/src/main/java/org/elasticsearch/rest/action/ingest/RestGetPipelineAction.java new file mode 100644 index 00000000000..b483a84c116 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/rest/action/ingest/RestGetPipelineAction.java @@ -0,0 +1,47 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.rest.action.ingest; + +import org.elasticsearch.action.ingest.GetPipelineRequest; +import org.elasticsearch.client.Client; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.rest.BaseRestHandler; +import org.elasticsearch.rest.RestChannel; +import org.elasticsearch.rest.RestController; +import org.elasticsearch.rest.RestRequest; +import org.elasticsearch.rest.action.support.RestStatusToXContentListener; + +public class RestGetPipelineAction extends BaseRestHandler { + + @Inject + public RestGetPipelineAction(Settings settings, RestController controller, Client client) { + super(settings, controller, client); + controller.registerHandler(RestRequest.Method.GET, "/_ingest/pipeline/{id}", this); + } + + @Override + protected void handleRequest(RestRequest restRequest, RestChannel channel, Client client) throws Exception { + GetPipelineRequest request = new GetPipelineRequest(Strings.splitStringByCommaToArray(restRequest.param("id"))); + request.masterNodeTimeout(restRequest.paramAsTime("master_timeout", request.masterNodeTimeout())); + client.getPipeline(request, new RestStatusToXContentListener<>(channel)); + } +} diff --git a/core/src/main/java/org/elasticsearch/rest/action/ingest/RestPutPipelineAction.java b/core/src/main/java/org/elasticsearch/rest/action/ingest/RestPutPipelineAction.java new file mode 100644 index 00000000000..5cdd9a893f2 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/rest/action/ingest/RestPutPipelineAction.java @@ -0,0 +1,48 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.rest.action.ingest; + +import org.elasticsearch.action.ingest.PutPipelineRequest; +import org.elasticsearch.client.Client; +import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.rest.BaseRestHandler; +import org.elasticsearch.rest.RestChannel; +import org.elasticsearch.rest.RestController; +import org.elasticsearch.rest.RestRequest; +import org.elasticsearch.rest.action.support.AcknowledgedRestListener; +import org.elasticsearch.rest.action.support.RestActions; + +public class RestPutPipelineAction extends BaseRestHandler { + + @Inject + public RestPutPipelineAction(Settings settings, RestController controller, Client client) { + super(settings, controller, client); + controller.registerHandler(RestRequest.Method.PUT, "/_ingest/pipeline/{id}", this); + } + + @Override + protected void handleRequest(RestRequest restRequest, RestChannel channel, Client client) throws Exception { + PutPipelineRequest request = new PutPipelineRequest(restRequest.param("id"), RestActions.getRestContent(restRequest)); + request.masterNodeTimeout(restRequest.paramAsTime("master_timeout", request.masterNodeTimeout())); + request.timeout(restRequest.paramAsTime("timeout", request.timeout())); + client.putPipeline(request, new AcknowledgedRestListener<>(channel)); + } +} diff --git a/core/src/main/java/org/elasticsearch/rest/action/ingest/RestSimulatePipelineAction.java b/core/src/main/java/org/elasticsearch/rest/action/ingest/RestSimulatePipelineAction.java new file mode 100644 index 00000000000..35cf43740a4 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/rest/action/ingest/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.rest.action.ingest; + +import org.elasticsearch.action.ingest.SimulatePipelineRequest; +import org.elasticsearch.client.Client; +import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.rest.BaseRestHandler; +import org.elasticsearch.rest.RestChannel; +import org.elasticsearch.rest.RestController; +import org.elasticsearch.rest.RestRequest; +import org.elasticsearch.rest.action.support.RestActions; +import org.elasticsearch.rest.action.support.RestToXContentListener; + +public class RestSimulatePipelineAction extends BaseRestHandler { + + @Inject + public RestSimulatePipelineAction(Settings settings, RestController controller, Client client) { + super(settings, controller, client); + controller.registerHandler(RestRequest.Method.POST, "/_ingest/pipeline/{id}/_simulate", this); + controller.registerHandler(RestRequest.Method.GET, "/_ingest/pipeline/{id}/_simulate", this); + controller.registerHandler(RestRequest.Method.POST, "/_ingest/pipeline/_simulate", this); + controller.registerHandler(RestRequest.Method.GET, "/_ingest/pipeline/_simulate", this); + } + + @Override + protected void handleRequest(RestRequest restRequest, RestChannel channel, Client client) throws Exception { + SimulatePipelineRequest request = new SimulatePipelineRequest(RestActions.getRestContent(restRequest)); + request.setId(restRequest.param("id")); + request.setVerbose(restRequest.paramAsBoolean("verbose", false)); + client.simulatePipeline(request, new RestToXContentListener<>(channel)); + } +} diff --git a/core/src/main/java/org/elasticsearch/script/ScriptContext.java b/core/src/main/java/org/elasticsearch/script/ScriptContext.java index 4b1b6de63f2..3ab2bb52c9b 100644 --- a/core/src/main/java/org/elasticsearch/script/ScriptContext.java +++ b/core/src/main/java/org/elasticsearch/script/ScriptContext.java @@ -37,7 +37,7 @@ public interface ScriptContext { */ enum Standard implements ScriptContext { - AGGS("aggs"), SEARCH("search"), UPDATE("update"); + AGGS("aggs"), SEARCH("search"), UPDATE("update"), INGEST("ingest"); private final String key; diff --git a/core/src/main/java/org/elasticsearch/search/sort/ScriptSortBuilder.java b/core/src/main/java/org/elasticsearch/search/sort/ScriptSortBuilder.java index 36b651a2220..e9a9c8df57c 100644 --- a/core/src/main/java/org/elasticsearch/search/sort/ScriptSortBuilder.java +++ b/core/src/main/java/org/elasticsearch/search/sort/ScriptSortBuilder.java @@ -22,11 +22,8 @@ package org.elasticsearch.search.sort; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.index.query.QueryBuilder; import org.elasticsearch.script.Script; -import org.elasticsearch.script.ScriptService.ScriptType; import java.io.IOException; -import java.util.HashMap; -import java.util.Map; /** * Script sort builder allows to sort based on a custom script expression. @@ -35,17 +32,8 @@ public class ScriptSortBuilder extends SortBuilder { private Script script; - @Deprecated - private String scriptString; - private final String type; - @Deprecated - private String lang; - - @Deprecated - private Map params; - private SortOrder order; private String sortMode; @@ -65,66 +53,6 @@ public class ScriptSortBuilder extends SortBuilder { this.type = type; } - /** - * Constructs a script sort builder with the script and the type. - * - * @param script - * The script to use. - * @param type - * The type, can either be "string" or "number". - * - * @deprecated Use {@link #ScriptSortBuilder(Script, String)} instead. - */ - @Deprecated - public ScriptSortBuilder(String script, String type) { - this.scriptString = script; - this.type = type; - } - - /** - * Adds a parameter to the script. - * - * @param name - * The name of the parameter. - * @param value - * The value of the parameter. - * - * @deprecated Use {@link #ScriptSortBuilder(Script, String)} instead. - */ - @Deprecated - public ScriptSortBuilder param(String name, Object value) { - if (params == null) { - params = new HashMap<>(); - } - params.put(name, value); - return this; - } - - /** - * Sets parameters for the script. - * - * @param params - * The script parameters - * - * @deprecated Use {@link #ScriptSortBuilder(Script, String)} instead. - */ - @Deprecated - public ScriptSortBuilder setParams(Map params) { - this.params = params; - return this; - } - - /** - * The language of the script. - * - * @deprecated Use {@link #ScriptSortBuilder(Script, String)} instead. - */ - @Deprecated - public ScriptSortBuilder lang(String lang) { - this.lang = lang; - return this; - } - /** * Sets the sort order. */ @@ -172,12 +100,7 @@ public class ScriptSortBuilder extends SortBuilder { @Override public XContentBuilder toXContent(XContentBuilder builder, Params builderParams) throws IOException { builder.startObject("_script"); - if (script == null) { - - builder.field("script", new Script(scriptString, ScriptType.INLINE, lang, params)); - } else { - builder.field("script", script); - } + builder.field("script", script); builder.field("type", type); if (order == SortOrder.DESC) { builder.field("reverse", true); @@ -189,7 +112,7 @@ public class ScriptSortBuilder extends SortBuilder { builder.field("nested_path", nestedPath); } if (nestedFilter != null) { - builder.field("nested_filter", nestedFilter, params); + builder.field("nested_filter", nestedFilter, builderParams); } builder.endObject(); return builder; diff --git a/core/src/main/java/org/elasticsearch/search/sort/SortBuilders.java b/core/src/main/java/org/elasticsearch/search/sort/SortBuilders.java index 01134ca2648..9a843c43f74 100644 --- a/core/src/main/java/org/elasticsearch/search/sort/SortBuilders.java +++ b/core/src/main/java/org/elasticsearch/search/sort/SortBuilders.java @@ -54,20 +54,6 @@ public class SortBuilders { return new ScriptSortBuilder(script, type); } - /** - * Constructs a new script based sort. - * - * @param script - * The script to use. - * @param type - * The type, can either be "string" or "number". - * @deprecated Use {@link #scriptSort(Script, String)} instead. - */ - @Deprecated - public static ScriptSortBuilder scriptSort(String script, String type) { - return new ScriptSortBuilder(script, type); - } - /** * A geo distance based sort. * diff --git a/core/src/main/resources/org/elasticsearch/plugins/plugin-install.help b/core/src/main/resources/org/elasticsearch/plugins/plugin-install.help index 8c73e3837a4..d46f7dca29a 100644 --- a/core/src/main/resources/org/elasticsearch/plugins/plugin-install.help +++ b/core/src/main/resources/org/elasticsearch/plugins/plugin-install.help @@ -43,6 +43,7 @@ OFFICIAL PLUGINS - discovery-ec2 - discovery-gce - discovery-multicast + - ingest-geoip - lang-javascript - lang-plan-a - lang-python diff --git a/core/src/test/java/org/elasticsearch/action/ingest/BulkRequestModifierTests.java b/core/src/test/java/org/elasticsearch/action/ingest/BulkRequestModifierTests.java new file mode 100644 index 00000000000..aa30c89ef59 --- /dev/null +++ b/core/src/test/java/org/elasticsearch/action/ingest/BulkRequestModifierTests.java @@ -0,0 +1,165 @@ +package org.elasticsearch.action.ingest; + +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.ActionRequest; +import org.elasticsearch.action.bulk.BulkItemResponse; +import org.elasticsearch.action.bulk.BulkRequest; +import org.elasticsearch.action.bulk.BulkResponse; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.index.IndexResponse; +import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.test.ESTestCase; +import org.hamcrest.Matchers; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.nullValue; +import static org.mockito.Mockito.mock; + +public class BulkRequestModifierTests extends ESTestCase { + + public void testBulkRequestModifier() { + int numRequests = scaledRandomIntBetween(8, 64); + BulkRequest bulkRequest = new BulkRequest(); + for (int i = 0; i < numRequests; i++) { + bulkRequest.add(new IndexRequest("_index", "_type", String.valueOf(i)).source("{}")); + } + CaptureActionListener actionListener = new CaptureActionListener(); + IngestActionFilter.BulkRequestModifier bulkRequestModifier = new IngestActionFilter.BulkRequestModifier(bulkRequest); + + int i = 0; + Set failedSlots = new HashSet<>(); + while (bulkRequestModifier.hasNext()) { + bulkRequestModifier.next(); + if (randomBoolean()) { + bulkRequestModifier.markCurrentItemAsFailed(new RuntimeException()); + failedSlots.add(i); + } + i++; + } + + assertThat(bulkRequestModifier.getBulkRequest().requests().size(), equalTo(numRequests - failedSlots.size())); + // simulate that we actually executed the modified bulk request: + ActionListener result = bulkRequestModifier.wrapActionListenerIfNeeded(actionListener); + result.onResponse(new BulkResponse(new BulkItemResponse[numRequests - failedSlots.size()], 0)); + + BulkResponse bulkResponse = actionListener.getResponse(); + for (int j = 0; j < bulkResponse.getItems().length; j++) { + if (failedSlots.contains(j)) { + BulkItemResponse item = bulkResponse.getItems()[j]; + assertThat(item.isFailed(), is(true)); + assertThat(item.getFailure().getIndex(), equalTo("_index")); + assertThat(item.getFailure().getType(), equalTo("_type")); + assertThat(item.getFailure().getId(), equalTo(String.valueOf(j))); + assertThat(item.getFailure().getMessage(), equalTo("java.lang.RuntimeException")); + } else { + assertThat(bulkResponse.getItems()[j], nullValue()); + } + } + } + + public void testPipelineFailures() { + BulkRequest originalBulkRequest = new BulkRequest(); + for (int i = 0; i < 32; i++) { + originalBulkRequest.add(new IndexRequest("index", "type", String.valueOf(i))); + } + + IngestActionFilter.BulkRequestModifier modifier = new IngestActionFilter.BulkRequestModifier(originalBulkRequest); + for (int i = 0; modifier.hasNext(); i++) { + modifier.next(); + if (i % 2 == 0) { + modifier.markCurrentItemAsFailed(new RuntimeException()); + } + } + + // So half of the requests have "failed", so only the successful requests are left: + BulkRequest bulkRequest = modifier.getBulkRequest(); + assertThat(bulkRequest.requests().size(), Matchers.equalTo(16)); + + List responses = new ArrayList<>(); + ActionListener bulkResponseListener = modifier.wrapActionListenerIfNeeded(new ActionListener() { + @Override + public void onResponse(BulkResponse bulkItemResponses) { + responses.addAll(Arrays.asList(bulkItemResponses.getItems())); + } + + @Override + public void onFailure(Throwable e) { + } + }); + + List originalResponses = new ArrayList<>(); + for (ActionRequest actionRequest : bulkRequest.requests()) { + IndexRequest indexRequest = (IndexRequest) actionRequest; + IndexResponse indexResponse = new IndexResponse(new ShardId("index", 0), indexRequest.type(), indexRequest.id(), 1, true); + originalResponses.add(new BulkItemResponse(Integer.parseInt(indexRequest.id()), indexRequest.opType().lowercase(), indexResponse)); + } + bulkResponseListener.onResponse(new BulkResponse(originalResponses.toArray(new BulkItemResponse[originalResponses.size()]), 0)); + + assertThat(responses.size(), Matchers.equalTo(32)); + for (int i = 0; i < 32; i++) { + assertThat(responses.get(i).getId(), Matchers.equalTo(String.valueOf(i))); + } + } + + public void testNoFailures() { + BulkRequest originalBulkRequest = new BulkRequest(); + for (int i = 0; i < 32; i++) { + originalBulkRequest.add(new IndexRequest("index", "type", String.valueOf(i))); + } + + IngestActionFilter.BulkRequestModifier modifier = new IngestActionFilter.BulkRequestModifier(originalBulkRequest); + while (modifier.hasNext()) { + modifier.next(); + } + + BulkRequest bulkRequest = modifier.getBulkRequest(); + assertThat(bulkRequest, Matchers.sameInstance(originalBulkRequest)); + @SuppressWarnings("unchecked") + ActionListener actionListener = mock(ActionListener.class); + assertThat(modifier.wrapActionListenerIfNeeded(actionListener), Matchers.sameInstance(actionListener)); + } + + private static class CaptureActionListener implements ActionListener { + + private BulkResponse response; + + @Override + public void onResponse(BulkResponse bulkItemResponses) { + this.response = bulkItemResponses ; + } + + @Override + public void onFailure(Throwable e) { + } + + public BulkResponse getResponse() { + return response; + } + } +} diff --git a/core/src/test/java/org/elasticsearch/action/ingest/IngestActionFilterTests.java b/core/src/test/java/org/elasticsearch/action/ingest/IngestActionFilterTests.java new file mode 100644 index 00000000000..e1ffe94e63d --- /dev/null +++ b/core/src/test/java/org/elasticsearch/action/ingest/IngestActionFilterTests.java @@ -0,0 +1,249 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.action.ingest; + +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.ActionRequest; +import org.elasticsearch.action.bulk.BulkAction; +import org.elasticsearch.action.bulk.BulkRequest; +import org.elasticsearch.action.delete.DeleteRequest; +import org.elasticsearch.action.index.IndexAction; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.support.ActionFilterChain; +import org.elasticsearch.action.update.UpdateRequest; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.ingest.IngestService; +import org.elasticsearch.ingest.PipelineExecutionService; +import org.elasticsearch.ingest.PipelineStore; +import org.elasticsearch.ingest.core.CompoundProcessor; +import org.elasticsearch.ingest.core.IngestDocument; +import org.elasticsearch.ingest.core.Pipeline; +import org.elasticsearch.ingest.core.Processor; +import org.elasticsearch.node.service.NodeService; +import org.elasticsearch.tasks.Task; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.threadpool.ThreadPool; +import org.junit.Before; +import org.mockito.stubbing.Answer; + +import java.util.function.Consumer; + +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.nullValue; +import static org.mockito.Matchers.same; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.verifyZeroInteractions; +import static org.mockito.Mockito.when; + +public class IngestActionFilterTests extends ESTestCase { + + private IngestActionFilter filter; + private PipelineExecutionService executionService; + + @Before + public void setup() { + executionService = mock(PipelineExecutionService.class); + IngestService ingestService = mock(IngestService.class); + when(ingestService.getPipelineExecutionService()).thenReturn(executionService); + NodeService nodeService = mock(NodeService.class); + when(nodeService.getIngestService()).thenReturn(ingestService); + filter = new IngestActionFilter(Settings.EMPTY, nodeService); + } + + public void testApplyNoPipelineId() throws Exception { + IndexRequest indexRequest = new IndexRequest(); + Task task = mock(Task.class); + ActionListener actionListener = mock(ActionListener.class); + ActionFilterChain actionFilterChain = mock(ActionFilterChain.class); + + filter.apply(task, IndexAction.NAME, indexRequest, actionListener, actionFilterChain); + + verify(actionFilterChain).proceed(task, IndexAction.NAME, indexRequest, actionListener); + verifyZeroInteractions(executionService, actionFilterChain); + } + + public void testApplyBulkNoPipelineId() throws Exception { + BulkRequest bulkRequest = new BulkRequest(); + bulkRequest.add(new IndexRequest()); + Task task = mock(Task.class); + ActionListener actionListener = mock(ActionListener.class); + ActionFilterChain actionFilterChain = mock(ActionFilterChain.class); + + filter.apply(task, BulkAction.NAME, bulkRequest, actionListener, actionFilterChain); + + verify(actionFilterChain).proceed(task, BulkAction.NAME, bulkRequest, actionListener); + verifyZeroInteractions(executionService, actionFilterChain); + } + + @SuppressWarnings("unchecked") + public void testApplyIngestIdViaRequestParam() throws Exception { + Task task = mock(Task.class); + IndexRequest indexRequest = new IndexRequest("_index", "_type", "_id").setPipeline("_id"); + indexRequest.source("field", "value"); + ActionListener actionListener = mock(ActionListener.class); + ActionFilterChain actionFilterChain = mock(ActionFilterChain.class); + + filter.apply(task, IndexAction.NAME, indexRequest, actionListener, actionFilterChain); + + verify(executionService).execute(same(indexRequest), any(Consumer.class), any(Consumer.class)); + verifyZeroInteractions(actionFilterChain); + } + + @SuppressWarnings("unchecked") + public void testApplyExecuted() throws Exception { + Task task = mock(Task.class); + IndexRequest indexRequest = new IndexRequest("_index", "_type", "_id").setPipeline("_id"); + indexRequest.source("field", "value"); + ActionListener actionListener = mock(ActionListener.class); + ActionFilterChain actionFilterChain = mock(ActionFilterChain.class); + + Answer answer = invocationOnMock -> { + @SuppressWarnings("unchecked") + Consumer listener = (Consumer) invocationOnMock.getArguments()[2]; + listener.accept(true); + return null; + }; + doAnswer(answer).when(executionService).execute(any(IndexRequest.class), any(Consumer.class), any(Consumer.class)); + filter.apply(task, IndexAction.NAME, indexRequest, actionListener, actionFilterChain); + + verify(executionService).execute(same(indexRequest), any(Consumer.class), any(Consumer.class)); + verify(actionFilterChain).proceed(task, IndexAction.NAME, indexRequest, actionListener); + verifyZeroInteractions(actionListener); + } + + @SuppressWarnings("unchecked") + public void testApplyFailed() throws Exception { + Task task = mock(Task.class); + IndexRequest indexRequest = new IndexRequest("_index", "_type", "_id").setPipeline("_id"); + indexRequest.source("field", "value"); + ActionListener actionListener = mock(ActionListener.class); + ActionFilterChain actionFilterChain = mock(ActionFilterChain.class); + + RuntimeException exception = new RuntimeException(); + Answer answer = invocationOnMock -> { + Consumer handler = (Consumer) invocationOnMock.getArguments()[1]; + handler.accept(exception); + return null; + }; + doAnswer(answer).when(executionService).execute(same(indexRequest), any(Consumer.class), any(Consumer.class)); + filter.apply(task, IndexAction.NAME, indexRequest, actionListener, actionFilterChain); + + verify(executionService).execute(same(indexRequest), any(Consumer.class), any(Consumer.class)); + verify(actionListener).onFailure(exception); + verifyZeroInteractions(actionFilterChain); + } + + public void testApplyWithBulkRequest() throws Exception { + Task task = mock(Task.class); + ThreadPool threadPool = mock(ThreadPool.class); + when(threadPool.executor(any())).thenReturn(Runnable::run); + PipelineStore store = mock(PipelineStore.class); + + Processor processor = new Processor() { + @Override + public void execute(IngestDocument ingestDocument) { + ingestDocument.setFieldValue("field2", "value2"); + } + + @Override + public String getType() { + return null; + } + + @Override + public String getTag() { + return null; + } + }; + when(store.get("_id")).thenReturn(new Pipeline("_id", "_description", new CompoundProcessor(processor))); + executionService = new PipelineExecutionService(store, threadPool); + IngestService ingestService = mock(IngestService.class); + when(ingestService.getPipelineExecutionService()).thenReturn(executionService); + NodeService nodeService = mock(NodeService.class); + when(nodeService.getIngestService()).thenReturn(ingestService); + filter = new IngestActionFilter(Settings.EMPTY, nodeService); + + BulkRequest bulkRequest = new BulkRequest(); + int numRequest = scaledRandomIntBetween(8, 64); + for (int i = 0; i < numRequest; i++) { + if (rarely()) { + ActionRequest request; + if (randomBoolean()) { + request = new DeleteRequest("_index", "_type", "_id"); + } else { + request = new UpdateRequest("_index", "_type", "_id"); + } + bulkRequest.add(request); + } else { + IndexRequest indexRequest = new IndexRequest("_index", "_type", "_id").setPipeline("_id"); + indexRequest.source("field1", "value1"); + bulkRequest.add(indexRequest); + } + } + + ActionListener actionListener = mock(ActionListener.class); + ActionFilterChain actionFilterChain = mock(ActionFilterChain.class); + + filter.apply(task, BulkAction.NAME, bulkRequest, actionListener, actionFilterChain); + + assertBusy(() -> { + verify(actionFilterChain).proceed(task, BulkAction.NAME, bulkRequest, actionListener); + verifyZeroInteractions(actionListener); + + int assertedRequests = 0; + for (ActionRequest actionRequest : bulkRequest.requests()) { + if (actionRequest instanceof IndexRequest) { + IndexRequest indexRequest = (IndexRequest) actionRequest; + assertThat(indexRequest.sourceAsMap().size(), equalTo(2)); + assertThat(indexRequest.sourceAsMap().get("field1"), equalTo("value1")); + assertThat(indexRequest.sourceAsMap().get("field2"), equalTo("value2")); + } + assertedRequests++; + } + assertThat(assertedRequests, equalTo(numRequest)); + }); + } + + @SuppressWarnings("unchecked") + public void testIndexApiSinglePipelineExecution() { + Answer answer = invocationOnMock -> { + @SuppressWarnings("unchecked") + Consumer listener = (Consumer) invocationOnMock.getArguments()[2]; + listener.accept(true); + return null; + }; + doAnswer(answer).when(executionService).execute(any(IndexRequest.class), any(Consumer.class), any(Consumer.class)); + + Task task = mock(Task.class); + ActionListener actionListener = mock(ActionListener.class); + ActionFilterChain actionFilterChain = mock(ActionFilterChain.class); + + IndexRequest indexRequest = new IndexRequest("_index", "_type", "_id").setPipeline("_id").source("field", "value"); + filter.apply(task, IndexAction.NAME, indexRequest, actionListener, actionFilterChain); + assertThat(indexRequest.getPipeline(), nullValue()); + filter.apply(task, IndexAction.NAME, indexRequest, actionListener, actionFilterChain); + verify(executionService, times(1)).execute(same(indexRequest), any(Consumer.class), any(Consumer.class)); + verify(actionFilterChain, times(2)).proceed(task, IndexAction.NAME, indexRequest, actionListener); + } +} diff --git a/core/src/test/java/org/elasticsearch/action/ingest/IngestProxyActionFilterTests.java b/core/src/test/java/org/elasticsearch/action/ingest/IngestProxyActionFilterTests.java new file mode 100644 index 00000000000..fa9728c4cd1 --- /dev/null +++ b/core/src/test/java/org/elasticsearch/action/ingest/IngestProxyActionFilterTests.java @@ -0,0 +1,251 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.action.ingest; + +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.ActionRequest; +import org.elasticsearch.action.bulk.BulkAction; +import org.elasticsearch.action.bulk.BulkRequest; +import org.elasticsearch.action.bulk.BulkResponse; +import org.elasticsearch.action.index.IndexAction; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.index.IndexResponse; +import org.elasticsearch.action.support.ActionFilterChain; +import org.elasticsearch.cluster.ClusterName; +import org.elasticsearch.cluster.ClusterService; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.cluster.node.DiscoveryNodes; +import org.elasticsearch.common.transport.DummyTransportAddress; +import org.elasticsearch.tasks.Task; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.test.VersionUtils; +import org.elasticsearch.transport.TransportException; +import org.elasticsearch.transport.TransportRequest; +import org.elasticsearch.transport.TransportResponse; +import org.elasticsearch.transport.TransportResponseHandler; +import org.elasticsearch.transport.TransportService; +import org.hamcrest.CustomTypeSafeMatcher; +import org.mockito.stubbing.Answer; + +import java.util.HashMap; +import java.util.Map; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.mockito.Matchers.any; +import static org.mockito.Matchers.argThat; +import static org.mockito.Matchers.eq; +import static org.mockito.Matchers.same; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.verifyZeroInteractions; +import static org.mockito.Mockito.when; + +public class IngestProxyActionFilterTests extends ESTestCase { + + private TransportService transportService; + + @SuppressWarnings("unchecked") + private IngestProxyActionFilter buildFilter(int ingestNodes, int totalNodes) { + ClusterState.Builder clusterState = new ClusterState.Builder(new ClusterName("_name")); + DiscoveryNodes.Builder builder = new DiscoveryNodes.Builder(); + DiscoveryNode localNode = null; + for (int i = 0; i < totalNodes; i++) { + String nodeId = "node" + i; + Map attributes = new HashMap<>(); + if (i >= ingestNodes) { + attributes.put("ingest", "false"); + } else if (randomBoolean()) { + attributes.put("ingest", "true"); + } + DiscoveryNode node = new DiscoveryNode(nodeId, nodeId, DummyTransportAddress.INSTANCE, attributes, VersionUtils.randomVersion(random())); + builder.put(node); + if (i == totalNodes - 1) { + localNode = node; + } + } + clusterState.nodes(builder); + ClusterService clusterService = mock(ClusterService.class); + when(clusterService.localNode()).thenReturn(localNode); + when(clusterService.state()).thenReturn(clusterState.build()); + transportService = mock(TransportService.class); + return new IngestProxyActionFilter(clusterService, transportService); + } + + public void testApplyNoIngestNodes() { + Task task = mock(Task.class); + ActionListener actionListener = mock(ActionListener.class); + ActionFilterChain actionFilterChain = mock(ActionFilterChain.class); + int totalNodes = randomIntBetween(1, 5); + IngestProxyActionFilter filter = buildFilter(0, totalNodes); + + String action; + ActionRequest request; + if (randomBoolean()) { + action = IndexAction.NAME; + request = new IndexRequest().setPipeline("_id"); + } else { + action = BulkAction.NAME; + request = new BulkRequest().add(new IndexRequest().setPipeline("_id")); + } + try { + filter.apply(task, action, request, actionListener, actionFilterChain); + fail("should have failed because there are no ingest nodes"); + } catch(IllegalStateException e) { + assertThat(e.getMessage(), equalTo("There are no ingest nodes in this cluster, unable to forward request to an ingest node.")); + } + verifyZeroInteractions(transportService); + verifyZeroInteractions(actionFilterChain); + verifyZeroInteractions(actionListener); + } + + public void testApplyNoPipelineId() { + Task task = mock(Task.class); + ActionListener actionListener = mock(ActionListener.class); + ActionFilterChain actionFilterChain = mock(ActionFilterChain.class); + int totalNodes = randomIntBetween(1, 5); + IngestProxyActionFilter filter = buildFilter(randomIntBetween(0, totalNodes - 1), totalNodes); + + String action; + ActionRequest request; + if (randomBoolean()) { + action = IndexAction.NAME; + request = new IndexRequest(); + } else { + action = BulkAction.NAME; + request = new BulkRequest().add(new IndexRequest()); + } + filter.apply(task, action, request, actionListener, actionFilterChain); + verifyZeroInteractions(transportService); + verify(actionFilterChain).proceed(any(Task.class), eq(action), same(request), same(actionListener)); + verifyZeroInteractions(actionListener); + } + + public void testApplyAnyAction() { + Task task = mock(Task.class); + ActionListener actionListener = mock(ActionListener.class); + ActionFilterChain actionFilterChain = mock(ActionFilterChain.class); + ActionRequest request = mock(ActionRequest.class); + int totalNodes = randomIntBetween(1, 5); + IngestProxyActionFilter filter = buildFilter(randomIntBetween(0, totalNodes - 1), totalNodes); + + String action = randomAsciiOfLengthBetween(1, 20); + filter.apply(task, action, request, actionListener, actionFilterChain); + verifyZeroInteractions(transportService); + verify(actionFilterChain).proceed(any(Task.class), eq(action), same(request), same(actionListener)); + verifyZeroInteractions(actionListener); + } + + @SuppressWarnings("unchecked") + public void testApplyIndexRedirect() { + Task task = mock(Task.class); + ActionListener actionListener = mock(ActionListener.class); + ActionFilterChain actionFilterChain = mock(ActionFilterChain.class); + int totalNodes = randomIntBetween(2, 5); + IngestProxyActionFilter filter = buildFilter(randomIntBetween(1, totalNodes - 1), totalNodes); + Answer answer = invocationOnMock -> { + TransportResponseHandler transportResponseHandler = (TransportResponseHandler) invocationOnMock.getArguments()[3]; + transportResponseHandler.handleResponse(new IndexResponse()); + return null; + }; + doAnswer(answer).when(transportService).sendRequest(any(DiscoveryNode.class), any(String.class), any(TransportRequest.class), any(TransportResponseHandler.class)); + + IndexRequest indexRequest = new IndexRequest().setPipeline("_id"); + filter.apply(task, IndexAction.NAME, indexRequest, actionListener, actionFilterChain); + + verify(transportService).sendRequest(argThat(new IngestNodeMatcher()), eq(IndexAction.NAME), same(indexRequest), any(TransportResponseHandler.class)); + verifyZeroInteractions(actionFilterChain); + verify(actionListener).onResponse(any(IndexResponse.class)); + verify(actionListener, never()).onFailure(any(TransportException.class)); + } + + @SuppressWarnings("unchecked") + public void testApplyBulkRedirect() { + Task task = mock(Task.class); + ActionListener actionListener = mock(ActionListener.class); + ActionFilterChain actionFilterChain = mock(ActionFilterChain.class); + int totalNodes = randomIntBetween(2, 5); + IngestProxyActionFilter filter = buildFilter(randomIntBetween(1, totalNodes - 1), totalNodes); + Answer answer = invocationOnMock -> { + TransportResponseHandler transportResponseHandler = (TransportResponseHandler) invocationOnMock.getArguments()[3]; + transportResponseHandler.handleResponse(new BulkResponse(null, -1)); + return null; + }; + doAnswer(answer).when(transportService).sendRequest(any(DiscoveryNode.class), any(String.class), any(TransportRequest.class), any(TransportResponseHandler.class)); + + BulkRequest bulkRequest = new BulkRequest(); + bulkRequest.add(new IndexRequest().setPipeline("_id")); + int numNoPipelineRequests = randomIntBetween(0, 10); + for (int i = 0; i < numNoPipelineRequests; i++) { + bulkRequest.add(new IndexRequest()); + } + filter.apply(task, BulkAction.NAME, bulkRequest, actionListener, actionFilterChain); + + verify(transportService).sendRequest(argThat(new IngestNodeMatcher()), eq(BulkAction.NAME), same(bulkRequest), any(TransportResponseHandler.class)); + verifyZeroInteractions(actionFilterChain); + verify(actionListener).onResponse(any(BulkResponse.class)); + verify(actionListener, never()).onFailure(any(TransportException.class)); + } + + @SuppressWarnings("unchecked") + public void testApplyFailures() { + Task task = mock(Task.class); + ActionListener actionListener = mock(ActionListener.class); + ActionFilterChain actionFilterChain = mock(ActionFilterChain.class); + int totalNodes = randomIntBetween(2, 5); + IngestProxyActionFilter filter = buildFilter(randomIntBetween(1, totalNodes - 1), totalNodes); + Answer answer = invocationOnMock -> { + TransportResponseHandler transportResponseHandler = (TransportResponseHandler) invocationOnMock.getArguments()[3]; + transportResponseHandler.handleException(new TransportException(new IllegalArgumentException())); + return null; + }; + doAnswer(answer).when(transportService).sendRequest(any(DiscoveryNode.class), any(String.class), any(TransportRequest.class), any(TransportResponseHandler.class)); + + String action; + ActionRequest request; + if (randomBoolean()) { + action = IndexAction.NAME; + request = new IndexRequest().setPipeline("_id"); + } else { + action = BulkAction.NAME; + request = new BulkRequest().add(new IndexRequest().setPipeline("_id")); + } + + filter.apply(task, action, request, actionListener, actionFilterChain); + + verify(transportService).sendRequest(argThat(new IngestNodeMatcher()), eq(action), same(request), any(TransportResponseHandler.class)); + verifyZeroInteractions(actionFilterChain); + verify(actionListener).onFailure(any(TransportException.class)); + verify(actionListener, never()).onResponse(any(TransportResponse.class)); + } + + private static class IngestNodeMatcher extends CustomTypeSafeMatcher { + private IngestNodeMatcher() { + super("discovery node should be an ingest node"); + } + + @Override + protected boolean matchesSafely(DiscoveryNode node) { + return node.isIngestNode(); + } + } +} diff --git a/core/src/test/java/org/elasticsearch/action/ingest/SimulateDocumentSimpleResultTests.java b/core/src/test/java/org/elasticsearch/action/ingest/SimulateDocumentSimpleResultTests.java new file mode 100644 index 00000000000..882fca72156 --- /dev/null +++ b/core/src/test/java/org/elasticsearch/action/ingest/SimulateDocumentSimpleResultTests.java @@ -0,0 +1,57 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.action.ingest; + +import org.elasticsearch.common.io.stream.BytesStreamOutput; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.ingest.core.IngestDocument; +import org.elasticsearch.ingest.RandomDocumentPicks; +import org.elasticsearch.test.ESTestCase; + +import java.io.IOException; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.instanceOf; + +public class SimulateDocumentSimpleResultTests extends ESTestCase { + + public void testSerialization() throws IOException { + boolean isFailure = randomBoolean(); + SimulateDocumentBaseResult simulateDocumentBaseResult; + if (isFailure) { + simulateDocumentBaseResult = new SimulateDocumentBaseResult(new IllegalArgumentException("test")); + } else { + IngestDocument ingestDocument = RandomDocumentPicks.randomIngestDocument(random()); + simulateDocumentBaseResult = new SimulateDocumentBaseResult(ingestDocument); + } + + BytesStreamOutput out = new BytesStreamOutput(); + simulateDocumentBaseResult.writeTo(out); + StreamInput streamInput = StreamInput.wrap(out.bytes()); + SimulateDocumentBaseResult otherSimulateDocumentBaseResult = SimulateDocumentBaseResult.readSimulateDocumentSimpleResult(streamInput); + + assertThat(otherSimulateDocumentBaseResult.getIngestDocument(), equalTo(simulateDocumentBaseResult.getIngestDocument())); + if (isFailure) { + assertThat(otherSimulateDocumentBaseResult.getFailure(), instanceOf(IllegalArgumentException.class)); + IllegalArgumentException e = (IllegalArgumentException) otherSimulateDocumentBaseResult.getFailure(); + assertThat(e.getMessage(), equalTo("test")); + } + } +} diff --git a/core/src/test/java/org/elasticsearch/action/ingest/SimulateExecutionServiceTests.java b/core/src/test/java/org/elasticsearch/action/ingest/SimulateExecutionServiceTests.java new file mode 100644 index 00000000000..d58b9bf850d --- /dev/null +++ b/core/src/test/java/org/elasticsearch/action/ingest/SimulateExecutionServiceTests.java @@ -0,0 +1,206 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.action.ingest; + +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.ingest.RandomDocumentPicks; +import org.elasticsearch.ingest.TestProcessor; +import org.elasticsearch.ingest.core.CompoundProcessor; +import org.elasticsearch.ingest.core.Processor; +import org.elasticsearch.ingest.core.IngestDocument; +import org.elasticsearch.ingest.core.Pipeline; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.threadpool.ThreadPool; +import org.junit.After; +import org.junit.Before; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.instanceOf; +import static org.hamcrest.Matchers.not; +import static org.hamcrest.Matchers.nullValue; +import static org.hamcrest.Matchers.sameInstance; + +public class SimulateExecutionServiceTests extends ESTestCase { + + private ThreadPool threadPool; + private SimulateExecutionService executionService; + private Pipeline pipeline; + private Processor processor; + private IngestDocument ingestDocument; + + @Before + public void setup() { + threadPool = new ThreadPool( + Settings.builder() + .put("name", getClass().getName()) + .build() + ); + executionService = new SimulateExecutionService(threadPool); + processor = new TestProcessor("id", "mock", ingestDocument -> {}); + pipeline = new Pipeline("_id", "_description", new CompoundProcessor(processor, processor)); + ingestDocument = RandomDocumentPicks.randomIngestDocument(random()); + } + + @After + public void destroy() { + threadPool.shutdown(); + } + + public void testExecuteVerboseDocumentSimple() throws Exception { + List processorResultList = new ArrayList<>(); + executionService.executeVerboseDocument(processor, ingestDocument, processorResultList); + SimulateProcessorResult result = new SimulateProcessorResult("id", ingestDocument); + assertThat(processorResultList.size(), equalTo(1)); + assertThat(processorResultList.get(0).getProcessorTag(), equalTo(result.getProcessorTag())); + assertThat(processorResultList.get(0).getIngestDocument(), equalTo(result.getIngestDocument())); + assertThat(processorResultList.get(0).getFailure(), nullValue()); + } + + public void testExecuteVerboseDocumentSimpleException() throws Exception { + RuntimeException exception = new RuntimeException("mock_exception"); + TestProcessor processor = new TestProcessor("id", "mock", ingestDocument -> { throw exception; }); + List processorResultList = new ArrayList<>(); + try { + executionService.executeVerboseDocument(processor, ingestDocument, processorResultList); + fail("should throw exception"); + } catch (RuntimeException e) { + assertThat(e.getMessage(), equalTo("mock_exception")); + } + SimulateProcessorResult result = new SimulateProcessorResult("id", exception); + assertThat(processorResultList.size(), equalTo(1)); + assertThat(processorResultList.get(0).getProcessorTag(), equalTo(result.getProcessorTag())); + assertThat(processorResultList.get(0).getFailure(), equalTo(result.getFailure())); + } + + public void testExecuteVerboseDocumentCompoundSuccess() throws Exception { + TestProcessor processor1 = new TestProcessor("p1", "mock", ingestDocument -> { }); + TestProcessor processor2 = new TestProcessor("p2", "mock", ingestDocument -> { }); + + Processor compoundProcessor = new CompoundProcessor(processor1, processor2); + List processorResultList = new ArrayList<>(); + executionService.executeVerboseDocument(compoundProcessor, ingestDocument, processorResultList); + assertThat(processor1.getInvokedCounter(), equalTo(1)); + assertThat(processor2.getInvokedCounter(), equalTo(1)); + assertThat(processorResultList.size(), equalTo(2)); + assertThat(processorResultList.get(0).getProcessorTag(), equalTo("p1")); + assertThat(processorResultList.get(0).getIngestDocument(), equalTo(ingestDocument)); + assertThat(processorResultList.get(0).getFailure(), nullValue()); + assertThat(processorResultList.get(1).getProcessorTag(), equalTo("p2")); + assertThat(processorResultList.get(1).getIngestDocument(), equalTo(ingestDocument)); + assertThat(processorResultList.get(1).getFailure(), nullValue()); + } + + public void testExecuteVerboseDocumentCompoundOnFailure() throws Exception { + TestProcessor processor1 = new TestProcessor("p1", "mock", ingestDocument -> { }); + TestProcessor processor2 = new TestProcessor("p2", "mock", ingestDocument -> { throw new RuntimeException("p2_exception"); }); + TestProcessor onFailureProcessor1 = new TestProcessor("fail_p1", "mock", ingestDocument -> { }); + TestProcessor onFailureProcessor2 = new TestProcessor("fail_p2", "mock", ingestDocument -> { throw new RuntimeException("fail_p2_exception"); }); + TestProcessor onFailureProcessor3 = new TestProcessor("fail_p3", "mock", ingestDocument -> { }); + CompoundProcessor onFailureCompoundProcessor = new CompoundProcessor(Collections.singletonList(onFailureProcessor2), Collections.singletonList(onFailureProcessor3)); + + Processor compoundProcessor = new CompoundProcessor(Arrays.asList(processor1, processor2), Arrays.asList(onFailureProcessor1, onFailureCompoundProcessor)); + List processorResultList = new ArrayList<>(); + executionService.executeVerboseDocument(compoundProcessor, ingestDocument, processorResultList); + assertThat(processor1.getInvokedCounter(), equalTo(1)); + assertThat(processor2.getInvokedCounter(), equalTo(1)); + assertThat(onFailureProcessor1.getInvokedCounter(), equalTo(1)); + assertThat(onFailureProcessor2.getInvokedCounter(), equalTo(1)); + assertThat(onFailureProcessor3.getInvokedCounter(), equalTo(1)); + assertThat(processorResultList.size(), equalTo(5)); + assertThat(processorResultList.get(0).getProcessorTag(), equalTo("p1")); + assertThat(processorResultList.get(1).getProcessorTag(), equalTo("p2")); + assertThat(processorResultList.get(2).getProcessorTag(), equalTo("fail_p1")); + assertThat(processorResultList.get(3).getProcessorTag(), equalTo("fail_p2")); + assertThat(processorResultList.get(4).getProcessorTag(), equalTo("fail_p3")); + } + + public void testExecuteVerboseItem() throws Exception { + TestProcessor processor = new TestProcessor("test-id", "mock", ingestDocument -> {}); + Pipeline pipeline = new Pipeline("_id", "_description", new CompoundProcessor(processor, processor)); + SimulateDocumentResult actualItemResponse = executionService.executeDocument(pipeline, ingestDocument, true); + assertThat(processor.getInvokedCounter(), equalTo(2)); + assertThat(actualItemResponse, instanceOf(SimulateDocumentVerboseResult.class)); + SimulateDocumentVerboseResult simulateDocumentVerboseResult = (SimulateDocumentVerboseResult) actualItemResponse; + assertThat(simulateDocumentVerboseResult.getProcessorResults().size(), equalTo(2)); + assertThat(simulateDocumentVerboseResult.getProcessorResults().get(0).getProcessorTag(), equalTo("test-id")); + assertThat(simulateDocumentVerboseResult.getProcessorResults().get(0).getIngestDocument(), not(sameInstance(ingestDocument))); + assertThat(simulateDocumentVerboseResult.getProcessorResults().get(0).getIngestDocument(), equalTo(ingestDocument)); + assertThat(simulateDocumentVerboseResult.getProcessorResults().get(0).getIngestDocument().getSourceAndMetadata(), not(sameInstance(ingestDocument.getSourceAndMetadata()))); + + assertThat(simulateDocumentVerboseResult.getProcessorResults().get(0).getFailure(), nullValue()); + assertThat(simulateDocumentVerboseResult.getProcessorResults().get(1).getProcessorTag(), equalTo("test-id")); + assertThat(simulateDocumentVerboseResult.getProcessorResults().get(1).getIngestDocument(), not(sameInstance(ingestDocument))); + assertThat(simulateDocumentVerboseResult.getProcessorResults().get(1).getIngestDocument(), equalTo(ingestDocument)); + assertThat(simulateDocumentVerboseResult.getProcessorResults().get(1).getIngestDocument().getSourceAndMetadata(), not(sameInstance(ingestDocument.getSourceAndMetadata()))); + assertThat(simulateDocumentVerboseResult.getProcessorResults().get(1).getIngestDocument().getSourceAndMetadata(), + not(sameInstance(simulateDocumentVerboseResult.getProcessorResults().get(0).getIngestDocument().getSourceAndMetadata()))); + assertThat(simulateDocumentVerboseResult.getProcessorResults().get(1).getFailure(), nullValue()); + } + + public void testExecuteItem() throws Exception { + TestProcessor processor = new TestProcessor("processor_0", "mock", ingestDocument -> {}); + Pipeline pipeline = new Pipeline("_id", "_description", new CompoundProcessor(processor, processor)); + SimulateDocumentResult actualItemResponse = executionService.executeDocument(pipeline, ingestDocument, false); + assertThat(processor.getInvokedCounter(), equalTo(2)); + assertThat(actualItemResponse, instanceOf(SimulateDocumentBaseResult.class)); + SimulateDocumentBaseResult simulateDocumentBaseResult = (SimulateDocumentBaseResult) actualItemResponse; + assertThat(simulateDocumentBaseResult.getIngestDocument(), equalTo(ingestDocument)); + assertThat(simulateDocumentBaseResult.getFailure(), nullValue()); + } + + public void testExecuteVerboseItemWithFailure() throws Exception { + TestProcessor processor1 = new TestProcessor("processor_0", "mock", ingestDocument -> { throw new RuntimeException("processor failed"); }); + TestProcessor processor2 = new TestProcessor("processor_1", "mock", ingestDocument -> {}); + Pipeline pipeline = new Pipeline("_id", "_description", new CompoundProcessor(Collections.singletonList(processor1), Collections.singletonList(processor2))); + SimulateDocumentResult actualItemResponse = executionService.executeDocument(pipeline, ingestDocument, true); + assertThat(processor1.getInvokedCounter(), equalTo(1)); + assertThat(processor2.getInvokedCounter(), equalTo(1)); + assertThat(actualItemResponse, instanceOf(SimulateDocumentVerboseResult.class)); + SimulateDocumentVerboseResult simulateDocumentVerboseResult = (SimulateDocumentVerboseResult) actualItemResponse; + assertThat(simulateDocumentVerboseResult.getProcessorResults().size(), equalTo(2)); + assertThat(simulateDocumentVerboseResult.getProcessorResults().get(0).getProcessorTag(), equalTo("processor_0")); + assertThat(simulateDocumentVerboseResult.getProcessorResults().get(0).getIngestDocument(), nullValue()); + assertThat(simulateDocumentVerboseResult.getProcessorResults().get(0).getFailure(), instanceOf(RuntimeException.class)); + RuntimeException runtimeException = (RuntimeException) simulateDocumentVerboseResult.getProcessorResults().get(0).getFailure(); + assertThat(runtimeException.getMessage(), equalTo("processor failed")); + assertThat(simulateDocumentVerboseResult.getProcessorResults().get(1).getProcessorTag(), equalTo("processor_1")); + assertThat(simulateDocumentVerboseResult.getProcessorResults().get(1).getIngestDocument(), not(sameInstance(ingestDocument))); + assertThat(simulateDocumentVerboseResult.getProcessorResults().get(1).getIngestDocument(), equalTo(ingestDocument)); + assertThat(simulateDocumentVerboseResult.getProcessorResults().get(1).getFailure(), nullValue()); + } + + public void testExecuteItemWithFailure() throws Exception { + TestProcessor processor = new TestProcessor(ingestDocument -> { throw new RuntimeException("processor failed"); }); + Pipeline pipeline = new Pipeline("_id", "_description", new CompoundProcessor(processor, processor)); + SimulateDocumentResult actualItemResponse = executionService.executeDocument(pipeline, ingestDocument, false); + assertThat(processor.getInvokedCounter(), equalTo(1)); + assertThat(actualItemResponse, instanceOf(SimulateDocumentBaseResult.class)); + SimulateDocumentBaseResult simulateDocumentBaseResult = (SimulateDocumentBaseResult) actualItemResponse; + assertThat(simulateDocumentBaseResult.getIngestDocument(), nullValue()); + assertThat(simulateDocumentBaseResult.getFailure(), instanceOf(RuntimeException.class)); + RuntimeException runtimeException = (RuntimeException) simulateDocumentBaseResult.getFailure(); + assertThat(runtimeException.getMessage(), equalTo("processor failed")); + } +} diff --git a/core/src/test/java/org/elasticsearch/action/ingest/SimulatePipelineRequestParsingTests.java b/core/src/test/java/org/elasticsearch/action/ingest/SimulatePipelineRequestParsingTests.java new file mode 100644 index 00000000000..c0e7d6921ac --- /dev/null +++ b/core/src/test/java/org/elasticsearch/action/ingest/SimulatePipelineRequestParsingTests.java @@ -0,0 +1,181 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.action.ingest; + +import org.elasticsearch.ingest.PipelineStore; +import org.elasticsearch.ingest.TestProcessor; +import org.elasticsearch.ingest.core.CompoundProcessor; +import org.elasticsearch.ingest.core.IngestDocument; +import org.elasticsearch.ingest.core.Pipeline; +import org.elasticsearch.ingest.core.Processor; +import org.elasticsearch.test.ESTestCase; +import org.junit.Before; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; + +import static org.elasticsearch.action.ingest.SimulatePipelineRequest.Fields; +import static org.elasticsearch.ingest.core.IngestDocument.MetaData.ID; +import static org.elasticsearch.ingest.core.IngestDocument.MetaData.INDEX; +import static org.elasticsearch.ingest.core.IngestDocument.MetaData.TYPE; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.nullValue; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class SimulatePipelineRequestParsingTests extends ESTestCase { + + private PipelineStore store; + + @Before + public void init() throws IOException { + TestProcessor processor = new TestProcessor(ingestDocument -> {}); + CompoundProcessor pipelineCompoundProcessor = new CompoundProcessor(processor); + Pipeline pipeline = new Pipeline(SimulatePipelineRequest.SIMULATED_PIPELINE_ID, null, pipelineCompoundProcessor); + Map processorRegistry = new HashMap<>(); + processorRegistry.put("mock_processor", mock(Processor.Factory.class)); + store = mock(PipelineStore.class); + when(store.get(SimulatePipelineRequest.SIMULATED_PIPELINE_ID)).thenReturn(pipeline); + when(store.getProcessorFactoryRegistry()).thenReturn(processorRegistry); + } + + public void testParseUsingPipelineStore() throws Exception { + int numDocs = randomIntBetween(1, 10); + + Map requestContent = new HashMap<>(); + List> docs = new ArrayList<>(); + 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(INDEX.getFieldName(), index); + doc.put(TYPE.getFieldName(), type); + doc.put(ID.getFieldName(), id); + String fieldName = randomAsciiOfLengthBetween(1, 10); + String fieldValue = randomAsciiOfLengthBetween(1, 10); + doc.put(Fields.SOURCE, Collections.singletonMap(fieldName, fieldValue)); + docs.add(doc); + Map expectedDoc = new HashMap<>(); + expectedDoc.put(INDEX.getFieldName(), index); + expectedDoc.put(TYPE.getFieldName(), type); + expectedDoc.put(ID.getFieldName(), id); + expectedDoc.put(Fields.SOURCE, Collections.singletonMap(fieldName, fieldValue)); + expectedDocs.add(expectedDoc); + } + + SimulatePipelineRequest.Parsed actualRequest = SimulatePipelineRequest.parseWithPipelineId(SimulatePipelineRequest.SIMULATED_PIPELINE_ID, requestContent, false, store); + assertThat(actualRequest.isVerbose(), equalTo(false)); + assertThat(actualRequest.getDocuments().size(), equalTo(numDocs)); + Iterator> expectedDocsIterator = expectedDocs.iterator(); + for (IngestDocument ingestDocument : actualRequest.getDocuments()) { + Map expectedDocument = expectedDocsIterator.next(); + Map metadataMap = ingestDocument.extractMetadata(); + assertThat(metadataMap.get(INDEX), equalTo(expectedDocument.get(INDEX.getFieldName()))); + assertThat(metadataMap.get(TYPE), equalTo(expectedDocument.get(TYPE.getFieldName()))); + assertThat(metadataMap.get(ID), equalTo(expectedDocument.get(ID.getFieldName()))); + assertThat(ingestDocument.getSourceAndMetadata(), equalTo(expectedDocument.get(Fields.SOURCE))); + } + + assertThat(actualRequest.getPipeline().getId(), equalTo(SimulatePipelineRequest.SIMULATED_PIPELINE_ID)); + assertThat(actualRequest.getPipeline().getDescription(), nullValue()); + assertThat(actualRequest.getPipeline().getProcessors().size(), equalTo(1)); + } + + public void testParseWithProvidedPipeline() throws Exception { + int numDocs = randomIntBetween(1, 10); + + Map requestContent = new HashMap<>(); + List> docs = new ArrayList<>(); + 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(INDEX.getFieldName(), index); + doc.put(TYPE.getFieldName(), type); + doc.put(ID.getFieldName(), id); + String fieldName = randomAsciiOfLengthBetween(1, 10); + String fieldValue = randomAsciiOfLengthBetween(1, 10); + doc.put(Fields.SOURCE, Collections.singletonMap(fieldName, fieldValue)); + docs.add(doc); + Map expectedDoc = new HashMap<>(); + expectedDoc.put(INDEX.getFieldName(), index); + expectedDoc.put(TYPE.getFieldName(), type); + expectedDoc.put(ID.getFieldName(), id); + expectedDoc.put(Fields.SOURCE, Collections.singletonMap(fieldName, fieldValue)); + expectedDocs.add(expectedDoc); + } + + Map pipelineConfig = new HashMap<>(); + List> processors = new ArrayList<>(); + int numProcessors = randomIntBetween(1, 10); + for (int i = 0; i < numProcessors; i++) { + Map processorConfig = new HashMap<>(); + List> onFailureProcessors = new ArrayList<>(); + int numOnFailureProcessors = randomIntBetween(0, 1); + for (int j = 0; j < numOnFailureProcessors; j++) { + onFailureProcessors.add(Collections.singletonMap("mock_processor", Collections.emptyMap())); + } + if (numOnFailureProcessors > 0) { + processorConfig.put("on_failure", onFailureProcessors); + } + processors.add(Collections.singletonMap("mock_processor", processorConfig)); + } + pipelineConfig.put("processors", processors); + + List> onFailureProcessors = new ArrayList<>(); + int numOnFailureProcessors = randomIntBetween(0, 1); + for (int i = 0; i < numOnFailureProcessors; i++) { + onFailureProcessors.add(Collections.singletonMap("mock_processor", Collections.emptyMap())); + } + if (numOnFailureProcessors > 0) { + pipelineConfig.put("on_failure", onFailureProcessors); + } + + requestContent.put(Fields.PIPELINE, pipelineConfig); + + SimulatePipelineRequest.Parsed actualRequest = SimulatePipelineRequest.parse(requestContent, false, store); + assertThat(actualRequest.isVerbose(), equalTo(false)); + assertThat(actualRequest.getDocuments().size(), equalTo(numDocs)); + Iterator> expectedDocsIterator = expectedDocs.iterator(); + for (IngestDocument ingestDocument : actualRequest.getDocuments()) { + Map expectedDocument = expectedDocsIterator.next(); + Map metadataMap = ingestDocument.extractMetadata(); + assertThat(metadataMap.get(INDEX), equalTo(expectedDocument.get(INDEX.getFieldName()))); + assertThat(metadataMap.get(TYPE), equalTo(expectedDocument.get(TYPE.getFieldName()))); + assertThat(metadataMap.get(ID), equalTo(expectedDocument.get(ID.getFieldName()))); + assertThat(ingestDocument.getSourceAndMetadata(), equalTo(expectedDocument.get(Fields.SOURCE))); + } + + assertThat(actualRequest.getPipeline().getId(), equalTo(SimulatePipelineRequest.SIMULATED_PIPELINE_ID)); + assertThat(actualRequest.getPipeline().getDescription(), nullValue()); + assertThat(actualRequest.getPipeline().getProcessors().size(), equalTo(numProcessors)); + } +} diff --git a/core/src/test/java/org/elasticsearch/action/ingest/SimulatePipelineResponseTests.java b/core/src/test/java/org/elasticsearch/action/ingest/SimulatePipelineResponseTests.java new file mode 100644 index 00000000000..12a62f0684a --- /dev/null +++ b/core/src/test/java/org/elasticsearch/action/ingest/SimulatePipelineResponseTests.java @@ -0,0 +1,117 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.action.ingest; + +import org.elasticsearch.common.io.stream.BytesStreamOutput; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.ingest.core.IngestDocument; +import org.elasticsearch.ingest.RandomDocumentPicks; +import org.elasticsearch.test.ESTestCase; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.instanceOf; +import static org.hamcrest.CoreMatchers.nullValue; + +public class SimulatePipelineResponseTests extends ESTestCase { + + public void testSerialization() throws IOException { + boolean isVerbose = randomBoolean(); + int numResults = randomIntBetween(1, 10); + List results = new ArrayList<>(numResults); + for (int i = 0; i < numResults; i++) { + boolean isFailure = randomBoolean(); + IngestDocument ingestDocument = RandomDocumentPicks.randomIngestDocument(random()); + if (isVerbose) { + int numProcessors = randomIntBetween(1, 10); + List processorResults = new ArrayList<>(numProcessors); + for (int j = 0; j < numProcessors; j++) { + String processorTag = randomAsciiOfLengthBetween(1, 10); + SimulateProcessorResult processorResult; + if (isFailure) { + processorResult = new SimulateProcessorResult(processorTag, new IllegalArgumentException("test")); + } else { + processorResult = new SimulateProcessorResult(processorTag, ingestDocument); + } + processorResults.add(processorResult); + } + results.add(new SimulateDocumentVerboseResult(processorResults)); + } else { + results.add(new SimulateDocumentBaseResult(ingestDocument)); + SimulateDocumentBaseResult simulateDocumentBaseResult; + if (isFailure) { + simulateDocumentBaseResult = new SimulateDocumentBaseResult(new IllegalArgumentException("test")); + } else { + simulateDocumentBaseResult = new SimulateDocumentBaseResult(ingestDocument); + } + results.add(simulateDocumentBaseResult); + } + } + + SimulatePipelineResponse response = new SimulatePipelineResponse(randomAsciiOfLengthBetween(1, 10), isVerbose, results); + BytesStreamOutput out = new BytesStreamOutput(); + response.writeTo(out); + StreamInput streamInput = StreamInput.wrap(out.bytes()); + SimulatePipelineResponse otherResponse = new SimulatePipelineResponse(); + otherResponse.readFrom(streamInput); + + assertThat(otherResponse.getPipelineId(), equalTo(response.getPipelineId())); + assertThat(otherResponse.getResults().size(), equalTo(response.getResults().size())); + + Iterator 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.getProcessorTag(), equalTo(expectedProcessorResult.getProcessorTag())); + assertThat(simulateProcessorResult.getIngestDocument(), equalTo(expectedProcessorResult.getIngestDocument())); + if (expectedProcessorResult.getFailure() == null) { + assertThat(simulateProcessorResult.getFailure(), nullValue()); + } else { + assertThat(simulateProcessorResult.getFailure(), instanceOf(IllegalArgumentException.class)); + IllegalArgumentException e = (IllegalArgumentException) simulateProcessorResult.getFailure(); + assertThat(e.getMessage(), equalTo("test")); + } + } + } else { + SimulateDocumentBaseResult expectedSimulateDocumentBaseResult = (SimulateDocumentBaseResult) expectedResultIterator.next(); + assertThat(result, instanceOf(SimulateDocumentBaseResult.class)); + SimulateDocumentBaseResult simulateDocumentBaseResult = (SimulateDocumentBaseResult) result; + assertThat(simulateDocumentBaseResult.getIngestDocument(), equalTo(expectedSimulateDocumentBaseResult.getIngestDocument())); + if (expectedSimulateDocumentBaseResult.getFailure() == null) { + assertThat(simulateDocumentBaseResult.getFailure(), nullValue()); + } else { + assertThat(simulateDocumentBaseResult.getFailure(), instanceOf(IllegalArgumentException.class)); + IllegalArgumentException e = (IllegalArgumentException) simulateDocumentBaseResult.getFailure(); + assertThat(e.getMessage(), equalTo("test")); + } + } + } + } +} diff --git a/core/src/test/java/org/elasticsearch/action/ingest/SimulateProcessorResultTests.java b/core/src/test/java/org/elasticsearch/action/ingest/SimulateProcessorResultTests.java new file mode 100644 index 00000000000..0885475adc5 --- /dev/null +++ b/core/src/test/java/org/elasticsearch/action/ingest/SimulateProcessorResultTests.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.action.ingest; + +import org.elasticsearch.common.io.stream.BytesStreamOutput; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.ingest.core.IngestDocument; +import org.elasticsearch.ingest.RandomDocumentPicks; +import org.elasticsearch.test.ESTestCase; + +import java.io.IOException; + +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.instanceOf; + +public class SimulateProcessorResultTests extends ESTestCase { + + public void testSerialization() throws IOException { + String processorTag = randomAsciiOfLengthBetween(1, 10); + boolean isFailure = randomBoolean(); + SimulateProcessorResult simulateProcessorResult; + if (isFailure) { + simulateProcessorResult = new SimulateProcessorResult(processorTag, new IllegalArgumentException("test")); + } else { + IngestDocument ingestDocument = RandomDocumentPicks.randomIngestDocument(random()); + simulateProcessorResult = new SimulateProcessorResult(processorTag, ingestDocument); + } + + BytesStreamOutput out = new BytesStreamOutput(); + simulateProcessorResult.writeTo(out); + StreamInput streamInput = StreamInput.wrap(out.bytes()); + SimulateProcessorResult otherSimulateProcessorResult = new SimulateProcessorResult(streamInput); + assertThat(otherSimulateProcessorResult.getProcessorTag(), equalTo(simulateProcessorResult.getProcessorTag())); + assertThat(otherSimulateProcessorResult.getIngestDocument(), equalTo(simulateProcessorResult.getIngestDocument())); + if (isFailure) { + assertThat(otherSimulateProcessorResult.getFailure(), instanceOf(IllegalArgumentException.class)); + IllegalArgumentException e = (IllegalArgumentException) otherSimulateProcessorResult.getFailure(); + assertThat(e.getMessage(), equalTo("test")); + } + } +} diff --git a/core/src/test/java/org/elasticsearch/action/ingest/WriteableIngestDocumentTests.java b/core/src/test/java/org/elasticsearch/action/ingest/WriteableIngestDocumentTests.java new file mode 100644 index 00000000000..8d3c812f4ce --- /dev/null +++ b/core/src/test/java/org/elasticsearch/action/ingest/WriteableIngestDocumentTests.java @@ -0,0 +1,114 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.action.ingest; + +import org.elasticsearch.common.io.stream.BytesStreamOutput; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.ingest.RandomDocumentPicks; +import org.elasticsearch.ingest.core.IngestDocument; +import org.elasticsearch.test.ESTestCase; + +import java.io.IOException; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.not; + +public class WriteableIngestDocumentTests extends ESTestCase { + + public void testEqualsAndHashcode() throws Exception { + Map sourceAndMetadata = RandomDocumentPicks.randomSource(random()); + int numFields = randomIntBetween(1, IngestDocument.MetaData.values().length); + for (int i = 0; i < numFields; i++) { + sourceAndMetadata.put(randomFrom(IngestDocument.MetaData.values()).getFieldName(), randomAsciiOfLengthBetween(5, 10)); + } + Map ingestMetadata = new HashMap<>(); + numFields = randomIntBetween(1, 5); + for (int i = 0; i < numFields; i++) { + ingestMetadata.put(randomAsciiOfLengthBetween(5, 10), randomAsciiOfLengthBetween(5, 10)); + } + WriteableIngestDocument ingestDocument = new WriteableIngestDocument(new IngestDocument(sourceAndMetadata, ingestMetadata)); + + boolean changed = false; + Map otherSourceAndMetadata; + if (randomBoolean()) { + otherSourceAndMetadata = RandomDocumentPicks.randomSource(random()); + changed = true; + } else { + otherSourceAndMetadata = new HashMap<>(sourceAndMetadata); + } + if (randomBoolean()) { + numFields = randomIntBetween(1, IngestDocument.MetaData.values().length); + for (int i = 0; i < numFields; i++) { + otherSourceAndMetadata.put(randomFrom(IngestDocument.MetaData.values()).getFieldName(), randomAsciiOfLengthBetween(5, 10)); + } + changed = true; + } + + Map otherIngestMetadata; + if (randomBoolean()) { + otherIngestMetadata = new HashMap<>(); + numFields = randomIntBetween(1, 5); + for (int i = 0; i < numFields; i++) { + otherIngestMetadata.put(randomAsciiOfLengthBetween(5, 10), randomAsciiOfLengthBetween(5, 10)); + } + changed = true; + } else { + otherIngestMetadata = Collections.unmodifiableMap(ingestMetadata); + } + + WriteableIngestDocument otherIngestDocument = new WriteableIngestDocument(new IngestDocument(otherSourceAndMetadata, otherIngestMetadata)); + if (changed) { + assertThat(ingestDocument, not(equalTo(otherIngestDocument))); + assertThat(otherIngestDocument, not(equalTo(ingestDocument))); + } else { + assertThat(ingestDocument, equalTo(otherIngestDocument)); + assertThat(otherIngestDocument, equalTo(ingestDocument)); + assertThat(ingestDocument.hashCode(), equalTo(otherIngestDocument.hashCode())); + WriteableIngestDocument thirdIngestDocument = new WriteableIngestDocument(new IngestDocument(Collections.unmodifiableMap(sourceAndMetadata), Collections.unmodifiableMap(ingestMetadata))); + assertThat(thirdIngestDocument, equalTo(ingestDocument)); + assertThat(ingestDocument, equalTo(thirdIngestDocument)); + assertThat(ingestDocument.hashCode(), equalTo(thirdIngestDocument.hashCode())); + } + } + + public void testSerialization() throws IOException { + Map sourceAndMetadata = RandomDocumentPicks.randomSource(random()); + int numFields = randomIntBetween(1, IngestDocument.MetaData.values().length); + for (int i = 0; i < numFields; i++) { + sourceAndMetadata.put(randomFrom(IngestDocument.MetaData.values()).getFieldName(), randomAsciiOfLengthBetween(5, 10)); + } + Map ingestMetadata = new HashMap<>(); + numFields = randomIntBetween(1, 5); + for (int i = 0; i < numFields; i++) { + ingestMetadata.put(randomAsciiOfLengthBetween(5, 10), randomAsciiOfLengthBetween(5, 10)); + } + Map document = RandomDocumentPicks.randomSource(random()); + WriteableIngestDocument writeableIngestDocument = new WriteableIngestDocument(new IngestDocument(sourceAndMetadata, ingestMetadata)); + + BytesStreamOutput out = new BytesStreamOutput(); + writeableIngestDocument.writeTo(out); + StreamInput streamInput = StreamInput.wrap(out.bytes()); + WriteableIngestDocument otherWriteableIngestDocument = new WriteableIngestDocument(streamInput); + assertThat(otherWriteableIngestDocument, equalTo(writeableIngestDocument)); + } +} diff --git a/core/src/test/java/org/elasticsearch/indices/template/SimpleIndexTemplateIT.java b/core/src/test/java/org/elasticsearch/indices/template/SimpleIndexTemplateIT.java index 5aaed6b5d94..0a51f5d4e2c 100644 --- a/core/src/test/java/org/elasticsearch/indices/template/SimpleIndexTemplateIT.java +++ b/core/src/test/java/org/elasticsearch/indices/template/SimpleIndexTemplateIT.java @@ -63,6 +63,7 @@ import static org.hamcrest.Matchers.nullValue; * */ public class SimpleIndexTemplateIT extends ESIntegTestCase { + public void testSimpleIndexTemplateTests() throws Exception { // clean all templates setup by the framework. client().admin().indices().prepareDeleteTemplate("*").get(); diff --git a/core/src/test/java/org/elasticsearch/ingest/IngestClientIT.java b/core/src/test/java/org/elasticsearch/ingest/IngestClientIT.java new file mode 100644 index 00000000000..ae724a59f78 --- /dev/null +++ b/core/src/test/java/org/elasticsearch/ingest/IngestClientIT.java @@ -0,0 +1,231 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.ingest; + +import org.elasticsearch.action.bulk.BulkItemResponse; +import org.elasticsearch.action.bulk.BulkRequest; +import org.elasticsearch.action.bulk.BulkResponse; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.index.IndexResponse; +import org.elasticsearch.action.ingest.DeletePipelineRequest; +import org.elasticsearch.action.ingest.GetPipelineRequest; +import org.elasticsearch.action.ingest.GetPipelineResponse; +import org.elasticsearch.action.ingest.PutPipelineRequest; +import org.elasticsearch.action.ingest.SimulateDocumentBaseResult; +import org.elasticsearch.action.ingest.SimulatePipelineRequest; +import org.elasticsearch.action.ingest.SimulatePipelineResponse; +import org.elasticsearch.action.ingest.WritePipelineResponse; +import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.ingest.core.IngestDocument; +import org.elasticsearch.node.NodeModule; +import org.elasticsearch.plugins.Plugin; +import org.elasticsearch.test.ESIntegTestCase; + +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.instanceOf; +import static org.hamcrest.Matchers.nullValue; +import static org.hamcrest.core.Is.is; + +@ESIntegTestCase.ClusterScope(minNumDataNodes = 2) +public class IngestClientIT extends ESIntegTestCase { + + @Override + protected Settings nodeSettings(int nodeOrdinal) { + // TODO: Remove this method once gets in: https://github.com/elastic/elasticsearch/issues/16019 + if (nodeOrdinal % 2 == 0) { + return Settings.builder().put("node.ingest", false).put(super.nodeSettings(nodeOrdinal)).build(); + } + return super.nodeSettings(nodeOrdinal); + } + + @Override + protected Collection> nodePlugins() { + return pluginList(IngestPlugin.class); + } + + public void testSimulate() throws Exception { + BytesReference pipelineSource = jsonBuilder().startObject() + .field("description", "my_pipeline") + .startArray("processors") + .startObject() + .startObject("test") + .endObject() + .endObject() + .endArray() + .endObject().bytes(); + client().preparePutPipeline("_id", pipelineSource) + .get(); + GetPipelineResponse getResponse = client().prepareGetPipeline("_id") + .get(); + assertThat(getResponse.isFound(), is(true)); + assertThat(getResponse.pipelines().size(), equalTo(1)); + assertThat(getResponse.pipelines().get(0).getId(), equalTo("_id")); + + BytesReference bytes = jsonBuilder().startObject() + .startArray("docs") + .startObject() + .field("_index", "index") + .field("_type", "type") + .field("_id", "id") + .startObject("_source") + .field("foo", "bar") + .field("fail", false) + .endObject() + .endObject() + .endArray() + .endObject().bytes(); + SimulatePipelineResponse response; + if (randomBoolean()) { + response = client().prepareSimulatePipeline(bytes) + .setId("_id").get(); + } else { + SimulatePipelineRequest request = new SimulatePipelineRequest(bytes); + request.setId("_id"); + response = client().simulatePipeline(request).get(); + } + assertThat(response.isVerbose(), equalTo(false)); + assertThat(response.getPipelineId(), equalTo("_id")); + assertThat(response.getResults().size(), equalTo(1)); + assertThat(response.getResults().get(0), instanceOf(SimulateDocumentBaseResult.class)); + SimulateDocumentBaseResult simulateDocumentBaseResult = (SimulateDocumentBaseResult) response.getResults().get(0); + Map source = new HashMap<>(); + source.put("foo", "bar"); + source.put("fail", false); + source.put("processed", true); + IngestDocument ingestDocument = new IngestDocument("index", "type", "id", null, null, null, null, source); + assertThat(simulateDocumentBaseResult.getIngestDocument().getSourceAndMetadata(), equalTo(ingestDocument.getSourceAndMetadata())); + assertThat(simulateDocumentBaseResult.getFailure(), nullValue()); + } + + public void testBulkWithIngestFailures() throws Exception { + createIndex("index"); + + BytesReference source = jsonBuilder().startObject() + .field("description", "my_pipeline") + .startArray("processors") + .startObject() + .startObject("test") + .endObject() + .endObject() + .endArray() + .endObject().bytes(); + PutPipelineRequest putPipelineRequest = new PutPipelineRequest("_id", source); + client().putPipeline(putPipelineRequest).get(); + + int numRequests = scaledRandomIntBetween(32, 128); + BulkRequest bulkRequest = new BulkRequest(); + for (int i = 0; i < numRequests; i++) { + IndexRequest indexRequest = new IndexRequest("index", "type", Integer.toString(i)).setPipeline("_id"); + indexRequest.source("field", "value", "fail", i % 2 == 0); + bulkRequest.add(indexRequest); + } + + BulkResponse response = client().bulk(bulkRequest).actionGet(); + assertThat(response.getItems().length, equalTo(bulkRequest.requests().size())); + for (int i = 0; i < bulkRequest.requests().size(); i++) { + BulkItemResponse itemResponse = response.getItems()[i]; + if (i % 2 == 0) { + BulkItemResponse.Failure failure = itemResponse.getFailure(); + assertThat(failure.getMessage(), equalTo("java.lang.IllegalArgumentException: test processor failed")); + } else { + IndexResponse indexResponse = itemResponse.getResponse(); + assertThat(indexResponse.getId(), equalTo(Integer.toString(i))); + assertThat(indexResponse.isCreated(), is(true)); + } + } + } + + public void test() throws Exception { + BytesReference source = jsonBuilder().startObject() + .field("description", "my_pipeline") + .startArray("processors") + .startObject() + .startObject("test") + .endObject() + .endObject() + .endArray() + .endObject().bytes(); + PutPipelineRequest putPipelineRequest = new PutPipelineRequest("_id", source); + client().putPipeline(putPipelineRequest).get(); + + GetPipelineRequest getPipelineRequest = new GetPipelineRequest("_id"); + GetPipelineResponse getResponse = client().getPipeline(getPipelineRequest).get(); + assertThat(getResponse.isFound(), is(true)); + assertThat(getResponse.pipelines().size(), equalTo(1)); + assertThat(getResponse.pipelines().get(0).getId(), equalTo("_id")); + + client().prepareIndex("test", "type", "1").setPipeline("_id").setSource("field", "value", "fail", false).get(); + + Map doc = client().prepareGet("test", "type", "1") + .get().getSourceAsMap(); + assertThat(doc.get("field"), equalTo("value")); + assertThat(doc.get("processed"), equalTo(true)); + + client().prepareBulk().add( + client().prepareIndex("test", "type", "2").setSource("field", "value2", "fail", false).setPipeline("_id")).get(); + doc = client().prepareGet("test", "type", "2").get().getSourceAsMap(); + assertThat(doc.get("field"), equalTo("value2")); + assertThat(doc.get("processed"), equalTo(true)); + + DeletePipelineRequest deletePipelineRequest = new DeletePipelineRequest("_id"); + WritePipelineResponse response = client().deletePipeline(deletePipelineRequest).get(); + assertThat(response.isAcknowledged(), is(true)); + + getResponse = client().prepareGetPipeline("_id").get(); + assertThat(getResponse.isFound(), is(false)); + assertThat(getResponse.pipelines().size(), equalTo(0)); + } + + @Override + protected Collection> getMockPlugins() { + return Collections.singletonList(TestSeedPlugin.class); + } + + public static class IngestPlugin extends Plugin { + + @Override + public String name() { + return "ingest"; + } + + @Override + public String description() { + return "ingest mock"; + } + + public void onModule(NodeModule nodeModule) { + nodeModule.registerProcessor("test", templateService -> config -> + new TestProcessor("id", "test", ingestDocument -> { + ingestDocument.setFieldValue("processed", true); + if (ingestDocument.getFieldValue("fail", Boolean.class)) { + throw new IllegalArgumentException("test processor failed"); + } + }) + ); + } + } +} diff --git a/core/src/test/java/org/elasticsearch/ingest/IngestMetadataTests.java b/core/src/test/java/org/elasticsearch/ingest/IngestMetadataTests.java new file mode 100644 index 00000000000..a6cf12389a0 --- /dev/null +++ b/core/src/test/java/org/elasticsearch/ingest/IngestMetadataTests.java @@ -0,0 +1,64 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.ingest; + +import org.elasticsearch.cluster.metadata.MetaData; +import org.elasticsearch.common.bytes.BytesArray; +import org.elasticsearch.common.xcontent.ToXContent; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentFactory; +import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.common.xcontent.XContentType; +import org.elasticsearch.test.ESTestCase; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +public class IngestMetadataTests extends ESTestCase { + + public void testFromXContent() throws IOException { + PipelineConfiguration pipeline = new PipelineConfiguration( + "1",new BytesArray("{\"processors\": [{\"set\" : {\"field\": \"_field\", \"value\": \"_value\"}}]}") + ); + PipelineConfiguration pipeline2 = new PipelineConfiguration( + "2",new BytesArray("{\"processors\": [{\"set\" : {\"field\": \"_field1\", \"value\": \"_value1\"}}]}") + ); + Map map = new HashMap<>(); + map.put(pipeline.getId(), pipeline); + map.put(pipeline2.getId(), pipeline2); + IngestMetadata ingestMetadata = new IngestMetadata(map); + XContentBuilder builder = XContentFactory.jsonBuilder(); + builder.prettyPrint(); + builder.startObject(); + ingestMetadata.toXContent(builder, ToXContent.EMPTY_PARAMS); + builder.endObject(); + String string = builder.string(); + final XContentParser parser = XContentFactory.xContent(XContentType.JSON).createParser(string); + MetaData.Custom custom = ingestMetadata.fromXContent(parser); + assertTrue(custom instanceof IngestMetadata); + IngestMetadata m = (IngestMetadata) custom; + assertEquals(2, m.getPipelines().size()); + assertEquals("1", m.getPipelines().get("1").getId()); + assertEquals("2", m.getPipelines().get("2").getId()); + assertEquals(pipeline.getConfigAsMap(), m.getPipelines().get("1").getConfigAsMap()); + assertEquals(pipeline2.getConfigAsMap(), m.getPipelines().get("2").getConfigAsMap()); + } +} diff --git a/core/src/test/java/org/elasticsearch/ingest/PipelineExecutionServiceTests.java b/core/src/test/java/org/elasticsearch/ingest/PipelineExecutionServiceTests.java new file mode 100644 index 00000000000..9126a513e6e --- /dev/null +++ b/core/src/test/java/org/elasticsearch/ingest/PipelineExecutionServiceTests.java @@ -0,0 +1,366 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.ingest; + +import org.elasticsearch.ElasticsearchParseException; +import org.elasticsearch.action.ActionRequest; +import org.elasticsearch.action.bulk.BulkRequest; +import org.elasticsearch.action.delete.DeleteRequest; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.update.UpdateRequest; +import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.ingest.core.CompoundProcessor; +import org.elasticsearch.ingest.core.IngestDocument; +import org.elasticsearch.ingest.core.Pipeline; +import org.elasticsearch.ingest.core.Processor; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.threadpool.ThreadPool; +import org.hamcrest.CustomTypeSafeMatcher; +import org.junit.Before; +import org.mockito.ArgumentMatcher; +import org.mockito.invocation.InvocationOnMock; + +import java.util.Collections; +import java.util.Map; +import java.util.Objects; +import java.util.function.BiConsumer; +import java.util.function.Consumer; + +import static org.hamcrest.Matchers.equalTo; +import static org.mockito.Matchers.eq; +import static org.mockito.Matchers.any; +import static org.mockito.Matchers.anyBoolean; +import static org.mockito.Matchers.anyString; +import static org.mockito.Matchers.argThat; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +public class PipelineExecutionServiceTests extends ESTestCase { + + private PipelineStore store; + private PipelineExecutionService executionService; + + @Before + public void setup() { + store = mock(PipelineStore.class); + ThreadPool threadPool = mock(ThreadPool.class); + when(threadPool.executor(anyString())).thenReturn(Runnable::run); + executionService = new PipelineExecutionService(store, threadPool); + } + + public void testExecuteIndexPipelineDoesNotExist() { + IndexRequest indexRequest = new IndexRequest("_index", "_type", "_id").source(Collections.emptyMap()).setPipeline("_id"); + @SuppressWarnings("unchecked") + Consumer failureHandler = mock(Consumer.class); + @SuppressWarnings("unchecked") + Consumer completionHandler = mock(Consumer.class); + try { + executionService.execute(indexRequest, failureHandler, completionHandler); + fail("IllegalArgumentException expected"); + } catch (IllegalArgumentException e) { + assertThat(e.getMessage(), equalTo("pipeline with id [_id] does not exist")); + } + verify(failureHandler, never()).accept(any(Throwable.class)); + verify(completionHandler, never()).accept(anyBoolean()); + } + + public void testExecuteBulkPipelineDoesNotExist() { + CompoundProcessor processor = mock(CompoundProcessor.class); + when(store.get("_id")).thenReturn(new Pipeline("_id", "_description", processor)); + BulkRequest bulkRequest = new BulkRequest(); + + IndexRequest indexRequest1 = new IndexRequest("_index", "_type", "_id").source(Collections.emptyMap()).setPipeline("_id"); + bulkRequest.add(indexRequest1); + IndexRequest indexRequest2 = new IndexRequest("_index", "_type", "_id").source(Collections.emptyMap()).setPipeline("does_not_exist"); + bulkRequest.add(indexRequest2); + @SuppressWarnings("unchecked") + BiConsumer failureHandler = mock(BiConsumer.class); + @SuppressWarnings("unchecked") + Consumer completionHandler = mock(Consumer.class); + executionService.execute(bulkRequest.requests(), failureHandler, completionHandler); + verify(failureHandler, times(1)).accept( + argThat(new CustomTypeSafeMatcher("failure handler was not called with the expected arguments") { + @Override + protected boolean matchesSafely(IndexRequest item) { + return item == indexRequest2; + } + + }), + argThat(new CustomTypeSafeMatcher("failure handler was not called with the expected arguments") { + @Override + protected boolean matchesSafely(IllegalArgumentException iae) { + return "pipeline with id [does_not_exist] does not exist".equals(iae.getMessage()); + } + }) + ); + verify(completionHandler, times(1)).accept(null); + } + + public void testExecuteSuccess() throws Exception { + CompoundProcessor processor = mock(CompoundProcessor.class); + when(store.get("_id")).thenReturn(new Pipeline("_id", "_description", processor)); + + IndexRequest indexRequest = new IndexRequest("_index", "_type", "_id").source(Collections.emptyMap()).setPipeline("_id"); + @SuppressWarnings("unchecked") + Consumer failureHandler = mock(Consumer.class); + @SuppressWarnings("unchecked") + Consumer completionHandler = mock(Consumer.class); + executionService.execute(indexRequest, failureHandler, completionHandler); + verify(failureHandler, never()).accept(any()); + verify(completionHandler, times(1)).accept(true); + } + + public void testExecutePropagateAllMetaDataUpdates() throws Exception { + CompoundProcessor processor = mock(CompoundProcessor.class); + doAnswer((InvocationOnMock invocationOnMock) -> { + IngestDocument ingestDocument = (IngestDocument) invocationOnMock.getArguments()[0]; + for (IngestDocument.MetaData metaData : IngestDocument.MetaData.values()) { + if (metaData == IngestDocument.MetaData.TTL) { + ingestDocument.setFieldValue(IngestDocument.MetaData.TTL.getFieldName(), "5w"); + } else { + ingestDocument.setFieldValue(metaData.getFieldName(), "update" + metaData.getFieldName()); + } + + } + return null; + }).when(processor).execute(any()); + when(store.get("_id")).thenReturn(new Pipeline("_id", "_description", processor)); + + IndexRequest indexRequest = new IndexRequest("_index", "_type", "_id").source(Collections.emptyMap()).setPipeline("_id"); + @SuppressWarnings("unchecked") + Consumer failureHandler = mock(Consumer.class); + @SuppressWarnings("unchecked") + Consumer completionHandler = mock(Consumer.class); + executionService.execute(indexRequest, failureHandler, completionHandler); + verify(processor).execute(any()); + verify(failureHandler, never()).accept(any()); + verify(completionHandler, times(1)).accept(true); + + assertThat(indexRequest.index(), equalTo("update_index")); + assertThat(indexRequest.type(), equalTo("update_type")); + assertThat(indexRequest.id(), equalTo("update_id")); + assertThat(indexRequest.routing(), equalTo("update_routing")); + assertThat(indexRequest.parent(), equalTo("update_parent")); + assertThat(indexRequest.timestamp(), equalTo("update_timestamp")); + assertThat(indexRequest.ttl(), equalTo(new TimeValue(3024000000L))); + } + + public void testExecuteFailure() throws Exception { + CompoundProcessor processor = mock(CompoundProcessor.class); + when(store.get("_id")).thenReturn(new Pipeline("_id", "_description", processor)); + IndexRequest indexRequest = new IndexRequest("_index", "_type", "_id").source(Collections.emptyMap()).setPipeline("_id"); + doThrow(new RuntimeException()).when(processor).execute(eqID("_index", "_type", "_id", Collections.emptyMap())); + @SuppressWarnings("unchecked") + Consumer failureHandler = mock(Consumer.class); + @SuppressWarnings("unchecked") + Consumer completionHandler = mock(Consumer.class); + executionService.execute(indexRequest, failureHandler, completionHandler); + verify(processor).execute(eqID("_index", "_type", "_id", Collections.emptyMap())); + verify(failureHandler, times(1)).accept(any(RuntimeException.class)); + verify(completionHandler, never()).accept(anyBoolean()); + } + + public void testExecuteSuccessWithOnFailure() throws Exception { + Processor processor = mock(Processor.class); + Processor onFailureProcessor = mock(Processor.class); + CompoundProcessor compoundProcessor = new CompoundProcessor(Collections.singletonList(processor), Collections.singletonList(new CompoundProcessor(onFailureProcessor))); + when(store.get("_id")).thenReturn(new Pipeline("_id", "_description", compoundProcessor)); + IndexRequest indexRequest = new IndexRequest("_index", "_type", "_id").source(Collections.emptyMap()).setPipeline("_id"); + doThrow(new RuntimeException()).when(processor).execute(eqID("_index", "_type", "_id", Collections.emptyMap())); + @SuppressWarnings("unchecked") + Consumer failureHandler = mock(Consumer.class); + @SuppressWarnings("unchecked") + Consumer completionHandler = mock(Consumer.class); + executionService.execute(indexRequest, failureHandler, completionHandler); + verify(failureHandler, never()).accept(any(RuntimeException.class)); + verify(completionHandler, times(1)).accept(true); + } + + public void testExecuteFailureWithOnFailure() throws Exception { + Processor processor = mock(Processor.class); + Processor onFailureProcessor = mock(Processor.class); + CompoundProcessor compoundProcessor = new CompoundProcessor(Collections.singletonList(processor), Collections.singletonList(new CompoundProcessor(onFailureProcessor))); + when(store.get("_id")).thenReturn(new Pipeline("_id", "_description", compoundProcessor)); + IndexRequest indexRequest = new IndexRequest("_index", "_type", "_id").source(Collections.emptyMap()).setPipeline("_id"); + doThrow(new RuntimeException()).when(processor).execute(eqID("_index", "_type", "_id", Collections.emptyMap())); + doThrow(new RuntimeException()).when(onFailureProcessor).execute(eqID("_index", "_type", "_id", Collections.emptyMap())); + @SuppressWarnings("unchecked") + Consumer failureHandler = mock(Consumer.class); + @SuppressWarnings("unchecked") + Consumer completionHandler = mock(Consumer.class); + executionService.execute(indexRequest, failureHandler, completionHandler); + verify(processor).execute(eqID("_index", "_type", "_id", Collections.emptyMap())); + verify(failureHandler, times(1)).accept(any(RuntimeException.class)); + verify(completionHandler, never()).accept(anyBoolean()); + } + + public void testExecuteFailureWithNestedOnFailure() throws Exception { + Processor processor = mock(Processor.class); + Processor onFailureProcessor = mock(Processor.class); + Processor onFailureOnFailureProcessor = mock(Processor.class); + CompoundProcessor compoundProcessor = new CompoundProcessor(Collections.singletonList(processor), + Collections.singletonList(new CompoundProcessor(Collections.singletonList(onFailureProcessor), Collections.singletonList(onFailureOnFailureProcessor)))); + when(store.get("_id")).thenReturn(new Pipeline("_id", "_description", compoundProcessor)); + IndexRequest indexRequest = new IndexRequest("_index", "_type", "_id").source(Collections.emptyMap()).setPipeline("_id"); + doThrow(new RuntimeException()).when(onFailureOnFailureProcessor).execute(eqID("_index", "_type", "_id", Collections.emptyMap())); + doThrow(new RuntimeException()).when(onFailureProcessor).execute(eqID("_index", "_type", "_id", Collections.emptyMap())); + doThrow(new RuntimeException()).when(processor).execute(eqID("_index", "_type", "_id", Collections.emptyMap())); + @SuppressWarnings("unchecked") + Consumer failureHandler = mock(Consumer.class); + @SuppressWarnings("unchecked") + Consumer completionHandler = mock(Consumer.class); + executionService.execute(indexRequest, failureHandler, completionHandler); + verify(processor).execute(eqID("_index", "_type", "_id", Collections.emptyMap())); + verify(failureHandler, times(1)).accept(any(RuntimeException.class)); + verify(completionHandler, never()).accept(anyBoolean()); + } + + public void testExecuteSetTTL() throws Exception { + Processor processor = new TestProcessor(ingestDocument -> ingestDocument.setFieldValue("_ttl", "5d")); + when(store.get("_id")).thenReturn(new Pipeline("_id", "_description", new CompoundProcessor(processor))); + + IndexRequest indexRequest = new IndexRequest("_index", "_type", "_id").source(Collections.emptyMap()).setPipeline("_id"); + @SuppressWarnings("unchecked") + Consumer failureHandler = mock(Consumer.class); + @SuppressWarnings("unchecked") + Consumer completionHandler = mock(Consumer.class); + executionService.execute(indexRequest, failureHandler, completionHandler); + + assertThat(indexRequest.ttl(), equalTo(TimeValue.parseTimeValue("5d", null, "ttl"))); + verify(failureHandler, never()).accept(any()); + verify(completionHandler, times(1)).accept(true); + } + + public void testExecuteSetInvalidTTL() throws Exception { + Processor processor = new TestProcessor(ingestDocument -> ingestDocument.setFieldValue("_ttl", "abc")); + when(store.get("_id")).thenReturn(new Pipeline("_id", "_description", new CompoundProcessor(processor))); + + IndexRequest indexRequest = new IndexRequest("_index", "_type", "_id").source(Collections.emptyMap()).setPipeline("_id"); + @SuppressWarnings("unchecked") + Consumer failureHandler = mock(Consumer.class); + @SuppressWarnings("unchecked") + Consumer completionHandler = mock(Consumer.class); + executionService.execute(indexRequest, failureHandler, completionHandler); + verify(failureHandler, times(1)).accept(any(ElasticsearchParseException.class)); + verify(completionHandler, never()).accept(anyBoolean()); + } + + public void testExecuteProvidedTTL() throws Exception { + when(store.get("_id")).thenReturn(new Pipeline("_id", "_description", mock(CompoundProcessor.class))); + + IndexRequest indexRequest = new IndexRequest("_index", "_type", "_id").setPipeline("_id") + .source(Collections.emptyMap()) + .ttl(1000L); + Consumer failureHandler = mock(Consumer.class); + Consumer completionHandler = mock(Consumer.class); + executionService.execute(indexRequest, failureHandler, completionHandler); + + assertThat(indexRequest.ttl(), equalTo(new TimeValue(1000L))); + verify(failureHandler, never()).accept(any()); + verify(completionHandler, times(1)).accept(true); + } + + public void testBulkRequestExecutionWithFailures() throws Exception { + BulkRequest bulkRequest = new BulkRequest(); + String pipelineId = "_id"; + + int numRequest = scaledRandomIntBetween(8, 64); + int numIndexRequests = 0; + for (int i = 0; i < numRequest; i++) { + ActionRequest request; + if (randomBoolean()) { + if (randomBoolean()) { + request = new DeleteRequest("_index", "_type", "_id"); + } else { + request = new UpdateRequest("_index", "_type", "_id"); + } + } else { + IndexRequest indexRequest = new IndexRequest("_index", "_type", "_id").setPipeline(pipelineId); + indexRequest.source("field1", "value1"); + request = indexRequest; + numIndexRequests++; + } + bulkRequest.add(request); + } + + CompoundProcessor processor = mock(CompoundProcessor.class); + Exception error = new RuntimeException(); + doThrow(error).when(processor).execute(any()); + when(store.get(pipelineId)).thenReturn(new Pipeline(pipelineId, null, processor)); + + BiConsumer requestItemErrorHandler = mock(BiConsumer.class); + Consumer completionHandler = mock(Consumer.class); + executionService.execute(bulkRequest.requests(), requestItemErrorHandler, completionHandler); + + verify(requestItemErrorHandler, times(numIndexRequests)).accept(any(IndexRequest.class), eq(error)); + verify(completionHandler, times(1)).accept(null); + } + + public void testBulkRequestExecution() throws Exception { + BulkRequest bulkRequest = new BulkRequest(); + String pipelineId = "_id"; + + int numRequest = scaledRandomIntBetween(8, 64); + for (int i = 0; i < numRequest; i++) { + IndexRequest indexRequest = new IndexRequest("_index", "_type", "_id").setPipeline(pipelineId); + indexRequest.source("field1", "value1"); + bulkRequest.add(indexRequest); + } + + when(store.get(pipelineId)).thenReturn(new Pipeline(pipelineId, null, new CompoundProcessor())); + + @SuppressWarnings("unchecked") + BiConsumer requestItemErrorHandler = mock(BiConsumer.class); + @SuppressWarnings("unchecked") + Consumer completionHandler = mock(Consumer.class); + executionService.execute(bulkRequest.requests(), requestItemErrorHandler, completionHandler); + + verify(requestItemErrorHandler, never()).accept(any(), any()); + verify(completionHandler, times(1)).accept(null); + } + + private IngestDocument eqID(String index, String type, String id, Map source) { + return argThat(new IngestDocumentMatcher(index, type, id, source)); + } + + private class IngestDocumentMatcher extends ArgumentMatcher { + + private final IngestDocument ingestDocument; + + public IngestDocumentMatcher(String index, String type, String id, Map source) { + this.ingestDocument = new IngestDocument(index, type, id, null, null, null, null, source); + } + + @Override + public boolean matches(Object o) { + if (o.getClass() == IngestDocument.class) { + IngestDocument otherIngestDocument = (IngestDocument) o; + //ingest metadata will not be the same (timestamp differs every time) + return Objects.equals(ingestDocument.getSourceAndMetadata(), otherIngestDocument.getSourceAndMetadata()); + } + return false; + } + } +} diff --git a/core/src/test/java/org/elasticsearch/ingest/PipelineStoreTests.java b/core/src/test/java/org/elasticsearch/ingest/PipelineStoreTests.java new file mode 100644 index 00000000000..117b95b2cd7 --- /dev/null +++ b/core/src/test/java/org/elasticsearch/ingest/PipelineStoreTests.java @@ -0,0 +1,182 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.ingest; + +import org.elasticsearch.ResourceNotFoundException; +import org.elasticsearch.action.ingest.DeletePipelineRequest; +import org.elasticsearch.action.ingest.PutPipelineRequest; +import org.elasticsearch.cluster.ClusterName; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.metadata.MetaData; +import org.elasticsearch.common.bytes.BytesArray; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.ingest.core.Pipeline; +import org.elasticsearch.ingest.processor.SetProcessor; +import org.elasticsearch.test.ESTestCase; +import org.junit.Before; + +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.notNullValue; +import static org.hamcrest.Matchers.nullValue; +import static org.mockito.Mockito.mock; + +public class PipelineStoreTests extends ESTestCase { + + private PipelineStore store; + + @Before + public void init() throws Exception { + store = new PipelineStore(Settings.EMPTY); + ProcessorsRegistry registry = new ProcessorsRegistry(); + registry.registerProcessor("set", (templateService) -> new SetProcessor.Factory(TestTemplateService.instance())); + store.buildProcessorFactoryRegistry(registry, null); + } + + public void testUpdatePipelines() { + ClusterState clusterState = ClusterState.builder(new ClusterName("_name")).build(); + store.innerUpdatePipelines(clusterState); + assertThat(store.pipelines.size(), is(0)); + + PipelineConfiguration pipeline = new PipelineConfiguration( + "_id",new BytesArray("{\"processors\": [{\"set\" : {\"field\": \"_field\", \"value\": \"_value\"}}]}") + ); + IngestMetadata ingestMetadata = new IngestMetadata(Collections.singletonMap("_id", pipeline)); + clusterState = ClusterState.builder(clusterState) + .metaData(MetaData.builder().putCustom(IngestMetadata.TYPE, ingestMetadata)) + .build(); + store.innerUpdatePipelines(clusterState); + assertThat(store.pipelines.size(), is(1)); + assertThat(store.pipelines.get("_id").getId(), equalTo("_id")); + assertThat(store.pipelines.get("_id").getDescription(), nullValue()); + assertThat(store.pipelines.get("_id").getProcessors().size(), equalTo(1)); + assertThat(store.pipelines.get("_id").getProcessors().get(0).getType(), equalTo("set")); + } + + public void testPut() { + String id = "_id"; + Pipeline pipeline = store.get(id); + assertThat(pipeline, nullValue()); + ClusterState clusterState = ClusterState.builder(new ClusterName("_name")).build(); + + // add a new pipeline: + PutPipelineRequest putRequest = new PutPipelineRequest(id, new BytesArray("{\"processors\": []}")); + clusterState = store.innerPut(putRequest, clusterState); + store.innerUpdatePipelines(clusterState); + pipeline = store.get(id); + assertThat(pipeline, notNullValue()); + assertThat(pipeline.getId(), equalTo(id)); + assertThat(pipeline.getDescription(), nullValue()); + assertThat(pipeline.getProcessors().size(), equalTo(0)); + + // overwrite existing pipeline: + putRequest = new PutPipelineRequest(id, new BytesArray("{\"processors\": [], \"description\": \"_description\"}")); + clusterState = store.innerPut(putRequest, clusterState); + store.innerUpdatePipelines(clusterState); + pipeline = store.get(id); + assertThat(pipeline, notNullValue()); + assertThat(pipeline.getId(), equalTo(id)); + assertThat(pipeline.getDescription(), equalTo("_description")); + assertThat(pipeline.getProcessors().size(), equalTo(0)); + } + + public void testDelete() { + PipelineConfiguration config = new PipelineConfiguration( + "_id",new BytesArray("{\"processors\": [{\"set\" : {\"field\": \"_field\", \"value\": \"_value\"}}]}") + ); + IngestMetadata ingestMetadata = new IngestMetadata(Collections.singletonMap("_id", config)); + ClusterState clusterState = ClusterState.builder(new ClusterName("_name")) + .metaData(MetaData.builder().putCustom(IngestMetadata.TYPE, ingestMetadata)) + .build(); + store.innerUpdatePipelines(clusterState); + assertThat(store.get("_id"), notNullValue()); + + // Delete pipeline: + DeletePipelineRequest deleteRequest = new DeletePipelineRequest("_id"); + clusterState = store.innerDelete(deleteRequest, clusterState); + store.innerUpdatePipelines(clusterState); + assertThat(store.get("_id"), nullValue()); + + // Delete existing pipeline: + try { + store.innerDelete(deleteRequest, clusterState); + fail("exception expected"); + } catch (ResourceNotFoundException e) { + assertThat(e.getMessage(), equalTo("pipeline [_id] is missing")); + } + } + + public void testGetPipelines() { + Map configs = new HashMap<>(); + configs.put("_id1", new PipelineConfiguration( + "_id1", new BytesArray("{\"processors\": []}") + )); + configs.put("_id2", new PipelineConfiguration( + "_id2", new BytesArray("{\"processors\": []}") + )); + + assertThat(store.innerGetPipelines(null, "_id1").isEmpty(), is(true)); + + IngestMetadata ingestMetadata = new IngestMetadata(configs); + List pipelines = store.innerGetPipelines(ingestMetadata, "_id1"); + assertThat(pipelines.size(), equalTo(1)); + assertThat(pipelines.get(0).getId(), equalTo("_id1")); + + pipelines = store.innerGetPipelines(ingestMetadata, "_id1", "_id2"); + assertThat(pipelines.size(), equalTo(2)); + assertThat(pipelines.get(0).getId(), equalTo("_id1")); + assertThat(pipelines.get(1).getId(), equalTo("_id2")); + + pipelines = store.innerGetPipelines(ingestMetadata, "_id*"); + pipelines.sort((o1, o2) -> o1.getId().compareTo(o2.getId())); + assertThat(pipelines.size(), equalTo(2)); + assertThat(pipelines.get(0).getId(), equalTo("_id1")); + assertThat(pipelines.get(1).getId(), equalTo("_id2")); + } + + public void testCrud() throws Exception { + String id = "_id"; + Pipeline pipeline = store.get(id); + assertThat(pipeline, nullValue()); + ClusterState clusterState = ClusterState.builder(new ClusterName("_name")).build(); // Start empty + + PutPipelineRequest putRequest = new PutPipelineRequest(id, new BytesArray("{\"processors\": [{\"set\" : {\"field\": \"_field\", \"value\": \"_value\"}}]}")); + clusterState = store.innerPut(putRequest, clusterState); + store.innerUpdatePipelines(clusterState); + pipeline = store.get(id); + assertThat(pipeline, notNullValue()); + assertThat(pipeline.getId(), equalTo(id)); + assertThat(pipeline.getDescription(), nullValue()); + assertThat(pipeline.getProcessors().size(), equalTo(1)); + assertThat(pipeline.getProcessors().get(0).getType(), equalTo("set")); + + DeletePipelineRequest deleteRequest = new DeletePipelineRequest(id); + clusterState = store.innerDelete(deleteRequest, clusterState); + store.innerUpdatePipelines(clusterState); + pipeline = store.get(id); + assertThat(pipeline, nullValue()); + } + +} diff --git a/core/src/test/java/org/elasticsearch/ingest/ProcessorsRegistryTests.java b/core/src/test/java/org/elasticsearch/ingest/ProcessorsRegistryTests.java new file mode 100644 index 00000000000..ad18488d990 --- /dev/null +++ b/core/src/test/java/org/elasticsearch/ingest/ProcessorsRegistryTests.java @@ -0,0 +1,60 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.ingest; + +import org.elasticsearch.ingest.core.Processor; +import org.elasticsearch.ingest.core.TemplateService; +import org.elasticsearch.test.ESTestCase; + +import java.util.Map; +import java.util.Set; +import java.util.function.Function; + +import static org.hamcrest.CoreMatchers.equalTo; + +public class ProcessorsRegistryTests extends ESTestCase { + + public void testAddProcessor() { + ProcessorsRegistry processorsRegistry = new ProcessorsRegistry(); + TestProcessor.Factory factory1 = new TestProcessor.Factory(); + processorsRegistry.registerProcessor("1", (templateService) -> factory1); + TestProcessor.Factory factory2 = new TestProcessor.Factory(); + processorsRegistry.registerProcessor("2", (templateService) -> factory2); + TestProcessor.Factory factory3 = new TestProcessor.Factory(); + try { + processorsRegistry.registerProcessor("1", (templateService) -> factory3); + fail("addProcessor should have failed"); + } catch(IllegalArgumentException e) { + assertThat(e.getMessage(), equalTo("Processor factory already registered for name [1]")); + } + + Set>>> entrySet = processorsRegistry.entrySet(); + assertThat(entrySet.size(), equalTo(2)); + for (Map.Entry>> entry : entrySet) { + if (entry.getKey().equals("1")) { + assertThat(entry.getValue().apply(null), equalTo(factory1)); + } else if (entry.getKey().equals("2")) { + assertThat(entry.getValue().apply(null), equalTo(factory2)); + } else { + fail("unexpected processor id [" + entry.getKey() + "]"); + } + } + } +} diff --git a/core/src/test/java/org/elasticsearch/ingest/core/CompoundProcessorTests.java b/core/src/test/java/org/elasticsearch/ingest/core/CompoundProcessorTests.java new file mode 100644 index 00000000000..f21644e6005 --- /dev/null +++ b/core/src/test/java/org/elasticsearch/ingest/core/CompoundProcessorTests.java @@ -0,0 +1,117 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.ingest.core; + +import org.elasticsearch.ingest.TestProcessor; +import org.elasticsearch.ingest.TestTemplateService; +import org.elasticsearch.ingest.processor.AppendProcessor; +import org.elasticsearch.ingest.processor.SetProcessor; +import org.elasticsearch.ingest.processor.SplitProcessor; +import org.elasticsearch.test.ESTestCase; +import org.junit.Before; + +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.Matchers.is; + +public class CompoundProcessorTests extends ESTestCase { + private IngestDocument ingestDocument; + + @Before + public void init() { + ingestDocument = new IngestDocument(new HashMap<>(), new HashMap<>()); + } + + public void testEmpty() throws Exception { + CompoundProcessor processor = new CompoundProcessor(); + assertThat(processor.getProcessors().isEmpty(), is(true)); + assertThat(processor.getOnFailureProcessors().isEmpty(), is(true)); + processor.execute(ingestDocument); + } + + public void testSingleProcessor() throws Exception { + TestProcessor processor = new TestProcessor(ingestDocument -> {}); + CompoundProcessor compoundProcessor = new CompoundProcessor(processor); + assertThat(compoundProcessor.getProcessors().size(), equalTo(1)); + assertThat(compoundProcessor.getProcessors().get(0), equalTo(processor)); + assertThat(compoundProcessor.getOnFailureProcessors().isEmpty(), is(true)); + compoundProcessor.execute(ingestDocument); + assertThat(processor.getInvokedCounter(), equalTo(1)); + } + + public void testSingleProcessorWithException() throws Exception { + TestProcessor processor = new TestProcessor(ingestDocument -> {throw new RuntimeException("error");}); + CompoundProcessor compoundProcessor = new CompoundProcessor(processor); + assertThat(compoundProcessor.getProcessors().size(), equalTo(1)); + assertThat(compoundProcessor.getProcessors().get(0), equalTo(processor)); + assertThat(compoundProcessor.getOnFailureProcessors().isEmpty(), is(true)); + try { + compoundProcessor.execute(ingestDocument); + fail("should throw exception"); + } catch (Exception e) { + assertThat(e.getMessage(), equalTo("error")); + } + assertThat(processor.getInvokedCounter(), equalTo(1)); + } + + public void testSingleProcessorWithOnFailureProcessor() throws Exception { + TestProcessor processor1 = new TestProcessor("id", "first", ingestDocument -> {throw new RuntimeException("error");}); + TestProcessor processor2 = new TestProcessor(ingestDocument -> { + Map ingestMetadata = ingestDocument.getIngestMetadata(); + assertThat(ingestMetadata.size(), equalTo(2)); + assertThat(ingestMetadata.get(CompoundProcessor.ON_FAILURE_MESSAGE_FIELD), equalTo("error")); + assertThat(ingestMetadata.get(CompoundProcessor.ON_FAILURE_PROCESSOR_FIELD), equalTo("first")); + }); + + CompoundProcessor compoundProcessor = new CompoundProcessor(Collections.singletonList(processor1), Collections.singletonList(processor2)); + compoundProcessor.execute(ingestDocument); + + assertThat(processor1.getInvokedCounter(), equalTo(1)); + assertThat(processor2.getInvokedCounter(), equalTo(1)); + } + + public void testSingleProcessorWithNestedFailures() throws Exception { + TestProcessor processor = new TestProcessor("id", "first", ingestDocument -> {throw new RuntimeException("error");}); + TestProcessor processorToFail = new TestProcessor("id", "second", ingestDocument -> { + Map ingestMetadata = ingestDocument.getIngestMetadata(); + assertThat(ingestMetadata.size(), equalTo(2)); + assertThat(ingestMetadata.get(CompoundProcessor.ON_FAILURE_MESSAGE_FIELD), equalTo("error")); + assertThat(ingestMetadata.get(CompoundProcessor.ON_FAILURE_PROCESSOR_FIELD), equalTo("first")); + throw new RuntimeException("error"); + }); + TestProcessor lastProcessor = new TestProcessor(ingestDocument -> { + Map ingestMetadata = ingestDocument.getIngestMetadata(); + assertThat(ingestMetadata.size(), equalTo(2)); + assertThat(ingestMetadata.get(CompoundProcessor.ON_FAILURE_MESSAGE_FIELD), equalTo("error")); + assertThat(ingestMetadata.get(CompoundProcessor.ON_FAILURE_PROCESSOR_FIELD), equalTo("second")); + }); + CompoundProcessor compoundOnFailProcessor = new CompoundProcessor(Collections.singletonList(processorToFail), Collections.singletonList(lastProcessor)); + CompoundProcessor compoundProcessor = new CompoundProcessor(Collections.singletonList(processor), Collections.singletonList(compoundOnFailProcessor)); + compoundProcessor.execute(ingestDocument); + + assertThat(processorToFail.getInvokedCounter(), equalTo(1)); + assertThat(lastProcessor.getInvokedCounter(), equalTo(1)); + } +} diff --git a/core/src/test/java/org/elasticsearch/ingest/core/ConfigurationUtilsTests.java b/core/src/test/java/org/elasticsearch/ingest/core/ConfigurationUtilsTests.java new file mode 100644 index 00000000000..958378f355a --- /dev/null +++ b/core/src/test/java/org/elasticsearch/ingest/core/ConfigurationUtilsTests.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.ingest.core; + +import org.elasticsearch.test.ESTestCase; +import org.junit.Before; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.hamcrest.Matchers.equalTo; + + +public class ConfigurationUtilsTests extends ESTestCase { + private Map config; + + @Before + public void setConfig() { + config = new HashMap<>(); + config.put("foo", "bar"); + config.put("arr", Arrays.asList("1", "2", "3")); + List list = new ArrayList<>(); + list.add(2); + config.put("int", list); + config.put("ip", "127.0.0.1"); + Map fizz = new HashMap<>(); + fizz.put("buzz", "hello world"); + config.put("fizz", fizz); + } + + public void testReadStringProperty() { + String val = ConfigurationUtils.readStringProperty(config, "foo"); + assertThat(val, equalTo("bar")); + } + + public void testReadStringPropertyInvalidType() { + try { + ConfigurationUtils.readStringProperty(config, "arr"); + } catch (IllegalArgumentException e) { + assertThat(e.getMessage(), equalTo("property [arr] isn't a string, but of type [java.util.Arrays$ArrayList]")); + } + } + + // TODO(talevy): Issue with generics. This test should fail, "int" is of type List + public void testOptional_InvalidType() { + List val = ConfigurationUtils.readList(config, "int"); + assertThat(val, equalTo(Arrays.asList(2))); + } +} diff --git a/core/src/test/java/org/elasticsearch/ingest/core/IngestDocumentTests.java b/core/src/test/java/org/elasticsearch/ingest/core/IngestDocumentTests.java new file mode 100644 index 00000000000..56d1fa76c64 --- /dev/null +++ b/core/src/test/java/org/elasticsearch/ingest/core/IngestDocumentTests.java @@ -0,0 +1,976 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.ingest.core; + +import org.elasticsearch.ingest.RandomDocumentPicks; +import org.elasticsearch.ingest.core.IngestDocument; +import org.elasticsearch.test.ESTestCase; +import org.junit.Before; + +import java.text.DateFormat; +import java.text.SimpleDateFormat; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.Date; +import java.util.HashMap; +import java.util.List; +import java.util.Locale; +import java.util.Map; + +import static org.hamcrest.Matchers.both; +import static org.hamcrest.Matchers.endsWith; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.greaterThanOrEqualTo; +import static org.hamcrest.Matchers.instanceOf; +import static org.hamcrest.Matchers.lessThanOrEqualTo; +import static org.hamcrest.Matchers.not; +import static org.hamcrest.Matchers.notNullValue; +import static org.hamcrest.Matchers.nullValue; +import static org.hamcrest.Matchers.sameInstance; + +public class IngestDocumentTests extends ESTestCase { + + private IngestDocument ingestDocument; + + @Before + public void setIngestDocument() { + Map document = new HashMap<>(); + Map ingestMap = new HashMap<>(); + ingestMap.put("timestamp", "bogus_timestamp"); + document.put("_ingest", ingestMap); + document.put("foo", "bar"); + document.put("int", 123); + Map innerObject = new HashMap<>(); + innerObject.put("buzz", "hello world"); + innerObject.put("foo_null", null); + innerObject.put("1", "bar"); + List innerInnerList = new ArrayList<>(); + innerInnerList.add("item1"); + List innerList = new ArrayList<>(); + innerList.add(innerInnerList); + innerObject.put("list", innerList); + document.put("fizz", innerObject); + List> list = new ArrayList<>(); + Map value = new HashMap<>(); + value.put("field", "value"); + list.add(value); + list.add(null); + + document.put("list", list); + ingestDocument = new IngestDocument("index", "type", "id", null, null, null, null, document); + } + + public void testSimpleGetFieldValue() { + assertThat(ingestDocument.getFieldValue("foo", String.class), equalTo("bar")); + assertThat(ingestDocument.getFieldValue("int", Integer.class), equalTo(123)); + assertThat(ingestDocument.getFieldValue("_source.foo", String.class), equalTo("bar")); + assertThat(ingestDocument.getFieldValue("_source.int", Integer.class), equalTo(123)); + assertThat(ingestDocument.getFieldValue("_index", String.class), equalTo("index")); + assertThat(ingestDocument.getFieldValue("_type", String.class), equalTo("type")); + assertThat(ingestDocument.getFieldValue("_id", String.class), equalTo("id")); + assertThat(ingestDocument.getFieldValue("_ingest.timestamp", String.class), both(notNullValue()).and(not(equalTo("bogus_timestamp")))); + assertThat(ingestDocument.getFieldValue("_source._ingest.timestamp", String.class), equalTo("bogus_timestamp")); + } + + public void testGetSourceObject() { + try { + ingestDocument.getFieldValue("_source", Object.class); + fail("get field value should have failed"); + } catch (IllegalArgumentException e) { + assertThat(e.getMessage(), equalTo("field [_source] not present as part of path [_source]")); + } + } + + public void testGetIngestObject() { + assertThat(ingestDocument.getFieldValue("_ingest", Map.class), notNullValue()); + } + + public void testGetEmptyPathAfterStrippingOutPrefix() { + try { + ingestDocument.getFieldValue("_source.", Object.class); + fail("get field value should have failed"); + } catch(IllegalArgumentException e) { + assertThat(e.getMessage(), equalTo("path [_source.] is not valid")); + } + + try { + ingestDocument.getFieldValue("_ingest.", Object.class); + fail("get field value should have failed"); + } catch(IllegalArgumentException e) { + assertThat(e.getMessage(), equalTo("path [_ingest.] is not valid")); + } + } + + public void testGetFieldValueNullValue() { + assertThat(ingestDocument.getFieldValue("fizz.foo_null", Object.class), nullValue()); + } + + public void testSimpleGetFieldValueTypeMismatch() { + try { + ingestDocument.getFieldValue("int", String.class); + fail("getFieldValue should have failed"); + } catch(IllegalArgumentException e) { + assertThat(e.getMessage(), equalTo("field [int] of type [java.lang.Integer] cannot be cast to [java.lang.String]")); + } + + try { + ingestDocument.getFieldValue("foo", Integer.class); + fail("getFieldValue should have failed"); + } catch(IllegalArgumentException e) { + assertThat(e.getMessage(), equalTo("field [foo] of type [java.lang.String] cannot be cast to [java.lang.Integer]")); + } + } + + public void testNestedGetFieldValue() { + assertThat(ingestDocument.getFieldValue("fizz.buzz", String.class), equalTo("hello world")); + assertThat(ingestDocument.getFieldValue("fizz.1", String.class), equalTo("bar")); + } + + public void testNestedGetFieldValueTypeMismatch() { + try { + ingestDocument.getFieldValue("foo.foo.bar", String.class); + } catch(IllegalArgumentException e) { + assertThat(e.getMessage(), equalTo("cannot resolve [foo] from object of type [java.lang.String] as part of path [foo.foo.bar]")); + } + } + + public void testListGetFieldValue() { + assertThat(ingestDocument.getFieldValue("list.0.field", String.class), equalTo("value")); + } + + public void testListGetFieldValueNull() { + assertThat(ingestDocument.getFieldValue("list.1", String.class), nullValue()); + } + + public void testListGetFieldValueIndexNotNumeric() { + try { + ingestDocument.getFieldValue("list.test.field", String.class); + } catch(IllegalArgumentException e) { + assertThat(e.getMessage(), equalTo("[test] is not an integer, cannot be used as an index as part of path [list.test.field]")); + } + } + + public void testListGetFieldValueIndexOutOfBounds() { + try { + ingestDocument.getFieldValue("list.10.field", String.class); + } catch(IllegalArgumentException e) { + assertThat(e.getMessage(), equalTo("[10] is out of bounds for array with length [2] as part of path [list.10.field]")); + } + } + + public void testGetFieldValueNotFound() { + try { + ingestDocument.getFieldValue("not.here", String.class); + fail("get field value should have failed"); + } catch(IllegalArgumentException e) { + assertThat(e.getMessage(), equalTo("field [not] not present as part of path [not.here]")); + } + } + + public void testGetFieldValueNotFoundNullParent() { + try { + ingestDocument.getFieldValue("fizz.foo_null.not_there", String.class); + fail("get field value should have failed"); + } catch(IllegalArgumentException e) { + assertThat(e.getMessage(), equalTo("cannot resolve [not_there] from null as part of path [fizz.foo_null.not_there]")); + } + } + + public void testGetFieldValueNull() { + try { + ingestDocument.getFieldValue(null, String.class); + fail("get field value should have failed"); + } catch(IllegalArgumentException e) { + assertThat(e.getMessage(), equalTo("path cannot be null nor empty")); + } + } + + public void testGetFieldValueEmpty() { + try { + ingestDocument.getFieldValue("", String.class); + fail("get field value should have failed"); + } catch(IllegalArgumentException e) { + assertThat(e.getMessage(), equalTo("path cannot be null nor empty")); + } + } + + public void testHasField() { + assertTrue(ingestDocument.hasField("fizz")); + assertTrue(ingestDocument.hasField("_index")); + assertTrue(ingestDocument.hasField("_type")); + assertTrue(ingestDocument.hasField("_id")); + assertTrue(ingestDocument.hasField("_source.fizz")); + assertTrue(ingestDocument.hasField("_ingest.timestamp")); + } + + public void testHasFieldNested() { + assertTrue(ingestDocument.hasField("fizz.buzz")); + assertTrue(ingestDocument.hasField("_source._ingest.timestamp")); + } + + public void testListHasField() { + assertTrue(ingestDocument.hasField("list.0.field")); + } + + public void testListHasFieldNull() { + assertTrue(ingestDocument.hasField("list.1")); + } + + public void testListHasFieldIndexOutOfBounds() { + assertFalse(ingestDocument.hasField("list.10")); + } + + public void testListHasFieldIndexNotNumeric() { + assertFalse(ingestDocument.hasField("list.test")); + } + + public void testNestedHasFieldTypeMismatch() { + assertFalse(ingestDocument.hasField("foo.foo.bar")); + } + + public void testHasFieldNotFound() { + assertFalse(ingestDocument.hasField("not.here")); + } + + public void testHasFieldNotFoundNullParent() { + assertFalse(ingestDocument.hasField("fizz.foo_null.not_there")); + } + + public void testHasFieldNestedNotFound() { + assertFalse(ingestDocument.hasField("fizz.doesnotexist")); + } + + public void testHasFieldNull() { + try { + ingestDocument.hasField(null); + fail("has field should have failed"); + } catch(IllegalArgumentException e) { + assertThat(e.getMessage(), equalTo("path cannot be null nor empty")); + } + } + + public void testHasFieldNullValue() { + assertTrue(ingestDocument.hasField("fizz.foo_null")); + } + + public void testHasFieldEmpty() { + try { + ingestDocument.hasField(""); + fail("has field should have failed"); + } catch(IllegalArgumentException e) { + assertThat(e.getMessage(), equalTo("path cannot be null nor empty")); + } + } + + public void testHasFieldSourceObject() { + assertThat(ingestDocument.hasField("_source"), equalTo(false)); + } + + public void testHasFieldIngestObject() { + assertThat(ingestDocument.hasField("_ingest"), equalTo(true)); + } + + public void testHasFieldEmptyPathAfterStrippingOutPrefix() { + try { + ingestDocument.hasField("_source."); + fail("has field value should have failed"); + } catch(IllegalArgumentException e) { + assertThat(e.getMessage(), equalTo("path [_source.] is not valid")); + } + + try { + ingestDocument.hasField("_ingest."); + fail("has field value should have failed"); + } catch(IllegalArgumentException e) { + assertThat(e.getMessage(), equalTo("path [_ingest.] is not valid")); + } + } + + public void testSimpleSetFieldValue() { + ingestDocument.setFieldValue("new_field", "foo"); + assertThat(ingestDocument.getSourceAndMetadata().get("new_field"), equalTo("foo")); + ingestDocument.setFieldValue("_ttl", "ttl"); + assertThat(ingestDocument.getSourceAndMetadata().get("_ttl"), equalTo("ttl")); + ingestDocument.setFieldValue("_source.another_field", "bar"); + assertThat(ingestDocument.getSourceAndMetadata().get("another_field"), equalTo("bar")); + ingestDocument.setFieldValue("_ingest.new_field", "new_value"); + assertThat(ingestDocument.getIngestMetadata().size(), equalTo(2)); + assertThat(ingestDocument.getIngestMetadata().get("new_field"), equalTo("new_value")); + ingestDocument.setFieldValue("_ingest.timestamp", "timestamp"); + assertThat(ingestDocument.getIngestMetadata().get("timestamp"), equalTo("timestamp")); + } + + public void testSetFieldValueNullValue() { + ingestDocument.setFieldValue("new_field", null); + assertThat(ingestDocument.getSourceAndMetadata().containsKey("new_field"), equalTo(true)); + assertThat(ingestDocument.getSourceAndMetadata().get("new_field"), nullValue()); + } + + @SuppressWarnings("unchecked") + public void testNestedSetFieldValue() { + ingestDocument.setFieldValue("a.b.c.d", "foo"); + assertThat(ingestDocument.getSourceAndMetadata().get("a"), instanceOf(Map.class)); + Map a = (Map) ingestDocument.getSourceAndMetadata().get("a"); + assertThat(a.get("b"), instanceOf(Map.class)); + Map b = (Map) a.get("b"); + assertThat(b.get("c"), instanceOf(Map.class)); + Map c = (Map) b.get("c"); + assertThat(c.get("d"), instanceOf(String.class)); + String d = (String) c.get("d"); + assertThat(d, equalTo("foo")); + } + + public void testSetFieldValueOnExistingField() { + ingestDocument.setFieldValue("foo", "newbar"); + assertThat(ingestDocument.getSourceAndMetadata().get("foo"), equalTo("newbar")); + } + + @SuppressWarnings("unchecked") + public void testSetFieldValueOnExistingParent() { + ingestDocument.setFieldValue("fizz.new", "bar"); + assertThat(ingestDocument.getSourceAndMetadata().get("fizz"), instanceOf(Map.class)); + Map innerMap = (Map) ingestDocument.getSourceAndMetadata().get("fizz"); + assertThat(innerMap.get("new"), instanceOf(String.class)); + String value = (String) innerMap.get("new"); + assertThat(value, equalTo("bar")); + } + + public void testSetFieldValueOnExistingParentTypeMismatch() { + try { + ingestDocument.setFieldValue("fizz.buzz.new", "bar"); + fail("add field should have failed"); + } catch(IllegalArgumentException e) { + assertThat(e.getMessage(), equalTo("cannot set [new] with parent object of type [java.lang.String] as part of path [fizz.buzz.new]")); + } + } + + public void testSetFieldValueOnExistingNullParent() { + try { + ingestDocument.setFieldValue("fizz.foo_null.test", "bar"); + fail("add field should have failed"); + } catch(IllegalArgumentException e) { + assertThat(e.getMessage(), equalTo("cannot set [test] with null parent as part of path [fizz.foo_null.test]")); + } + } + + public void testSetFieldValueNullName() { + try { + ingestDocument.setFieldValue(null, "bar"); + fail("add field should have failed"); + } catch(IllegalArgumentException e) { + assertThat(e.getMessage(), equalTo("path cannot be null nor empty")); + } + } + + public void testSetSourceObject() { + ingestDocument.setFieldValue("_source", "value"); + assertThat(ingestDocument.getSourceAndMetadata().get("_source"), equalTo("value")); + } + + public void testSetIngestObject() { + ingestDocument.setFieldValue("_ingest", "value"); + assertThat(ingestDocument.getSourceAndMetadata().get("_ingest"), equalTo("value")); + } + + public void testSetIngestSourceObject() { + //test that we don't strip out the _source prefix when _ingest is used + ingestDocument.setFieldValue("_ingest._source", "value"); + assertThat(ingestDocument.getIngestMetadata().get("_source"), equalTo("value")); + } + + public void testSetEmptyPathAfterStrippingOutPrefix() { + try { + ingestDocument.setFieldValue("_source.", "value"); + fail("set field value should have failed"); + } catch(IllegalArgumentException e) { + assertThat(e.getMessage(), equalTo("path [_source.] is not valid")); + } + + try { + ingestDocument.setFieldValue("_ingest.", "_value"); + fail("set field value should have failed"); + } catch(IllegalArgumentException e) { + assertThat(e.getMessage(), equalTo("path [_ingest.] is not valid")); + } + } + + public void testListSetFieldValueNoIndexProvided() { + ingestDocument.setFieldValue("list", "value"); + Object object = ingestDocument.getSourceAndMetadata().get("list"); + assertThat(object, instanceOf(String.class)); + assertThat(object, equalTo("value")); + } + + public void testListAppendFieldValue() { + ingestDocument.appendFieldValue("list", "new_value"); + Object object = ingestDocument.getSourceAndMetadata().get("list"); + assertThat(object, instanceOf(List.class)); + @SuppressWarnings("unchecked") + List list = (List) object; + assertThat(list.size(), equalTo(3)); + assertThat(list.get(0), equalTo(Collections.singletonMap("field", "value"))); + assertThat(list.get(1), nullValue()); + assertThat(list.get(2), equalTo("new_value")); + } + + public void testListAppendFieldValues() { + ingestDocument.appendFieldValue("list", Arrays.asList("item1", "item2", "item3")); + Object object = ingestDocument.getSourceAndMetadata().get("list"); + assertThat(object, instanceOf(List.class)); + @SuppressWarnings("unchecked") + List list = (List) object; + assertThat(list.size(), equalTo(5)); + assertThat(list.get(0), equalTo(Collections.singletonMap("field", "value"))); + assertThat(list.get(1), nullValue()); + assertThat(list.get(2), equalTo("item1")); + assertThat(list.get(3), equalTo("item2")); + assertThat(list.get(4), equalTo("item3")); + } + + public void testAppendFieldValueToNonExistingList() { + ingestDocument.appendFieldValue("non_existing_list", "new_value"); + Object object = ingestDocument.getSourceAndMetadata().get("non_existing_list"); + assertThat(object, instanceOf(List.class)); + @SuppressWarnings("unchecked") + List list = (List) object; + assertThat(list.size(), equalTo(1)); + assertThat(list.get(0), equalTo("new_value")); + } + + public void testAppendFieldValuesToNonExistingList() { + ingestDocument.appendFieldValue("non_existing_list", Arrays.asList("item1", "item2", "item3")); + Object object = ingestDocument.getSourceAndMetadata().get("non_existing_list"); + assertThat(object, instanceOf(List.class)); + @SuppressWarnings("unchecked") + List list = (List) object; + assertThat(list.size(), equalTo(3)); + assertThat(list.get(0), equalTo("item1")); + assertThat(list.get(1), equalTo("item2")); + assertThat(list.get(2), equalTo("item3")); + } + + public void testAppendFieldValueConvertStringToList() { + ingestDocument.appendFieldValue("fizz.buzz", "new_value"); + Object object = ingestDocument.getSourceAndMetadata().get("fizz"); + assertThat(object, instanceOf(Map.class)); + @SuppressWarnings("unchecked") + Map map = (Map) object; + object = map.get("buzz"); + assertThat(object, instanceOf(List.class)); + @SuppressWarnings("unchecked") + List list = (List) object; + assertThat(list.size(), equalTo(2)); + assertThat(list.get(0), equalTo("hello world")); + assertThat(list.get(1), equalTo("new_value")); + } + + public void testAppendFieldValuesConvertStringToList() { + ingestDocument.appendFieldValue("fizz.buzz", Arrays.asList("item1", "item2", "item3")); + Object object = ingestDocument.getSourceAndMetadata().get("fizz"); + assertThat(object, instanceOf(Map.class)); + @SuppressWarnings("unchecked") + Map map = (Map) object; + object = map.get("buzz"); + assertThat(object, instanceOf(List.class)); + @SuppressWarnings("unchecked") + List list = (List) object; + assertThat(list.size(), equalTo(4)); + assertThat(list.get(0), equalTo("hello world")); + assertThat(list.get(1), equalTo("item1")); + assertThat(list.get(2), equalTo("item2")); + assertThat(list.get(3), equalTo("item3")); + } + + public void testAppendFieldValueConvertIntegerToList() { + ingestDocument.appendFieldValue("int", 456); + Object object = ingestDocument.getSourceAndMetadata().get("int"); + assertThat(object, instanceOf(List.class)); + @SuppressWarnings("unchecked") + List list = (List) object; + assertThat(list.size(), equalTo(2)); + assertThat(list.get(0), equalTo(123)); + assertThat(list.get(1), equalTo(456)); + } + + public void testAppendFieldValuesConvertIntegerToList() { + ingestDocument.appendFieldValue("int", Arrays.asList(456, 789)); + Object object = ingestDocument.getSourceAndMetadata().get("int"); + assertThat(object, instanceOf(List.class)); + @SuppressWarnings("unchecked") + List list = (List) object; + assertThat(list.size(), equalTo(3)); + assertThat(list.get(0), equalTo(123)); + assertThat(list.get(1), equalTo(456)); + assertThat(list.get(2), equalTo(789)); + } + + public void testAppendFieldValueConvertMapToList() { + ingestDocument.appendFieldValue("fizz", Collections.singletonMap("field", "value")); + Object object = ingestDocument.getSourceAndMetadata().get("fizz"); + assertThat(object, instanceOf(List.class)); + List list = (List) object; + assertThat(list.size(), equalTo(2)); + assertThat(list.get(0), instanceOf(Map.class)); + @SuppressWarnings("unchecked") + Map map = (Map) list.get(0); + assertThat(map.size(), equalTo(4)); + assertThat(list.get(1), equalTo(Collections.singletonMap("field", "value"))); + } + + public void testAppendFieldValueToNull() { + ingestDocument.appendFieldValue("fizz.foo_null", "new_value"); + Object object = ingestDocument.getSourceAndMetadata().get("fizz"); + assertThat(object, instanceOf(Map.class)); + @SuppressWarnings("unchecked") + Map map = (Map) object; + object = map.get("foo_null"); + assertThat(object, instanceOf(List.class)); + List list = (List) object; + assertThat(list.size(), equalTo(2)); + assertThat(list.get(0), nullValue()); + assertThat(list.get(1), equalTo("new_value")); + } + + public void testAppendFieldValueToListElement() { + ingestDocument.appendFieldValue("fizz.list.0", "item2"); + Object object = ingestDocument.getSourceAndMetadata().get("fizz"); + assertThat(object, instanceOf(Map.class)); + @SuppressWarnings("unchecked") + Map map = (Map) object; + object = map.get("list"); + assertThat(object, instanceOf(List.class)); + @SuppressWarnings("unchecked") + List list = (List) object; + assertThat(list.size(), equalTo(1)); + object = list.get(0); + assertThat(object, instanceOf(List.class)); + @SuppressWarnings("unchecked") + List innerList = (List) object; + assertThat(innerList.size(), equalTo(2)); + assertThat(innerList.get(0), equalTo("item1")); + assertThat(innerList.get(1), equalTo("item2")); + } + + public void testAppendFieldValuesToListElement() { + ingestDocument.appendFieldValue("fizz.list.0", Arrays.asList("item2", "item3", "item4")); + Object object = ingestDocument.getSourceAndMetadata().get("fizz"); + assertThat(object, instanceOf(Map.class)); + @SuppressWarnings("unchecked") + Map map = (Map) object; + object = map.get("list"); + assertThat(object, instanceOf(List.class)); + @SuppressWarnings("unchecked") + List list = (List) object; + assertThat(list.size(), equalTo(1)); + object = list.get(0); + assertThat(object, instanceOf(List.class)); + @SuppressWarnings("unchecked") + List innerList = (List) object; + assertThat(innerList.size(), equalTo(4)); + assertThat(innerList.get(0), equalTo("item1")); + assertThat(innerList.get(1), equalTo("item2")); + assertThat(innerList.get(2), equalTo("item3")); + assertThat(innerList.get(3), equalTo("item4")); + } + + public void testAppendFieldValueConvertStringListElementToList() { + ingestDocument.appendFieldValue("fizz.list.0.0", "new_value"); + Object object = ingestDocument.getSourceAndMetadata().get("fizz"); + assertThat(object, instanceOf(Map.class)); + @SuppressWarnings("unchecked") + Map map = (Map) object; + object = map.get("list"); + assertThat(object, instanceOf(List.class)); + @SuppressWarnings("unchecked") + List list = (List) object; + assertThat(list.size(), equalTo(1)); + object = list.get(0); + assertThat(object, instanceOf(List.class)); + @SuppressWarnings("unchecked") + List innerList = (List) object; + object = innerList.get(0); + assertThat(object, instanceOf(List.class)); + @SuppressWarnings("unchecked") + List innerInnerList = (List) object; + assertThat(innerInnerList.size(), equalTo(2)); + assertThat(innerInnerList.get(0), equalTo("item1")); + assertThat(innerInnerList.get(1), equalTo("new_value")); + } + + public void testAppendFieldValuesConvertStringListElementToList() { + ingestDocument.appendFieldValue("fizz.list.0.0", Arrays.asList("item2", "item3", "item4")); + Object object = ingestDocument.getSourceAndMetadata().get("fizz"); + assertThat(object, instanceOf(Map.class)); + @SuppressWarnings("unchecked") + Map map = (Map) object; + object = map.get("list"); + assertThat(object, instanceOf(List.class)); + @SuppressWarnings("unchecked") + List list = (List) object; + assertThat(list.size(), equalTo(1)); + object = list.get(0); + assertThat(object, instanceOf(List.class)); + @SuppressWarnings("unchecked") + List innerList = (List) object; + object = innerList.get(0); + assertThat(object, instanceOf(List.class)); + @SuppressWarnings("unchecked") + List innerInnerList = (List) object; + assertThat(innerInnerList.size(), equalTo(4)); + assertThat(innerInnerList.get(0), equalTo("item1")); + assertThat(innerInnerList.get(1), equalTo("item2")); + assertThat(innerInnerList.get(2), equalTo("item3")); + assertThat(innerInnerList.get(3), equalTo("item4")); + } + + public void testAppendFieldValueListElementConvertMapToList() { + ingestDocument.appendFieldValue("list.0", Collections.singletonMap("item2", "value2")); + Object object = ingestDocument.getSourceAndMetadata().get("list"); + assertThat(object, instanceOf(List.class)); + List list = (List) object; + assertThat(list.size(), equalTo(2)); + assertThat(list.get(0), instanceOf(List.class)); + assertThat(list.get(1), nullValue()); + list = (List) list.get(0); + assertThat(list.size(), equalTo(2)); + assertThat(list.get(0), equalTo(Collections.singletonMap("field", "value"))); + assertThat(list.get(1), equalTo(Collections.singletonMap("item2", "value2"))); + } + + public void testAppendFieldValueToNullListElement() { + ingestDocument.appendFieldValue("list.1", "new_value"); + Object object = ingestDocument.getSourceAndMetadata().get("list"); + assertThat(object, instanceOf(List.class)); + List list = (List) object; + assertThat(list.get(1), instanceOf(List.class)); + list = (List) list.get(1); + assertThat(list.size(), equalTo(2)); + assertThat(list.get(0), nullValue()); + assertThat(list.get(1), equalTo("new_value")); + } + + public void testAppendFieldValueToListOfMaps() { + ingestDocument.appendFieldValue("list", Collections.singletonMap("item2", "value2")); + Object object = ingestDocument.getSourceAndMetadata().get("list"); + assertThat(object, instanceOf(List.class)); + @SuppressWarnings("unchecked") + List list = (List) object; + assertThat(list.size(), equalTo(3)); + assertThat(list.get(0), equalTo(Collections.singletonMap("field", "value"))); + assertThat(list.get(1), nullValue()); + assertThat(list.get(2), equalTo(Collections.singletonMap("item2", "value2"))); + } + + public void testListSetFieldValueIndexProvided() { + ingestDocument.setFieldValue("list.1", "value"); + Object object = ingestDocument.getSourceAndMetadata().get("list"); + assertThat(object, instanceOf(List.class)); + @SuppressWarnings("unchecked") + List list = (List) object; + assertThat(list.size(), equalTo(2)); + assertThat(list.get(0), equalTo(Collections.singletonMap("field", "value"))); + assertThat(list.get(1), equalTo("value")); + } + + public void testSetFieldValueListAsPartOfPath() { + ingestDocument.setFieldValue("list.0.field", "new_value"); + Object object = ingestDocument.getSourceAndMetadata().get("list"); + assertThat(object, instanceOf(List.class)); + @SuppressWarnings("unchecked") + List list = (List) object; + assertThat(list.size(), equalTo(2)); + assertThat(list.get(0), equalTo(Collections.singletonMap("field", "new_value"))); + assertThat(list.get(1), nullValue()); + } + + public void testListSetFieldValueIndexNotNumeric() { + try { + ingestDocument.setFieldValue("list.test", "value"); + } catch(IllegalArgumentException e) { + assertThat(e.getMessage(), equalTo("[test] is not an integer, cannot be used as an index as part of path [list.test]")); + } + + try { + ingestDocument.setFieldValue("list.test.field", "new_value"); + } catch(IllegalArgumentException e) { + assertThat(e.getMessage(), equalTo("[test] is not an integer, cannot be used as an index as part of path [list.test.field]")); + } + } + + public void testListSetFieldValueIndexOutOfBounds() { + try { + ingestDocument.setFieldValue("list.10", "value"); + } catch(IllegalArgumentException e) { + assertThat(e.getMessage(), equalTo("[10] is out of bounds for array with length [2] as part of path [list.10]")); + } + + try { + ingestDocument.setFieldValue("list.10.field", "value"); + } catch(IllegalArgumentException e) { + assertThat(e.getMessage(), equalTo("[10] is out of bounds for array with length [2] as part of path [list.10.field]")); + } + } + + public void testSetFieldValueEmptyName() { + try { + ingestDocument.setFieldValue("", "bar"); + fail("add field should have failed"); + } catch(IllegalArgumentException e) { + assertThat(e.getMessage(), equalTo("path cannot be null nor empty")); + } + } + + public void testRemoveField() { + ingestDocument.removeField("foo"); + assertThat(ingestDocument.getSourceAndMetadata().size(), equalTo(7)); + assertThat(ingestDocument.getSourceAndMetadata().containsKey("foo"), equalTo(false)); + ingestDocument.removeField("_index"); + assertThat(ingestDocument.getSourceAndMetadata().size(), equalTo(6)); + assertThat(ingestDocument.getSourceAndMetadata().containsKey("_index"), equalTo(false)); + ingestDocument.removeField("_source.fizz"); + assertThat(ingestDocument.getSourceAndMetadata().size(), equalTo(5)); + assertThat(ingestDocument.getSourceAndMetadata().containsKey("fizz"), equalTo(false)); + assertThat(ingestDocument.getIngestMetadata().size(), equalTo(1)); + ingestDocument.removeField("_ingest.timestamp"); + assertThat(ingestDocument.getSourceAndMetadata().size(), equalTo(5)); + assertThat(ingestDocument.getIngestMetadata().size(), equalTo(0)); + } + + public void testRemoveInnerField() { + ingestDocument.removeField("fizz.buzz"); + assertThat(ingestDocument.getSourceAndMetadata().size(), equalTo(8)); + assertThat(ingestDocument.getSourceAndMetadata().get("fizz"), instanceOf(Map.class)); + @SuppressWarnings("unchecked") + Map map = (Map) ingestDocument.getSourceAndMetadata().get("fizz"); + assertThat(map.size(), equalTo(3)); + assertThat(map.containsKey("buzz"), equalTo(false)); + + ingestDocument.removeField("fizz.foo_null"); + assertThat(map.size(), equalTo(2)); + assertThat(ingestDocument.getSourceAndMetadata().size(), equalTo(8)); + assertThat(ingestDocument.getSourceAndMetadata().containsKey("fizz"), equalTo(true)); + + ingestDocument.removeField("fizz.1"); + assertThat(map.size(), equalTo(1)); + assertThat(ingestDocument.getSourceAndMetadata().size(), equalTo(8)); + assertThat(ingestDocument.getSourceAndMetadata().containsKey("fizz"), equalTo(true)); + + ingestDocument.removeField("fizz.list"); + assertThat(map.size(), equalTo(0)); + assertThat(ingestDocument.getSourceAndMetadata().size(), equalTo(8)); + assertThat(ingestDocument.getSourceAndMetadata().containsKey("fizz"), equalTo(true)); + } + + public void testRemoveNonExistingField() { + try { + ingestDocument.removeField("does_not_exist"); + fail("remove field should have failed"); + } catch(IllegalArgumentException e) { + assertThat(e.getMessage(), equalTo("field [does_not_exist] not present as part of path [does_not_exist]")); + } + } + + public void testRemoveExistingParentTypeMismatch() { + try { + ingestDocument.removeField("foo.foo.bar"); + fail("remove field should have failed"); + } catch(IllegalArgumentException e) { + assertThat(e.getMessage(), equalTo("cannot resolve [foo] from object of type [java.lang.String] as part of path [foo.foo.bar]")); + } + } + + public void testRemoveSourceObject() { + try { + ingestDocument.removeField("_source"); + fail("remove field should have failed"); + } catch (IllegalArgumentException e) { + assertThat(e.getMessage(), equalTo("field [_source] not present as part of path [_source]")); + } + } + + public void testRemoveIngestObject() { + ingestDocument.removeField("_ingest"); + assertThat(ingestDocument.getSourceAndMetadata().size(), equalTo(7)); + assertThat(ingestDocument.getSourceAndMetadata().containsKey("_ingest"), equalTo(false)); + } + + public void testRemoveEmptyPathAfterStrippingOutPrefix() { + try { + ingestDocument.removeField("_source."); + fail("set field value should have failed"); + } catch(IllegalArgumentException e) { + assertThat(e.getMessage(), equalTo("path [_source.] is not valid")); + } + + try { + ingestDocument.removeField("_ingest."); + fail("set field value should have failed"); + } catch(IllegalArgumentException e) { + assertThat(e.getMessage(), equalTo("path [_ingest.] is not valid")); + } + } + + public void testListRemoveField() { + ingestDocument.removeField("list.0.field"); + assertThat(ingestDocument.getSourceAndMetadata().size(), equalTo(8)); + assertThat(ingestDocument.getSourceAndMetadata().containsKey("list"), equalTo(true)); + Object object = ingestDocument.getSourceAndMetadata().get("list"); + assertThat(object, instanceOf(List.class)); + @SuppressWarnings("unchecked") + List list = (List) object; + assertThat(list.size(), equalTo(2)); + object = list.get(0); + assertThat(object, instanceOf(Map.class)); + @SuppressWarnings("unchecked") + Map map = (Map) object; + assertThat(map.size(), equalTo(0)); + ingestDocument.removeField("list.0"); + assertThat(list.size(), equalTo(1)); + assertThat(list.get(0), nullValue()); + } + + public void testRemoveFieldValueNotFoundNullParent() { + try { + ingestDocument.removeField("fizz.foo_null.not_there"); + fail("get field value should have failed"); + } catch(IllegalArgumentException e) { + assertThat(e.getMessage(), equalTo("cannot remove [not_there] from null as part of path [fizz.foo_null.not_there]")); + } + } + + public void testNestedRemoveFieldTypeMismatch() { + try { + ingestDocument.removeField("fizz.1.bar"); + } catch(IllegalArgumentException e) { + assertThat(e.getMessage(), equalTo("cannot remove [bar] from object of type [java.lang.String] as part of path [fizz.1.bar]")); + } + } + + public void testListRemoveFieldIndexNotNumeric() { + try { + ingestDocument.removeField("list.test"); + } catch(IllegalArgumentException e) { + assertThat(e.getMessage(), equalTo("[test] is not an integer, cannot be used as an index as part of path [list.test]")); + } + } + + public void testListRemoveFieldIndexOutOfBounds() { + try { + ingestDocument.removeField("list.10"); + } catch(IllegalArgumentException e) { + assertThat(e.getMessage(), equalTo("[10] is out of bounds for array with length [2] as part of path [list.10]")); + } + } + + public void testRemoveNullField() { + try { + ingestDocument.removeField((String) null); + fail("remove field should have failed"); + } catch(IllegalArgumentException e) { + assertThat(e.getMessage(), equalTo("path cannot be null nor empty")); + } + } + + public void testRemoveEmptyField() { + try { + ingestDocument.removeField(""); + fail("remove field should have failed"); + } catch(IllegalArgumentException e) { + assertThat(e.getMessage(), equalTo("path cannot be null nor empty")); + } + } + + public void testEqualsAndHashcode() throws Exception { + Map sourceAndMetadata = RandomDocumentPicks.randomSource(random()); + int numFields = randomIntBetween(1, IngestDocument.MetaData.values().length); + for (int i = 0; i < numFields; i++) { + sourceAndMetadata.put(randomFrom(IngestDocument.MetaData.values()).getFieldName(), randomAsciiOfLengthBetween(5, 10)); + } + Map ingestMetadata = new HashMap<>(); + numFields = randomIntBetween(1, 5); + for (int i = 0; i < numFields; i++) { + ingestMetadata.put(randomAsciiOfLengthBetween(5, 10), randomAsciiOfLengthBetween(5, 10)); + } + IngestDocument ingestDocument = new IngestDocument(sourceAndMetadata, ingestMetadata); + + boolean changed = false; + Map otherSourceAndMetadata; + if (randomBoolean()) { + otherSourceAndMetadata = RandomDocumentPicks.randomSource(random()); + changed = true; + } else { + otherSourceAndMetadata = new HashMap<>(sourceAndMetadata); + } + if (randomBoolean()) { + numFields = randomIntBetween(1, IngestDocument.MetaData.values().length); + for (int i = 0; i < numFields; i++) { + otherSourceAndMetadata.put(randomFrom(IngestDocument.MetaData.values()).getFieldName(), randomAsciiOfLengthBetween(5, 10)); + } + changed = true; + } + + Map otherIngestMetadata; + if (randomBoolean()) { + otherIngestMetadata = new HashMap<>(); + numFields = randomIntBetween(1, 5); + for (int i = 0; i < numFields; i++) { + otherIngestMetadata.put(randomAsciiOfLengthBetween(5, 10), randomAsciiOfLengthBetween(5, 10)); + } + changed = true; + } else { + otherIngestMetadata = Collections.unmodifiableMap(ingestMetadata); + } + + IngestDocument otherIngestDocument = new IngestDocument(otherSourceAndMetadata, otherIngestMetadata); + if (changed) { + assertThat(ingestDocument, not(equalTo(otherIngestDocument))); + assertThat(otherIngestDocument, not(equalTo(ingestDocument))); + } else { + assertThat(ingestDocument, equalTo(otherIngestDocument)); + assertThat(otherIngestDocument, equalTo(ingestDocument)); + assertThat(ingestDocument.hashCode(), equalTo(otherIngestDocument.hashCode())); + IngestDocument thirdIngestDocument = new IngestDocument(Collections.unmodifiableMap(sourceAndMetadata), Collections.unmodifiableMap(ingestMetadata)); + assertThat(thirdIngestDocument, equalTo(ingestDocument)); + assertThat(ingestDocument, equalTo(thirdIngestDocument)); + assertThat(ingestDocument.hashCode(), equalTo(thirdIngestDocument.hashCode())); + } + } + + public void testIngestMetadataTimestamp() throws Exception { + long before = System.currentTimeMillis(); + IngestDocument ingestDocument = RandomDocumentPicks.randomIngestDocument(random()); + long after = System.currentTimeMillis(); + String timestampString = ingestDocument.getIngestMetadata().get("timestamp"); + assertThat(timestampString, notNullValue()); + assertThat(timestampString, endsWith("+0000")); + DateFormat df = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSSZZ", Locale.ROOT); + Date timestamp = df.parse(timestampString); + assertThat(timestamp.getTime(), greaterThanOrEqualTo(before)); + assertThat(timestamp.getTime(), lessThanOrEqualTo(after)); + } + + public void testCopyConstructor() { + IngestDocument ingestDocument = RandomDocumentPicks.randomIngestDocument(random()); + IngestDocument copy = new IngestDocument(ingestDocument); + assertThat(ingestDocument.getSourceAndMetadata(), not(sameInstance(copy.getSourceAndMetadata()))); + assertThat(ingestDocument.getSourceAndMetadata(), equalTo(copy.getSourceAndMetadata())); + } +} diff --git a/core/src/test/java/org/elasticsearch/ingest/core/PipelineFactoryTests.java b/core/src/test/java/org/elasticsearch/ingest/core/PipelineFactoryTests.java new file mode 100644 index 00000000000..229290372b6 --- /dev/null +++ b/core/src/test/java/org/elasticsearch/ingest/core/PipelineFactoryTests.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.ingest.core; + +import org.elasticsearch.ingest.TestProcessor; +import org.elasticsearch.test.ESTestCase; + +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.Matchers.nullValue; + +public class PipelineFactoryTests extends ESTestCase { + + public void testCreate() throws Exception { + Map processorConfig0 = new HashMap<>(); + Map processorConfig1 = new HashMap<>(); + processorConfig0.put(AbstractProcessorFactory.TAG_KEY, "first-processor"); + Map pipelineConfig = new HashMap<>(); + pipelineConfig.put(Pipeline.DESCRIPTION_KEY, "_description"); + pipelineConfig.put(Pipeline.PROCESSORS_KEY, Arrays.asList(Collections.singletonMap("test", processorConfig0), Collections.singletonMap("test", processorConfig1))); + Pipeline.Factory factory = new Pipeline.Factory(); + Map processorRegistry = Collections.singletonMap("test", new TestProcessor.Factory()); + Pipeline pipeline = factory.create("_id", pipelineConfig, processorRegistry); + assertThat(pipeline.getId(), equalTo("_id")); + assertThat(pipeline.getDescription(), equalTo("_description")); + assertThat(pipeline.getProcessors().size(), equalTo(2)); + assertThat(pipeline.getProcessors().get(0).getType(), equalTo("test-processor")); + assertThat(pipeline.getProcessors().get(0).getTag(), equalTo("first-processor")); + assertThat(pipeline.getProcessors().get(1).getType(), equalTo("test-processor")); + assertThat(pipeline.getProcessors().get(1).getTag(), nullValue()); + } + + public void testCreateWithPipelineOnFailure() throws Exception { + Map processorConfig = new HashMap<>(); + Map pipelineConfig = new HashMap<>(); + pipelineConfig.put(Pipeline.DESCRIPTION_KEY, "_description"); + pipelineConfig.put(Pipeline.PROCESSORS_KEY, Collections.singletonList(Collections.singletonMap("test", processorConfig))); + pipelineConfig.put(Pipeline.ON_FAILURE_KEY, Collections.singletonList(Collections.singletonMap("test", processorConfig))); + Pipeline.Factory factory = new Pipeline.Factory(); + Map processorRegistry = Collections.singletonMap("test", new TestProcessor.Factory()); + Pipeline pipeline = factory.create("_id", pipelineConfig, processorRegistry); + assertThat(pipeline.getId(), equalTo("_id")); + assertThat(pipeline.getDescription(), equalTo("_description")); + assertThat(pipeline.getProcessors().size(), equalTo(1)); + assertThat(pipeline.getProcessors().get(0).getType(), equalTo("test-processor")); + assertThat(pipeline.getOnFailureProcessors().size(), equalTo(1)); + assertThat(pipeline.getOnFailureProcessors().get(0).getType(), equalTo("test-processor")); + } + + public void testCreateUnusedProcessorOptions() throws Exception { + Map processorConfig = new HashMap<>(); + processorConfig.put("unused", "value"); + Map pipelineConfig = new HashMap<>(); + pipelineConfig.put(Pipeline.DESCRIPTION_KEY, "_description"); + pipelineConfig.put(Pipeline.PROCESSORS_KEY, Collections.singletonList(Collections.singletonMap("test", processorConfig))); + Pipeline.Factory factory = new Pipeline.Factory(); + Map processorRegistry = Collections.singletonMap("test", new TestProcessor.Factory()); + try { + factory.create("_id", pipelineConfig, processorRegistry); + } catch (IllegalArgumentException e) { + assertThat(e.getMessage(), equalTo("processor [test] doesn't support one or more provided configuration parameters [unused]")); + } + } + + public void testCreateProcessorsWithOnFailureProperties() throws Exception { + Map processorConfig = new HashMap<>(); + processorConfig.put(Pipeline.ON_FAILURE_KEY, Collections.singletonList(Collections.singletonMap("test", new HashMap<>()))); + + Map pipelineConfig = new HashMap<>(); + pipelineConfig.put(Pipeline.DESCRIPTION_KEY, "_description"); + pipelineConfig.put(Pipeline.PROCESSORS_KEY, Collections.singletonList(Collections.singletonMap("test", processorConfig))); + Pipeline.Factory factory = new Pipeline.Factory(); + Map processorRegistry = Collections.singletonMap("test", new TestProcessor.Factory()); + Pipeline pipeline = factory.create("_id", pipelineConfig, processorRegistry); + assertThat(pipeline.getId(), equalTo("_id")); + assertThat(pipeline.getDescription(), equalTo("_description")); + assertThat(pipeline.getProcessors().size(), equalTo(1)); + assertThat(pipeline.getProcessors().get(0).getType(), equalTo("compound")); + } +} diff --git a/core/src/test/java/org/elasticsearch/ingest/core/ValueSourceTests.java b/core/src/test/java/org/elasticsearch/ingest/core/ValueSourceTests.java new file mode 100644 index 00000000000..f2aa9f32bcd --- /dev/null +++ b/core/src/test/java/org/elasticsearch/ingest/core/ValueSourceTests.java @@ -0,0 +1,72 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.ingest.core; + +import org.elasticsearch.ingest.RandomDocumentPicks; +import org.elasticsearch.ingest.TestTemplateService; +import org.elasticsearch.test.ESTestCase; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.not; +import static org.hamcrest.Matchers.sameInstance; + +public class ValueSourceTests extends ESTestCase { + + public void testDeepCopy() { + int iterations = scaledRandomIntBetween(8, 64); + for (int i = 0; i < iterations; i++) { + Map map = RandomDocumentPicks.randomSource(random()); + ValueSource valueSource = ValueSource.wrap(map, TestTemplateService.instance()); + Object copy = valueSource.copyAndResolve(Collections.emptyMap()); + assertThat("iteration: " + i, copy, equalTo(map)); + assertThat("iteration: " + i, copy, not(sameInstance(map))); + } + } + + public void testCopyDoesNotChangeProvidedMap() { + Map myPreciousMap = new HashMap<>(); + myPreciousMap.put("field2", "value2"); + + IngestDocument ingestDocument = new IngestDocument(new HashMap<>(), new HashMap<>()); + ingestDocument.setFieldValue(TestTemplateService.instance().compile("field1"), ValueSource.wrap(myPreciousMap, TestTemplateService.instance())); + ingestDocument.removeField("field1.field2"); + + assertThat(myPreciousMap.size(), equalTo(1)); + assertThat(myPreciousMap.get("field2"), equalTo("value2")); + } + + public void testCopyDoesNotChangeProvidedList() { + List myPreciousList = new ArrayList<>(); + myPreciousList.add("value"); + + IngestDocument ingestDocument = new IngestDocument(new HashMap<>(), new HashMap<>()); + ingestDocument.setFieldValue(TestTemplateService.instance().compile("field1"), ValueSource.wrap(myPreciousList, TestTemplateService.instance())); + ingestDocument.removeField("field1.0"); + + assertThat(myPreciousList.size(), equalTo(1)); + assertThat(myPreciousList.get(0), equalTo("value")); + } +} diff --git a/core/src/test/java/org/elasticsearch/ingest/processor/AbstractStringProcessorTestCase.java b/core/src/test/java/org/elasticsearch/ingest/processor/AbstractStringProcessorTestCase.java new file mode 100644 index 00000000000..1113a4b402f --- /dev/null +++ b/core/src/test/java/org/elasticsearch/ingest/processor/AbstractStringProcessorTestCase.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.ingest.processor; + +import org.elasticsearch.ingest.core.IngestDocument; +import org.elasticsearch.ingest.core.Processor; +import org.elasticsearch.ingest.RandomDocumentPicks; +import org.elasticsearch.test.ESTestCase; + +import java.util.Collections; +import java.util.HashMap; + +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.equalTo; + +public abstract class AbstractStringProcessorTestCase extends ESTestCase { + + protected abstract AbstractStringProcessor newProcessor(String field); + + protected String modifyInput(String input) { + return input; + } + + protected abstract String expectedResult(String input); + + public void testProcessor() throws Exception { + IngestDocument ingestDocument = RandomDocumentPicks.randomIngestDocument(random()); + String fieldValue = RandomDocumentPicks.randomString(random()); + String fieldName = RandomDocumentPicks.addRandomField(random(), ingestDocument, modifyInput(fieldValue)); + Processor processor = newProcessor(fieldName); + processor.execute(ingestDocument); + assertThat(ingestDocument.getFieldValue(fieldName, String.class), equalTo(expectedResult(fieldValue))); + } + + public void testFieldNotFound() throws Exception { + String fieldName = RandomDocumentPicks.randomFieldName(random()); + Processor processor = newProcessor(fieldName); + IngestDocument ingestDocument = RandomDocumentPicks.randomIngestDocument(random(), new HashMap<>()); + try { + processor.execute(ingestDocument); + fail("processor should have failed"); + } catch (IllegalArgumentException e) { + assertThat(e.getMessage(), containsString("not present as part of path [" + fieldName + "]")); + } + } + + public void testNullValue() throws Exception { + Processor processor = newProcessor("field"); + IngestDocument ingestDocument = RandomDocumentPicks.randomIngestDocument(random(), Collections.singletonMap("field", null)); + try { + processor.execute(ingestDocument); + fail("processor should have failed"); + } catch (IllegalArgumentException e) { + assertThat(e.getMessage(), equalTo("field [field] is null, cannot process it.")); + } + } + + public void testNonStringValue() throws Exception { + String fieldName = RandomDocumentPicks.randomFieldName(random()); + Processor processor = newProcessor(fieldName); + IngestDocument ingestDocument = RandomDocumentPicks.randomIngestDocument(random(), new HashMap<>()); + ingestDocument.setFieldValue(fieldName, randomInt()); + try { + processor.execute(ingestDocument); + fail("processor should have failed"); + } catch (IllegalArgumentException e) { + assertThat(e.getMessage(), equalTo("field [" + fieldName + "] of type [java.lang.Integer] cannot be cast to [java.lang.String]")); + } + } +} diff --git a/core/src/test/java/org/elasticsearch/ingest/processor/AppendProcessorFactoryTests.java b/core/src/test/java/org/elasticsearch/ingest/processor/AppendProcessorFactoryTests.java new file mode 100644 index 00000000000..b72c144605f --- /dev/null +++ b/core/src/test/java/org/elasticsearch/ingest/processor/AppendProcessorFactoryTests.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.ingest.processor; + +import org.elasticsearch.ingest.TestTemplateService; +import org.elasticsearch.ingest.core.AbstractProcessorFactory; +import org.elasticsearch.test.ESTestCase; +import org.junit.Before; + +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +import static org.hamcrest.CoreMatchers.equalTo; + +public class AppendProcessorFactoryTests extends ESTestCase { + + private AppendProcessor.Factory factory; + + @Before + public void init() { + factory = new AppendProcessor.Factory(TestTemplateService.instance()); + } + + public void testCreate() throws Exception { + Map config = new HashMap<>(); + config.put("field", "field1"); + Object value; + if (randomBoolean()) { + value = "value1"; + } else { + value = Arrays.asList("value1", "value2", "value3"); + } + config.put("value", value); + String processorTag = randomAsciiOfLength(10); + config.put(AbstractProcessorFactory.TAG_KEY, processorTag); + AppendProcessor appendProcessor = factory.create(config); + assertThat(appendProcessor.getTag(), equalTo(processorTag)); + assertThat(appendProcessor.getField().execute(Collections.emptyMap()), equalTo("field1")); + assertThat(appendProcessor.getValue().copyAndResolve(Collections.emptyMap()), equalTo(value)); + } + + public void testCreateNoFieldPresent() throws Exception { + Map config = new HashMap<>(); + config.put("value", "value1"); + try { + factory.create(config); + fail("factory create should have failed"); + } catch(IllegalArgumentException e) { + assertThat(e.getMessage(), equalTo("required property [field] is missing")); + } + } + + public void testCreateNoValuePresent() throws Exception { + Map config = new HashMap<>(); + config.put("field", "field1"); + try { + factory.create(config); + fail("factory create should have failed"); + } catch(IllegalArgumentException e) { + assertThat(e.getMessage(), equalTo("required property [value] is missing")); + } + } + + public void testCreateNullValue() throws Exception { + Map config = new HashMap<>(); + config.put("field", "field1"); + config.put("value", null); + try { + factory.create(config); + fail("factory create should have failed"); + } catch(IllegalArgumentException e) { + assertThat(e.getMessage(), equalTo("required property [value] is missing")); + } + } +} diff --git a/core/src/test/java/org/elasticsearch/ingest/processor/AppendProcessorTests.java b/core/src/test/java/org/elasticsearch/ingest/processor/AppendProcessorTests.java new file mode 100644 index 00000000000..4a78ba621ce --- /dev/null +++ b/core/src/test/java/org/elasticsearch/ingest/processor/AppendProcessorTests.java @@ -0,0 +1,209 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.ingest.processor; + +import org.elasticsearch.ingest.core.IngestDocument; +import org.elasticsearch.ingest.RandomDocumentPicks; +import org.elasticsearch.ingest.core.TemplateService; +import org.elasticsearch.ingest.TestTemplateService; +import org.elasticsearch.ingest.core.ValueSource; +import org.elasticsearch.ingest.core.Processor; +import org.elasticsearch.test.ESTestCase; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.not; +import static org.hamcrest.CoreMatchers.sameInstance; + +public class AppendProcessorTests extends ESTestCase { + + public void testAppendValuesToExistingList() throws Exception { + IngestDocument ingestDocument = RandomDocumentPicks.randomIngestDocument(random()); + Scalar scalar = randomFrom(Scalar.values()); + List list = new ArrayList<>(); + int size = randomIntBetween(0, 10); + for (int i = 0; i < size; i++) { + list.add(scalar.randomValue()); + } + List checkList = new ArrayList<>(list); + String field = RandomDocumentPicks.addRandomField(random(), ingestDocument, list); + List values = new ArrayList<>(); + Processor appendProcessor; + if (randomBoolean()) { + Object value = scalar.randomValue(); + values.add(value); + appendProcessor = createAppendProcessor(field, value); + } else { + int valuesSize = randomIntBetween(0, 10); + for (int i = 0; i < valuesSize; i++) { + values.add(scalar.randomValue()); + } + appendProcessor = createAppendProcessor(field, values); + } + appendProcessor.execute(ingestDocument); + Object fieldValue = ingestDocument.getFieldValue(field, Object.class); + assertThat(fieldValue, sameInstance(list)); + assertThat(list.size(), equalTo(size + values.size())); + for (int i = 0; i < size; i++) { + assertThat(list.get(i), equalTo(checkList.get(i))); + } + for (int i = size; i < size + values.size(); i++) { + assertThat(list.get(i), equalTo(values.get(i - size))); + } + } + + public void testAppendValuesToNonExistingList() throws Exception { + IngestDocument ingestDocument = RandomDocumentPicks.randomIngestDocument(random(), new HashMap<>()); + String field = RandomDocumentPicks.randomFieldName(random()); + Scalar scalar = randomFrom(Scalar.values()); + List values = new ArrayList<>(); + Processor appendProcessor; + if (randomBoolean()) { + Object value = scalar.randomValue(); + values.add(value); + appendProcessor = createAppendProcessor(field, value); + } else { + int valuesSize = randomIntBetween(0, 10); + for (int i = 0; i < valuesSize; i++) { + values.add(scalar.randomValue()); + } + appendProcessor = createAppendProcessor(field, values); + } + appendProcessor.execute(ingestDocument); + List list = ingestDocument.getFieldValue(field, List.class); + assertThat(list, not(sameInstance(values))); + assertThat(list, equalTo(values)); + } + + public void testConvertScalarToList() throws Exception { + IngestDocument ingestDocument = RandomDocumentPicks.randomIngestDocument(random()); + Scalar scalar = randomFrom(Scalar.values()); + Object initialValue = scalar.randomValue(); + String field = RandomDocumentPicks.addRandomField(random(), ingestDocument, initialValue); + List values = new ArrayList<>(); + Processor appendProcessor; + if (randomBoolean()) { + Object value = scalar.randomValue(); + values.add(value); + appendProcessor = createAppendProcessor(field, value); + } else { + int valuesSize = randomIntBetween(0, 10); + for (int i = 0; i < valuesSize; i++) { + values.add(scalar.randomValue()); + } + appendProcessor = createAppendProcessor(field, values); + } + appendProcessor.execute(ingestDocument); + List fieldValue = ingestDocument.getFieldValue(field, List.class); + assertThat(fieldValue.size(), equalTo(values.size() + 1)); + assertThat(fieldValue.get(0), equalTo(initialValue)); + for (int i = 1; i < values.size() + 1; i++) { + assertThat(fieldValue.get(i), equalTo(values.get(i - 1))); + } + } + + public void testAppendMetadata() throws Exception { + //here any metadata field value becomes a list, which won't make sense in most of the cases, + // but support for append is streamlined like for set so we test it + IngestDocument.MetaData randomMetaData = randomFrom(IngestDocument.MetaData.values()); + List values = new ArrayList<>(); + Processor appendProcessor; + if (randomBoolean()) { + String value = randomAsciiOfLengthBetween(1, 10); + values.add(value); + appendProcessor = createAppendProcessor(randomMetaData.getFieldName(), value); + } else { + int valuesSize = randomIntBetween(0, 10); + for (int i = 0; i < valuesSize; i++) { + values.add(randomAsciiOfLengthBetween(1, 10)); + } + appendProcessor = createAppendProcessor(randomMetaData.getFieldName(), values); + } + + IngestDocument ingestDocument = RandomDocumentPicks.randomIngestDocument(random()); + Object initialValue = ingestDocument.getSourceAndMetadata().get(randomMetaData.getFieldName()); + appendProcessor.execute(ingestDocument); + List list = ingestDocument.getFieldValue(randomMetaData.getFieldName(), List.class); + if (initialValue == null) { + assertThat(list, equalTo(values)); + } else { + assertThat(list.size(), equalTo(values.size() + 1)); + assertThat(list.get(0), equalTo(initialValue)); + for (int i = 1; i < list.size(); i++) { + assertThat(list.get(i), equalTo(values.get(i - 1))); + } + } + } + + private static Processor createAppendProcessor(String fieldName, Object fieldValue) { + TemplateService templateService = TestTemplateService.instance(); + return new AppendProcessor(randomAsciiOfLength(10), templateService.compile(fieldName), ValueSource.wrap(fieldValue, templateService)); + } + + private enum Scalar { + INTEGER { + @Override + Object randomValue() { + return randomInt(); + } + }, DOUBLE { + @Override + Object randomValue() { + return randomDouble(); + } + }, FLOAT { + @Override + Object randomValue() { + return randomFloat(); + } + }, BOOLEAN { + @Override + Object randomValue() { + return randomBoolean(); + } + }, STRING { + @Override + Object randomValue() { + return randomAsciiOfLengthBetween(1, 10); + } + }, MAP { + @Override + Object randomValue() { + int numItems = randomIntBetween(1, 10); + Map map = new HashMap<>(numItems); + for (int i = 0; i < numItems; i++) { + map.put(randomAsciiOfLengthBetween(1, 10), randomFrom(Scalar.values()).randomValue()); + } + return map; + } + }, NULL { + @Override + Object randomValue() { + return null; + } + }; + + abstract Object randomValue(); + } +} diff --git a/core/src/test/java/org/elasticsearch/ingest/processor/ConvertProcessorFactoryTests.java b/core/src/test/java/org/elasticsearch/ingest/processor/ConvertProcessorFactoryTests.java new file mode 100644 index 00000000000..706433141d4 --- /dev/null +++ b/core/src/test/java/org/elasticsearch/ingest/processor/ConvertProcessorFactoryTests.java @@ -0,0 +1,85 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.ingest.processor; + +import org.elasticsearch.ingest.core.AbstractProcessorFactory; +import org.elasticsearch.test.ESTestCase; +import org.hamcrest.Matchers; + +import java.util.HashMap; +import java.util.Map; + +import static org.hamcrest.CoreMatchers.equalTo; + +public class ConvertProcessorFactoryTests extends ESTestCase { + + public void testCreate() throws Exception { + ConvertProcessor.Factory factory = new ConvertProcessor.Factory(); + Map config = new HashMap<>(); + ConvertProcessor.Type type = randomFrom(ConvertProcessor.Type.values()); + config.put("field", "field1"); + config.put("type", type.toString()); + String processorTag = randomAsciiOfLength(10); + config.put(AbstractProcessorFactory.TAG_KEY, processorTag); + ConvertProcessor convertProcessor = factory.create(config); + assertThat(convertProcessor.getTag(), equalTo(processorTag)); + assertThat(convertProcessor.getField(), equalTo("field1")); + assertThat(convertProcessor.getConvertType(), equalTo(type)); + } + + public void testCreateUnsupportedType() throws Exception { + ConvertProcessor.Factory factory = new ConvertProcessor.Factory(); + Map config = new HashMap<>(); + String type = "type-" + randomAsciiOfLengthBetween(1, 10); + config.put("field", "field1"); + config.put("type", type); + try { + factory.create(config); + fail("factory create should have failed"); + } catch (IllegalArgumentException e) { + assertThat(e.getMessage(), Matchers.equalTo("type [" + type + "] not supported, cannot convert field.")); + } + } + + public void testCreateNoFieldPresent() throws Exception { + ConvertProcessor.Factory factory = new ConvertProcessor.Factory(); + Map config = new HashMap<>(); + String type = "type-" + randomAsciiOfLengthBetween(1, 10); + config.put("type", type); + try { + factory.create(config); + fail("factory create should have failed"); + } catch (IllegalArgumentException e) { + assertThat(e.getMessage(), Matchers.equalTo("required property [field] is missing")); + } + } + + public void testCreateNoTypePresent() throws Exception { + ConvertProcessor.Factory factory = new ConvertProcessor.Factory(); + Map config = new HashMap<>(); + config.put("field", "field1"); + try { + factory.create(config); + fail("factory create should have failed"); + } catch (IllegalArgumentException e) { + assertThat(e.getMessage(), Matchers.equalTo("required property [type] is missing")); + } + } +} diff --git a/core/src/test/java/org/elasticsearch/ingest/processor/ConvertProcessorTests.java b/core/src/test/java/org/elasticsearch/ingest/processor/ConvertProcessorTests.java new file mode 100644 index 00000000000..1350ebab601 --- /dev/null +++ b/core/src/test/java/org/elasticsearch/ingest/processor/ConvertProcessorTests.java @@ -0,0 +1,268 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.ingest.processor; + +import org.elasticsearch.ingest.core.IngestDocument; +import org.elasticsearch.ingest.RandomDocumentPicks; +import org.elasticsearch.ingest.core.Processor; +import org.elasticsearch.test.ESTestCase; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Locale; +import java.util.Map; + +import static org.elasticsearch.ingest.processor.ConvertProcessor.Type; +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.equalTo; + +public class ConvertProcessorTests extends ESTestCase { + + public void testConvertInt() throws Exception { + IngestDocument ingestDocument = RandomDocumentPicks.randomIngestDocument(random()); + int randomInt = randomInt(); + String fieldName = RandomDocumentPicks.addRandomField(random(), ingestDocument, randomInt); + Processor processor = new ConvertProcessor(randomAsciiOfLength(10), fieldName, Type.INTEGER); + processor.execute(ingestDocument); + assertThat(ingestDocument.getFieldValue(fieldName, Integer.class), equalTo(randomInt)); + } + + public void testConvertIntList() throws Exception { + IngestDocument ingestDocument = RandomDocumentPicks.randomIngestDocument(random()); + int numItems = randomIntBetween(1, 10); + List fieldValue = new ArrayList<>(); + List expectedList = new ArrayList<>(); + for (int j = 0; j < numItems; j++) { + int randomInt = randomInt(); + fieldValue.add(Integer.toString(randomInt)); + expectedList.add(randomInt); + } + String fieldName = RandomDocumentPicks.addRandomField(random(), ingestDocument, fieldValue); + Processor processor = new ConvertProcessor(randomAsciiOfLength(10), fieldName, Type.INTEGER); + processor.execute(ingestDocument); + assertThat(ingestDocument.getFieldValue(fieldName, List.class), equalTo(expectedList)); + } + + public void testConvertIntError() throws Exception { + IngestDocument ingestDocument = RandomDocumentPicks.randomIngestDocument(random(), new HashMap<>()); + String fieldName = RandomDocumentPicks.randomFieldName(random()); + String value = "string-" + randomAsciiOfLengthBetween(1, 10); + ingestDocument.setFieldValue(fieldName, value); + + Processor processor = new ConvertProcessor(randomAsciiOfLength(10), fieldName, Type.INTEGER); + try { + processor.execute(ingestDocument); + fail("processor execute should have failed"); + } catch(IllegalArgumentException e) { + assertThat(e.getMessage(), equalTo("unable to convert [" + value + "] to integer")); + } + } + + public void testConvertFloat() throws Exception { + IngestDocument ingestDocument = RandomDocumentPicks.randomIngestDocument(random()); + Map expectedResult = new HashMap<>(); + float randomFloat = randomFloat(); + String fieldName = RandomDocumentPicks.addRandomField(random(), ingestDocument, randomFloat); + expectedResult.put(fieldName, randomFloat); + + Processor processor = new ConvertProcessor(randomAsciiOfLength(10), fieldName, Type.FLOAT); + processor.execute(ingestDocument); + assertThat(ingestDocument.getFieldValue(fieldName, Float.class), equalTo(randomFloat)); + } + + public void testConvertFloatList() throws Exception { + IngestDocument ingestDocument = RandomDocumentPicks.randomIngestDocument(random()); + int numItems = randomIntBetween(1, 10); + List fieldValue = new ArrayList<>(); + List expectedList = new ArrayList<>(); + for (int j = 0; j < numItems; j++) { + float randomFloat = randomFloat(); + fieldValue.add(Float.toString(randomFloat)); + expectedList.add(randomFloat); + } + String fieldName = RandomDocumentPicks.addRandomField(random(), ingestDocument, fieldValue); + Processor processor = new ConvertProcessor(randomAsciiOfLength(10), fieldName, Type.FLOAT); + processor.execute(ingestDocument); + assertThat(ingestDocument.getFieldValue(fieldName, List.class), equalTo(expectedList)); + } + + public void testConvertFloatError() throws Exception { + IngestDocument ingestDocument = RandomDocumentPicks.randomIngestDocument(random(), new HashMap<>()); + String fieldName = RandomDocumentPicks.randomFieldName(random()); + String value = "string-" + randomAsciiOfLengthBetween(1, 10); + ingestDocument.setFieldValue(fieldName, value); + + Processor processor = new ConvertProcessor(randomAsciiOfLength(10), fieldName, Type.FLOAT); + try { + processor.execute(ingestDocument); + fail("processor execute should have failed"); + } catch(IllegalArgumentException e) { + assertThat(e.getMessage(), equalTo("unable to convert [" + value + "] to float")); + } + } + + public void testConvertBoolean() throws Exception { + IngestDocument ingestDocument = RandomDocumentPicks.randomIngestDocument(random()); + boolean randomBoolean = randomBoolean(); + String booleanString = Boolean.toString(randomBoolean); + if (randomBoolean) { + booleanString = booleanString.toUpperCase(Locale.ROOT); + } + String fieldName = RandomDocumentPicks.addRandomField(random(), ingestDocument, booleanString); + + Processor processor = new ConvertProcessor(randomAsciiOfLength(10), fieldName, Type.BOOLEAN); + processor.execute(ingestDocument); + assertThat(ingestDocument.getFieldValue(fieldName, Boolean.class), equalTo(randomBoolean)); + } + + public void testConvertBooleanList() throws Exception { + IngestDocument ingestDocument = RandomDocumentPicks.randomIngestDocument(random()); + int numItems = randomIntBetween(1, 10); + List fieldValue = new ArrayList<>(); + List expectedList = new ArrayList<>(); + for (int j = 0; j < numItems; j++) { + boolean randomBoolean = randomBoolean(); + String booleanString = Boolean.toString(randomBoolean); + if (randomBoolean) { + booleanString = booleanString.toUpperCase(Locale.ROOT); + } + fieldValue.add(booleanString); + expectedList.add(randomBoolean); + } + String fieldName = RandomDocumentPicks.addRandomField(random(), ingestDocument, fieldValue); + Processor processor = new ConvertProcessor(randomAsciiOfLength(10), fieldName, Type.BOOLEAN); + processor.execute(ingestDocument); + assertThat(ingestDocument.getFieldValue(fieldName, List.class), equalTo(expectedList)); + } + + public void testConvertBooleanError() throws Exception { + IngestDocument ingestDocument = RandomDocumentPicks.randomIngestDocument(random(), new HashMap<>()); + String fieldName = RandomDocumentPicks.randomFieldName(random()); + String fieldValue; + if (randomBoolean()) { + fieldValue = "string-" + randomAsciiOfLengthBetween(1, 10); + } else { + //verify that only proper boolean values are supported and we are strict about it + fieldValue = randomFrom("on", "off", "yes", "no", "0", "1"); + } + ingestDocument.setFieldValue(fieldName, fieldValue); + + Processor processor = new ConvertProcessor(randomAsciiOfLength(10), fieldName, Type.BOOLEAN); + try { + processor.execute(ingestDocument); + fail("processor execute should have failed"); + } catch(Exception e) { + assertThat(e.getMessage(), equalTo("[" + fieldValue + "] is not a boolean value, cannot convert to boolean")); + } + } + + public void testConvertString() throws Exception { + IngestDocument ingestDocument = RandomDocumentPicks.randomIngestDocument(random()); + Object fieldValue; + String expectedFieldValue; + switch(randomIntBetween(0, 2)) { + case 0: + float randomFloat = randomFloat(); + fieldValue = randomFloat; + expectedFieldValue = Float.toString(randomFloat); + break; + case 1: + int randomInt = randomInt(); + fieldValue = randomInt; + expectedFieldValue = Integer.toString(randomInt); + break; + case 2: + boolean randomBoolean = randomBoolean(); + fieldValue = randomBoolean; + expectedFieldValue = Boolean.toString(randomBoolean); + break; + default: + throw new UnsupportedOperationException(); + } + String fieldName = RandomDocumentPicks.addRandomField(random(), ingestDocument, fieldValue); + + Processor processor = new ConvertProcessor(randomAsciiOfLength(10), fieldName, Type.STRING); + processor.execute(ingestDocument); + assertThat(ingestDocument.getFieldValue(fieldName, String.class), equalTo(expectedFieldValue)); + } + + public void testConvertStringList() throws Exception { + IngestDocument ingestDocument = RandomDocumentPicks.randomIngestDocument(random()); + int numItems = randomIntBetween(1, 10); + List fieldValue = new ArrayList<>(); + List expectedList = new ArrayList<>(); + for (int j = 0; j < numItems; j++) { + Object randomValue; + String randomValueString; + switch(randomIntBetween(0, 2)) { + case 0: + float randomFloat = randomFloat(); + randomValue = randomFloat; + randomValueString = Float.toString(randomFloat); + break; + case 1: + int randomInt = randomInt(); + randomValue = randomInt; + randomValueString = Integer.toString(randomInt); + break; + case 2: + boolean randomBoolean = randomBoolean(); + randomValue = randomBoolean; + randomValueString = Boolean.toString(randomBoolean); + break; + default: + throw new UnsupportedOperationException(); + } + fieldValue.add(randomValue); + expectedList.add(randomValueString); + } + String fieldName = RandomDocumentPicks.addRandomField(random(), ingestDocument, fieldValue); + Processor processor = new ConvertProcessor(randomAsciiOfLength(10), fieldName, Type.STRING); + processor.execute(ingestDocument); + assertThat(ingestDocument.getFieldValue(fieldName, List.class), equalTo(expectedList)); + } + + public void testConvertNonExistingField() throws Exception { + IngestDocument ingestDocument = RandomDocumentPicks.randomIngestDocument(random(), new HashMap<>()); + String fieldName = RandomDocumentPicks.randomFieldName(random()); + Type type = randomFrom(Type.values()); + Processor processor = new ConvertProcessor(randomAsciiOfLength(10), fieldName, type); + try { + processor.execute(ingestDocument); + fail("processor execute should have failed"); + } catch (IllegalArgumentException e) { + assertThat(e.getMessage(), containsString("not present as part of path [" + fieldName + "]")); + } + } + + public void testConvertNullField() throws Exception { + IngestDocument ingestDocument = RandomDocumentPicks.randomIngestDocument(random(), Collections.singletonMap("field", null)); + Type type = randomFrom(Type.values()); + Processor processor = new ConvertProcessor(randomAsciiOfLength(10), "field", type); + try { + processor.execute(ingestDocument); + fail("processor execute should have failed"); + } catch (IllegalArgumentException e) { + assertThat(e.getMessage(), equalTo("Field [field] is null, cannot be converted to type [" + type + "]")); + } + } +} diff --git a/core/src/test/java/org/elasticsearch/ingest/processor/DateFormatTests.java b/core/src/test/java/org/elasticsearch/ingest/processor/DateFormatTests.java new file mode 100644 index 00000000000..401dd44d44a --- /dev/null +++ b/core/src/test/java/org/elasticsearch/ingest/processor/DateFormatTests.java @@ -0,0 +1,83 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.ingest.processor; + +import org.elasticsearch.test.ESTestCase; +import org.joda.time.DateTime; +import org.joda.time.DateTimeZone; + +import java.time.Instant; +import java.time.ZoneId; +import java.time.format.DateTimeFormatter; +import java.util.Locale; +import java.util.function.Function; + +import static org.hamcrest.core.IsEqual.equalTo; + +public class DateFormatTests extends ESTestCase { + + public void testParseJoda() { + Function jodaFunction = DateFormat.Joda.getFunction("MMM dd HH:mm:ss Z", DateTimeZone.forOffsetHours(-8), Locale.ENGLISH); + assertThat(Instant.ofEpochMilli(jodaFunction.apply("Nov 24 01:29:01 -0800").getMillis()) + .atZone(ZoneId.of("GMT-8")) + .format(DateTimeFormatter.ofPattern("MM dd HH:mm:ss", Locale.ENGLISH)), + equalTo("11 24 01:29:01")); + } + + public void testParseUnixMs() { + assertThat(DateFormat.UnixMs.getFunction(null, DateTimeZone.UTC, null).apply("1000500").getMillis(), equalTo(1000500L)); + } + + public void testParseUnix() { + assertThat(DateFormat.Unix.getFunction(null, DateTimeZone.UTC, null).apply("1000.5").getMillis(), equalTo(1000500L)); + } + + public void testParseISO8601() { + assertThat(DateFormat.Iso8601.getFunction(null, DateTimeZone.UTC, null).apply("2001-01-01T00:00:00-0800").getMillis(), equalTo(978336000000L)); + } + + public void testParseISO8601Failure() { + Function function = DateFormat.Iso8601.getFunction(null, DateTimeZone.UTC, null); + try { + function.apply("2001-01-0:00-0800"); + fail("parse should have failed"); + } catch(IllegalArgumentException e) { + //all good + } + } + + public void testTAI64NParse() { + String input = "4000000050d506482dbdf024"; + String expected = "2012-12-22T03:00:46.767+02:00"; + assertThat(DateFormat.Tai64n.getFunction(null, DateTimeZone.forOffsetHours(2), null).apply((randomBoolean() ? "@" : "") + input).toString(), equalTo(expected)); + } + + public void testFromString() { + assertThat(DateFormat.fromString("UNIX_MS"), equalTo(DateFormat.UnixMs)); + assertThat(DateFormat.fromString("unix_ms"), equalTo(DateFormat.Joda)); + assertThat(DateFormat.fromString("UNIX"), equalTo(DateFormat.Unix)); + assertThat(DateFormat.fromString("unix"), equalTo(DateFormat.Joda)); + assertThat(DateFormat.fromString("ISO8601"), equalTo(DateFormat.Iso8601)); + assertThat(DateFormat.fromString("iso8601"), equalTo(DateFormat.Joda)); + assertThat(DateFormat.fromString("TAI64N"), equalTo(DateFormat.Tai64n)); + assertThat(DateFormat.fromString("tai64n"), equalTo(DateFormat.Joda)); + assertThat(DateFormat.fromString("prefix-" + randomAsciiOfLengthBetween(1, 10)), equalTo(DateFormat.Joda)); + } +} diff --git a/core/src/test/java/org/elasticsearch/ingest/processor/DateProcessorFactoryTests.java b/core/src/test/java/org/elasticsearch/ingest/processor/DateProcessorFactoryTests.java new file mode 100644 index 00000000000..a145a7c5149 --- /dev/null +++ b/core/src/test/java/org/elasticsearch/ingest/processor/DateProcessorFactoryTests.java @@ -0,0 +1,189 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.ingest.processor; + +import org.elasticsearch.ingest.core.AbstractProcessorFactory; +import org.elasticsearch.test.ESTestCase; +import org.joda.time.DateTimeZone; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Locale; +import java.util.Map; + +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.equalTo; + +public class DateProcessorFactoryTests extends ESTestCase { + + public void testBuildDefaults() throws Exception { + DateProcessor.Factory factory = new DateProcessor.Factory(); + Map config = new HashMap<>(); + String sourceField = randomAsciiOfLengthBetween(1, 10); + config.put("match_field", sourceField); + config.put("match_formats", Collections.singletonList("dd/MM/yyyyy")); + String processorTag = randomAsciiOfLength(10); + config.put(AbstractProcessorFactory.TAG_KEY, processorTag); + DateProcessor processor = factory.create(config); + assertThat(processor.getTag(), equalTo(processorTag)); + assertThat(processor.getMatchField(), equalTo(sourceField)); + assertThat(processor.getTargetField(), equalTo(DateProcessor.DEFAULT_TARGET_FIELD)); + assertThat(processor.getMatchFormats(), equalTo(Collections.singletonList("dd/MM/yyyyy"))); + assertThat(processor.getLocale(), equalTo(Locale.ENGLISH)); + assertThat(processor.getTimezone(), equalTo(DateTimeZone.UTC)); + } + + public void testMatchFieldIsMandatory() throws Exception { + DateProcessor.Factory factory = new DateProcessor.Factory(); + Map config = new HashMap<>(); + String targetField = randomAsciiOfLengthBetween(1, 10); + config.put("target_field", targetField); + config.put("match_formats", Collections.singletonList("dd/MM/yyyyy")); + + try { + factory.create(config); + fail("processor creation should have failed"); + } catch(IllegalArgumentException e) { + assertThat(e.getMessage(), containsString("required property [match_field] is missing")); + } + } + + public void testMatchFormatsIsMandatory() throws Exception { + DateProcessor.Factory factory = new DateProcessor.Factory(); + Map config = new HashMap<>(); + String sourceField = randomAsciiOfLengthBetween(1, 10); + String targetField = randomAsciiOfLengthBetween(1, 10); + config.put("match_field", sourceField); + config.put("target_field", targetField); + + try { + factory.create(config); + fail("processor creation should have failed"); + } catch(IllegalArgumentException e) { + assertThat(e.getMessage(), containsString("required property [match_formats] is missing")); + } + } + + public void testParseLocale() throws Exception { + DateProcessor.Factory factory = new DateProcessor.Factory(); + Map config = new HashMap<>(); + String sourceField = randomAsciiOfLengthBetween(1, 10); + config.put("match_field", sourceField); + config.put("match_formats", Collections.singletonList("dd/MM/yyyyy")); + Locale locale = randomLocale(random()); + config.put("locale", locale.toLanguageTag()); + + DateProcessor processor = factory.create(config); + assertThat(processor.getLocale().toLanguageTag(), equalTo(locale.toLanguageTag())); + } + + public void testParseInvalidLocale() throws Exception { + DateProcessor.Factory factory = new DateProcessor.Factory(); + Map config = new HashMap<>(); + String sourceField = randomAsciiOfLengthBetween(1, 10); + config.put("match_field", sourceField); + config.put("match_formats", Collections.singletonList("dd/MM/yyyyy")); + config.put("locale", "invalid_locale"); + try { + factory.create(config); + fail("should fail with invalid locale"); + } catch (IllegalArgumentException e) { + assertThat(e.getMessage(), equalTo("Invalid language tag specified: invalid_locale")); + } + } + + public void testParseTimezone() throws Exception { + DateProcessor.Factory factory = new DateProcessor.Factory(); + Map config = new HashMap<>(); + String sourceField = randomAsciiOfLengthBetween(1, 10); + config.put("match_field", sourceField); + config.put("match_formats", Collections.singletonList("dd/MM/yyyyy")); + + DateTimeZone timezone = randomTimezone(); + config.put("timezone", timezone.getID()); + DateProcessor processor = factory.create(config); + assertThat(processor.getTimezone(), equalTo(timezone)); + } + + public void testParseInvalidTimezone() throws Exception { + DateProcessor.Factory factory = new DateProcessor.Factory(); + Map config = new HashMap<>(); + String sourceField = randomAsciiOfLengthBetween(1, 10); + config.put("match_field", sourceField); + config.put("match_formats", Collections.singletonList("dd/MM/yyyyy")); + config.put("timezone", "invalid_timezone"); + try { + factory.create(config); + fail("invalid timezone should fail"); + } catch (IllegalArgumentException e) { + assertThat(e.getMessage(), equalTo("The datetime zone id 'invalid_timezone' is not recognised")); + } + } + + //we generate a timezone out of the available ones in joda, some available in the jdk are not available in joda by default + private static DateTimeZone randomTimezone() { + List ids = new ArrayList<>(DateTimeZone.getAvailableIDs()); + Collections.sort(ids); + return DateTimeZone.forID(randomFrom(ids)); + } + + + public void testParseMatchFormats() throws Exception { + DateProcessor.Factory factory = new DateProcessor.Factory(); + Map config = new HashMap<>(); + String sourceField = randomAsciiOfLengthBetween(1, 10); + config.put("match_field", sourceField); + config.put("match_formats", Arrays.asList("dd/MM/yyyy", "dd-MM-yyyy")); + + DateProcessor processor = factory.create(config); + assertThat(processor.getMatchFormats(), equalTo(Arrays.asList("dd/MM/yyyy", "dd-MM-yyyy"))); + } + + public void testParseMatchFormatsFailure() throws Exception { + DateProcessor.Factory factory = new DateProcessor.Factory(); + Map config = new HashMap<>(); + String sourceField = randomAsciiOfLengthBetween(1, 10); + config.put("match_field", sourceField); + config.put("match_formats", "dd/MM/yyyy"); + + try { + factory.create(config); + fail("processor creation should have failed"); + } catch(IllegalArgumentException e) { + assertThat(e.getMessage(), containsString("property [match_formats] isn't a list, but of type [java.lang.String]")); + } + } + + public void testParseTargetField() throws Exception { + DateProcessor.Factory factory = new DateProcessor.Factory(); + Map config = new HashMap<>(); + String sourceField = randomAsciiOfLengthBetween(1, 10); + String targetField = randomAsciiOfLengthBetween(1, 10); + config.put("match_field", sourceField); + config.put("target_field", targetField); + config.put("match_formats", Arrays.asList("dd/MM/yyyy", "dd-MM-yyyy")); + + DateProcessor processor = factory.create(config); + assertThat(processor.getTargetField(), equalTo(targetField)); + } +} diff --git a/core/src/test/java/org/elasticsearch/ingest/processor/DateProcessorTests.java b/core/src/test/java/org/elasticsearch/ingest/processor/DateProcessorTests.java new file mode 100644 index 00000000000..5daab95a5d0 --- /dev/null +++ b/core/src/test/java/org/elasticsearch/ingest/processor/DateProcessorTests.java @@ -0,0 +1,147 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.ingest.processor; + +import org.elasticsearch.ingest.core.IngestDocument; +import org.elasticsearch.ingest.RandomDocumentPicks; +import org.elasticsearch.test.ESTestCase; +import org.joda.time.DateTime; +import org.joda.time.DateTimeZone; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Locale; +import java.util.Map; + +import static org.hamcrest.CoreMatchers.containsString; +import static org.hamcrest.CoreMatchers.equalTo; + +public class DateProcessorTests extends ESTestCase { + + public void testJodaPattern() { + DateProcessor dateProcessor = new DateProcessor(randomAsciiOfLength(10), DateTimeZone.forID("Europe/Amsterdam"), Locale.ENGLISH, + "date_as_string", Collections.singletonList("yyyy dd MM hh:mm:ss"), "date_as_date"); + Map document = new HashMap<>(); + document.put("date_as_string", "2010 12 06 11:05:15"); + IngestDocument ingestDocument = RandomDocumentPicks.randomIngestDocument(random(), document); + dateProcessor.execute(ingestDocument); + assertThat(ingestDocument.getFieldValue("date_as_date", String.class), equalTo("2010-06-12T11:05:15.000+02:00")); + } + + public void testJodaPatternMultipleFormats() { + List matchFormats = new ArrayList<>(); + matchFormats.add("yyyy dd MM"); + matchFormats.add("dd/MM/yyyy"); + matchFormats.add("dd-MM-yyyy"); + DateProcessor dateProcessor = new DateProcessor(randomAsciiOfLength(10), DateTimeZone.forID("Europe/Amsterdam"), Locale.ENGLISH, + "date_as_string", matchFormats, "date_as_date"); + + Map document = new HashMap<>(); + document.put("date_as_string", "2010 12 06"); + IngestDocument ingestDocument = RandomDocumentPicks.randomIngestDocument(random(), document); + dateProcessor.execute(ingestDocument); + assertThat(ingestDocument.getFieldValue("date_as_date", String.class), equalTo("2010-06-12T00:00:00.000+02:00")); + + document = new HashMap<>(); + document.put("date_as_string", "12/06/2010"); + ingestDocument = RandomDocumentPicks.randomIngestDocument(random(), document); + dateProcessor.execute(ingestDocument); + assertThat(ingestDocument.getFieldValue("date_as_date", String.class), equalTo("2010-06-12T00:00:00.000+02:00")); + + document = new HashMap<>(); + document.put("date_as_string", "12-06-2010"); + ingestDocument = RandomDocumentPicks.randomIngestDocument(random(), document); + dateProcessor.execute(ingestDocument); + assertThat(ingestDocument.getFieldValue("date_as_date", String.class), equalTo("2010-06-12T00:00:00.000+02:00")); + + document = new HashMap<>(); + document.put("date_as_string", "2010"); + ingestDocument = RandomDocumentPicks.randomIngestDocument(random(), document); + try { + dateProcessor.execute(ingestDocument); + fail("processor should have failed due to not supported date format"); + } catch(IllegalArgumentException e) { + assertThat(e.getMessage(), containsString("unable to parse date [2010]")); + } + } + + public void testInvalidJodaPattern() { + try { + new DateProcessor(randomAsciiOfLength(10), DateTimeZone.UTC, randomLocale(random()), + "date_as_string", Collections.singletonList("invalid pattern"), "date_as_date"); + fail("date processor initialization should have failed"); + } catch(IllegalArgumentException e) { + assertThat(e.getMessage(), equalTo("Illegal pattern component: i")); + } + } + + public void testJodaPatternLocale() { + DateProcessor dateProcessor = new DateProcessor(randomAsciiOfLength(10), DateTimeZone.forID("Europe/Amsterdam"), Locale.ITALIAN, + "date_as_string", Collections.singletonList("yyyy dd MMM"), "date_as_date"); + Map document = new HashMap<>(); + document.put("date_as_string", "2010 12 giugno"); + IngestDocument ingestDocument = RandomDocumentPicks.randomIngestDocument(random(), document); + dateProcessor.execute(ingestDocument); + assertThat(ingestDocument.getFieldValue("date_as_date", String.class), equalTo("2010-06-12T00:00:00.000+02:00")); + } + + public void testJodaPatternDefaultYear() { + DateProcessor dateProcessor = new DateProcessor(randomAsciiOfLength(10), DateTimeZone.forID("Europe/Amsterdam"), Locale.ENGLISH, + "date_as_string", Collections.singletonList("dd/MM"), "date_as_date"); + Map document = new HashMap<>(); + document.put("date_as_string", "12/06"); + IngestDocument ingestDocument = RandomDocumentPicks.randomIngestDocument(random(), document); + dateProcessor.execute(ingestDocument); + assertThat(ingestDocument.getFieldValue("date_as_date", String.class), equalTo(DateTime.now().getYear() + "-06-12T00:00:00.000+02:00")); + } + + public void testTAI64N() { + DateProcessor dateProcessor = new DateProcessor(randomAsciiOfLength(10), DateTimeZone.forOffsetHours(2), randomLocale(random()), + "date_as_string", Collections.singletonList("TAI64N"), "date_as_date"); + Map document = new HashMap<>(); + String dateAsString = (randomBoolean() ? "@" : "") + "4000000050d506482dbdf024"; + document.put("date_as_string", dateAsString); + IngestDocument ingestDocument = RandomDocumentPicks.randomIngestDocument(random(), document); + dateProcessor.execute(ingestDocument); + assertThat(ingestDocument.getFieldValue("date_as_date", String.class), equalTo("2012-12-22T03:00:46.767+02:00")); + } + + public void testUnixMs() { + DateProcessor dateProcessor = new DateProcessor(randomAsciiOfLength(10), DateTimeZone.UTC, randomLocale(random()), + "date_as_string", Collections.singletonList("UNIX_MS"), "date_as_date"); + Map document = new HashMap<>(); + document.put("date_as_string", "1000500"); + IngestDocument ingestDocument = RandomDocumentPicks.randomIngestDocument(random(), document); + dateProcessor.execute(ingestDocument); + assertThat(ingestDocument.getFieldValue("date_as_date", String.class), equalTo("1970-01-01T00:16:40.500Z")); + } + + public void testUnix() { + DateProcessor dateProcessor = new DateProcessor(randomAsciiOfLength(10), DateTimeZone.UTC, randomLocale(random()), + "date_as_string", Collections.singletonList("UNIX"), "date_as_date"); + Map document = new HashMap<>(); + document.put("date_as_string", "1000.5"); + IngestDocument ingestDocument = RandomDocumentPicks.randomIngestDocument(random(), document); + dateProcessor.execute(ingestDocument); + assertThat(ingestDocument.getFieldValue("date_as_date", String.class), equalTo("1970-01-01T00:16:40.500Z")); + } +} diff --git a/core/src/test/java/org/elasticsearch/ingest/processor/DeDotProcessorFactoryTests.java b/core/src/test/java/org/elasticsearch/ingest/processor/DeDotProcessorFactoryTests.java new file mode 100644 index 00000000000..63eee56cc68 --- /dev/null +++ b/core/src/test/java/org/elasticsearch/ingest/processor/DeDotProcessorFactoryTests.java @@ -0,0 +1,56 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.ingest.processor; + +import org.elasticsearch.ingest.core.AbstractProcessorFactory; +import org.elasticsearch.test.ESTestCase; +import org.junit.Before; + +import java.util.HashMap; +import java.util.Map; + +import static org.hamcrest.CoreMatchers.equalTo; + +public class DeDotProcessorFactoryTests extends ESTestCase { + + private DeDotProcessor.Factory factory; + + @Before + public void init() { + factory = new DeDotProcessor.Factory(); + } + + public void testCreate() throws Exception { + Map config = new HashMap<>(); + config.put("separator", "_"); + String processorTag = randomAsciiOfLength(10); + config.put(AbstractProcessorFactory.TAG_KEY, processorTag); + DeDotProcessor deDotProcessor = factory.create(config); + assertThat(deDotProcessor.getSeparator(), equalTo("_")); + assertThat(deDotProcessor.getTag(), equalTo(processorTag)); + } + + public void testCreateMissingSeparatorField() throws Exception { + Map config = new HashMap<>(); + DeDotProcessor deDotProcessor = factory.create(config); + assertThat(deDotProcessor.getSeparator(), equalTo(DeDotProcessor.DEFAULT_SEPARATOR)); + } + +} diff --git a/core/src/test/java/org/elasticsearch/ingest/processor/DeDotProcessorTests.java b/core/src/test/java/org/elasticsearch/ingest/processor/DeDotProcessorTests.java new file mode 100644 index 00000000000..a0c87d7a16b --- /dev/null +++ b/core/src/test/java/org/elasticsearch/ingest/processor/DeDotProcessorTests.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.ingest.processor; + +import org.elasticsearch.ingest.core.IngestDocument; +import org.elasticsearch.ingest.core.Processor; +import org.elasticsearch.test.ESTestCase; + +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +import static org.hamcrest.Matchers.equalTo; + +public class DeDotProcessorTests extends ESTestCase { + + public void testSimple() throws Exception { + Map source = new HashMap<>(); + source.put("a.b", "hello world!"); + IngestDocument ingestDocument = new IngestDocument(source, Collections.emptyMap()); + String separator = randomUnicodeOfCodepointLengthBetween(1, 10); + Processor processor = new DeDotProcessor(randomAsciiOfLength(10), separator); + processor.execute(ingestDocument); + assertThat(ingestDocument.getSourceAndMetadata().get("a" + separator + "b" ), equalTo("hello world!")); + } + + public void testSimpleMap() throws Exception { + Map source = new HashMap<>(); + Map subField = new HashMap<>(); + subField.put("b.c", "hello world!"); + source.put("a", subField); + IngestDocument ingestDocument = new IngestDocument(source, Collections.emptyMap()); + Processor processor = new DeDotProcessor(randomAsciiOfLength(10), "_"); + processor.execute(ingestDocument); + + IngestDocument expectedDocument = new IngestDocument( + Collections.singletonMap("a", Collections.singletonMap("b_c", "hello world!")), + Collections.emptyMap()); + assertThat(ingestDocument, equalTo(expectedDocument)); + } + + public void testSimpleList() throws Exception { + Map source = new HashMap<>(); + Map subField = new HashMap<>(); + subField.put("b.c", "hello world!"); + source.put("a", Arrays.asList(subField)); + IngestDocument ingestDocument = new IngestDocument(source, Collections.emptyMap()); + Processor processor = new DeDotProcessor(randomAsciiOfLength(10), "_"); + processor.execute(ingestDocument); + + IngestDocument expectedDocument = new IngestDocument( + Collections.singletonMap("a", + Collections.singletonList(Collections.singletonMap("b_c", "hello world!"))), + Collections.emptyMap()); + assertThat(ingestDocument, equalTo(expectedDocument)); + } +} diff --git a/core/src/test/java/org/elasticsearch/ingest/processor/FailProcessorFactoryTests.java b/core/src/test/java/org/elasticsearch/ingest/processor/FailProcessorFactoryTests.java new file mode 100644 index 00000000000..993c7ccd904 --- /dev/null +++ b/core/src/test/java/org/elasticsearch/ingest/processor/FailProcessorFactoryTests.java @@ -0,0 +1,62 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.ingest.processor; + +import org.elasticsearch.ingest.TestTemplateService; +import org.elasticsearch.ingest.core.AbstractProcessorFactory; +import org.elasticsearch.test.ESTestCase; +import org.junit.Before; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +import static org.hamcrest.CoreMatchers.equalTo; + +public class FailProcessorFactoryTests extends ESTestCase { + + private FailProcessor.Factory factory; + + @Before + public void init() { + factory = new FailProcessor.Factory(TestTemplateService.instance()); + } + + public void testCreate() throws Exception { + Map config = new HashMap<>(); + config.put("message", "error"); + String processorTag = randomAsciiOfLength(10); + config.put(AbstractProcessorFactory.TAG_KEY, processorTag); + FailProcessor failProcessor = factory.create(config); + assertThat(failProcessor.getTag(), equalTo(processorTag)); + assertThat(failProcessor.getMessage().execute(Collections.emptyMap()), equalTo("error")); + } + + public void testCreateMissingMessageField() throws Exception { + Map config = new HashMap<>(); + try { + factory.create(config); + fail("factory create should have failed"); + } catch(IllegalArgumentException e) { + assertThat(e.getMessage(), equalTo("required property [message] is missing")); + } + } + +} diff --git a/core/src/test/java/org/elasticsearch/ingest/processor/FailProcessorTests.java b/core/src/test/java/org/elasticsearch/ingest/processor/FailProcessorTests.java new file mode 100644 index 00000000000..3fdc2073c09 --- /dev/null +++ b/core/src/test/java/org/elasticsearch/ingest/processor/FailProcessorTests.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.ingest.processor; + +import org.elasticsearch.ingest.core.IngestDocument; +import org.elasticsearch.ingest.core.Processor; +import org.elasticsearch.ingest.RandomDocumentPicks; +import org.elasticsearch.ingest.TestTemplateService; +import org.elasticsearch.test.ESTestCase; + +import static org.hamcrest.Matchers.equalTo; + +public class FailProcessorTests extends ESTestCase { + + public void test() throws Exception { + IngestDocument ingestDocument = RandomDocumentPicks.randomIngestDocument(random()); + String message = randomAsciiOfLength(10); + Processor processor = new FailProcessor(randomAsciiOfLength(10), new TestTemplateService.MockTemplate(message)); + try { + processor.execute(ingestDocument); + fail("fail processor should throw an exception"); + } catch (FailProcessorException e) { + assertThat(e.getMessage(), equalTo(message)); + } + } +} diff --git a/core/src/test/java/org/elasticsearch/ingest/processor/GsubProcessorFactoryTests.java b/core/src/test/java/org/elasticsearch/ingest/processor/GsubProcessorFactoryTests.java new file mode 100644 index 00000000000..fd62f6cdeac --- /dev/null +++ b/core/src/test/java/org/elasticsearch/ingest/processor/GsubProcessorFactoryTests.java @@ -0,0 +1,85 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.ingest.processor; + +import org.elasticsearch.ingest.core.AbstractProcessorFactory; +import org.elasticsearch.test.ESTestCase; + +import java.util.HashMap; +import java.util.Map; + +import static org.hamcrest.CoreMatchers.equalTo; + +public class GsubProcessorFactoryTests extends ESTestCase { + + public void testCreate() throws Exception { + GsubProcessor.Factory factory = new GsubProcessor.Factory(); + Map config = new HashMap<>(); + config.put("field", "field1"); + config.put("pattern", "\\."); + config.put("replacement", "-"); + String processorTag = randomAsciiOfLength(10); + config.put(AbstractProcessorFactory.TAG_KEY, processorTag); + GsubProcessor gsubProcessor = factory.create(config); + assertThat(gsubProcessor.getTag(), equalTo(processorTag)); + assertThat(gsubProcessor.getField(), equalTo("field1")); + assertThat(gsubProcessor.getPattern().toString(), equalTo("\\.")); + assertThat(gsubProcessor.getReplacement(), equalTo("-")); + } + + public void testCreateNoFieldPresent() throws Exception { + GsubProcessor.Factory factory = new GsubProcessor.Factory(); + Map config = new HashMap<>(); + config.put("pattern", "\\."); + config.put("replacement", "-"); + try { + factory.create(config); + fail("factory create should have failed"); + } catch(IllegalArgumentException e) { + assertThat(e.getMessage(), equalTo("required property [field] is missing")); + } + } + + public void testCreateNoPatternPresent() throws Exception { + GsubProcessor.Factory factory = new GsubProcessor.Factory(); + Map config = new HashMap<>(); + config.put("field", "field1"); + config.put("replacement", "-"); + try { + factory.create(config); + fail("factory create should have failed"); + } catch(IllegalArgumentException e) { + assertThat(e.getMessage(), equalTo("required property [pattern] is missing")); + } + } + + public void testCreateNoReplacementPresent() throws Exception { + GsubProcessor.Factory factory = new GsubProcessor.Factory(); + Map config = new HashMap<>(); + config.put("field", "field1"); + config.put("pattern", "\\."); + try { + factory.create(config); + fail("factory create should have failed"); + } catch(IllegalArgumentException e) { + assertThat(e.getMessage(), equalTo("required property [replacement] is missing")); + } + } +} diff --git a/core/src/test/java/org/elasticsearch/ingest/processor/GsubProcessorTests.java b/core/src/test/java/org/elasticsearch/ingest/processor/GsubProcessorTests.java new file mode 100644 index 00000000000..fe44f33b614 --- /dev/null +++ b/core/src/test/java/org/elasticsearch/ingest/processor/GsubProcessorTests.java @@ -0,0 +1,79 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.ingest.processor; + +import org.elasticsearch.ingest.core.IngestDocument; +import org.elasticsearch.ingest.RandomDocumentPicks; +import org.elasticsearch.ingest.core.Processor; +import org.elasticsearch.test.ESTestCase; + +import java.util.Collections; +import java.util.HashMap; +import java.util.regex.Pattern; + +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.equalTo; + +public class GsubProcessorTests extends ESTestCase { + + public void testGsub() throws Exception { + IngestDocument ingestDocument = RandomDocumentPicks.randomIngestDocument(random()); + String fieldName = RandomDocumentPicks.addRandomField(random(), ingestDocument, "127.0.0.1"); + Processor processor = new GsubProcessor(randomAsciiOfLength(10), fieldName, Pattern.compile("\\."), "-"); + processor.execute(ingestDocument); + assertThat(ingestDocument.getFieldValue(fieldName, String.class), equalTo("127-0-0-1")); + } + + public void testGsubNotAStringValue() throws Exception { + IngestDocument ingestDocument = RandomDocumentPicks.randomIngestDocument(random(), new HashMap<>()); + String fieldName = RandomDocumentPicks.randomFieldName(random()); + ingestDocument.setFieldValue(fieldName, 123); + Processor processor = new GsubProcessor(randomAsciiOfLength(10), fieldName, Pattern.compile("\\."), "-"); + try { + processor.execute(ingestDocument); + fail("processor execution should have failed"); + } catch (IllegalArgumentException e) { + assertThat(e.getMessage(), equalTo("field [" + fieldName + "] of type [java.lang.Integer] cannot be cast to [java.lang.String]")); + } + } + + public void testGsubFieldNotFound() throws Exception { + IngestDocument ingestDocument = RandomDocumentPicks.randomIngestDocument(random(), new HashMap<>()); + String fieldName = RandomDocumentPicks.randomFieldName(random()); + Processor processor = new GsubProcessor(randomAsciiOfLength(10), fieldName, Pattern.compile("\\."), "-"); + try { + processor.execute(ingestDocument); + fail("processor execution should have failed"); + } catch (IllegalArgumentException e) { + assertThat(e.getMessage(), containsString("not present as part of path [" + fieldName + "]")); + } + } + + public void testGsubNullValue() throws Exception { + IngestDocument ingestDocument = RandomDocumentPicks.randomIngestDocument(random(), Collections.singletonMap("field", null)); + Processor processor = new GsubProcessor(randomAsciiOfLength(10), "field", Pattern.compile("\\."), "-"); + try { + processor.execute(ingestDocument); + fail("processor execution should have failed"); + } catch (IllegalArgumentException e) { + assertThat(e.getMessage(), equalTo("field [field] is null, cannot match pattern.")); + } + } +} diff --git a/core/src/test/java/org/elasticsearch/ingest/processor/JoinProcessorFactoryTests.java b/core/src/test/java/org/elasticsearch/ingest/processor/JoinProcessorFactoryTests.java new file mode 100644 index 00000000000..2af2b096417 --- /dev/null +++ b/core/src/test/java/org/elasticsearch/ingest/processor/JoinProcessorFactoryTests.java @@ -0,0 +1,68 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.ingest.processor; + +import org.elasticsearch.ingest.core.AbstractProcessorFactory; +import org.elasticsearch.test.ESTestCase; + +import java.util.HashMap; +import java.util.Map; + +import static org.hamcrest.CoreMatchers.equalTo; + +public class JoinProcessorFactoryTests extends ESTestCase { + + public void testCreate() throws Exception { + JoinProcessor.Factory factory = new JoinProcessor.Factory(); + Map config = new HashMap<>(); + config.put("field", "field1"); + config.put("separator", "-"); + String processorTag = randomAsciiOfLength(10); + config.put(AbstractProcessorFactory.TAG_KEY, processorTag); + JoinProcessor joinProcessor = factory.create(config); + assertThat(joinProcessor.getTag(), equalTo(processorTag)); + assertThat(joinProcessor.getField(), equalTo("field1")); + assertThat(joinProcessor.getSeparator(), equalTo("-")); + } + + public void testCreateNoFieldPresent() throws Exception { + JoinProcessor.Factory factory = new JoinProcessor.Factory(); + Map config = new HashMap<>(); + config.put("separator", "-"); + try { + factory.create(config); + fail("factory create should have failed"); + } catch (IllegalArgumentException e) { + assertThat(e.getMessage(), equalTo("required property [field] is missing")); + } + } + + public void testCreateNoSeparatorPresent() throws Exception { + JoinProcessor.Factory factory = new JoinProcessor.Factory(); + Map config = new HashMap<>(); + config.put("field", "field1"); + try { + factory.create(config); + fail("factory create should have failed"); + } catch (IllegalArgumentException e) { + assertThat(e.getMessage(), equalTo("required property [separator] is missing")); + } + } +} diff --git a/core/src/test/java/org/elasticsearch/ingest/processor/JoinProcessorTests.java b/core/src/test/java/org/elasticsearch/ingest/processor/JoinProcessorTests.java new file mode 100644 index 00000000000..2aa3ac2e035 --- /dev/null +++ b/core/src/test/java/org/elasticsearch/ingest/processor/JoinProcessorTests.java @@ -0,0 +1,111 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.ingest.processor; + +import org.elasticsearch.ingest.core.IngestDocument; +import org.elasticsearch.ingest.RandomDocumentPicks; +import org.elasticsearch.ingest.core.Processor; +import org.elasticsearch.test.ESTestCase; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; + +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.equalTo; + +public class JoinProcessorTests extends ESTestCase { + + private static final String[] SEPARATORS = new String[]{"-", "_", "."}; + + public void testJoinStrings() throws Exception { + IngestDocument ingestDocument = RandomDocumentPicks.randomIngestDocument(random()); + int numItems = randomIntBetween(1, 10); + String separator = randomFrom(SEPARATORS); + List fieldValue = new ArrayList<>(numItems); + String expectedResult = ""; + for (int j = 0; j < numItems; j++) { + String value = randomAsciiOfLengthBetween(1, 10); + fieldValue.add(value); + expectedResult += value; + if (j < numItems - 1) { + expectedResult += separator; + } + } + String fieldName = RandomDocumentPicks.addRandomField(random(), ingestDocument, fieldValue); + Processor processor = new JoinProcessor(randomAsciiOfLength(10), fieldName, separator); + processor.execute(ingestDocument); + assertThat(ingestDocument.getFieldValue(fieldName, String.class), equalTo(expectedResult)); + } + + public void testJoinIntegers() throws Exception { + IngestDocument ingestDocument = RandomDocumentPicks.randomIngestDocument(random()); + int numItems = randomIntBetween(1, 10); + String separator = randomFrom(SEPARATORS); + List fieldValue = new ArrayList<>(numItems); + String expectedResult = ""; + for (int j = 0; j < numItems; j++) { + int value = randomInt(); + fieldValue.add(value); + expectedResult += value; + if (j < numItems - 1) { + expectedResult += separator; + } + } + String fieldName = RandomDocumentPicks.addRandomField(random(), ingestDocument, fieldValue); + Processor processor = new JoinProcessor(randomAsciiOfLength(10), fieldName, separator); + processor.execute(ingestDocument); + assertThat(ingestDocument.getFieldValue(fieldName, String.class), equalTo(expectedResult)); + } + + public void testJoinNonListField() throws Exception { + IngestDocument ingestDocument = RandomDocumentPicks.randomIngestDocument(random(), new HashMap<>()); + String fieldName = RandomDocumentPicks.randomFieldName(random()); + ingestDocument.setFieldValue(fieldName, randomAsciiOfLengthBetween(1, 10)); + Processor processor = new JoinProcessor(randomAsciiOfLength(10), fieldName, "-"); + try { + processor.execute(ingestDocument); + } catch(IllegalArgumentException e) { + assertThat(e.getMessage(), equalTo("field [" + fieldName + "] of type [java.lang.String] cannot be cast to [java.util.List]")); + } + } + + public void testJoinNonExistingField() throws Exception { + IngestDocument ingestDocument = RandomDocumentPicks.randomIngestDocument(random(), new HashMap<>()); + String fieldName = RandomDocumentPicks.randomFieldName(random()); + Processor processor = new JoinProcessor(randomAsciiOfLength(10), fieldName, "-"); + try { + processor.execute(ingestDocument); + } catch(IllegalArgumentException e) { + assertThat(e.getMessage(), containsString("not present as part of path [" + fieldName + "]")); + } + } + + public void testJoinNullValue() throws Exception { + IngestDocument ingestDocument = RandomDocumentPicks.randomIngestDocument(random(), Collections.singletonMap("field", null)); + Processor processor = new JoinProcessor(randomAsciiOfLength(10), "field", "-"); + try { + processor.execute(ingestDocument); + } catch(IllegalArgumentException e) { + assertThat(e.getMessage(), equalTo("field [field] is null, cannot join.")); + } + } +} diff --git a/core/src/test/java/org/elasticsearch/ingest/processor/LowercaseProcessorFactoryTests.java b/core/src/test/java/org/elasticsearch/ingest/processor/LowercaseProcessorFactoryTests.java new file mode 100644 index 00000000000..6a4a67e40cf --- /dev/null +++ b/core/src/test/java/org/elasticsearch/ingest/processor/LowercaseProcessorFactoryTests.java @@ -0,0 +1,53 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.ingest.processor; + +import org.elasticsearch.ingest.core.AbstractProcessorFactory; +import org.elasticsearch.test.ESTestCase; + +import java.util.HashMap; +import java.util.Map; + +import static org.hamcrest.CoreMatchers.equalTo; + +public class LowercaseProcessorFactoryTests extends ESTestCase { + + public void testCreate() throws Exception { + LowercaseProcessor.Factory factory = new LowercaseProcessor.Factory(); + Map config = new HashMap<>(); + config.put("field", "field1"); + String processorTag = randomAsciiOfLength(10); + config.put(AbstractProcessorFactory.TAG_KEY, processorTag); + LowercaseProcessor uppercaseProcessor = factory.create(config); + assertThat(uppercaseProcessor.getTag(), equalTo(processorTag)); + assertThat(uppercaseProcessor.getField(), equalTo("field1")); + } + + public void testCreateMissingField() throws Exception { + LowercaseProcessor.Factory factory = new LowercaseProcessor.Factory(); + Map config = new HashMap<>(); + try { + factory.create(config); + fail("factory create should have failed"); + } catch(IllegalArgumentException e) { + assertThat(e.getMessage(), equalTo("required property [field] is missing")); + } + } +} diff --git a/core/src/test/java/org/elasticsearch/ingest/processor/LowercaseProcessorTests.java b/core/src/test/java/org/elasticsearch/ingest/processor/LowercaseProcessorTests.java new file mode 100644 index 00000000000..77e22b0ca1e --- /dev/null +++ b/core/src/test/java/org/elasticsearch/ingest/processor/LowercaseProcessorTests.java @@ -0,0 +1,34 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.ingest.processor; + +import java.util.Locale; + +public class LowercaseProcessorTests extends AbstractStringProcessorTestCase { + @Override + protected AbstractStringProcessor newProcessor(String field) { + return new LowercaseProcessor(randomAsciiOfLength(10), field); + } + + @Override + protected String expectedResult(String input) { + return input.toLowerCase(Locale.ROOT); + } +} diff --git a/core/src/test/java/org/elasticsearch/ingest/processor/RemoveProcessorFactoryTests.java b/core/src/test/java/org/elasticsearch/ingest/processor/RemoveProcessorFactoryTests.java new file mode 100644 index 00000000000..0b03150adb6 --- /dev/null +++ b/core/src/test/java/org/elasticsearch/ingest/processor/RemoveProcessorFactoryTests.java @@ -0,0 +1,62 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.ingest.processor; + +import org.elasticsearch.ingest.TestTemplateService; +import org.elasticsearch.ingest.core.AbstractProcessorFactory; +import org.elasticsearch.test.ESTestCase; +import org.junit.Before; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +import static org.hamcrest.CoreMatchers.equalTo; + +public class RemoveProcessorFactoryTests extends ESTestCase { + + private RemoveProcessor.Factory factory; + + @Before + public void init() { + factory = new RemoveProcessor.Factory(TestTemplateService.instance()); + } + + public void testCreate() throws Exception { + Map config = new HashMap<>(); + config.put("field", "field1"); + String processorTag = randomAsciiOfLength(10); + config.put(AbstractProcessorFactory.TAG_KEY, processorTag); + RemoveProcessor removeProcessor = factory.create(config); + assertThat(removeProcessor.getTag(), equalTo(processorTag)); + assertThat(removeProcessor.getField().execute(Collections.emptyMap()), equalTo("field1")); + } + + public void testCreateMissingField() throws Exception { + Map config = new HashMap<>(); + try { + factory.create(config); + fail("factory create should have failed"); + } catch(IllegalArgumentException e) { + assertThat(e.getMessage(), equalTo("required property [field] is missing")); + } + } + +} diff --git a/core/src/test/java/org/elasticsearch/ingest/processor/RemoveProcessorTests.java b/core/src/test/java/org/elasticsearch/ingest/processor/RemoveProcessorTests.java new file mode 100644 index 00000000000..d134b0213eb --- /dev/null +++ b/core/src/test/java/org/elasticsearch/ingest/processor/RemoveProcessorTests.java @@ -0,0 +1,54 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.ingest.processor; + +import org.elasticsearch.ingest.core.IngestDocument; +import org.elasticsearch.ingest.RandomDocumentPicks; +import org.elasticsearch.ingest.TestTemplateService; +import org.elasticsearch.ingest.core.Processor; +import org.elasticsearch.test.ESTestCase; + +import java.util.HashMap; + +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.equalTo; + +public class RemoveProcessorTests extends ESTestCase { + + public void testRemoveFields() throws Exception { + IngestDocument ingestDocument = RandomDocumentPicks.randomIngestDocument(random()); + String field = RandomDocumentPicks.randomExistingFieldName(random(), ingestDocument); + Processor processor = new RemoveProcessor(randomAsciiOfLength(10), new TestTemplateService.MockTemplate(field)); + processor.execute(ingestDocument); + assertThat(ingestDocument.hasField(field), equalTo(false)); + } + + public void testRemoveNonExistingField() throws Exception { + IngestDocument ingestDocument = RandomDocumentPicks.randomIngestDocument(random(), new HashMap<>()); + String fieldName = RandomDocumentPicks.randomFieldName(random()); + Processor processor = new RemoveProcessor(randomAsciiOfLength(10), new TestTemplateService.MockTemplate(fieldName)); + try { + processor.execute(ingestDocument); + fail("remove field should have failed"); + } catch(IllegalArgumentException e) { + assertThat(e.getMessage(), containsString("not present as part of path [" + fieldName + "]")); + } + } +} diff --git a/core/src/test/java/org/elasticsearch/ingest/processor/RenameProcessorFactoryTests.java b/core/src/test/java/org/elasticsearch/ingest/processor/RenameProcessorFactoryTests.java new file mode 100644 index 00000000000..21f5c663671 --- /dev/null +++ b/core/src/test/java/org/elasticsearch/ingest/processor/RenameProcessorFactoryTests.java @@ -0,0 +1,68 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.ingest.processor; + +import org.elasticsearch.ingest.core.AbstractProcessorFactory; +import org.elasticsearch.test.ESTestCase; + +import java.util.HashMap; +import java.util.Map; + +import static org.hamcrest.CoreMatchers.equalTo; + +public class RenameProcessorFactoryTests extends ESTestCase { + + public void testCreate() throws Exception { + RenameProcessor.Factory factory = new RenameProcessor.Factory(); + Map config = new HashMap<>(); + config.put("field", "old_field"); + config.put("to", "new_field"); + String processorTag = randomAsciiOfLength(10); + config.put(AbstractProcessorFactory.TAG_KEY, processorTag); + RenameProcessor renameProcessor = factory.create(config); + assertThat(renameProcessor.getTag(), equalTo(processorTag)); + assertThat(renameProcessor.getOldFieldName(), equalTo("old_field")); + assertThat(renameProcessor.getNewFieldName(), equalTo("new_field")); + } + + public void testCreateNoFieldPresent() throws Exception { + RenameProcessor.Factory factory = new RenameProcessor.Factory(); + Map config = new HashMap<>(); + config.put("to", "new_field"); + try { + factory.create(config); + fail("factory create should have failed"); + } catch(IllegalArgumentException e) { + assertThat(e.getMessage(), equalTo("required property [field] is missing")); + } + } + + public void testCreateNoToPresent() throws Exception { + RenameProcessor.Factory factory = new RenameProcessor.Factory(); + Map config = new HashMap<>(); + config.put("field", "old_field"); + try { + factory.create(config); + fail("factory create should have failed"); + } catch(IllegalArgumentException e) { + assertThat(e.getMessage(), equalTo("required property [to] is missing")); + } + } +} diff --git a/core/src/test/java/org/elasticsearch/ingest/processor/RenameProcessorTests.java b/core/src/test/java/org/elasticsearch/ingest/processor/RenameProcessorTests.java new file mode 100644 index 00000000000..1f9bddaac6b --- /dev/null +++ b/core/src/test/java/org/elasticsearch/ingest/processor/RenameProcessorTests.java @@ -0,0 +1,173 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.ingest.processor; + +import org.elasticsearch.ingest.core.IngestDocument; +import org.elasticsearch.ingest.RandomDocumentPicks; +import org.elasticsearch.ingest.core.Processor; +import org.elasticsearch.test.ESTestCase; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.instanceOf; +import static org.hamcrest.Matchers.nullValue; + +public class RenameProcessorTests extends ESTestCase { + + public void testRename() throws Exception { + IngestDocument ingestDocument = RandomDocumentPicks.randomIngestDocument(random()); + String fieldName = RandomDocumentPicks.randomExistingFieldName(random(), ingestDocument); + Object fieldValue = ingestDocument.getFieldValue(fieldName, Object.class); + String newFieldName; + do { + newFieldName = RandomDocumentPicks.randomFieldName(random()); + } while (RandomDocumentPicks.canAddField(newFieldName, ingestDocument) == false || newFieldName.equals(fieldName)); + Processor processor = new RenameProcessor(randomAsciiOfLength(10), fieldName, newFieldName); + processor.execute(ingestDocument); + assertThat(ingestDocument.getFieldValue(newFieldName, Object.class), equalTo(fieldValue)); + } + + public void testRenameArrayElement() throws Exception { + Map document = new HashMap<>(); + List list = new ArrayList<>(); + list.add("item1"); + list.add("item2"); + list.add("item3"); + document.put("list", list); + List> one = new ArrayList<>(); + one.add(Collections.singletonMap("one", "one")); + one.add(Collections.singletonMap("two", "two")); + document.put("one", one); + IngestDocument ingestDocument = RandomDocumentPicks.randomIngestDocument(random(), document); + + Processor processor = new RenameProcessor(randomAsciiOfLength(10), "list.0", "item"); + processor.execute(ingestDocument); + Object actualObject = ingestDocument.getSourceAndMetadata().get("list"); + assertThat(actualObject, instanceOf(List.class)); + @SuppressWarnings("unchecked") + List actualList = (List) actualObject; + assertThat(actualList.size(), equalTo(2)); + assertThat(actualList.get(0), equalTo("item2")); + assertThat(actualList.get(1), equalTo("item3")); + actualObject = ingestDocument.getSourceAndMetadata().get("item"); + assertThat(actualObject, instanceOf(String.class)); + assertThat(actualObject, equalTo("item1")); + + processor = new RenameProcessor(randomAsciiOfLength(10), "list.0", "list.3"); + try { + processor.execute(ingestDocument); + fail("processor execute should have failed"); + } catch(IllegalArgumentException e) { + assertThat(e.getMessage(), equalTo("[3] is out of bounds for array with length [2] as part of path [list.3]")); + assertThat(actualList.size(), equalTo(2)); + assertThat(actualList.get(0), equalTo("item2")); + assertThat(actualList.get(1), equalTo("item3")); + } + } + + public void testRenameNonExistingField() throws Exception { + IngestDocument ingestDocument = RandomDocumentPicks.randomIngestDocument(random(), new HashMap<>()); + String fieldName = RandomDocumentPicks.randomFieldName(random()); + Processor processor = new RenameProcessor(randomAsciiOfLength(10), fieldName, RandomDocumentPicks.randomFieldName(random())); + try { + processor.execute(ingestDocument); + fail("processor execute should have failed"); + } catch(IllegalArgumentException e) { + assertThat(e.getMessage(), equalTo("field [" + fieldName + "] doesn't exist")); + } + } + + public void testRenameNewFieldAlreadyExists() throws Exception { + IngestDocument ingestDocument = RandomDocumentPicks.randomIngestDocument(random()); + String fieldName = RandomDocumentPicks.randomExistingFieldName(random(), ingestDocument); + Processor processor = new RenameProcessor(randomAsciiOfLength(10), RandomDocumentPicks.randomExistingFieldName(random(), ingestDocument), fieldName); + try { + processor.execute(ingestDocument); + fail("processor execute should have failed"); + } catch(IllegalArgumentException e) { + assertThat(e.getMessage(), equalTo("field [" + fieldName + "] already exists")); + } + } + + public void testRenameExistingFieldNullValue() throws Exception { + IngestDocument ingestDocument = RandomDocumentPicks.randomIngestDocument(random(), new HashMap<>()); + String fieldName = RandomDocumentPicks.randomFieldName(random()); + ingestDocument.setFieldValue(fieldName, null); + String newFieldName = RandomDocumentPicks.randomFieldName(random()); + Processor processor = new RenameProcessor(randomAsciiOfLength(10), fieldName, newFieldName); + processor.execute(ingestDocument); + assertThat(ingestDocument.hasField(fieldName), equalTo(false)); + assertThat(ingestDocument.hasField(newFieldName), equalTo(true)); + assertThat(ingestDocument.getFieldValue(newFieldName, Object.class), nullValue()); + } + + public void testRenameAtomicOperationSetFails() throws Exception { + Map source = new HashMap() { + @Override + public Object put(String key, Object value) { + if (key.equals("new_field")) { + throw new UnsupportedOperationException(); + } + return super.put(key, value); + } + }; + source.put("list", Collections.singletonList("item")); + + IngestDocument ingestDocument = new IngestDocument(source, Collections.emptyMap()); + Processor processor = new RenameProcessor(randomAsciiOfLength(10), "list", "new_field"); + try { + processor.execute(ingestDocument); + fail("processor execute should have failed"); + } catch(UnsupportedOperationException e) { + //the set failed, the old field has not been removed + assertThat(ingestDocument.getSourceAndMetadata().containsKey("list"), equalTo(true)); + assertThat(ingestDocument.getSourceAndMetadata().containsKey("new_field"), equalTo(false)); + } + } + + public void testRenameAtomicOperationRemoveFails() throws Exception { + Map source = new HashMap() { + @Override + public Object remove(Object key) { + if (key.equals("list")) { + throw new UnsupportedOperationException(); + } + return super.remove(key); + } + }; + source.put("list", Collections.singletonList("item")); + + IngestDocument ingestDocument = new IngestDocument(source, Collections.emptyMap()); + Processor processor = new RenameProcessor(randomAsciiOfLength(10), "list", "new_field"); + try { + processor.execute(ingestDocument); + fail("processor execute should have failed"); + } catch (UnsupportedOperationException e) { + //the set failed, the old field has not been removed + assertThat(ingestDocument.getSourceAndMetadata().containsKey("list"), equalTo(true)); + assertThat(ingestDocument.getSourceAndMetadata().containsKey("new_field"), equalTo(false)); + } + } +} diff --git a/core/src/test/java/org/elasticsearch/ingest/processor/SetProcessorFactoryTests.java b/core/src/test/java/org/elasticsearch/ingest/processor/SetProcessorFactoryTests.java new file mode 100644 index 00000000000..a58ee491a7c --- /dev/null +++ b/core/src/test/java/org/elasticsearch/ingest/processor/SetProcessorFactoryTests.java @@ -0,0 +1,88 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.ingest.processor; + +import org.elasticsearch.ingest.TestTemplateService; +import org.elasticsearch.ingest.core.AbstractProcessorFactory; +import org.elasticsearch.test.ESTestCase; +import org.junit.Before; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +import static org.hamcrest.CoreMatchers.equalTo; + +public class SetProcessorFactoryTests extends ESTestCase { + + private SetProcessor.Factory factory; + + @Before + public void init() { + factory = new SetProcessor.Factory(TestTemplateService.instance()); + } + + public void testCreate() throws Exception { + Map config = new HashMap<>(); + config.put("field", "field1"); + config.put("value", "value1"); + String processorTag = randomAsciiOfLength(10); + config.put(AbstractProcessorFactory.TAG_KEY, processorTag); + SetProcessor setProcessor = factory.create(config); + assertThat(setProcessor.getTag(), equalTo(processorTag)); + assertThat(setProcessor.getField().execute(Collections.emptyMap()), equalTo("field1")); + assertThat(setProcessor.getValue().copyAndResolve(Collections.emptyMap()), equalTo("value1")); + } + + public void testCreateNoFieldPresent() throws Exception { + Map config = new HashMap<>(); + config.put("value", "value1"); + try { + factory.create(config); + fail("factory create should have failed"); + } catch(IllegalArgumentException e) { + assertThat(e.getMessage(), equalTo("required property [field] is missing")); + } + } + + public void testCreateNoValuePresent() throws Exception { + Map config = new HashMap<>(); + config.put("field", "field1"); + try { + factory.create(config); + fail("factory create should have failed"); + } catch(IllegalArgumentException e) { + assertThat(e.getMessage(), equalTo("required property [value] is missing")); + } + } + + public void testCreateNullValue() throws Exception { + Map config = new HashMap<>(); + config.put("field", "field1"); + config.put("value", null); + try { + factory.create(config); + fail("factory create should have failed"); + } catch(IllegalArgumentException e) { + assertThat(e.getMessage(), equalTo("required property [value] is missing")); + } + } + +} diff --git a/core/src/test/java/org/elasticsearch/ingest/processor/SetProcessorTests.java b/core/src/test/java/org/elasticsearch/ingest/processor/SetProcessorTests.java new file mode 100644 index 00000000000..283825cdad8 --- /dev/null +++ b/core/src/test/java/org/elasticsearch/ingest/processor/SetProcessorTests.java @@ -0,0 +1,83 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.ingest.processor; + +import org.elasticsearch.ingest.core.IngestDocument; +import org.elasticsearch.ingest.RandomDocumentPicks; +import org.elasticsearch.ingest.core.TemplateService; +import org.elasticsearch.ingest.TestTemplateService; +import org.elasticsearch.ingest.core.ValueSource; +import org.elasticsearch.ingest.core.Processor; +import org.elasticsearch.test.ESTestCase; +import org.hamcrest.Matchers; + +import java.util.HashMap; + +import static org.hamcrest.Matchers.equalTo; + +public class SetProcessorTests extends ESTestCase { + + public void testSetExistingFields() throws Exception { + IngestDocument ingestDocument = RandomDocumentPicks.randomIngestDocument(random()); + String fieldName = RandomDocumentPicks.randomExistingFieldName(random(), ingestDocument); + Object fieldValue = RandomDocumentPicks.randomFieldValue(random()); + Processor processor = createSetProcessor(fieldName, fieldValue); + processor.execute(ingestDocument); + assertThat(ingestDocument.hasField(fieldName), equalTo(true)); + assertThat(ingestDocument.getFieldValue(fieldName, Object.class), equalTo(fieldValue)); + } + + public void testSetNewFields() throws Exception { + IngestDocument ingestDocument = RandomDocumentPicks.randomIngestDocument(random(), new HashMap<>()); + //used to verify that there are no conflicts between subsequent fields going to be added + IngestDocument testIngestDocument = RandomDocumentPicks.randomIngestDocument(random(), new HashMap<>()); + Object fieldValue = RandomDocumentPicks.randomFieldValue(random()); + String fieldName = RandomDocumentPicks.addRandomField(random(), testIngestDocument, fieldValue); + Processor processor = createSetProcessor(fieldName, fieldValue); + processor.execute(ingestDocument); + assertThat(ingestDocument.hasField(fieldName), equalTo(true)); + assertThat(ingestDocument.getFieldValue(fieldName, Object.class), equalTo(fieldValue)); + } + + public void testSetFieldsTypeMismatch() throws Exception { + IngestDocument ingestDocument = RandomDocumentPicks.randomIngestDocument(random(), new HashMap<>()); + ingestDocument.setFieldValue("field", "value"); + Processor processor = createSetProcessor("field.inner", "value"); + try { + processor.execute(ingestDocument); + fail("processor execute should have failed"); + } catch(IllegalArgumentException e) { + assertThat(e.getMessage(), equalTo("cannot set [inner] with parent object of type [java.lang.String] as part of path [field.inner]")); + } + } + + public void testSetMetadata() throws Exception { + IngestDocument.MetaData randomMetaData = randomFrom(IngestDocument.MetaData.values()); + Processor processor = createSetProcessor(randomMetaData.getFieldName(), "_value"); + IngestDocument ingestDocument = RandomDocumentPicks.randomIngestDocument(random()); + processor.execute(ingestDocument); + assertThat(ingestDocument.getFieldValue(randomMetaData.getFieldName(), String.class), Matchers.equalTo("_value")); + } + + private static Processor createSetProcessor(String fieldName, Object fieldValue) { + TemplateService templateService = TestTemplateService.instance(); + return new SetProcessor(randomAsciiOfLength(10), templateService.compile(fieldName), ValueSource.wrap(fieldValue, templateService)); + } +} diff --git a/core/src/test/java/org/elasticsearch/ingest/processor/SplitProcessorFactoryTests.java b/core/src/test/java/org/elasticsearch/ingest/processor/SplitProcessorFactoryTests.java new file mode 100644 index 00000000000..7267544c1ff --- /dev/null +++ b/core/src/test/java/org/elasticsearch/ingest/processor/SplitProcessorFactoryTests.java @@ -0,0 +1,68 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.ingest.processor; + +import org.elasticsearch.ingest.core.AbstractProcessorFactory; +import org.elasticsearch.test.ESTestCase; + +import java.util.HashMap; +import java.util.Map; + +import static org.hamcrest.CoreMatchers.equalTo; + +public class SplitProcessorFactoryTests extends ESTestCase { + + public void testCreate() throws Exception { + SplitProcessor.Factory factory = new SplitProcessor.Factory(); + Map config = new HashMap<>(); + config.put("field", "field1"); + config.put("separator", "\\."); + String processorTag = randomAsciiOfLength(10); + config.put(AbstractProcessorFactory.TAG_KEY, processorTag); + SplitProcessor splitProcessor = factory.create(config); + assertThat(splitProcessor.getTag(), equalTo(processorTag)); + assertThat(splitProcessor.getField(), equalTo("field1")); + assertThat(splitProcessor.getSeparator(), equalTo("\\.")); + } + + public void testCreateNoFieldPresent() throws Exception { + SplitProcessor.Factory factory = new SplitProcessor.Factory(); + Map config = new HashMap<>(); + config.put("separator", "\\."); + try { + factory.create(config); + fail("factory create should have failed"); + } catch(IllegalArgumentException e) { + assertThat(e.getMessage(), equalTo("required property [field] is missing")); + } + } + + public void testCreateNoSeparatorPresent() throws Exception { + SplitProcessor.Factory factory = new SplitProcessor.Factory(); + Map config = new HashMap<>(); + config.put("field", "field1"); + try { + factory.create(config); + fail("factory create should have failed"); + } catch(IllegalArgumentException e) { + assertThat(e.getMessage(), equalTo("required property [separator] is missing")); + } + } +} diff --git a/core/src/test/java/org/elasticsearch/ingest/processor/SplitProcessorTests.java b/core/src/test/java/org/elasticsearch/ingest/processor/SplitProcessorTests.java new file mode 100644 index 00000000000..e1c8a626a4b --- /dev/null +++ b/core/src/test/java/org/elasticsearch/ingest/processor/SplitProcessorTests.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.ingest.processor; + +import org.elasticsearch.ingest.RandomDocumentPicks; +import org.elasticsearch.ingest.core.IngestDocument; +import org.elasticsearch.ingest.core.Processor; +import org.elasticsearch.test.ESTestCase; + +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.equalTo; + +public class SplitProcessorTests extends ESTestCase { + + public void testSplit() throws Exception { + IngestDocument ingestDocument = RandomDocumentPicks.randomIngestDocument(random()); + String fieldName = RandomDocumentPicks.addRandomField(random(), ingestDocument, "127.0.0.1"); + Processor processor = new SplitProcessor(randomAsciiOfLength(10), fieldName, "\\."); + processor.execute(ingestDocument); + assertThat(ingestDocument.getFieldValue(fieldName, List.class), equalTo(Arrays.asList("127", "0", "0", "1"))); + } + + public void testSplitFieldNotFound() throws Exception { + IngestDocument ingestDocument = RandomDocumentPicks.randomIngestDocument(random(), new HashMap<>()); + String fieldName = RandomDocumentPicks.randomFieldName(random()); + Processor processor = new SplitProcessor(randomAsciiOfLength(10), fieldName, "\\."); + try { + processor.execute(ingestDocument); + fail("split processor should have failed"); + } catch (IllegalArgumentException e) { + assertThat(e.getMessage(), containsString("not present as part of path [" + fieldName + "]")); + } + } + + public void testSplitNullValue() throws Exception { + IngestDocument ingestDocument = RandomDocumentPicks.randomIngestDocument(random(), Collections.singletonMap("field", null)); + Processor processor = new SplitProcessor(randomAsciiOfLength(10), "field", "\\."); + try { + processor.execute(ingestDocument); + fail("split processor should have failed"); + } catch (IllegalArgumentException e) { + assertThat(e.getMessage(), equalTo("field [field] is null, cannot split.")); + } + } + + public void testSplitNonStringValue() throws Exception { + IngestDocument ingestDocument = RandomDocumentPicks.randomIngestDocument(random(), new HashMap<>()); + String fieldName = RandomDocumentPicks.randomFieldName(random()); + ingestDocument.setFieldValue(fieldName, randomInt()); + Processor processor = new SplitProcessor(randomAsciiOfLength(10), fieldName, "\\."); + try { + processor.execute(ingestDocument); + fail("split processor should have failed"); + } catch (IllegalArgumentException e) { + assertThat(e.getMessage(), equalTo("field [" + fieldName + "] of type [java.lang.Integer] cannot be cast to [java.lang.String]")); + } + } + + public void testSplitAppendable() throws Exception { + Map splitConfig = new HashMap<>(); + splitConfig.put("field", "flags"); + splitConfig.put("separator", "\\|"); + Processor splitProcessor = (new SplitProcessor.Factory()).create(splitConfig); + Map source = new HashMap<>(); + source.put("flags", "new|hot|super|fun|interesting"); + IngestDocument ingestDocument = new IngestDocument(source, new HashMap<>()); + splitProcessor.execute(ingestDocument); + @SuppressWarnings("unchecked") + List flags = (List)ingestDocument.getFieldValue("flags", List.class); + assertThat(flags, equalTo(Arrays.asList("new", "hot", "super", "fun", "interesting"))); + ingestDocument.appendFieldValue("flags", "additional_flag"); + assertThat(ingestDocument.getFieldValue("flags", List.class), equalTo(Arrays.asList("new", "hot", "super", "fun", "interesting", "additional_flag"))); + } +} diff --git a/core/src/test/java/org/elasticsearch/ingest/processor/TrimProcessorFactoryTests.java b/core/src/test/java/org/elasticsearch/ingest/processor/TrimProcessorFactoryTests.java new file mode 100644 index 00000000000..350aaa66e6d --- /dev/null +++ b/core/src/test/java/org/elasticsearch/ingest/processor/TrimProcessorFactoryTests.java @@ -0,0 +1,53 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.ingest.processor; + +import org.elasticsearch.ingest.core.AbstractProcessorFactory; +import org.elasticsearch.test.ESTestCase; + +import java.util.HashMap; +import java.util.Map; + +import static org.hamcrest.CoreMatchers.equalTo; + +public class TrimProcessorFactoryTests extends ESTestCase { + + public void testCreate() throws Exception { + TrimProcessor.Factory factory = new TrimProcessor.Factory(); + Map config = new HashMap<>(); + config.put("field", "field1"); + String processorTag = randomAsciiOfLength(10); + config.put(AbstractProcessorFactory.TAG_KEY, processorTag); + TrimProcessor uppercaseProcessor = factory.create(config); + assertThat(uppercaseProcessor.getTag(), equalTo(processorTag)); + assertThat(uppercaseProcessor.getField(), equalTo("field1")); + } + + public void testCreateMissingField() throws Exception { + TrimProcessor.Factory factory = new TrimProcessor.Factory(); + Map config = new HashMap<>(); + try { + factory.create(config); + fail("factory create should have failed"); + } catch(IllegalArgumentException e) { + assertThat(e.getMessage(), equalTo("required property [field] is missing")); + } + } +} diff --git a/core/src/test/java/org/elasticsearch/ingest/processor/TrimProcessorTests.java b/core/src/test/java/org/elasticsearch/ingest/processor/TrimProcessorTests.java new file mode 100644 index 00000000000..a0e5fdeab2b --- /dev/null +++ b/core/src/test/java/org/elasticsearch/ingest/processor/TrimProcessorTests.java @@ -0,0 +1,50 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.ingest.processor; + +public class TrimProcessorTests extends AbstractStringProcessorTestCase { + + @Override + protected AbstractStringProcessor newProcessor(String field) { + return new TrimProcessor(randomAsciiOfLength(10), field); + } + + @Override + protected String modifyInput(String input) { + String updatedFieldValue = ""; + updatedFieldValue = addWhitespaces(updatedFieldValue); + updatedFieldValue += input; + updatedFieldValue = addWhitespaces(updatedFieldValue); + return updatedFieldValue; + } + + @Override + protected String expectedResult(String input) { + return input.trim(); + } + + private static String addWhitespaces(String input) { + int prefixLength = randomIntBetween(0, 10); + for (int i = 0; i < prefixLength; i++) { + input += ' '; + } + return input; + } +} diff --git a/core/src/test/java/org/elasticsearch/ingest/processor/UppercaseProcessorFactoryTests.java b/core/src/test/java/org/elasticsearch/ingest/processor/UppercaseProcessorFactoryTests.java new file mode 100644 index 00000000000..2220438c75f --- /dev/null +++ b/core/src/test/java/org/elasticsearch/ingest/processor/UppercaseProcessorFactoryTests.java @@ -0,0 +1,53 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.ingest.processor; + +import org.elasticsearch.ingest.core.AbstractProcessorFactory; +import org.elasticsearch.test.ESTestCase; + +import java.util.HashMap; +import java.util.Map; + +import static org.hamcrest.CoreMatchers.equalTo; + +public class UppercaseProcessorFactoryTests extends ESTestCase { + + public void testCreate() throws Exception { + UppercaseProcessor.Factory factory = new UppercaseProcessor.Factory(); + Map config = new HashMap<>(); + config.put("field", "field1"); + String processorTag = randomAsciiOfLength(10); + config.put(AbstractProcessorFactory.TAG_KEY, processorTag); + UppercaseProcessor uppercaseProcessor = factory.create(config); + assertThat(uppercaseProcessor.getTag(), equalTo(processorTag)); + assertThat(uppercaseProcessor.getField(), equalTo("field1")); + } + + public void testCreateMissingField() throws Exception { + UppercaseProcessor.Factory factory = new UppercaseProcessor.Factory(); + Map config = new HashMap<>(); + try { + factory.create(config); + fail("factory create should have failed"); + } catch(IllegalArgumentException e) { + assertThat(e.getMessage(), equalTo("required property [field] is missing")); + } + } +} diff --git a/core/src/test/java/org/elasticsearch/ingest/processor/UppercaseProcessorTests.java b/core/src/test/java/org/elasticsearch/ingest/processor/UppercaseProcessorTests.java new file mode 100644 index 00000000000..4ab61f7b5e3 --- /dev/null +++ b/core/src/test/java/org/elasticsearch/ingest/processor/UppercaseProcessorTests.java @@ -0,0 +1,35 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.ingest.processor; + +import java.util.Locale; + +public class UppercaseProcessorTests extends AbstractStringProcessorTestCase { + + @Override + protected AbstractStringProcessor newProcessor(String field) { + return new UppercaseProcessor(randomAsciiOfLength(10), field); + } + + @Override + protected String expectedResult(String input) { + return input.toUpperCase(Locale.ROOT); + } +} diff --git a/core/src/test/java/org/elasticsearch/percolator/PercolateDocumentParserTests.java b/core/src/test/java/org/elasticsearch/percolator/PercolateDocumentParserTests.java index a5d7ab5ba90..ba503b47640 100644 --- a/core/src/test/java/org/elasticsearch/percolator/PercolateDocumentParserTests.java +++ b/core/src/test/java/org/elasticsearch/percolator/PercolateDocumentParserTests.java @@ -23,7 +23,6 @@ import org.apache.lucene.index.Term; import org.apache.lucene.search.TermQuery; import org.elasticsearch.Version; import org.elasticsearch.action.percolate.PercolateShardRequest; -import org.elasticsearch.cluster.action.index.MappingUpdatedAction; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.settings.Settings; @@ -93,10 +92,7 @@ public class PercolateDocumentParserTests extends ESTestCase { new NamedWriteableRegistry()); AggregationPhase aggregationPhase = new AggregationPhase(new AggregationParseElement(aggregatorParsers, indicesQueriesRegistry), new AggregationBinaryParseElement(aggregatorParsers, indicesQueriesRegistry)); - MappingUpdatedAction mappingUpdatedAction = Mockito.mock(MappingUpdatedAction.class); - parser = new PercolateDocumentParser( - highlightPhase, new SortParseElement(), aggregationPhase, mappingUpdatedAction - ); + parser = new PercolateDocumentParser(highlightPhase, new SortParseElement(), aggregationPhase); request = Mockito.mock(PercolateShardRequest.class); Mockito.when(request.shardId()).thenReturn(new ShardId(new Index("_index"), 0)); diff --git a/core/src/test/java/org/elasticsearch/percolator/PercolatorIT.java b/core/src/test/java/org/elasticsearch/percolator/PercolatorIT.java index 4a15b65382c..22183bdcefc 100644 --- a/core/src/test/java/org/elasticsearch/percolator/PercolatorIT.java +++ b/core/src/test/java/org/elasticsearch/percolator/PercolatorIT.java @@ -175,7 +175,7 @@ public class PercolatorIT extends ESIntegTestCase { } public void testSimple2() throws Exception { - assertAcked(prepareCreate("test").addMapping("type1", "field1", "type=long,doc_values=true")); + assertAcked(prepareCreate("test").addMapping("type1", "field1", "type=long,doc_values=true", "field2", "type=string")); ensureGreen(); // introduce the doc @@ -1577,92 +1577,6 @@ public class PercolatorIT extends ESIntegTestCase { assertEquals(response.getMatches()[0].getId().string(), "Q"); } - public void testPercolationWithDynamicTemplates() throws Exception { - assertAcked(prepareCreate("idx").addMapping("type", jsonBuilder().startObject().startObject("type") - .field("dynamic", false) - .startObject("properties") - .startObject("custom") - .field("dynamic", true) - .field("type", "object") - .field("include_in_all", false) - .endObject() - .endObject() - .startArray("dynamic_templates") - .startObject() - .startObject("custom_fields") - .field("path_match", "custom.*") - .startObject("mapping") - .field("index", "not_analyzed") - .endObject() - .endObject() - .endObject() - .endArray() - .endObject().endObject())); - ensureGreen("idx"); - - try { - client().prepareIndex("idx", PercolatorService.TYPE_NAME, "1") - .setSource(jsonBuilder().startObject().field("query", QueryBuilders.queryStringQuery("color:red")).endObject()) - .get(); - fail(); - } catch (MapperParsingException e) { - } - refresh(); - - PercolateResponse percolateResponse = client().preparePercolate().setDocumentType("type") - .setPercolateDoc(new PercolateSourceBuilder.DocBuilder().setDoc(jsonBuilder().startObject().startObject("custom").field("color", "blue").endObject().endObject())) - .get(); - - assertMatchCount(percolateResponse, 0l); - assertThat(percolateResponse.getMatches(), arrayWithSize(0)); - - // The previous percolate request introduced the custom.color field, so now we register the query again - // and the field name `color` will be resolved to `custom.color` field in mapping via smart field mapping resolving. - client().prepareIndex("idx", PercolatorService.TYPE_NAME, "1") - .setSource(jsonBuilder().startObject().field("query", QueryBuilders.queryStringQuery("custom.color:red")).endObject()) - .get(); - client().prepareIndex("idx", PercolatorService.TYPE_NAME, "2") - .setSource(jsonBuilder().startObject().field("query", QueryBuilders.queryStringQuery("custom.color:blue")).field("type", "type").endObject()) - .get(); - refresh(); - - // The second request will yield a match, since the query during the proper field during parsing. - percolateResponse = client().preparePercolate().setDocumentType("type") - .setPercolateDoc(new PercolateSourceBuilder.DocBuilder().setDoc(jsonBuilder().startObject().startObject("custom").field("color", "blue").endObject().endObject())) - .get(); - - assertMatchCount(percolateResponse, 1l); - assertThat(percolateResponse.getMatches()[0].getId().string(), equalTo("2")); - } - - public void testUpdateMappingDynamicallyWhilePercolating() throws Exception { - createIndex("test"); - ensureSearchable(); - - // percolation source - XContentBuilder percolateDocumentSource = XContentFactory.jsonBuilder().startObject().startObject("doc") - .field("field1", 1) - .field("field2", "value") - .endObject().endObject(); - - PercolateResponse response = client().preparePercolate() - .setIndices("test").setDocumentType("type1") - .setSource(percolateDocumentSource).execute().actionGet(); - assertAllSuccessful(response); - assertMatchCount(response, 0l); - assertThat(response.getMatches(), arrayWithSize(0)); - - assertMappingOnMaster("test", "type1"); - - GetMappingsResponse mappingsResponse = client().admin().indices().prepareGetMappings("test").get(); - assertThat(mappingsResponse.getMappings().get("test"), notNullValue()); - assertThat(mappingsResponse.getMappings().get("test").get("type1"), notNullValue()); - assertThat(mappingsResponse.getMappings().get("test").get("type1").getSourceAsMap().isEmpty(), is(false)); - Map properties = (Map) mappingsResponse.getMappings().get("test").get("type1").getSourceAsMap().get("properties"); - assertThat(((Map) properties.get("field1")).get("type"), equalTo("long")); - assertThat(((Map) properties.get("field2")).get("type"), equalTo("string")); - } - public void testDontReportDeletedPercolatorDocs() throws Exception { client().admin().indices().prepareCreate("test").execute().actionGet(); ensureGreen(); diff --git a/core/src/test/java/org/elasticsearch/script/FileScriptTests.java b/core/src/test/java/org/elasticsearch/script/FileScriptTests.java index 8ef7bcc41f5..d639411db8c 100644 --- a/core/src/test/java/org/elasticsearch/script/FileScriptTests.java +++ b/core/src/test/java/org/elasticsearch/script/FileScriptTests.java @@ -63,7 +63,8 @@ public class FileScriptTests extends ESTestCase { .put("script.engine." + MockScriptEngine.NAME + ".file.aggs", false) .put("script.engine." + MockScriptEngine.NAME + ".file.search", false) .put("script.engine." + MockScriptEngine.NAME + ".file.mapping", false) - .put("script.engine." + MockScriptEngine.NAME + ".file.update", false).build(); + .put("script.engine." + MockScriptEngine.NAME + ".file.update", false) + .put("script.engine." + MockScriptEngine.NAME + ".file.ingest", false).build(); ScriptService scriptService = makeScriptService(settings); Script script = new Script("script1", ScriptService.ScriptType.FILE, MockScriptEngine.NAME, null); for (ScriptContext context : ScriptContext.Standard.values()) { diff --git a/core/src/test/java/org/elasticsearch/search/basic/SearchWhileCreatingIndexIT.java b/core/src/test/java/org/elasticsearch/search/basic/SearchWhileCreatingIndexIT.java index 28874d2e2a4..35dbde26aba 100644 --- a/core/src/test/java/org/elasticsearch/search/basic/SearchWhileCreatingIndexIT.java +++ b/core/src/test/java/org/elasticsearch/search/basic/SearchWhileCreatingIndexIT.java @@ -29,7 +29,6 @@ import org.elasticsearch.test.ESIntegTestCase; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount; import static org.hamcrest.Matchers.greaterThanOrEqualTo; - /** * This test basically verifies that search with a single shard active (cause we indexed to it) and other * shards possibly not active at all (cause they haven't allocated) will still work. @@ -58,39 +57,44 @@ public class SearchWhileCreatingIndexIT extends ESIntegTestCase { int shardsNo = numberOfReplicas + 1; int neededNodes = shardsNo <= 2 ? 1 : shardsNo / 2 + 1; internalCluster().ensureAtLeastNumDataNodes(randomIntBetween(neededNodes, shardsNo)); - for (int i = 0; i < 20; i++) { - logger.info("running iteration {}", i); - if (createIndex) { - createIndex("test"); - } - client().prepareIndex("test", "type1", randomAsciiOfLength(5)).setSource("field", "test").execute().actionGet(); - RefreshResponse refreshResponse = client().admin().indices().prepareRefresh("test").execute().actionGet(); - assertThat(refreshResponse.getSuccessfulShards(), greaterThanOrEqualTo(1)); // at least one shard should be successful when refreshing - // we want to make sure that while recovery happens, and a replica gets recovered, its properly refreshed - ClusterHealthStatus status = ClusterHealthStatus.RED; - while (status != ClusterHealthStatus.GREEN) { - // first, verify that search on the primary search works - SearchResponse searchResponse = client().prepareSearch("test").setPreference("_primary").setQuery(QueryBuilders.termQuery("field", "test")).execute().actionGet(); - assertHitCount(searchResponse, 1); - // now, let it go to primary or replica, though in a randomized re-creatable manner - String preference = randomAsciiOfLength(5); - Client client = client(); - searchResponse = client.prepareSearch("test").setPreference(preference).setQuery(QueryBuilders.termQuery("field", "test")).execute().actionGet(); - if (searchResponse.getHits().getTotalHits() != 1) { - refresh(); - SearchResponse searchResponseAfterRefresh = client.prepareSearch("test").setPreference(preference).setQuery(QueryBuilders.termQuery("field", "test")).execute().actionGet(); - logger.info("hits count mismatch on any shard search failed, post explicit refresh hits are {}", searchResponseAfterRefresh.getHits().getTotalHits()); - ensureGreen(); - SearchResponse searchResponseAfterGreen = client.prepareSearch("test").setPreference(preference).setQuery(QueryBuilders.termQuery("field", "test")).execute().actionGet(); - logger.info("hits count mismatch on any shard search failed, post explicit wait for green hits are {}", searchResponseAfterGreen.getHits().getTotalHits()); - assertHitCount(searchResponse, 1); - } - assertHitCount(searchResponse, 1); - status = client().admin().cluster().prepareHealth("test").get().getStatus(); - internalCluster().ensureAtLeastNumDataNodes(numberOfReplicas + 1); - } - cluster().wipeIndices("test"); + String id = randomAsciiOfLength(5); + // we will go the primary or the replica, but in a + // randomized re-creatable manner + int counter = 0; + String preference = randomAsciiOfLength(5); + + logger.info("running iteration for id {}, preference {}", id, preference); + + if (createIndex) { + createIndex("test"); } + client().prepareIndex("test", "type1", id).setSource("field", "test").execute().actionGet(); + RefreshResponse refreshResponse = client().admin().indices().prepareRefresh("test").execute().actionGet(); + assertThat(refreshResponse.getSuccessfulShards(), greaterThanOrEqualTo(1)); // at least one shard should be successful when refreshing + + logger.info("using preference {}", preference); + // we want to make sure that while recovery happens, and a replica gets recovered, its properly refreshed + ClusterHealthStatus status = ClusterHealthStatus.RED; + while (status != ClusterHealthStatus.GREEN) { + // first, verify that search on the primary search works + SearchResponse searchResponse = client().prepareSearch("test").setPreference("_primary").setQuery(QueryBuilders.termQuery("field", "test")).execute().actionGet(); + assertHitCount(searchResponse, 1); + Client client = client(); + searchResponse = client.prepareSearch("test").setPreference(preference + Integer.toString(counter++)).setQuery(QueryBuilders.termQuery("field", "test")).execute().actionGet(); + if (searchResponse.getHits().getTotalHits() != 1) { + refresh(); + SearchResponse searchResponseAfterRefresh = client.prepareSearch("test").setPreference(preference).setQuery(QueryBuilders.termQuery("field", "test")).execute().actionGet(); + logger.info("hits count mismatch on any shard search failed, post explicit refresh hits are {}", searchResponseAfterRefresh.getHits().getTotalHits()); + ensureGreen(); + SearchResponse searchResponseAfterGreen = client.prepareSearch("test").setPreference(preference).setQuery(QueryBuilders.termQuery("field", "test")).execute().actionGet(); + logger.info("hits count mismatch on any shard search failed, post explicit wait for green hits are {}", searchResponseAfterGreen.getHits().getTotalHits()); + assertHitCount(searchResponse, 1); + } + assertHitCount(searchResponse, 1); + status = client().admin().cluster().prepareHealth("test").get().getStatus(); + internalCluster().ensureAtLeastNumDataNodes(numberOfReplicas + 1); + } + cluster().wipeIndices("test"); } } diff --git a/core/src/test/java/org/elasticsearch/search/suggest/CompletionSuggestSearchIT.java b/core/src/test/java/org/elasticsearch/search/suggest/CompletionSuggestSearchIT.java index fac7f71446a..1543433be32 100644 --- a/core/src/test/java/org/elasticsearch/search/suggest/CompletionSuggestSearchIT.java +++ b/core/src/test/java/org/elasticsearch/search/suggest/CompletionSuggestSearchIT.java @@ -230,7 +230,7 @@ public class CompletionSuggestSearchIT extends ESIntegTestCase { SuggestResponse suggestResponse = client().suggest(request).get(); assertThat(suggestResponse.getSuccessfulShards(), equalTo(0)); for (ShardOperationFailedException exception : suggestResponse.getShardFailures()) { - assertThat(exception.reason(), containsString("ParsingException[[completion] failed to parse field [payload]]; nested: IllegalStateException[expected value but got [START_OBJECT]]")); + assertThat(exception.reason(), containsString("ParsingException[[completion] failed to parse field [payload]]; nested: IllegalStateException[Can't get text on a START_OBJECT")); } } diff --git a/core/src/test/java/org/elasticsearch/threadpool/UpdateThreadPoolSettingsTests.java b/core/src/test/java/org/elasticsearch/threadpool/UpdateThreadPoolSettingsTests.java index 09653c12e07..2fe11b5875c 100644 --- a/core/src/test/java/org/elasticsearch/threadpool/UpdateThreadPoolSettingsTests.java +++ b/core/src/test/java/org/elasticsearch/threadpool/UpdateThreadPoolSettingsTests.java @@ -29,6 +29,7 @@ import org.elasticsearch.threadpool.ThreadPool.Names; import java.lang.reflect.Field; import java.util.Arrays; import java.util.HashSet; +import java.util.Map; import java.util.Set; import java.util.concurrent.CountDownLatch; import java.util.concurrent.Executor; @@ -47,6 +48,7 @@ import static org.hamcrest.Matchers.sameInstance; /** */ public class UpdateThreadPoolSettingsTests extends ESTestCase { + public void testCorrectThreadPoolTypePermittedInSettings() throws InterruptedException { String threadPoolName = randomThreadPoolName(); ThreadPool.ThreadPoolType correctThreadPoolType = ThreadPool.THREAD_POOL_TYPES.get(threadPoolName); @@ -452,11 +454,10 @@ public class UpdateThreadPoolSettingsTests extends ESTestCase { Set set = new HashSet<>(); set.addAll(Arrays.asList(ThreadPool.ThreadPoolType.values())); set.remove(ThreadPool.THREAD_POOL_TYPES.get(threadPoolName)); - ThreadPool.ThreadPoolType invalidThreadPoolType = randomFrom(set.toArray(new ThreadPool.ThreadPoolType[set.size()])); - return invalidThreadPoolType; + return randomFrom(set.toArray(new ThreadPool.ThreadPoolType[set.size()])); } private String randomThreadPool(ThreadPool.ThreadPoolType type) { - return randomFrom(ThreadPool.THREAD_POOL_TYPES.entrySet().stream().filter(t -> t.getValue().equals(type)).map(t -> t.getKey()).collect(Collectors.toList())); + return randomFrom(ThreadPool.THREAD_POOL_TYPES.entrySet().stream().filter(t -> t.getValue().equals(type)).map(Map.Entry::getKey).collect(Collectors.toList())); } } diff --git a/docs/plugins/ingest-geoip.asciidoc b/docs/plugins/ingest-geoip.asciidoc new file mode 100644 index 00000000000..539c29971a4 --- /dev/null +++ b/docs/plugins/ingest-geoip.asciidoc @@ -0,0 +1,64 @@ +[[ingest-geoip]] +== Ingest Geoip Processor Plugin + +The GeoIP processor adds information about the geographical location of IP addresses, based on data from the Maxmind databases. +This processor adds this information by default under the `geoip` field. + +The ingest plugin ships by default with the GeoLite2 City and GeoLite2 Country geoip2 databases from Maxmind made available +under the CCA-ShareAlike 3.0 license. For more details see, http://dev.maxmind.com/geoip/geoip2/geolite2/ + +The GeoIP processor can run with other geoip2 databases from Maxmind. The files must be copied into the geoip config directory +and the `database_file` option should be used to specify the filename of the custom database. The geoip config directory +is located at `$ES_HOME/config/ingest/geoip` and holds the shipped databases too. + +[[geoip-options]] +.Geoip options +[options="header"] +|====== +| Name | Required | Default | Description +| `source_field` | yes | - | The field to get the ip address or hostname from for the geographical lookup. +| `target_field` | no | geoip | The field that will hold the geographical information looked up from the Maxmind database. +| `database_file` | no | GeoLite2-City.mmdb | The database filename in the geoip config directory. The ingest plugin ships with the GeoLite2-City.mmdb and GeoLite2-Country.mmdb files. +| `fields` | no | [`continent_name`, `country_iso_code`, `region_name`, `city_name`, `location`] <1> | Controls what properties are added to the `target_field` based on the geoip lookup. +|====== + +<1> Depends on what is available in `database_field`: +* If the GeoLite2 City database is used then the following fields may be added under the `target_field`: `ip`, +`country_iso_code`, `country_name`, `continent_name`, `region_name`, `city_name`, `timezone`, `latitude`, `longitude` +and `location`. The fields actually added depend on what has been found and which fields were configured in `fields`. +* If the GeoLite2 Country database is used then the following fields may be added under the `target_field`: `ip`, +`country_iso_code`, `country_name` and `continent_name`.The fields actually added depend on what has been found and which fields were configured in `fields`. + +An example that uses the default city database and adds the geographical information to the `geoip` field based on the `ip` field: + +[source,js] +-------------------------------------------------- +{ + "description" : "...", + "processors" : [ + { + "geoip" : { + "source_field" : "ip" + } + } + ] +} +-------------------------------------------------- + +An example that uses the default country database and add the geographical information to the `geo` field based on the `ip` field`: + +[source,js] +-------------------------------------------------- +{ + "description" : "...", + "processors" : [ + { + "geoip" : { + "source_field" : "ip", + "target_field" : "geo", + "database_file" : "GeoLite2-Country.mmdb" + } + } + ] +} +-------------------------------------------------- diff --git a/docs/reference/aggregations/bucket/filters-aggregation.asciidoc b/docs/reference/aggregations/bucket/filters-aggregation.asciidoc index 322dccb790f..a7e07acc660 100644 --- a/docs/reference/aggregations/bucket/filters-aggregation.asciidoc +++ b/docs/reference/aggregations/bucket/filters-aggregation.asciidoc @@ -146,7 +146,7 @@ The following snippet shows a response where the `other` bucket is requested to "aggs" : { "messages" : { "filters" : { - "other_bucket": "other_messages", + "other_bucket_key": "other_messages", "filters" : { "errors" : { "term" : { "body" : "error" }}, "warnings" : { "term" : { "body" : "warning" }} diff --git a/docs/reference/cluster/stats.asciidoc b/docs/reference/cluster/stats.asciidoc index 8093dd32d7d..3f36ad6df25 100644 --- a/docs/reference/cluster/stats.asciidoc +++ b/docs/reference/cluster/stats.asciidoc @@ -57,15 +57,11 @@ Will return, for example: "memory_size_in_bytes": 0, "evictions": 0 }, - "filter_cache": { + "query_cache": { "memory_size": "0b", "memory_size_in_bytes": 0, "evictions": 0 }, - "id_cache": { - "memory_size": "0b", - "memory_size_in_bytes": 0 - }, "completion": { "size": "0b", "size_in_bytes": 0 diff --git a/docs/reference/docs/bulk.asciidoc b/docs/reference/docs/bulk.asciidoc index ef066eb9bb5..b9b7d4751e6 100644 --- a/docs/reference/docs/bulk.asciidoc +++ b/docs/reference/docs/bulk.asciidoc @@ -131,6 +131,8 @@ operation based on the `_parent` / `_routing` mapping. [[bulk-timestamp]] === Timestamp +deprecated[2.0.0,The `_timestamp` field is deprecated. Instead, use a normal <> field and set its value explicitly] + Each bulk item can include the timestamp value using the `_timestamp`/`timestamp` field. It automatically follows the behavior of the index operation based on the `_timestamp` mapping. @@ -139,6 +141,8 @@ the index operation based on the `_timestamp` mapping. [[bulk-ttl]] === TTL +deprecated[2.0.0,The current `_ttl` implementation is deprecated and will be replaced with a different implementation in a future version] + Each bulk item can include the ttl value using the `_ttl`/`ttl` field. It automatically follows the behavior of the index operation based on the `_ttl` mapping. diff --git a/docs/reference/docs/index_.asciidoc b/docs/reference/docs/index_.asciidoc index 5f79efbcc60..27ac85b9595 100644 --- a/docs/reference/docs/index_.asciidoc +++ b/docs/reference/docs/index_.asciidoc @@ -258,6 +258,8 @@ specified using the `routing` parameter. [[index-timestamp]] === Timestamp +deprecated[2.0.0,The `_timestamp` field is deprecated. Instead, use a normal <> field and set its value explicitly] + A document can be indexed with a `timestamp` associated with it. The `timestamp` value of a document can be set using the `timestamp` parameter. For example: @@ -280,6 +282,8 @@ page>>. [[index-ttl]] === TTL +deprecated[2.0.0,The current `_ttl` implementation is deprecated and will be replaced with a different implementation in a future version] + A document can be indexed with a `ttl` (time to live) associated with it. Expired documents will be expunged automatically. The expiration diff --git a/docs/reference/ingest/ingest.asciidoc b/docs/reference/ingest/ingest.asciidoc new file mode 100644 index 00000000000..0c049f82b69 --- /dev/null +++ b/docs/reference/ingest/ingest.asciidoc @@ -0,0 +1,1103 @@ +[[ingest]] +== Ingest Plugin + +The ingest plugin can be used to pre-process documents before the actual indexing takes place. +This pre-processing happens by the ingest plugin that intercepts bulk and index requests, applies the +transformations and then passes the documents back to the index or bulk APIs. + +The ingest plugin is disabled by default. In order to enable the ingest plugin the following +setting should be configured in the elasticsearch.yml file: + +[source,yaml] +-------------------------------------------------- +node.ingest: true +-------------------------------------------------- + +The ingest plugin can be installed and enabled on any node. It is possible to run ingest +on an master and or data node or have dedicated client nodes that run with ingest. + +In order to pre-process document before indexing the `pipeline` parameter should be used +on an index or bulk request to tell the ingest plugin what pipeline is going to be used. + +[source,js] +-------------------------------------------------- +PUT /my-index/my-type/my-id?pipeline=my_pipeline_id +{ + ... +} +-------------------------------------------------- +// AUTOSENSE + +=== Pipeline Definition + +A pipeline is a definition of a series of processors that are to be +executed in the same sequential order as they are declared. +[source,js] +-------------------------------------------------- +{ + "description" : "...", + "processors" : [ ... ] +} +-------------------------------------------------- + +The `description` is a special field to store a helpful description of +what the pipeline attempts to achieve. + +The `processors` parameter defines a list of processors to be executed in +order. + +=== Processors + +All processors are defined in the following way within a pipeline definition: + +[source,js] +-------------------------------------------------- +{ + "PROCESSOR_NAME" : { + ... processor configuration options ... + } +} +-------------------------------------------------- + +Each processor defines its own configuration parameters, but all processors have +the ability to declare `tag` and `on_failure` fields. These fields are optional. + +A `tag` is simply a string identifier of the specific instatiation of a certain +processor in a pipeline. The `tag` field does not affect any processor's behavior, +but is very useful for bookkeeping and tracing errors to specific processors. + +See <> to learn more about the `on_failure` field and error handling in pipelines. + +==== Set processor +Sets one field and associates it with the specified value. If the field already exists, +its value will be replaced with the provided one. + +[[set-options]] +.Set Options +[options="header"] +|====== +| Name | Required | Default | Description +| `field` | yes | - | The field to insert, upsert, or update +| `value` | yes | - | The value to be set for the field +|====== + +[source,js] +-------------------------------------------------- +{ + "set": { + "field": "field1", + "value": 582.1 + } +} +-------------------------------------------------- + +==== Append processor +Appends one or more values to an existing array if the field already exists and it is an array. +Converts a scalar to an array and appends one or more values to it if the field exists and it is a scalar. +Creates an array containing the provided values if the fields doesn't exist. +Accepts a single value or an array of values. + +[[append-options]] +.Append Options +[options="header"] +|====== +| Name | Required | Default | Description +| `field` | yes | - | The field to be appended to +| `value` | yes | - | The value to be appended +|====== + +[source,js] +-------------------------------------------------- +{ + "append": { + "field": "field1" + "value": ["item2", "item3", "item4"] + } +} +-------------------------------------------------- + +==== Remove processor +Removes an existing field. If the field doesn't exist, an exception will be thrown + +[[remove-options]] +.Remove Options +[options="header"] +|====== +| Name | Required | Default | Description +| `field` | yes | - | The field to be removed +|====== + +[source,js] +-------------------------------------------------- +{ + "remove": { + "field": "foo" + } +} +-------------------------------------------------- + +==== Rename processor +Renames an existing field. If the field doesn't exist, an exception will be thrown. Also, the new field +name must not exist. + +[[rename-options]] +.Rename Options +[options="header"] +|====== +| Name | Required | Default | Description +| `field` | yes | - | The field to be renamed +| `to` | yes | - | The new name of the field +|====== + +[source,js] +-------------------------------------------------- +{ + "rename": { + "field": "foo", + "to": "foobar" + } +} +-------------------------------------------------- + + +==== Convert processor +Converts an existing field's value to a different type, like turning a string to an integer. +If the field value is an array, all members will be converted. + +The supported types include: `integer`, `float`, `string`, and `boolean`. + +`boolean` will set the field to true if its string value is equal to `true` (ignore case), to +false if its string value is equal to `false` (ignore case) and it will throw exception otherwise. + +[[convert-options]] +.Convert Options +[options="header"] +|====== +| Name | Required | Default | Description +| `field` | yes | - | The field whose value is to be converted +| `type` | yes | - | The type to convert the existing value to +|====== + +[source,js] +-------------------------------------------------- +{ + "convert": { + "field" : "foo" + "type": "integer" + } +} +-------------------------------------------------- + +==== Gsub processor +Converts a string field by applying a regular expression and a replacement. +If the field is not a string, the processor will throw an exception. + +[[gsub-options]] +.Gsub Options +[options="header"] +|====== +| Name | Required | Default | Description +| `field` | yes | - | The field apply the replacement for +| `pattern` | yes | - | The pattern to be replaced +| `replacement` | yes | - | The string to replace the matching patterns with. +|====== + +[source,js] +-------------------------------------------------- +{ + "gsub": { + "field": "field1", + "pattern": "\.", + "replacement": "-" + } +} +-------------------------------------------------- + +==== Join processor +Joins each element of an array into a single string using a separator character between each element. +Throws error when the field is not an array. + +[[join-options]] +.Join Options +[options="header"] +|====== +| Name | Required | Default | Description +| `field` | yes | - | The field to be separated +| `separator` | yes | - | The separator character +|====== + +[source,js] +-------------------------------------------------- +{ + "join": { + "field": "joined_array_field", + "separator": "-" + } +} +-------------------------------------------------- + +==== Split processor +Split a field to an array using a separator character. Only works on string fields. + +[[split-options]] +.Split Options +[options="header"] +|====== +| Name | Required | Default | Description +| `field` | yes | - | The field to split +|====== + +[source,js] +-------------------------------------------------- +{ + "split": { + "field": "," + } +} +-------------------------------------------------- + +==== Lowercase processor +Converts a string to its lowercase equivalent. + +[[lowercase-options]] +.Lowercase Options +[options="header"] +|====== +| Name | Required | Default | Description +| `field` | yes | - | The field to lowercase +|====== + +[source,js] +-------------------------------------------------- +{ + "lowercase": { + "field": "foo" + } +} +-------------------------------------------------- + +==== Uppercase processor +Converts a string to its uppercase equivalent. + +[[uppercase-options]] +.Uppercase Options +[options="header"] +|====== +| Name | Required | Default | Description +| `field` | yes | - | The field to uppercase +|====== + +[source,js] +-------------------------------------------------- +{ + "uppercase": { + "field": "foo" + } +} +-------------------------------------------------- + +==== Trim processor +Trims whitespace from field. NOTE: this only works on leading and trailing whitespaces. + +[[trim-options]] +.Trim Options +[options="header"] +|====== +| Name | Required | Default | Description +| `field` | yes | - | The string-valued field to trim whitespace from +|====== + +[source,js] +-------------------------------------------------- +{ + "trim": { + "field": "foo" + } +} +-------------------------------------------------- + +==== Grok Processor + +The Grok Processor extracts structured fields out of a single text field within a document. You choose which field to +extract matched fields from, as well as the Grok Pattern you expect will match. A Grok Pattern is like a regular +expression that supports aliased expressions that can be reused. + +This tool is perfect for syslog logs, apache and other webserver logs, mysql logs, and in general, any log format +that is generally written for humans and not computer consumption. + +The processor comes packaged with over 120 reusable patterns that are located at `$ES_HOME/config/ingest/grok/patterns`. +Here, you can add your own custom grok pattern files with custom grok expressions to be used by the processor. + +If you need help building patterns to match your logs, you will find the and + applications quite useful! + +===== Grok Basics + +Grok sits on top of regular expressions, so any regular expressions are valid in grok as well. +The regular expression library is Oniguruma, and you can see the full supported regexp syntax +https://github.com/kkos/oniguruma/blob/master/doc/RE[on the Onigiruma site]. + +Grok works by leveraging this regular expression language to allow naming existing patterns and combining them into more +complex patterns that match your fields. + +The syntax for re-using a grok pattern comes in three forms: `%{SYNTAX:SEMANTIC}`, `%{SYNTAX}`, `%{SYNTAX:SEMANTIC:TYPE}`. + +The `SYNTAX` is the name of the pattern that will match your text. For example, `3.44` will be matched by the `NUMBER` +pattern and `55.3.244.1` will be matched by the `IP` pattern. The syntax is how you match. `NUMBER` and `IP` are both +patterns that are provided within the default patterns set. + +The `SEMANTIC` is the identifier you give to the piece of text being matched. For example, `3.44` could be the +duration of an event, so you could call it simply `duration`. Further, a string `55.3.244.1` might identify +the `client` making a request. + +The `TYPE` is the type you wish to cast your named field. `int` and `float` are currently the only types supported for coercion. + +For example, here is a grok pattern that would match the above example given. We would like to match a text with the following +contents: + +[source,js] +-------------------------------------------------- +3.44 55.3.244.1 +-------------------------------------------------- + +We may know that the above message is a number followed by an IP-address. We can match this text with the following +Grok expression. + +[source,js] +-------------------------------------------------- +%{NUMBER:duration} %{IP:client} +-------------------------------------------------- + +===== Custom Patterns and Pattern Files + +The Grok Processor comes pre-packaged with a base set of pattern files. These patterns may not always have +what you are looking for. These pattern files have a very basic format. Each line describes a named pattern with +the following format: + +[source,js] +-------------------------------------------------- +NAME ' '+ PATTERN '\n' +-------------------------------------------------- + +You can add this pattern to an existing file, or add your own file in the patterns directory here: `$ES_HOME/config/ingest/grok/patterns`. +The Ingest Plugin will pick up files in this directory to be loaded into the grok processor's known patterns. These patterns are loaded +at startup, so you will need to do a restart your ingest node if you wish to update these files while running. + +Example snippet of pattern definitions found in the `grok-patterns` patterns file: + +[source,js] +-------------------------------------------------- +YEAR (?>\d\d){1,2} +HOUR (?:2[0123]|[01]?[0-9]) +MINUTE (?:[0-5][0-9]) +SECOND (?:(?:[0-5]?[0-9]|60)(?:[:.,][0-9]+)?) +TIME (?!<[0-9])%{HOUR}:%{MINUTE}(?::%{SECOND})(?![0-9]) +-------------------------------------------------- + +===== Using Grok Processor in a Pipeline + +[[grok-options]] +.Grok Options +[options="header"] +|====== +| Name | Required | Default | Description +| `match_field` | yes | - | The field to use for grok expression parsing +| `match_pattern` | yes | - | The grok expression to match and extract named captures with +| `pattern_definitions` | no | - | A map of pattern-name and pattern tuples defining custom patterns to be used by the current processor. Patterns matching existing names will override the pre-existing definition. +|====== + +Here is an example of using the provided patterns to extract out and name structured fields from a string field in +a document. + +[source,js] +-------------------------------------------------- +{ + "message": "55.3.244.1 GET /index.html 15824 0.043" +} +-------------------------------------------------- + +The pattern for this could be + +[source] +-------------------------------------------------- +%{IP:client} %{WORD:method} %{URIPATHPARAM:request} %{NUMBER:bytes} %{NUMBER:duration} +-------------------------------------------------- + +An example pipeline for processing the above document using Grok: + +[source,js] +-------------------------------------------------- +{ + "description" : "...", + "processors": [ + { + "grok": { + "match_field": "message", + "match_pattern": "%{IP:client} %{WORD:method} %{URIPATHPARAM:request} %{NUMBER:bytes} %{NUMBER:duration}" + } + } + ] +} +-------------------------------------------------- + +This pipeline will insert these named captures as new fields within the document, like so: + +[source,js] +-------------------------------------------------- +{ + "message": "55.3.244.1 GET /index.html 15824 0.043", + "client": "55.3.244.1", + "method": "GET", + "request": "/index.html", + "bytes": 15824, + "duration": "0.043" +} +-------------------------------------------------- + +An example of a pipeline specifying custom pattern definitions: + +[source,js] +-------------------------------------------------- +{ + "description" : "...", + "processors": [ + { + "grok": { + "match_field": "message", + "match_pattern": "my %{FAVORITE_DOG:dog} is colored %{RGB:color}" + "pattern_definitions" : { + "FAVORITE_DOG" : "beagle", + "RGB" : "RED|GREEN|BLUE" + } + } + } + ] +} +-------------------------------------------------- + +==== Date processor + +The date processor is used for parsing dates from fields, and then using that date or timestamp as the timestamp for that document. +The date processor adds by default the parsed date as a new field called `@timestamp`, configurable by setting the `target_field` +configuration parameter. Multiple date formats are supported as part of the same date processor definition. They will be used +sequentially to attempt parsing the date field, in the same order they were defined as part of the processor definition. + +[[date-options]] +.Date options +[options="header"] +|====== +| Name | Required | Default | Description +| `match_field` | yes | - | The field to get the date from. +| `target_field` | no | @timestamp | The field that will hold the parsed date. +| `match_formats` | yes | - | Array of the expected date formats. Can be a joda pattern or one of the following formats: ISO8601, UNIX, UNIX_MS, TAI64N. +| `timezone` | no | UTC | The timezone to use when parsing the date. +| `locale` | no | ENGLISH | The locale to use when parsing the date, relevant when parsing month names or week days. +|====== + +An example that adds the parsed date to the `timestamp` field based on the `initial_date` field: + +[source,js] +-------------------------------------------------- +{ + "description" : "...", + "processors" : [ + { + "date" : { + "match_field" : "initial_date", + "target_field" : "timestamp", + "match_formats" : ["dd/MM/yyyy hh:mm:ss"], + "timezone" : "Europe/Amsterdam" + } + } + ] +} +-------------------------------------------------- + +==== Fail processor +The Fail Processor is used to raise an exception. This is useful for when +a user expects a pipeline to fail and wishes to relay a specific message +to the requester. + +[[fail-options]] +.Fail Options +[options="header"] +|====== +| Name | Required | Default | Description +| `message` | yes | - | The error message of the `FailException` thrown by the processor +|====== + +[source,js] +-------------------------------------------------- +{ + "fail": { + "message": "an error message" + } +} +-------------------------------------------------- + +==== DeDot Processor +The DeDot Processor is used to remove dots (".") from field names and +replace them with a specific `separator` string. + +[[dedot-options]] +.DeDot Options +[options="header"] +|====== +| Name | Required | Default | Description +| `separator` | yes | "_" | The string to replace dots with in all field names +|====== + +[source,js] +-------------------------------------------------- +{ + "dedot": { + "separator": "_" + } +} +-------------------------------------------------- + + +=== Accessing data in pipelines + +Processors in pipelines have read and write access to documents that pass through the pipeline. +The fields in the source of a document and its metadata fields are accessible. + +Accessing a field in the source is straightforward and one can refer to fields by +their name. For example: + +[source,js] +-------------------------------------------------- +{ + "set": { + "field": "my_field" + "value": 582.1 + } +} +-------------------------------------------------- + +On top of this fields from the source are always accessible via the `_source` prefix: + +[source,js] +-------------------------------------------------- +{ + "set": { + "field": "_source.my_field" + "value": 582.1 + } +} +-------------------------------------------------- + +Metadata fields can also be accessed in the same way as fields from the source. This +is possible because Elasticsearch doesn't allow fields in the source that have the +same name as metadata fields. + +The following example sets the id of a document to `1`: + +[source,js] +-------------------------------------------------- +{ + "set": { + "field": "_id" + "value": "1" + } +} +-------------------------------------------------- + +The following metadata fields are accessible by a processor: `_index`, `_type`, `_id`, `_routing`, `_parent`, +`_timestamp` and `_ttl`. + +Beyond metadata fields and source fields, the ingest plugin also adds ingest metadata to documents being processed. +These metadata properties are accessible under the `_ingest` key. Currently the ingest plugin adds the ingest timestamp +under `_ingest.timestamp` key to the ingest metadata, which is the time the ingest plugin received the index or bulk +request to pre-process. But any processor is free to add more ingest related metadata to it. Ingest metadata is transient +and is lost after a document has been processed by the pipeline and thus ingest metadata won't be indexed. + +The following example adds a field with the name `received` and the value is the ingest timestamp: + +[source,js] +-------------------------------------------------- +{ + "set": { + "field": "received" + "value": "{{_ingest.timestamp}}" + } +} +-------------------------------------------------- + +As opposed to Elasticsearch metadata fields, the ingest metadata field name _ingest can be used as a valid field name +in the source of a document. Use _source._ingest to refer to it, otherwise _ingest will be interpreted as ingest +metadata fields by the ingest plugin. + +A number of processor settings also support templating. Settings that support templating can have zero or more +template snippets. A template snippet begins with `{{` and ends with `}}`. +Accessing fields and metafields in templates is exactly the same as via regular processor field settings. + +In this example a field by the name `field_c` is added and its value is a concatenation of +the values of `field_a` and `field_b`. + +[source,js] +-------------------------------------------------- +{ + "set": { + "field": "field_c" + "value": "{{field_a}} {{field_b}}" + } +} +-------------------------------------------------- + +The following example changes the index a document is going to be indexed into. The index a document will be redirected +to depends on the field in the source with name `geoip.country_iso_code`. + +[source,js] +-------------------------------------------------- +{ + "set": { + "field": "_index" + "value": "{{geoip.country_iso_code}}" + } +} +-------------------------------------------------- + +==== Handling Failure in Pipelines + +In its simplest case, pipelines describe a list of processors which +are executed sequentially and processing halts at the first exception. This +may not be desirable when failures are expected. For example, not all your logs +may match a certain grok expression and you may wish to index such documents into +a separate index. + +To enable this behavior, you can utilize the `on_failure` parameter. `on_failure` +defines a list of processors to be executed immediately following the failed processor. +This parameter can be supplied at the pipeline level, as well as at the processor +level. If a processor has an `on_failure` configuration option provided, whether +it is empty or not, any exceptions that are thrown by it will be caught and the +pipeline will continue executing the proceeding processors defined. Since further processors +are defined within the scope of an `on_failure` statement, failure handling can be nested. + +Example: In the following example we define a pipeline that hopes to rename documents with +a field named `foo` to `bar`. If the document does not contain the `foo` field, we +go ahead and attach an error message within the document for later analysis within +Elasticsearch. + +[source,js] +-------------------------------------------------- +{ + "description" : "my first pipeline with handled exceptions", + "processors" : [ + { + "rename" : { + "field" : "foo", + "to" : "bar", + "on_failure" : [ + { + "set" : { + "field" : "error", + "value" : "field \"foo\" does not exist, cannot rename to \"bar\"" + } + } + ] + } + } + ] +} +-------------------------------------------------- + +Example: Here we define an `on_failure` block on a whole pipeline to change +the index for which failed documents get sent. + +[source,js] +-------------------------------------------------- +{ + "description" : "my first pipeline with handled exceptions", + "processors" : [ ... ], + "on_failure" : [ + { + "set" : { + "field" : "_index", + "value" : "failed-{{ _index }}" + } + } + ] +} +-------------------------------------------------- + + +===== Accessing Error Metadata From Processors Handling Exceptions + +Sometimes you may want to retrieve the actual error message that was thrown +by a failed processor. To do so you can access metadata fields called +`on_failure_message` and `on_failure_processor`. These fields are only accessible +from within the context of an `on_failure` block. Here is an updated version of +our first example which leverages these fields to provide the error message instead +of manually setting it. + +[source,js] +-------------------------------------------------- +{ + "description" : "my first pipeline with handled exceptions", + "processors" : [ + { + "rename" : { + "field" : "foo", + "to" : "bar", + "on_failure" : [ + { + "set" : { + "field" : "error", + "value" : "{{ _ingest.on_failure_message }}" + } + } + ] + } + } + ] +} +-------------------------------------------------- + + +=== Ingest APIs + +==== Put pipeline API + +The put pipeline api adds pipelines and updates existing pipelines in the cluster. + +[source,js] +-------------------------------------------------- +PUT _ingest/pipeline/my-pipeline-id +{ + "description" : "describe pipeline", + "processors" : [ + { + "simple" : { + // settings + } + }, + // other processors + ] +} +-------------------------------------------------- +// AUTOSENSE + +NOTE: The put pipeline api also instructs all ingest nodes to reload their in-memory representation of pipelines, so that + pipeline changes take immediately in effect. + +==== Get pipeline API + +The get pipeline api returns pipelines based on id. This api always returns a local reference of the pipeline. + +[source,js] +-------------------------------------------------- +GET _ingest/pipeline/my-pipeline-id +-------------------------------------------------- +// AUTOSENSE + +Example response: + +[source,js] +-------------------------------------------------- +{ + "my-pipeline-id": { + "_source" : { + "description": "describe pipeline", + "processors": [ + { + "simple" : { + // settings + } + }, + // other processors + ] + }, + "_version" : 0 + } +} +-------------------------------------------------- + +For each returned pipeline the source and the version is returned. +The version is useful for knowing what version of the pipeline the node has. +Multiple ids can be provided at the same time. Also wildcards are supported. + +==== Delete pipeline API + +The delete pipeline api deletes pipelines by id. + +[source,js] +-------------------------------------------------- +DELETE _ingest/pipeline/my-pipeline-id +-------------------------------------------------- +// AUTOSENSE + +==== Simulate pipeline API + +The simulate pipeline api executes a specific pipeline against +the set of documents provided in the body of the request. + +A simulate request may call upon an existing pipeline to be executed +against the provided documents, or supply a pipeline definition in +the body of the request. + +Here is the structure of a simulate request with a provided pipeline: + +[source,js] +-------------------------------------------------- +POST _ingest/pipeline/_simulate +{ + "pipeline" : { + // pipeline definition here + }, + "docs" : [ + { /** first document **/ }, + { /** second document **/ }, + // ... + ] +} +-------------------------------------------------- + +Here is the structure of a simulate request against a pre-existing pipeline: + +[source,js] +-------------------------------------------------- +POST _ingest/pipeline/my-pipeline-id/_simulate +{ + "docs" : [ + { /** first document **/ }, + { /** second document **/ }, + // ... + ] +} +-------------------------------------------------- + + +Here is an example simulate request with a provided pipeline and its response: + +[source,js] +-------------------------------------------------- +POST _ingest/pipeline/_simulate +{ + "pipeline" : + { + "description": "_description", + "processors": [ + { + "set" : { + "field" : "field2", + "value" : "_value" + } + } + ] + }, + "docs": [ + { + "_index": "index", + "_type": "type", + "_id": "id", + "_source": { + "foo": "bar" + } + }, + { + "_index": "index", + "_type": "type", + "_id": "id", + "_source": { + "foo": "rab" + } + } + ] +} +-------------------------------------------------- +// AUTOSENSE + +response: + +[source,js] +-------------------------------------------------- +{ + "docs": [ + { + "doc": { + "_id": "id", + "_ttl": null, + "_parent": null, + "_index": "index", + "_routing": null, + "_type": "type", + "_timestamp": null, + "_source": { + "field2": "_value", + "foo": "bar" + }, + "_ingest": { + "timestamp": "2016-01-04T23:53:27.186+0000" + } + } + }, + { + "doc": { + "_id": "id", + "_ttl": null, + "_parent": null, + "_index": "index", + "_routing": null, + "_type": "type", + "_timestamp": null, + "_source": { + "field2": "_value", + "foo": "rab" + }, + "_ingest": { + "timestamp": "2016-01-04T23:53:27.186+0000" + } + } + } + ] +} +-------------------------------------------------- + +It is often useful to see how each processor affects the ingest document +as it is passed through the pipeline. To see the intermediate results of +each processor in the simulat request, a `verbose` parameter may be added +to the request + +Here is an example verbose request and its response: + + +[source,js] +-------------------------------------------------- +POST _ingest/pipeline/_simulate?verbose +{ + "pipeline" : + { + "description": "_description", + "processors": [ + { + "set" : { + "field" : "field2", + "value" : "_value2" + } + }, + { + "set" : { + "field" : "field3", + "value" : "_value3" + } + } + ] + }, + "docs": [ + { + "_index": "index", + "_type": "type", + "_id": "id", + "_source": { + "foo": "bar" + } + }, + { + "_index": "index", + "_type": "type", + "_id": "id", + "_source": { + "foo": "rab" + } + } + ] +} +-------------------------------------------------- +// AUTOSENSE + +response: + +[source,js] +-------------------------------------------------- +{ + "docs": [ + { + "processor_results": [ + { + "processor_id": "processor[set]-0", + "doc": { + "_id": "id", + "_ttl": null, + "_parent": null, + "_index": "index", + "_routing": null, + "_type": "type", + "_timestamp": null, + "_source": { + "field2": "_value2", + "foo": "bar" + }, + "_ingest": { + "timestamp": "2016-01-05T00:02:51.383+0000" + } + } + }, + { + "processor_id": "processor[set]-1", + "doc": { + "_id": "id", + "_ttl": null, + "_parent": null, + "_index": "index", + "_routing": null, + "_type": "type", + "_timestamp": null, + "_source": { + "field3": "_value3", + "field2": "_value2", + "foo": "bar" + }, + "_ingest": { + "timestamp": "2016-01-05T00:02:51.383+0000" + } + } + } + ] + }, + { + "processor_results": [ + { + "processor_id": "processor[set]-0", + "doc": { + "_id": "id", + "_ttl": null, + "_parent": null, + "_index": "index", + "_routing": null, + "_type": "type", + "_timestamp": null, + "_source": { + "field2": "_value2", + "foo": "rab" + }, + "_ingest": { + "timestamp": "2016-01-05T00:02:51.384+0000" + } + } + }, + { + "processor_id": "processor[set]-1", + "doc": { + "_id": "id", + "_ttl": null, + "_parent": null, + "_index": "index", + "_routing": null, + "_type": "type", + "_timestamp": null, + "_source": { + "field3": "_value3", + "field2": "_value2", + "foo": "rab" + }, + "_ingest": { + "timestamp": "2016-01-05T00:02:51.384+0000" + } + } + } + ] + } + ] +} +-------------------------------------------------- diff --git a/docs/reference/mapping/fields/timestamp-field.asciidoc b/docs/reference/mapping/fields/timestamp-field.asciidoc index 5971a02c771..3f4bf8a8134 100644 --- a/docs/reference/mapping/fields/timestamp-field.asciidoc +++ b/docs/reference/mapping/fields/timestamp-field.asciidoc @@ -1,6 +1,8 @@ [[mapping-timestamp-field]] === `_timestamp` field +deprecated[2.0.0,The `_timestamp` field is deprecated. Instead, use a normal <> field and set its value explicitly] + The `_timestamp` field, when enabled, allows a timestamp to be indexed and stored with a document. The timestamp may be specified manually, generated automatically, or set to a default value: diff --git a/docs/reference/mapping/fields/ttl-field.asciidoc b/docs/reference/mapping/fields/ttl-field.asciidoc index d81582c9078..9bfdc72b4c4 100644 --- a/docs/reference/mapping/fields/ttl-field.asciidoc +++ b/docs/reference/mapping/fields/ttl-field.asciidoc @@ -1,6 +1,8 @@ [[mapping-ttl-field]] === `_ttl` field +deprecated[2.0.0,The current `_ttl` implementation is deprecated and will be replaced with a different implementation in a future version] + Some types of documents, such as session data or special offers, come with an expiration date. The `_ttl` field allows you to specify the minimum time a document should live, after which time the document is deleted automatically. diff --git a/docs/reference/migration/migrate_3_0.asciidoc b/docs/reference/migration/migrate_3_0.asciidoc index 76b1ddb417e..b4aa2d6d844 100644 --- a/docs/reference/migration/migrate_3_0.asciidoc +++ b/docs/reference/migration/migrate_3_0.asciidoc @@ -644,6 +644,10 @@ The percolate api can no longer accept documents that have fields that don't exi When percolating an existing document then specifying a document in the source of the percolate request is not allowed any more. +The percolate api no longer modifies the mappings. Before the percolate api could be used to dynamically introduce new +fields to the mappings based on the fields in the document being percolated. This no longer works, because these +unmapped fields are not persisted in the mapping. + Percolator documents are no longer excluded from the search response. [[breaking_30_packaging]] diff --git a/docs/reference/search/percolate.asciidoc b/docs/reference/search/percolate.asciidoc index 7f160d1a503..4ac1b6b35ea 100644 --- a/docs/reference/search/percolate.asciidoc +++ b/docs/reference/search/percolate.asciidoc @@ -20,14 +20,8 @@ in a request to the percolate API. ===================================== Fields referred to in a percolator query must *already* exist in the mapping -associated with the index used for percolation. -There are two ways to make sure that a field mapping exist: - -* Add or update a mapping via the <> or - <> APIs. -* Percolate a document before registering a query. Percolating a document can - add field mappings dynamically, in the same way as happens when indexing a - document. +associated with the index used for percolation. In order to make sure these fields exist, +add or update a mapping via the <> or <> APIs. ===================================== diff --git a/modules/ingest-grok/build.gradle b/modules/ingest-grok/build.gradle new file mode 100644 index 00000000000..26722345544 --- /dev/null +++ b/modules/ingest-grok/build.gradle @@ -0,0 +1,39 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +esplugin { + description 'Ingest processor that uses grok patterns to split text' + classname 'org.elasticsearch.ingest.grok.IngestGrokPlugin' +} + +dependencies { + compile 'org.jruby.joni:joni:2.1.6' + // joni dependencies: + compile 'org.jruby.jcodings:jcodings:1.0.12' +} + +compileJava.options.compilerArgs << "-Xlint:-rawtypes,-unchecked,-serial" +compileTestJava.options.compilerArgs << "-Xlint:-rawtypes,-unchecked" + +thirdPartyAudit.excludes = [ + // joni has AsmCompilerSupport, but that isn't being used: + 'org.objectweb.asm.ClassWriter', + 'org.objectweb.asm.MethodVisitor', + 'org.objectweb.asm.Opcodes', +] \ No newline at end of file diff --git a/modules/ingest-grok/licenses/jcodings-1.0.12.jar.sha1 b/modules/ingest-grok/licenses/jcodings-1.0.12.jar.sha1 new file mode 100644 index 00000000000..b097e32ece4 --- /dev/null +++ b/modules/ingest-grok/licenses/jcodings-1.0.12.jar.sha1 @@ -0,0 +1 @@ +6bc17079fcaa8823ea8cd0d4c66516335b558db8 \ No newline at end of file diff --git a/modules/ingest-grok/licenses/jcodings-LICENSE.txt b/modules/ingest-grok/licenses/jcodings-LICENSE.txt new file mode 100644 index 00000000000..a3fdf73d562 --- /dev/null +++ b/modules/ingest-grok/licenses/jcodings-LICENSE.txt @@ -0,0 +1,17 @@ +Permission is hereby granted, free of charge, to any person obtaining a copy of +this software and associated documentation files (the "Software"), to deal in +the Software without restriction, including without limitation the rights to +use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies +of the Software, and to permit persons to whom the Software is furnished to do +so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in all +copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE +SOFTWARE. \ No newline at end of file diff --git a/modules/ingest-grok/licenses/jcodings-NOTICE.txt b/modules/ingest-grok/licenses/jcodings-NOTICE.txt new file mode 100644 index 00000000000..f6c494861fd --- /dev/null +++ b/modules/ingest-grok/licenses/jcodings-NOTICE.txt @@ -0,0 +1 @@ +JCodings is released under the MIT License. \ No newline at end of file diff --git a/modules/ingest-grok/licenses/joni-2.1.6.jar.sha1 b/modules/ingest-grok/licenses/joni-2.1.6.jar.sha1 new file mode 100644 index 00000000000..48abe138a8f --- /dev/null +++ b/modules/ingest-grok/licenses/joni-2.1.6.jar.sha1 @@ -0,0 +1 @@ +0f23c95a06eaecbc8c74c7458a8bfd13e4fd2d3a \ No newline at end of file diff --git a/modules/ingest-grok/licenses/joni-LICENSE.txt b/modules/ingest-grok/licenses/joni-LICENSE.txt new file mode 100644 index 00000000000..a3fdf73d562 --- /dev/null +++ b/modules/ingest-grok/licenses/joni-LICENSE.txt @@ -0,0 +1,17 @@ +Permission is hereby granted, free of charge, to any person obtaining a copy of +this software and associated documentation files (the "Software"), to deal in +the Software without restriction, including without limitation the rights to +use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies +of the Software, and to permit persons to whom the Software is furnished to do +so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in all +copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE +SOFTWARE. \ No newline at end of file diff --git a/modules/ingest-grok/licenses/joni-NOTICE.txt b/modules/ingest-grok/licenses/joni-NOTICE.txt new file mode 100644 index 00000000000..45bc517b9e2 --- /dev/null +++ b/modules/ingest-grok/licenses/joni-NOTICE.txt @@ -0,0 +1 @@ +Joni is released under the MIT License. diff --git a/modules/ingest-grok/src/main/java/org/elasticsearch/ingest/grok/Grok.java b/modules/ingest-grok/src/main/java/org/elasticsearch/ingest/grok/Grok.java new file mode 100644 index 00000000000..abed8413287 --- /dev/null +++ b/modules/ingest-grok/src/main/java/org/elasticsearch/ingest/grok/Grok.java @@ -0,0 +1,158 @@ +/* + * 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.grok; + +import org.jcodings.specific.UTF8Encoding; +import org.joni.Matcher; +import org.joni.NameEntry; +import org.joni.Option; +import org.joni.Regex; +import org.joni.Region; +import org.joni.Syntax; +import org.joni.exception.ValueException; + +import java.nio.charset.StandardCharsets; +import java.util.HashMap; +import java.util.Iterator; +import java.util.Locale; +import java.util.Map; + +final class Grok { + + private static final String NAME_GROUP = "name"; + private static final String SUBNAME_GROUP = "subname"; + private static final String PATTERN_GROUP = "pattern"; + private static final String DEFINITION_GROUP = "definition"; + private static final String GROK_PATTERN = + "%\\{" + + "(?" + + "(?[A-z0-9]+)" + + "(?::(?[A-z0-9_:.-]+))?" + + ")" + + "(?:=(?" + + "(?:" + + "(?:[^{}]+|\\.+)+" + + ")+" + + ")" + + ")?" + "\\}"; + private static final Regex GROK_PATTERN_REGEX = new Regex(GROK_PATTERN.getBytes(StandardCharsets.UTF_8), 0, GROK_PATTERN.getBytes(StandardCharsets.UTF_8).length, Option.NONE, UTF8Encoding.INSTANCE, Syntax.DEFAULT); + private final Map patternBank; + private final boolean namedCaptures; + private final Regex compiledExpression; + private final String expression; + + + public Grok(Map patternBank, String grokPattern) { + this(patternBank, grokPattern, true); + } + + @SuppressWarnings("unchecked") + public Grok(Map patternBank, String grokPattern, boolean namedCaptures) { + this.patternBank = patternBank; + this.namedCaptures = namedCaptures; + + this.expression = toRegex(grokPattern); + byte[] expressionBytes = expression.getBytes(StandardCharsets.UTF_8); + this.compiledExpression = new Regex(expressionBytes, 0, expressionBytes.length, Option.DEFAULT, UTF8Encoding.INSTANCE); + } + + + public String groupMatch(String name, Region region, String pattern) { + try { + int number = GROK_PATTERN_REGEX.nameToBackrefNumber(name.getBytes(StandardCharsets.UTF_8), 0, name.getBytes(StandardCharsets.UTF_8).length, region); + int begin = region.beg[number]; + int end = region.end[number]; + return new String(pattern.getBytes(StandardCharsets.UTF_8), begin, end - begin, StandardCharsets.UTF_8); + } catch (StringIndexOutOfBoundsException e) { + return null; + } catch (ValueException e) { + return null; + } + } + + /** + * converts a grok expression into a named regex expression + * + * @return named regex expression + */ + public String toRegex(String grokPattern) { + byte[] grokPatternBytes = grokPattern.getBytes(StandardCharsets.UTF_8); + Matcher matcher = GROK_PATTERN_REGEX.matcher(grokPatternBytes); + + int result = matcher.search(0, grokPatternBytes.length, Option.NONE); + if (result != -1) { + Region region = matcher.getEagerRegion(); + String namedPatternRef = groupMatch(NAME_GROUP, region, grokPattern); + String subName = groupMatch(SUBNAME_GROUP, region, grokPattern); + // TODO(tal): Support definitions + String definition = groupMatch(DEFINITION_GROUP, region, grokPattern); + String patternName = groupMatch(PATTERN_GROUP, region, grokPattern); + String pattern = patternBank.get(patternName); + + String grokPart; + if (namedCaptures && subName != null) { + grokPart = String.format(Locale.US, "(?<%s>%s)", namedPatternRef, pattern); + } else if (!namedCaptures) { + grokPart = String.format(Locale.US, "(?<%s>%s)", patternName + "_" + String.valueOf(result), pattern); + } else { + grokPart = String.format(Locale.US, "(?:%s)", pattern); + } + + String start = new String(grokPatternBytes, 0, result, StandardCharsets.UTF_8); + String rest = new String(grokPatternBytes, region.end[0], grokPatternBytes.length - region.end[0], StandardCharsets.UTF_8); + return start + toRegex(grokPart + rest); + } + + return grokPattern; + } + + public boolean match(String text) { + Matcher matcher = compiledExpression.matcher(text.getBytes(StandardCharsets.UTF_8)); + int result = matcher.search(0, text.length(), Option.DEFAULT); + return (result != -1); + } + + public Map captures(String text) { + byte[] textAsBytes = text.getBytes(StandardCharsets.UTF_8); + Map fields = new HashMap<>(); + Matcher matcher = compiledExpression.matcher(textAsBytes); + int result = matcher.search(0, textAsBytes.length, Option.DEFAULT); + if (result != -1 && compiledExpression.numberOfNames() > 0) { + Region region = matcher.getEagerRegion(); + for (Iterator entry = compiledExpression.namedBackrefIterator(); entry.hasNext();) { + NameEntry e = entry.next(); + int number = e.getBackRefs()[0]; + + String groupName = new String(e.name, e.nameP, e.nameEnd - e.nameP, StandardCharsets.UTF_8); + String matchValue = null; + if (region.beg[number] >= 0) { + matchValue = new String(textAsBytes, region.beg[number], region.end[number] - region.beg[number], StandardCharsets.UTF_8); + } + GrokMatchGroup match = new GrokMatchGroup(groupName, matchValue); + fields.put(match.getName(), match.getValue()); + } + return fields; + } else if (result != -1) { + return fields; + } + return null; + } +} + diff --git a/modules/ingest-grok/src/main/java/org/elasticsearch/ingest/grok/GrokMatchGroup.java b/modules/ingest-grok/src/main/java/org/elasticsearch/ingest/grok/GrokMatchGroup.java new file mode 100644 index 00000000000..2cebf620c96 --- /dev/null +++ b/modules/ingest-grok/src/main/java/org/elasticsearch/ingest/grok/GrokMatchGroup.java @@ -0,0 +1,62 @@ +/* + * 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.grok; + +final class GrokMatchGroup { + private static final String DEFAULT_TYPE = "string"; + private final String patternName; + private final String fieldName; + private final String type; + private final String groupValue; + + public GrokMatchGroup(String groupName, String groupValue) { + String[] parts = groupName.split(":"); + patternName = parts[0]; + if (parts.length >= 2) { + fieldName = parts[1]; + } else { + fieldName = null; + } + + if (parts.length == 3) { + type = parts[2]; + } else { + type = DEFAULT_TYPE; + } + this.groupValue = groupValue; + } + + public String getName() { + return (fieldName == null) ? patternName : fieldName; + } + + public Object getValue() { + if (groupValue == null) { return null; } + + switch(type) { + case "int": + return Integer.parseInt(groupValue); + case "float": + return Float.parseFloat(groupValue); + default: + return groupValue; + } + } +} diff --git a/modules/ingest-grok/src/main/java/org/elasticsearch/ingest/grok/GrokProcessor.java b/modules/ingest-grok/src/main/java/org/elasticsearch/ingest/grok/GrokProcessor.java new file mode 100644 index 00000000000..4df8d673072 --- /dev/null +++ b/modules/ingest-grok/src/main/java/org/elasticsearch/ingest/grok/GrokProcessor.java @@ -0,0 +1,91 @@ +/* + * 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.grok; + +import org.elasticsearch.ingest.core.AbstractProcessor; +import org.elasticsearch.ingest.core.AbstractProcessorFactory; +import org.elasticsearch.ingest.core.ConfigurationUtils; +import org.elasticsearch.ingest.core.IngestDocument; + +import java.util.HashMap; +import java.util.Map; + +public final class GrokProcessor extends AbstractProcessor { + + public static final String TYPE = "grok"; + + private final String matchField; + private final Grok grok; + + public GrokProcessor(String tag, Grok grok, String matchField) { + super(tag); + this.matchField = matchField; + this.grok = grok; + } + + @Override + public void execute(IngestDocument ingestDocument) throws Exception { + String fieldValue = ingestDocument.getFieldValue(matchField, String.class); + Map matches = grok.captures(fieldValue); + if (matches != null) { + matches.forEach((k, v) -> ingestDocument.setFieldValue(k, v)); + } else { + throw new IllegalArgumentException("Grok expression does not match field value: [" + fieldValue + "]"); + } + } + + @Override + public String getType() { + return TYPE; + } + + String getMatchField() { + return matchField; + } + + Grok getGrok() { + return grok; + } + + public final static class Factory extends AbstractProcessorFactory { + + private final Map builtinPatterns; + + public Factory(Map builtinPatterns) { + this.builtinPatterns = builtinPatterns; + } + + @Override + public GrokProcessor doCreate(String processorTag, Map config) throws Exception { + String matchField = ConfigurationUtils.readStringProperty(config, "field"); + String matchPattern = ConfigurationUtils.readStringProperty(config, "pattern"); + Map customPatternBank = ConfigurationUtils.readOptionalMap(config, "pattern_definitions"); + Map patternBank = new HashMap<>(builtinPatterns); + if (customPatternBank != null) { + patternBank.putAll(customPatternBank); + } + + Grok grok = new Grok(patternBank, matchPattern); + return new GrokProcessor(processorTag, grok, matchField); + } + + } + +} diff --git a/modules/ingest-grok/src/main/java/org/elasticsearch/ingest/grok/IngestGrokPlugin.java b/modules/ingest-grok/src/main/java/org/elasticsearch/ingest/grok/IngestGrokPlugin.java new file mode 100644 index 00000000000..54800ac1603 --- /dev/null +++ b/modules/ingest-grok/src/main/java/org/elasticsearch/ingest/grok/IngestGrokPlugin.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.ingest.grok; + +import org.elasticsearch.node.NodeModule; +import org.elasticsearch.plugins.Plugin; + +import java.io.BufferedReader; +import java.io.IOException; +import java.io.InputStream; +import java.io.InputStreamReader; +import java.nio.charset.StandardCharsets; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +public class IngestGrokPlugin extends Plugin { + + private static final String[] PATTERN_NAMES = new String[] { + "aws", "bacula", "bro", "exim", "firewalls", "grok-patterns", "haproxy", + "java", "junos", "linux-syslog", "mcollective-patterns", "mongodb", "nagios", + "postgresql", "rails", "redis", "ruby" + }; + + private final Map builtinPatterns; + + public IngestGrokPlugin() throws IOException { + this.builtinPatterns = loadBuiltinPatterns(); + } + + @Override + public String name() { + return "ingest-grok"; + } + + @Override + public String description() { + return "Ingest processor that uses grok patterns to split text"; + } + + public void onModule(NodeModule nodeModule) { + nodeModule.registerProcessor(GrokProcessor.TYPE, (templateService) -> new GrokProcessor.Factory(builtinPatterns)); + } + + static Map loadBuiltinPatterns() throws IOException { + Map builtinPatterns = new HashMap<>(); + for (String pattern : PATTERN_NAMES) { + try(InputStream is = IngestGrokPlugin.class.getResourceAsStream("/patterns/" + pattern)) { + loadPatterns(builtinPatterns, is); + } + } + return Collections.unmodifiableMap(builtinPatterns); + } + + private static void loadPatterns(Map patternBank, InputStream inputStream) throws IOException { + String line; + BufferedReader br = new BufferedReader(new InputStreamReader(inputStream, StandardCharsets.UTF_8)); + while ((line = br.readLine()) != null) { + String trimmedLine = line.replaceAll("^\\s+", ""); + if (trimmedLine.startsWith("#") || trimmedLine.length() == 0) { + continue; + } + + String[] parts = trimmedLine.split("\\s+", 2); + if (parts.length == 2) { + patternBank.put(parts[0], parts[1]); + } + } + } +} diff --git a/modules/ingest-grok/src/main/resources/patterns/aws b/modules/ingest-grok/src/main/resources/patterns/aws new file mode 100644 index 00000000000..71edbc9f296 --- /dev/null +++ b/modules/ingest-grok/src/main/resources/patterns/aws @@ -0,0 +1,11 @@ +S3_REQUEST_LINE (?:%{WORD:verb} %{NOTSPACE:request}(?: HTTP/%{NUMBER:httpversion})?|%{DATA:rawrequest}) + +S3_ACCESS_LOG %{WORD:owner} %{NOTSPACE:bucket} \[%{HTTPDATE:timestamp}\] %{IP:clientip} %{NOTSPACE:requester} %{NOTSPACE:request_id} %{NOTSPACE:operation} %{NOTSPACE:key} (?:"%{S3_REQUEST_LINE}"|-) (?:%{INT:response:int}|-) (?:-|%{NOTSPACE:error_code}) (?:%{INT:bytes:int}|-) (?:%{INT:object_size:int}|-) (?:%{INT:request_time_ms:int}|-) (?:%{INT:turnaround_time_ms:int}|-) (?:%{QS:referrer}|-) (?:"?%{QS:agent}"?|-) (?:-|%{NOTSPACE:version_id}) + +ELB_URIPATHPARAM %{URIPATH:path}(?:%{URIPARAM:params})? + +ELB_URI %{URIPROTO:proto}://(?:%{USER}(?::[^@]*)?@)?(?:%{URIHOST:urihost})?(?:%{ELB_URIPATHPARAM})? + +ELB_REQUEST_LINE (?:%{WORD:verb} %{ELB_URI:request}(?: HTTP/%{NUMBER:httpversion})?|%{DATA:rawrequest}) + +ELB_ACCESS_LOG %{TIMESTAMP_ISO8601:timestamp} %{NOTSPACE:elb} %{IP:clientip}:%{INT:clientport:int} (?:(%{IP:backendip}:?:%{INT:backendport:int})|-) %{NUMBER:request_processing_time:float} %{NUMBER:backend_processing_time:float} %{NUMBER:response_processing_time:float} %{INT:response:int} %{INT:backend_response:int} %{INT:received_bytes:int} %{INT:bytes:int} "%{ELB_REQUEST_LINE}" diff --git a/modules/ingest-grok/src/main/resources/patterns/bacula b/modules/ingest-grok/src/main/resources/patterns/bacula new file mode 100644 index 00000000000..d80dfe5de4c --- /dev/null +++ b/modules/ingest-grok/src/main/resources/patterns/bacula @@ -0,0 +1,50 @@ +BACULA_TIMESTAMP %{MONTHDAY}-%{MONTH} %{HOUR}:%{MINUTE} +BACULA_HOST [a-zA-Z0-9-]+ +BACULA_VOLUME %{USER} +BACULA_DEVICE %{USER} +BACULA_DEVICEPATH %{UNIXPATH} +BACULA_CAPACITY %{INT}{1,3}(,%{INT}{3})* +BACULA_VERSION %{USER} +BACULA_JOB %{USER} + +BACULA_LOG_MAX_CAPACITY User defined maximum volume capacity %{BACULA_CAPACITY} exceeded on device \"%{BACULA_DEVICE:device}\" \(%{BACULA_DEVICEPATH}\) +BACULA_LOG_END_VOLUME End of medium on Volume \"%{BACULA_VOLUME:volume}\" Bytes=%{BACULA_CAPACITY} Blocks=%{BACULA_CAPACITY} at %{MONTHDAY}-%{MONTH}-%{YEAR} %{HOUR}:%{MINUTE}. +BACULA_LOG_NEW_VOLUME Created new Volume \"%{BACULA_VOLUME:volume}\" in catalog. +BACULA_LOG_NEW_LABEL Labeled new Volume \"%{BACULA_VOLUME:volume}\" on device \"%{BACULA_DEVICE:device}\" \(%{BACULA_DEVICEPATH}\). +BACULA_LOG_WROTE_LABEL Wrote label to prelabeled Volume \"%{BACULA_VOLUME:volume}\" on device \"%{BACULA_DEVICE}\" \(%{BACULA_DEVICEPATH}\) +BACULA_LOG_NEW_MOUNT New volume \"%{BACULA_VOLUME:volume}\" mounted on device \"%{BACULA_DEVICE:device}\" \(%{BACULA_DEVICEPATH}\) at %{MONTHDAY}-%{MONTH}-%{YEAR} %{HOUR}:%{MINUTE}. +BACULA_LOG_NOOPEN \s+Cannot open %{DATA}: ERR=%{GREEDYDATA:berror} +BACULA_LOG_NOOPENDIR \s+Could not open directory %{DATA}: ERR=%{GREEDYDATA:berror} +BACULA_LOG_NOSTAT \s+Could not stat %{DATA}: ERR=%{GREEDYDATA:berror} +BACULA_LOG_NOJOBS There are no more Jobs associated with Volume \"%{BACULA_VOLUME:volume}\". Marking it purged. +BACULA_LOG_ALL_RECORDS_PRUNED All records pruned from Volume \"%{BACULA_VOLUME:volume}\"; marking it \"Purged\" +BACULA_LOG_BEGIN_PRUNE_JOBS Begin pruning Jobs older than %{INT} month %{INT} days . +BACULA_LOG_BEGIN_PRUNE_FILES Begin pruning Files. +BACULA_LOG_PRUNED_JOBS Pruned %{INT} Jobs* for client %{BACULA_HOST:client} from catalog. +BACULA_LOG_PRUNED_FILES Pruned Files from %{INT} Jobs* for client %{BACULA_HOST:client} from catalog. +BACULA_LOG_ENDPRUNE End auto prune. +BACULA_LOG_STARTJOB Start Backup JobId %{INT}, Job=%{BACULA_JOB:job} +BACULA_LOG_STARTRESTORE Start Restore Job %{BACULA_JOB:job} +BACULA_LOG_USEDEVICE Using Device \"%{BACULA_DEVICE:device}\" +BACULA_LOG_DIFF_FS \s+%{UNIXPATH} is a different filesystem. Will not descend from %{UNIXPATH} into it. +BACULA_LOG_JOBEND Job write elapsed time = %{DATA:elapsed}, Transfer rate = %{NUMBER} (K|M|G)? Bytes/second +BACULA_LOG_NOPRUNE_JOBS No Jobs found to prune. +BACULA_LOG_NOPRUNE_FILES No Files found to prune. +BACULA_LOG_VOLUME_PREVWRITTEN Volume \"%{BACULA_VOLUME:volume}\" previously written, moving to end of data. +BACULA_LOG_READYAPPEND Ready to append to end of Volume \"%{BACULA_VOLUME:volume}\" size=%{INT} +BACULA_LOG_CANCELLING Cancelling duplicate JobId=%{INT}. +BACULA_LOG_MARKCANCEL JobId %{INT}, Job %{BACULA_JOB:job} marked to be canceled. +BACULA_LOG_CLIENT_RBJ shell command: run ClientRunBeforeJob \"%{GREEDYDATA:runjob}\" +BACULA_LOG_VSS (Generate )?VSS (Writer)? +BACULA_LOG_MAXSTART Fatal error: Job canceled because max start delay time exceeded. +BACULA_LOG_DUPLICATE Fatal error: JobId %{INT:duplicate} already running. Duplicate job not allowed. +BACULA_LOG_NOJOBSTAT Fatal error: No Job status returned from FD. +BACULA_LOG_FATAL_CONN Fatal error: bsock.c:133 Unable to connect to (Client: %{BACULA_HOST:client}|Storage daemon) on %{HOSTNAME}:%{POSINT}. ERR=(?%{GREEDYDATA}) +BACULA_LOG_NO_CONNECT Warning: bsock.c:127 Could not connect to (Client: %{BACULA_HOST:client}|Storage daemon) on %{HOSTNAME}:%{POSINT}. ERR=(?%{GREEDYDATA}) +BACULA_LOG_NO_AUTH Fatal error: Unable to authenticate with File daemon at %{HOSTNAME}. Possible causes: +BACULA_LOG_NOSUIT No prior or suitable Full backup found in catalog. Doing FULL backup. +BACULA_LOG_NOPRIOR No prior Full backup Job record found. + +BACULA_LOG_JOB (Error: )?Bacula %{BACULA_HOST} %{BACULA_VERSION} \(%{BACULA_VERSION}\): + +BACULA_LOGLINE %{BACULA_TIMESTAMP:bts} %{BACULA_HOST:hostname} JobId %{INT:jobid}: (%{BACULA_LOG_MAX_CAPACITY}|%{BACULA_LOG_END_VOLUME}|%{BACULA_LOG_NEW_VOLUME}|%{BACULA_LOG_NEW_LABEL}|%{BACULA_LOG_WROTE_LABEL}|%{BACULA_LOG_NEW_MOUNT}|%{BACULA_LOG_NOOPEN}|%{BACULA_LOG_NOOPENDIR}|%{BACULA_LOG_NOSTAT}|%{BACULA_LOG_NOJOBS}|%{BACULA_LOG_ALL_RECORDS_PRUNED}|%{BACULA_LOG_BEGIN_PRUNE_JOBS}|%{BACULA_LOG_BEGIN_PRUNE_FILES}|%{BACULA_LOG_PRUNED_JOBS}|%{BACULA_LOG_PRUNED_FILES}|%{BACULA_LOG_ENDPRUNE}|%{BACULA_LOG_STARTJOB}|%{BACULA_LOG_STARTRESTORE}|%{BACULA_LOG_USEDEVICE}|%{BACULA_LOG_DIFF_FS}|%{BACULA_LOG_JOBEND}|%{BACULA_LOG_NOPRUNE_JOBS}|%{BACULA_LOG_NOPRUNE_FILES}|%{BACULA_LOG_VOLUME_PREVWRITTEN}|%{BACULA_LOG_READYAPPEND}|%{BACULA_LOG_CANCELLING}|%{BACULA_LOG_MARKCANCEL}|%{BACULA_LOG_CLIENT_RBJ}|%{BACULA_LOG_VSS}|%{BACULA_LOG_MAXSTART}|%{BACULA_LOG_DUPLICATE}|%{BACULA_LOG_NOJOBSTAT}|%{BACULA_LOG_FATAL_CONN}|%{BACULA_LOG_NO_CONNECT}|%{BACULA_LOG_NO_AUTH}|%{BACULA_LOG_NOSUIT}|%{BACULA_LOG_JOB}|%{BACULA_LOG_NOPRIOR}) diff --git a/modules/ingest-grok/src/main/resources/patterns/bro b/modules/ingest-grok/src/main/resources/patterns/bro new file mode 100644 index 00000000000..31b138b39e2 --- /dev/null +++ b/modules/ingest-grok/src/main/resources/patterns/bro @@ -0,0 +1,13 @@ +# https://www.bro.org/sphinx/script-reference/log-files.html + +# http.log +BRO_HTTP %{NUMBER:ts}\t%{NOTSPACE:uid}\t%{IP:orig_h}\t%{INT:orig_p}\t%{IP:resp_h}\t%{INT:resp_p}\t%{INT:trans_depth}\t%{GREEDYDATA:method}\t%{GREEDYDATA:domain}\t%{GREEDYDATA:uri}\t%{GREEDYDATA:referrer}\t%{GREEDYDATA:user_agent}\t%{NUMBER:request_body_len}\t%{NUMBER:response_body_len}\t%{GREEDYDATA:status_code}\t%{GREEDYDATA:status_msg}\t%{GREEDYDATA:info_code}\t%{GREEDYDATA:info_msg}\t%{GREEDYDATA:filename}\t%{GREEDYDATA:bro_tags}\t%{GREEDYDATA:username}\t%{GREEDYDATA:password}\t%{GREEDYDATA:proxied}\t%{GREEDYDATA:orig_fuids}\t%{GREEDYDATA:orig_mime_types}\t%{GREEDYDATA:resp_fuids}\t%{GREEDYDATA:resp_mime_types} + +# dns.log +BRO_DNS %{NUMBER:ts}\t%{NOTSPACE:uid}\t%{IP:orig_h}\t%{INT:orig_p}\t%{IP:resp_h}\t%{INT:resp_p}\t%{WORD:proto}\t%{INT:trans_id}\t%{GREEDYDATA:query}\t%{GREEDYDATA:qclass}\t%{GREEDYDATA:qclass_name}\t%{GREEDYDATA:qtype}\t%{GREEDYDATA:qtype_name}\t%{GREEDYDATA:rcode}\t%{GREEDYDATA:rcode_name}\t%{GREEDYDATA:AA}\t%{GREEDYDATA:TC}\t%{GREEDYDATA:RD}\t%{GREEDYDATA:RA}\t%{GREEDYDATA:Z}\t%{GREEDYDATA:answers}\t%{GREEDYDATA:TTLs}\t%{GREEDYDATA:rejected} + +# conn.log +BRO_CONN %{NUMBER:ts}\t%{NOTSPACE:uid}\t%{IP:orig_h}\t%{INT:orig_p}\t%{IP:resp_h}\t%{INT:resp_p}\t%{WORD:proto}\t%{GREEDYDATA:service}\t%{NUMBER:duration}\t%{NUMBER:orig_bytes}\t%{NUMBER:resp_bytes}\t%{GREEDYDATA:conn_state}\t%{GREEDYDATA:local_orig}\t%{GREEDYDATA:missed_bytes}\t%{GREEDYDATA:history}\t%{GREEDYDATA:orig_pkts}\t%{GREEDYDATA:orig_ip_bytes}\t%{GREEDYDATA:resp_pkts}\t%{GREEDYDATA:resp_ip_bytes}\t%{GREEDYDATA:tunnel_parents} + +# files.log +BRO_FILES %{NUMBER:ts}\t%{NOTSPACE:fuid}\t%{IP:tx_hosts}\t%{IP:rx_hosts}\t%{NOTSPACE:conn_uids}\t%{GREEDYDATA:source}\t%{GREEDYDATA:depth}\t%{GREEDYDATA:analyzers}\t%{GREEDYDATA:mime_type}\t%{GREEDYDATA:filename}\t%{GREEDYDATA:duration}\t%{GREEDYDATA:local_orig}\t%{GREEDYDATA:is_orig}\t%{GREEDYDATA:seen_bytes}\t%{GREEDYDATA:total_bytes}\t%{GREEDYDATA:missing_bytes}\t%{GREEDYDATA:overflow_bytes}\t%{GREEDYDATA:timedout}\t%{GREEDYDATA:parent_fuid}\t%{GREEDYDATA:md5}\t%{GREEDYDATA:sha1}\t%{GREEDYDATA:sha256}\t%{GREEDYDATA:extracted} diff --git a/modules/ingest-grok/src/main/resources/patterns/exim b/modules/ingest-grok/src/main/resources/patterns/exim new file mode 100644 index 00000000000..68c4e5cd7d0 --- /dev/null +++ b/modules/ingest-grok/src/main/resources/patterns/exim @@ -0,0 +1,13 @@ +EXIM_MSGID [0-9A-Za-z]{6}-[0-9A-Za-z]{6}-[0-9A-Za-z]{2} +EXIM_FLAGS (<=|[-=>*]>|[*]{2}|==) +EXIM_DATE %{YEAR:exim_year}-%{MONTHNUM:exim_month}-%{MONTHDAY:exim_day} %{TIME:exim_time} +EXIM_PID \[%{POSINT}\] +EXIM_QT ((\d+y)?(\d+w)?(\d+d)?(\d+h)?(\d+m)?(\d+s)?) +EXIM_EXCLUDE_TERMS (Message is frozen|(Start|End) queue run| Warning: | retry time not reached | no (IP address|host name) found for (IP address|host) | unexpected disconnection while reading SMTP command | no immediate delivery: |another process is handling this message) +EXIM_REMOTE_HOST (H=(%{NOTSPACE:remote_hostname} )?(\(%{NOTSPACE:remote_heloname}\) )?\[%{IP:remote_host}\]) +EXIM_INTERFACE (I=\[%{IP:exim_interface}\](:%{NUMBER:exim_interface_port})) +EXIM_PROTOCOL (P=%{NOTSPACE:protocol}) +EXIM_MSG_SIZE (S=%{NUMBER:exim_msg_size}) +EXIM_HEADER_ID (id=%{NOTSPACE:exim_header_id}) +EXIM_SUBJECT (T=%{QS:exim_subject}) + diff --git a/modules/ingest-grok/src/main/resources/patterns/firewalls b/modules/ingest-grok/src/main/resources/patterns/firewalls new file mode 100644 index 00000000000..03c3e5aff0c --- /dev/null +++ b/modules/ingest-grok/src/main/resources/patterns/firewalls @@ -0,0 +1,86 @@ +# NetScreen firewall logs +NETSCREENSESSIONLOG %{SYSLOGTIMESTAMP:date} %{IPORHOST:device} %{IPORHOST}: NetScreen device_id=%{WORD:device_id}%{DATA}: start_time=%{QUOTEDSTRING:start_time} duration=%{INT:duration} policy_id=%{INT:policy_id} service=%{DATA:service} proto=%{INT:proto} src zone=%{WORD:src_zone} dst zone=%{WORD:dst_zone} action=%{WORD:action} sent=%{INT:sent} rcvd=%{INT:rcvd} src=%{IPORHOST:src_ip} dst=%{IPORHOST:dst_ip} src_port=%{INT:src_port} dst_port=%{INT:dst_port} src-xlated ip=%{IPORHOST:src_xlated_ip} port=%{INT:src_xlated_port} dst-xlated ip=%{IPORHOST:dst_xlated_ip} port=%{INT:dst_xlated_port} session_id=%{INT:session_id} reason=%{GREEDYDATA:reason} + +#== Cisco ASA == +CISCO_TAGGED_SYSLOG ^<%{POSINT:syslog_pri}>%{CISCOTIMESTAMP:timestamp}( %{SYSLOGHOST:sysloghost})? ?: %%{CISCOTAG:ciscotag}: +CISCOTIMESTAMP %{MONTH} +%{MONTHDAY}(?: %{YEAR})? %{TIME} +CISCOTAG [A-Z0-9]+-%{INT}-(?:[A-Z0-9_]+) +# Common Particles +CISCO_ACTION Built|Teardown|Deny|Denied|denied|requested|permitted|denied by ACL|discarded|est-allowed|Dropping|created|deleted +CISCO_REASON Duplicate TCP SYN|Failed to locate egress interface|Invalid transport field|No matching connection|DNS Response|DNS Query|(?:%{WORD}\s*)* +CISCO_DIRECTION Inbound|inbound|Outbound|outbound +CISCO_INTERVAL first hit|%{INT}-second interval +CISCO_XLATE_TYPE static|dynamic +# ASA-1-104001 +CISCOFW104001 \((?:Primary|Secondary)\) Switching to ACTIVE - %{GREEDYDATA:switch_reason} +# ASA-1-104002 +CISCOFW104002 \((?:Primary|Secondary)\) Switching to STANDBY - %{GREEDYDATA:switch_reason} +# ASA-1-104003 +CISCOFW104003 \((?:Primary|Secondary)\) Switching to FAILED\. +# ASA-1-104004 +CISCOFW104004 \((?:Primary|Secondary)\) Switching to OK\. +# ASA-1-105003 +CISCOFW105003 \((?:Primary|Secondary)\) Monitoring on [Ii]nterface %{GREEDYDATA:interface_name} waiting +# ASA-1-105004 +CISCOFW105004 \((?:Primary|Secondary)\) Monitoring on [Ii]nterface %{GREEDYDATA:interface_name} normal +# ASA-1-105005 +CISCOFW105005 \((?:Primary|Secondary)\) Lost Failover communications with mate on [Ii]nterface %{GREEDYDATA:interface_name} +# ASA-1-105008 +CISCOFW105008 \((?:Primary|Secondary)\) Testing [Ii]nterface %{GREEDYDATA:interface_name} +# ASA-1-105009 +CISCOFW105009 \((?:Primary|Secondary)\) Testing on [Ii]nterface %{GREEDYDATA:interface_name} (?:Passed|Failed) +# ASA-2-106001 +CISCOFW106001 %{CISCO_DIRECTION:direction} %{WORD:protocol} connection %{CISCO_ACTION:action} from %{IP:src_ip}/%{INT:src_port} to %{IP:dst_ip}/%{INT:dst_port} flags %{GREEDYDATA:tcp_flags} on interface %{GREEDYDATA:interface} +# ASA-2-106006, ASA-2-106007, ASA-2-106010 +CISCOFW106006_106007_106010 %{CISCO_ACTION:action} %{CISCO_DIRECTION:direction} %{WORD:protocol} (?:from|src) %{IP:src_ip}/%{INT:src_port}(\(%{DATA:src_fwuser}\))? (?:to|dst) %{IP:dst_ip}/%{INT:dst_port}(\(%{DATA:dst_fwuser}\))? (?:on interface %{DATA:interface}|due to %{CISCO_REASON:reason}) +# ASA-3-106014 +CISCOFW106014 %{CISCO_ACTION:action} %{CISCO_DIRECTION:direction} %{WORD:protocol} src %{DATA:src_interface}:%{IP:src_ip}(\(%{DATA:src_fwuser}\))? dst %{DATA:dst_interface}:%{IP:dst_ip}(\(%{DATA:dst_fwuser}\))? \(type %{INT:icmp_type}, code %{INT:icmp_code}\) +# ASA-6-106015 +CISCOFW106015 %{CISCO_ACTION:action} %{WORD:protocol} \(%{DATA:policy_id}\) from %{IP:src_ip}/%{INT:src_port} to %{IP:dst_ip}/%{INT:dst_port} flags %{DATA:tcp_flags} on interface %{GREEDYDATA:interface} +# ASA-1-106021 +CISCOFW106021 %{CISCO_ACTION:action} %{WORD:protocol} reverse path check from %{IP:src_ip} to %{IP:dst_ip} on interface %{GREEDYDATA:interface} +# ASA-4-106023 +CISCOFW106023 %{CISCO_ACTION:action}( protocol)? %{WORD:protocol} src %{DATA:src_interface}:%{DATA:src_ip}(/%{INT:src_port})?(\(%{DATA:src_fwuser}\))? dst %{DATA:dst_interface}:%{DATA:dst_ip}(/%{INT:dst_port})?(\(%{DATA:dst_fwuser}\))?( \(type %{INT:icmp_type}, code %{INT:icmp_code}\))? by access-group "?%{DATA:policy_id}"? \[%{DATA:hashcode1}, %{DATA:hashcode2}\] +# ASA-4-106100, ASA-4-106102, ASA-4-106103 +CISCOFW106100_2_3 access-list %{NOTSPACE:policy_id} %{CISCO_ACTION:action} %{WORD:protocol} for user '%{DATA:src_fwuser}' %{DATA:src_interface}/%{IP:src_ip}\(%{INT:src_port}\) -> %{DATA:dst_interface}/%{IP:dst_ip}\(%{INT:dst_port}\) hit-cnt %{INT:hit_count} %{CISCO_INTERVAL:interval} \[%{DATA:hashcode1}, %{DATA:hashcode2}\] +# ASA-5-106100 +CISCOFW106100 access-list %{NOTSPACE:policy_id} %{CISCO_ACTION:action} %{WORD:protocol} %{DATA:src_interface}/%{IP:src_ip}\(%{INT:src_port}\)(\(%{DATA:src_fwuser}\))? -> %{DATA:dst_interface}/%{IP:dst_ip}\(%{INT:dst_port}\)(\(%{DATA:src_fwuser}\))? hit-cnt %{INT:hit_count} %{CISCO_INTERVAL:interval} \[%{DATA:hashcode1}, %{DATA:hashcode2}\] +# ASA-6-110002 +CISCOFW110002 %{CISCO_REASON:reason} for %{WORD:protocol} from %{DATA:src_interface}:%{IP:src_ip}/%{INT:src_port} to %{IP:dst_ip}/%{INT:dst_port} +# ASA-6-302010 +CISCOFW302010 %{INT:connection_count} in use, %{INT:connection_count_max} most used +# ASA-6-302013, ASA-6-302014, ASA-6-302015, ASA-6-302016 +CISCOFW302013_302014_302015_302016 %{CISCO_ACTION:action}(?: %{CISCO_DIRECTION:direction})? %{WORD:protocol} connection %{INT:connection_id} for %{DATA:src_interface}:%{IP:src_ip}/%{INT:src_port}( \(%{IP:src_mapped_ip}/%{INT:src_mapped_port}\))?(\(%{DATA:src_fwuser}\))? to %{DATA:dst_interface}:%{IP:dst_ip}/%{INT:dst_port}( \(%{IP:dst_mapped_ip}/%{INT:dst_mapped_port}\))?(\(%{DATA:dst_fwuser}\))?( duration %{TIME:duration} bytes %{INT:bytes})?(?: %{CISCO_REASON:reason})?( \(%{DATA:user}\))? +# ASA-6-302020, ASA-6-302021 +CISCOFW302020_302021 %{CISCO_ACTION:action}(?: %{CISCO_DIRECTION:direction})? %{WORD:protocol} connection for faddr %{IP:dst_ip}/%{INT:icmp_seq_num}(?:\(%{DATA:fwuser}\))? gaddr %{IP:src_xlated_ip}/%{INT:icmp_code_xlated} laddr %{IP:src_ip}/%{INT:icmp_code}( \(%{DATA:user}\))? +# ASA-6-305011 +CISCOFW305011 %{CISCO_ACTION:action} %{CISCO_XLATE_TYPE:xlate_type} %{WORD:protocol} translation from %{DATA:src_interface}:%{IP:src_ip}(/%{INT:src_port})?(\(%{DATA:src_fwuser}\))? to %{DATA:src_xlated_interface}:%{IP:src_xlated_ip}/%{DATA:src_xlated_port} +# ASA-3-313001, ASA-3-313004, ASA-3-313008 +CISCOFW313001_313004_313008 %{CISCO_ACTION:action} %{WORD:protocol} type=%{INT:icmp_type}, code=%{INT:icmp_code} from %{IP:src_ip} on interface %{DATA:interface}( to %{IP:dst_ip})? +# ASA-4-313005 +CISCOFW313005 %{CISCO_REASON:reason} for %{WORD:protocol} error message: %{WORD:err_protocol} src %{DATA:err_src_interface}:%{IP:err_src_ip}(\(%{DATA:err_src_fwuser}\))? dst %{DATA:err_dst_interface}:%{IP:err_dst_ip}(\(%{DATA:err_dst_fwuser}\))? \(type %{INT:err_icmp_type}, code %{INT:err_icmp_code}\) on %{DATA:interface} interface\. Original IP payload: %{WORD:protocol} src %{IP:orig_src_ip}/%{INT:orig_src_port}(\(%{DATA:orig_src_fwuser}\))? dst %{IP:orig_dst_ip}/%{INT:orig_dst_port}(\(%{DATA:orig_dst_fwuser}\))? +# ASA-5-321001 +CISCOFW321001 Resource '%{WORD:resource_name}' limit of %{POSINT:resource_limit} reached for system +# ASA-4-402117 +CISCOFW402117 %{WORD:protocol}: Received a non-IPSec packet \(protocol= %{WORD:orig_protocol}\) from %{IP:src_ip} to %{IP:dst_ip} +# ASA-4-402119 +CISCOFW402119 %{WORD:protocol}: Received an %{WORD:orig_protocol} packet \(SPI= %{DATA:spi}, sequence number= %{DATA:seq_num}\) from %{IP:src_ip} \(user= %{DATA:user}\) to %{IP:dst_ip} that failed anti-replay checking +# ASA-4-419001 +CISCOFW419001 %{CISCO_ACTION:action} %{WORD:protocol} packet from %{DATA:src_interface}:%{IP:src_ip}/%{INT:src_port} to %{DATA:dst_interface}:%{IP:dst_ip}/%{INT:dst_port}, reason: %{GREEDYDATA:reason} +# ASA-4-419002 +CISCOFW419002 %{CISCO_REASON:reason} from %{DATA:src_interface}:%{IP:src_ip}/%{INT:src_port} to %{DATA:dst_interface}:%{IP:dst_ip}/%{INT:dst_port} with different initial sequence number +# ASA-4-500004 +CISCOFW500004 %{CISCO_REASON:reason} for protocol=%{WORD:protocol}, from %{IP:src_ip}/%{INT:src_port} to %{IP:dst_ip}/%{INT:dst_port} +# ASA-6-602303, ASA-6-602304 +CISCOFW602303_602304 %{WORD:protocol}: An %{CISCO_DIRECTION:direction} %{GREEDYDATA:tunnel_type} SA \(SPI= %{DATA:spi}\) between %{IP:src_ip} and %{IP:dst_ip} \(user= %{DATA:user}\) has been %{CISCO_ACTION:action} +# ASA-7-710001, ASA-7-710002, ASA-7-710003, ASA-7-710005, ASA-7-710006 +CISCOFW710001_710002_710003_710005_710006 %{WORD:protocol} (?:request|access) %{CISCO_ACTION:action} from %{IP:src_ip}/%{INT:src_port} to %{DATA:dst_interface}:%{IP:dst_ip}/%{INT:dst_port} +# ASA-6-713172 +CISCOFW713172 Group = %{GREEDYDATA:group}, IP = %{IP:src_ip}, Automatic NAT Detection Status:\s+Remote end\s*%{DATA:is_remote_natted}\s*behind a NAT device\s+This\s+end\s*%{DATA:is_local_natted}\s*behind a NAT device +# ASA-4-733100 +CISCOFW733100 \[\s*%{DATA:drop_type}\s*\] drop %{DATA:drop_rate_id} exceeded. Current burst rate is %{INT:drop_rate_current_burst} per second, max configured rate is %{INT:drop_rate_max_burst}; Current average rate is %{INT:drop_rate_current_avg} per second, max configured rate is %{INT:drop_rate_max_avg}; Cumulative total count is %{INT:drop_total_count} +#== End Cisco ASA == + +# Shorewall firewall logs +SHOREWALL (%{SYSLOGTIMESTAMP:timestamp}) (%{WORD:nf_host}) kernel:.*Shorewall:(%{WORD:nf_action1})?:(%{WORD:nf_action2})?.*IN=(%{USERNAME:nf_in_interface})?.*(OUT= *MAC=(%{COMMONMAC:nf_dst_mac}):(%{COMMONMAC:nf_src_mac})?|OUT=%{USERNAME:nf_out_interface}).*SRC=(%{IPV4:nf_src_ip}).*DST=(%{IPV4:nf_dst_ip}).*LEN=(%{WORD:nf_len}).?*TOS=(%{WORD:nf_tos}).?*PREC=(%{WORD:nf_prec}).?*TTL=(%{INT:nf_ttl}).?*ID=(%{INT:nf_id}).?*PROTO=(%{WORD:nf_protocol}).?*SPT=(%{INT:nf_src_port}?.*DPT=%{INT:nf_dst_port}?.*) +#== End Shorewall diff --git a/modules/ingest-grok/src/main/resources/patterns/grok-patterns b/modules/ingest-grok/src/main/resources/patterns/grok-patterns new file mode 100644 index 00000000000..cb4c3fffc6a --- /dev/null +++ b/modules/ingest-grok/src/main/resources/patterns/grok-patterns @@ -0,0 +1,102 @@ +USERNAME [a-zA-Z0-9._-]+ +USER %{USERNAME} +EMAILLOCALPART [a-zA-Z][a-zA-Z0-9_.+-=:]+ +EMAILADDRESS %{EMAILLOCALPART}@%{HOSTNAME} +HTTPDUSER %{EMAILADDRESS}|%{USER} +INT (?:[+-]?(?:[0-9]+)) +BASE10NUM (?[+-]?(?:(?:[0-9]+(?:\.[0-9]+)?)|(?:\.[0-9]+))) +NUMBER (?:%{BASE10NUM}) +BASE16NUM (?(?"(?>\\.|[^\\"]+)+"|""|(?>'(?>\\.|[^\\']+)+')|''|(?>`(?>\\.|[^\\`]+)+`)|``)) +UUID [A-Fa-f0-9]{8}-(?:[A-Fa-f0-9]{4}-){3}[A-Fa-f0-9]{12} + +# Networking +MAC (?:%{CISCOMAC}|%{WINDOWSMAC}|%{COMMONMAC}) +CISCOMAC (?:(?:[A-Fa-f0-9]{4}\.){2}[A-Fa-f0-9]{4}) +WINDOWSMAC (?:(?:[A-Fa-f0-9]{2}-){5}[A-Fa-f0-9]{2}) +COMMONMAC (?:(?:[A-Fa-f0-9]{2}:){5}[A-Fa-f0-9]{2}) +IPV6 ((([0-9A-Fa-f]{1,4}:){7}([0-9A-Fa-f]{1,4}|:))|(([0-9A-Fa-f]{1,4}:){6}(:[0-9A-Fa-f]{1,4}|((25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)(\.(25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)){3})|:))|(([0-9A-Fa-f]{1,4}:){5}(((:[0-9A-Fa-f]{1,4}){1,2})|:((25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)(\.(25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)){3})|:))|(([0-9A-Fa-f]{1,4}:){4}(((:[0-9A-Fa-f]{1,4}){1,3})|((:[0-9A-Fa-f]{1,4})?:((25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)(\.(25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)){3}))|:))|(([0-9A-Fa-f]{1,4}:){3}(((:[0-9A-Fa-f]{1,4}){1,4})|((:[0-9A-Fa-f]{1,4}){0,2}:((25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)(\.(25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)){3}))|:))|(([0-9A-Fa-f]{1,4}:){2}(((:[0-9A-Fa-f]{1,4}){1,5})|((:[0-9A-Fa-f]{1,4}){0,3}:((25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)(\.(25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)){3}))|:))|(([0-9A-Fa-f]{1,4}:){1}(((:[0-9A-Fa-f]{1,4}){1,6})|((:[0-9A-Fa-f]{1,4}){0,4}:((25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)(\.(25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)){3}))|:))|(:(((:[0-9A-Fa-f]{1,4}){1,7})|((:[0-9A-Fa-f]{1,4}){0,5}:((25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)(\.(25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)){3}))|:)))(%.+)? +IPV4 (?[A-Za-z]+:|\\)(?:\\[^\\?*]*)+ +URIPROTO [A-Za-z]+(\+[A-Za-z+]+)? +URIHOST %{IPORHOST}(?::%{POSINT:port})? +# uripath comes loosely from RFC1738, but mostly from what Firefox +# doesn't turn into %XX +URIPATH (?:/[A-Za-z0-9$.+!*'(){},~:;=@#%_\-]*)+ +#URIPARAM \?(?:[A-Za-z0-9]+(?:=(?:[^&]*))?(?:&(?:[A-Za-z0-9]+(?:=(?:[^&]*))?)?)*)? +URIPARAM \?[A-Za-z0-9$.+!*'|(){},~@#%&/=:;_?\-\[\]<>]* +URIPATHPARAM %{URIPATH}(?:%{URIPARAM})? +URI %{URIPROTO}://(?:%{USER}(?::[^@]*)?@)?(?:%{URIHOST})?(?:%{URIPATHPARAM})? + +# Months: January, Feb, 3, 03, 12, December +MONTH \b(?:Jan(?:uary|uar)?|Feb(?:ruary|ruar)?|M(?:a|ä)?r(?:ch|z)?|Apr(?:il)?|Ma(?:y|i)?|Jun(?:e|i)?|Jul(?:y)?|Aug(?:ust)?|Sep(?:tember)?|O(?:c|k)?t(?:ober)?|Nov(?:ember)?|De(?:c|z)(?:ember)?)\b +MONTHNUM (?:0?[1-9]|1[0-2]) +MONTHNUM2 (?:0[1-9]|1[0-2]) +MONTHDAY (?:(?:0[1-9])|(?:[12][0-9])|(?:3[01])|[1-9]) + +# Days: Monday, Tue, Thu, etc... +DAY (?:Mon(?:day)?|Tue(?:sday)?|Wed(?:nesday)?|Thu(?:rsday)?|Fri(?:day)?|Sat(?:urday)?|Sun(?:day)?) + +# Years? +YEAR (?>\d\d){1,2} +HOUR (?:2[0123]|[01]?[0-9]) +MINUTE (?:[0-5][0-9]) +# '60' is a leap second in most time standards and thus is valid. +SECOND (?:(?:[0-5]?[0-9]|60)(?:[:.,][0-9]+)?) +TIME (?!<[0-9])%{HOUR}:%{MINUTE}(?::%{SECOND})(?![0-9]) +# datestamp is YYYY/MM/DD-HH:MM:SS.UUUU (or something like it) +DATE_US %{MONTHNUM}[/-]%{MONTHDAY}[/-]%{YEAR} +DATE_EU %{MONTHDAY}[./-]%{MONTHNUM}[./-]%{YEAR} +ISO8601_TIMEZONE (?:Z|[+-]%{HOUR}(?::?%{MINUTE})) +ISO8601_SECOND (?:%{SECOND}|60) +ISO8601_HOUR (?:2[0123]|[01][0-9]) +TIMESTAMP_ISO8601 %{YEAR}-%{MONTHNUM}-%{MONTHDAY}[T ]%{ISO8601_HOUR}:?%{MINUTE}(?::?%{SECOND})?%{ISO8601_TIMEZONE}? +DATE %{DATE_US}|%{DATE_EU} +DATESTAMP %{DATE}[- ]%{TIME} +TZ (?:[PMCE][SD]T|UTC) +DATESTAMP_RFC822 %{DAY} %{MONTH} %{MONTHDAY} %{YEAR} %{TIME} %{TZ} +DATESTAMP_RFC2822 %{DAY}, %{MONTHDAY} %{MONTH} %{YEAR} %{TIME} %{ISO8601_TIMEZONE} +DATESTAMP_OTHER %{DAY} %{MONTH} %{MONTHDAY} %{TIME} %{TZ} %{YEAR} +DATESTAMP_EVENTLOG %{YEAR}%{MONTHNUM2}%{MONTHDAY}%{HOUR}%{MINUTE}%{SECOND} +HTTPDERROR_DATE %{DAY} %{MONTH} %{MONTHDAY} %{TIME} %{YEAR} + +# Syslog Dates: Month Day HH:MM:SS +SYSLOGTIMESTAMP %{MONTH} +%{MONTHDAY} %{TIME} +PROG [\x21-\x5a\x5c\x5e-\x7e]+ +SYSLOGPROG %{PROG:program}(?:\[%{POSINT:pid}\])? +SYSLOGHOST %{IPORHOST} +SYSLOGFACILITY <%{NONNEGINT:facility}.%{NONNEGINT:priority}> +HTTPDATE %{MONTHDAY}/%{MONTH}/%{YEAR}:%{TIME} %{INT} + +# Shortcuts +QS %{QUOTEDSTRING} + +# Log formats +SYSLOGBASE %{SYSLOGTIMESTAMP:timestamp} (?:%{SYSLOGFACILITY} )?%{SYSLOGHOST:logsource} %{SYSLOGPROG}: +COMMONAPACHELOG %{IPORHOST:clientip} %{HTTPDUSER:ident} %{USER:auth} \[%{HTTPDATE:timestamp}\] "(?:%{WORD:verb} %{NOTSPACE:request}(?: HTTP/%{NUMBER:httpversion})?|%{DATA:rawrequest})" %{NUMBER:response} (?:%{NUMBER:bytes}|-) +COMBINEDAPACHELOG %{COMMONAPACHELOG} %{QS:referrer} %{QS:agent} +HTTPD20_ERRORLOG \[%{HTTPDERROR_DATE:timestamp}\] \[%{LOGLEVEL:loglevel}\] (?:\[client %{IPORHOST:clientip}\] ){0,1}%{GREEDYDATA:errormsg} +HTTPD24_ERRORLOG \[%{HTTPDERROR_DATE:timestamp}\] \[%{WORD:module}:%{LOGLEVEL:loglevel}\] \[pid %{POSINT:pid}:tid %{NUMBER:tid}\]( \(%{POSINT:proxy_errorcode}\)%{DATA:proxy_errormessage}:)?( \[client %{IPORHOST:client}:%{POSINT:clientport}\])? %{DATA:errorcode}: %{GREEDYDATA:message} +HTTPD_ERRORLOG %{HTTPD20_ERRORLOG}|%{HTTPD24_ERRORLOG} + + +# Log Levels +LOGLEVEL ([Aa]lert|ALERT|[Tt]race|TRACE|[Dd]ebug|DEBUG|[Nn]otice|NOTICE|[Ii]nfo|INFO|[Ww]arn?(?:ing)?|WARN?(?:ING)?|[Ee]rr?(?:or)?|ERR?(?:OR)?|[Cc]rit?(?:ical)?|CRIT?(?:ICAL)?|[Ff]atal|FATAL|[Ss]evere|SEVERE|EMERG(?:ENCY)?|[Ee]merg(?:ency)?) diff --git a/modules/ingest-grok/src/main/resources/patterns/haproxy b/modules/ingest-grok/src/main/resources/patterns/haproxy new file mode 100644 index 00000000000..ddabd193f0c --- /dev/null +++ b/modules/ingest-grok/src/main/resources/patterns/haproxy @@ -0,0 +1,39 @@ +## These patterns were tested w/ haproxy-1.4.15 + +## Documentation of the haproxy log formats can be found at the following links: +## http://code.google.com/p/haproxy-docs/wiki/HTTPLogFormat +## http://code.google.com/p/haproxy-docs/wiki/TCPLogFormat + +HAPROXYTIME (?!<[0-9])%{HOUR:haproxy_hour}:%{MINUTE:haproxy_minute}(?::%{SECOND:haproxy_second})(?![0-9]) +HAPROXYDATE %{MONTHDAY:haproxy_monthday}/%{MONTH:haproxy_month}/%{YEAR:haproxy_year}:%{HAPROXYTIME:haproxy_time}.%{INT:haproxy_milliseconds} + +# Override these default patterns to parse out what is captured in your haproxy.cfg +HAPROXYCAPTUREDREQUESTHEADERS %{DATA:captured_request_headers} +HAPROXYCAPTUREDRESPONSEHEADERS %{DATA:captured_response_headers} + +# Example: +# These haproxy config lines will add data to the logs that are captured +# by the patterns below. Place them in your custom patterns directory to +# override the defaults. +# +# capture request header Host len 40 +# capture request header X-Forwarded-For len 50 +# capture request header Accept-Language len 50 +# capture request header Referer len 200 +# capture request header User-Agent len 200 +# +# capture response header Content-Type len 30 +# capture response header Content-Encoding len 10 +# capture response header Cache-Control len 200 +# capture response header Last-Modified len 200 +# +# HAPROXYCAPTUREDREQUESTHEADERS %{DATA:request_header_host}\|%{DATA:request_header_x_forwarded_for}\|%{DATA:request_header_accept_language}\|%{DATA:request_header_referer}\|%{DATA:request_header_user_agent} +# HAPROXYCAPTUREDRESPONSEHEADERS %{DATA:response_header_content_type}\|%{DATA:response_header_content_encoding}\|%{DATA:response_header_cache_control}\|%{DATA:response_header_last_modified} + +# parse a haproxy 'httplog' line +HAPROXYHTTPBASE %{IP:client_ip}:%{INT:client_port} \[%{HAPROXYDATE:accept_date}\] %{NOTSPACE:frontend_name} %{NOTSPACE:backend_name}/%{NOTSPACE:server_name} %{INT:time_request}/%{INT:time_queue}/%{INT:time_backend_connect}/%{INT:time_backend_response}/%{NOTSPACE:time_duration} %{INT:http_status_code} %{NOTSPACE:bytes_read} %{DATA:captured_request_cookie} %{DATA:captured_response_cookie} %{NOTSPACE:termination_state} %{INT:actconn}/%{INT:feconn}/%{INT:beconn}/%{INT:srvconn}/%{NOTSPACE:retries} %{INT:srv_queue}/%{INT:backend_queue} (\{%{HAPROXYCAPTUREDREQUESTHEADERS}\})?( )?(\{%{HAPROXYCAPTUREDRESPONSEHEADERS}\})?( )?"(|(%{WORD:http_verb} (%{URIPROTO:http_proto}://)?(?:%{USER:http_user}(?::[^@]*)?@)?(?:%{URIHOST:http_host})?(?:%{URIPATHPARAM:http_request})?( HTTP/%{NUMBER:http_version})?))?" + +HAPROXYHTTP (?:%{SYSLOGTIMESTAMP:syslog_timestamp}|%{TIMESTAMP_ISO8601:timestamp8601}) %{IPORHOST:syslog_server} %{SYSLOGPROG}: %{HAPROXYHTTPBASE} + +# parse a haproxy 'tcplog' line +HAPROXYTCP (?:%{SYSLOGTIMESTAMP:syslog_timestamp}|%{TIMESTAMP_ISO8601:timestamp8601}) %{IPORHOST:syslog_server} %{SYSLOGPROG}: %{IP:client_ip}:%{INT:client_port} \[%{HAPROXYDATE:accept_date}\] %{NOTSPACE:frontend_name} %{NOTSPACE:backend_name}/%{NOTSPACE:server_name} %{INT:time_queue}/%{INT:time_backend_connect}/%{NOTSPACE:time_duration} %{NOTSPACE:bytes_read} %{NOTSPACE:termination_state} %{INT:actconn}/%{INT:feconn}/%{INT:beconn}/%{INT:srvconn}/%{NOTSPACE:retries} %{INT:srv_queue}/%{INT:backend_queue} diff --git a/modules/ingest-grok/src/main/resources/patterns/java b/modules/ingest-grok/src/main/resources/patterns/java new file mode 100644 index 00000000000..e968006ad2d --- /dev/null +++ b/modules/ingest-grok/src/main/resources/patterns/java @@ -0,0 +1,20 @@ +JAVACLASS (?:[a-zA-Z$_][a-zA-Z$_0-9]*\.)*[a-zA-Z$_][a-zA-Z$_0-9]* +#Space is an allowed character to match special cases like 'Native Method' or 'Unknown Source' +JAVAFILE (?:[A-Za-z0-9_. -]+) +#Allow special method +JAVAMETHOD (?:()|[a-zA-Z$_][a-zA-Z$_0-9]*) +#Line number is optional in special cases 'Native method' or 'Unknown source' +JAVASTACKTRACEPART %{SPACE}at %{JAVACLASS:class}\.%{JAVAMETHOD:method}\(%{JAVAFILE:file}(?::%{NUMBER:line})?\) +# Java Logs +JAVATHREAD (?:[A-Z]{2}-Processor[\d]+) +JAVACLASS (?:[a-zA-Z0-9-]+\.)+[A-Za-z0-9$]+ +JAVAFILE (?:[A-Za-z0-9_.-]+) +JAVASTACKTRACEPART at %{JAVACLASS:class}\.%{WORD:method}\(%{JAVAFILE:file}:%{NUMBER:line}\) +JAVALOGMESSAGE (.*) +# MMM dd, yyyy HH:mm:ss eg: Jan 9, 2014 7:13:13 AM +CATALINA_DATESTAMP %{MONTH} %{MONTHDAY}, 20%{YEAR} %{HOUR}:?%{MINUTE}(?::?%{SECOND}) (?:AM|PM) +# yyyy-MM-dd HH:mm:ss,SSS ZZZ eg: 2014-01-09 17:32:25,527 -0800 +TOMCAT_DATESTAMP 20%{YEAR}-%{MONTHNUM}-%{MONTHDAY} %{HOUR}:?%{MINUTE}(?::?%{SECOND}) %{ISO8601_TIMEZONE} +CATALINALOG %{CATALINA_DATESTAMP:timestamp} %{JAVACLASS:class} %{JAVALOGMESSAGE:logmessage} +# 2014-01-09 20:03:28,269 -0800 | ERROR | com.example.service.ExampleService - something compeletely unexpected happened... +TOMCATLOG %{TOMCAT_DATESTAMP:timestamp} \| %{LOGLEVEL:level} \| %{JAVACLASS:class} - %{JAVALOGMESSAGE:logmessage} diff --git a/modules/ingest-grok/src/main/resources/patterns/junos b/modules/ingest-grok/src/main/resources/patterns/junos new file mode 100644 index 00000000000..4eea59d08cc --- /dev/null +++ b/modules/ingest-grok/src/main/resources/patterns/junos @@ -0,0 +1,9 @@ +# JUNOS 11.4 RT_FLOW patterns +RT_FLOW_EVENT (RT_FLOW_SESSION_CREATE|RT_FLOW_SESSION_CLOSE|RT_FLOW_SESSION_DENY) + +RT_FLOW1 %{RT_FLOW_EVENT:event}: %{GREEDYDATA:close-reason}: %{IP:src-ip}/%{INT:src-port}->%{IP:dst-ip}/%{INT:dst-port} %{DATA:service} %{IP:nat-src-ip}/%{INT:nat-src-port}->%{IP:nat-dst-ip}/%{INT:nat-dst-port} %{DATA:src-nat-rule-name} %{DATA:dst-nat-rule-name} %{INT:protocol-id} %{DATA:policy-name} %{DATA:from-zone} %{DATA:to-zone} %{INT:session-id} \d+\(%{DATA:sent}\) \d+\(%{DATA:received}\) %{INT:elapsed-time} .* + +RT_FLOW2 %{RT_FLOW_EVENT:event}: session created %{IP:src-ip}/%{INT:src-port}->%{IP:dst-ip}/%{INT:dst-port} %{DATA:service} %{IP:nat-src-ip}/%{INT:nat-src-port}->%{IP:nat-dst-ip}/%{INT:nat-dst-port} %{DATA:src-nat-rule-name} %{DATA:dst-nat-rule-name} %{INT:protocol-id} %{DATA:policy-name} %{DATA:from-zone} %{DATA:to-zone} %{INT:session-id} .* + +RT_FLOW3 %{RT_FLOW_EVENT:event}: session denied %{IP:src-ip}/%{INT:src-port}->%{IP:dst-ip}/%{INT:dst-port} %{DATA:service} %{INT:protocol-id}\(\d\) %{DATA:policy-name} %{DATA:from-zone} %{DATA:to-zone} .* + diff --git a/modules/ingest-grok/src/main/resources/patterns/linux-syslog b/modules/ingest-grok/src/main/resources/patterns/linux-syslog new file mode 100644 index 00000000000..dcffb41ba8f --- /dev/null +++ b/modules/ingest-grok/src/main/resources/patterns/linux-syslog @@ -0,0 +1,16 @@ +SYSLOG5424PRINTASCII [!-~]+ + +SYSLOGBASE2 (?:%{SYSLOGTIMESTAMP:timestamp}|%{TIMESTAMP_ISO8601:timestamp8601}) (?:%{SYSLOGFACILITY} )?%{SYSLOGHOST:logsource}+(?: %{SYSLOGPROG}:|) +SYSLOGPAMSESSION %{SYSLOGBASE} (?=%{GREEDYDATA:message})%{WORD:pam_module}\(%{DATA:pam_caller}\): session %{WORD:pam_session_state} for user %{USERNAME:username}(?: by %{GREEDYDATA:pam_by})? + +CRON_ACTION [A-Z ]+ +CRONLOG %{SYSLOGBASE} \(%{USER:user}\) %{CRON_ACTION:action} \(%{DATA:message}\) + +SYSLOGLINE %{SYSLOGBASE2} %{GREEDYDATA:message} + +# IETF 5424 syslog(8) format (see http://www.rfc-editor.org/info/rfc5424) +SYSLOG5424PRI <%{NONNEGINT:syslog5424_pri}> +SYSLOG5424SD \[%{DATA}\]+ +SYSLOG5424BASE %{SYSLOG5424PRI}%{NONNEGINT:syslog5424_ver} +(?:%{TIMESTAMP_ISO8601:syslog5424_ts}|-) +(?:%{HOSTNAME:syslog5424_host}|-) +(-|%{SYSLOG5424PRINTASCII:syslog5424_app}) +(-|%{SYSLOG5424PRINTASCII:syslog5424_proc}) +(-|%{SYSLOG5424PRINTASCII:syslog5424_msgid}) +(?:%{SYSLOG5424SD:syslog5424_sd}|-|) + +SYSLOG5424LINE %{SYSLOG5424BASE} +%{GREEDYDATA:syslog5424_msg} diff --git a/modules/ingest-grok/src/main/resources/patterns/mcollective-patterns b/modules/ingest-grok/src/main/resources/patterns/mcollective-patterns new file mode 100644 index 00000000000..bb2f7f9bc82 --- /dev/null +++ b/modules/ingest-grok/src/main/resources/patterns/mcollective-patterns @@ -0,0 +1,4 @@ +# Remember, these can be multi-line events. +MCOLLECTIVE ., \[%{TIMESTAMP_ISO8601:timestamp} #%{POSINT:pid}\]%{SPACE}%{LOGLEVEL:event_level} + +MCOLLECTIVEAUDIT %{TIMESTAMP_ISO8601:timestamp}: diff --git a/modules/ingest-grok/src/main/resources/patterns/mongodb b/modules/ingest-grok/src/main/resources/patterns/mongodb new file mode 100644 index 00000000000..78a43007c37 --- /dev/null +++ b/modules/ingest-grok/src/main/resources/patterns/mongodb @@ -0,0 +1,7 @@ +MONGO_LOG %{SYSLOGTIMESTAMP:timestamp} \[%{WORD:component}\] %{GREEDYDATA:message} +MONGO_QUERY \{ (?<={ ).*(?= } ntoreturn:) \} +MONGO_SLOWQUERY %{WORD} %{MONGO_WORDDASH:database}\.%{MONGO_WORDDASH:collection} %{WORD}: %{MONGO_QUERY:query} %{WORD}:%{NONNEGINT:ntoreturn} %{WORD}:%{NONNEGINT:ntoskip} %{WORD}:%{NONNEGINT:nscanned}.*nreturned:%{NONNEGINT:nreturned}..+ (?[0-9]+)ms +MONGO_WORDDASH \b[\w-]+\b +MONGO3_SEVERITY \w +MONGO3_COMPONENT %{WORD}|- +MONGO3_LOG %{TIMESTAMP_ISO8601:timestamp} %{MONGO3_SEVERITY:severity} %{MONGO3_COMPONENT:component}%{SPACE}(?:\[%{DATA:context}\])? %{GREEDYDATA:message} diff --git a/modules/ingest-grok/src/main/resources/patterns/nagios b/modules/ingest-grok/src/main/resources/patterns/nagios new file mode 100644 index 00000000000..f4a98bf533e --- /dev/null +++ b/modules/ingest-grok/src/main/resources/patterns/nagios @@ -0,0 +1,124 @@ +################################################################################## +################################################################################## +# Chop Nagios log files to smithereens! +# +# A set of GROK filters to process logfiles generated by Nagios. +# While it does not, this set intends to cover all possible Nagios logs. +# +# Some more work needs to be done to cover all External Commands: +# http://old.nagios.org/developerinfo/externalcommands/commandlist.php +# +# If you need some support on these rules please contact: +# Jelle Smet http://smetj.net +# +################################################################################# +################################################################################# + +NAGIOSTIME \[%{NUMBER:nagios_epoch}\] + +############################################### +######## Begin nagios log types +############################################### +NAGIOS_TYPE_CURRENT_SERVICE_STATE CURRENT SERVICE STATE +NAGIOS_TYPE_CURRENT_HOST_STATE CURRENT HOST STATE + +NAGIOS_TYPE_SERVICE_NOTIFICATION SERVICE NOTIFICATION +NAGIOS_TYPE_HOST_NOTIFICATION HOST NOTIFICATION + +NAGIOS_TYPE_SERVICE_ALERT SERVICE ALERT +NAGIOS_TYPE_HOST_ALERT HOST ALERT + +NAGIOS_TYPE_SERVICE_FLAPPING_ALERT SERVICE FLAPPING ALERT +NAGIOS_TYPE_HOST_FLAPPING_ALERT HOST FLAPPING ALERT + +NAGIOS_TYPE_SERVICE_DOWNTIME_ALERT SERVICE DOWNTIME ALERT +NAGIOS_TYPE_HOST_DOWNTIME_ALERT HOST DOWNTIME ALERT + +NAGIOS_TYPE_PASSIVE_SERVICE_CHECK PASSIVE SERVICE CHECK +NAGIOS_TYPE_PASSIVE_HOST_CHECK PASSIVE HOST CHECK + +NAGIOS_TYPE_SERVICE_EVENT_HANDLER SERVICE EVENT HANDLER +NAGIOS_TYPE_HOST_EVENT_HANDLER HOST EVENT HANDLER + +NAGIOS_TYPE_EXTERNAL_COMMAND EXTERNAL COMMAND +NAGIOS_TYPE_TIMEPERIOD_TRANSITION TIMEPERIOD TRANSITION +############################################### +######## End nagios log types +############################################### + +############################################### +######## Begin external check types +############################################### +NAGIOS_EC_DISABLE_SVC_CHECK DISABLE_SVC_CHECK +NAGIOS_EC_ENABLE_SVC_CHECK ENABLE_SVC_CHECK +NAGIOS_EC_DISABLE_HOST_CHECK DISABLE_HOST_CHECK +NAGIOS_EC_ENABLE_HOST_CHECK ENABLE_HOST_CHECK +NAGIOS_EC_PROCESS_SERVICE_CHECK_RESULT PROCESS_SERVICE_CHECK_RESULT +NAGIOS_EC_PROCESS_HOST_CHECK_RESULT PROCESS_HOST_CHECK_RESULT +NAGIOS_EC_SCHEDULE_SERVICE_DOWNTIME SCHEDULE_SERVICE_DOWNTIME +NAGIOS_EC_SCHEDULE_HOST_DOWNTIME SCHEDULE_HOST_DOWNTIME +NAGIOS_EC_DISABLE_HOST_SVC_NOTIFICATIONS DISABLE_HOST_SVC_NOTIFICATIONS +NAGIOS_EC_ENABLE_HOST_SVC_NOTIFICATIONS ENABLE_HOST_SVC_NOTIFICATIONS +NAGIOS_EC_DISABLE_HOST_NOTIFICATIONS DISABLE_HOST_NOTIFICATIONS +NAGIOS_EC_ENABLE_HOST_NOTIFICATIONS ENABLE_HOST_NOTIFICATIONS +NAGIOS_EC_DISABLE_SVC_NOTIFICATIONS DISABLE_SVC_NOTIFICATIONS +NAGIOS_EC_ENABLE_SVC_NOTIFICATIONS ENABLE_SVC_NOTIFICATIONS +############################################### +######## End external check types +############################################### +NAGIOS_WARNING Warning:%{SPACE}%{GREEDYDATA:nagios_message} + +NAGIOS_CURRENT_SERVICE_STATE %{NAGIOS_TYPE_CURRENT_SERVICE_STATE:nagios_type}: %{DATA:nagios_hostname};%{DATA:nagios_service};%{DATA:nagios_state};%{DATA:nagios_statetype};%{DATA:nagios_statecode};%{GREEDYDATA:nagios_message} +NAGIOS_CURRENT_HOST_STATE %{NAGIOS_TYPE_CURRENT_HOST_STATE:nagios_type}: %{DATA:nagios_hostname};%{DATA:nagios_state};%{DATA:nagios_statetype};%{DATA:nagios_statecode};%{GREEDYDATA:nagios_message} + +NAGIOS_SERVICE_NOTIFICATION %{NAGIOS_TYPE_SERVICE_NOTIFICATION:nagios_type}: %{DATA:nagios_notifyname};%{DATA:nagios_hostname};%{DATA:nagios_service};%{DATA:nagios_state};%{DATA:nagios_contact};%{GREEDYDATA:nagios_message} +NAGIOS_HOST_NOTIFICATION %{NAGIOS_TYPE_HOST_NOTIFICATION:nagios_type}: %{DATA:nagios_notifyname};%{DATA:nagios_hostname};%{DATA:nagios_state};%{DATA:nagios_contact};%{GREEDYDATA:nagios_message} + +NAGIOS_SERVICE_ALERT %{NAGIOS_TYPE_SERVICE_ALERT:nagios_type}: %{DATA:nagios_hostname};%{DATA:nagios_service};%{DATA:nagios_state};%{DATA:nagios_statelevel};%{NUMBER:nagios_attempt};%{GREEDYDATA:nagios_message} +NAGIOS_HOST_ALERT %{NAGIOS_TYPE_HOST_ALERT:nagios_type}: %{DATA:nagios_hostname};%{DATA:nagios_state};%{DATA:nagios_statelevel};%{NUMBER:nagios_attempt};%{GREEDYDATA:nagios_message} + +NAGIOS_SERVICE_FLAPPING_ALERT %{NAGIOS_TYPE_SERVICE_FLAPPING_ALERT:nagios_type}: %{DATA:nagios_hostname};%{DATA:nagios_service};%{DATA:nagios_state};%{GREEDYDATA:nagios_message} +NAGIOS_HOST_FLAPPING_ALERT %{NAGIOS_TYPE_HOST_FLAPPING_ALERT:nagios_type}: %{DATA:nagios_hostname};%{DATA:nagios_state};%{GREEDYDATA:nagios_message} + +NAGIOS_SERVICE_DOWNTIME_ALERT %{NAGIOS_TYPE_SERVICE_DOWNTIME_ALERT:nagios_type}: %{DATA:nagios_hostname};%{DATA:nagios_service};%{DATA:nagios_state};%{GREEDYDATA:nagios_comment} +NAGIOS_HOST_DOWNTIME_ALERT %{NAGIOS_TYPE_HOST_DOWNTIME_ALERT:nagios_type}: %{DATA:nagios_hostname};%{DATA:nagios_state};%{GREEDYDATA:nagios_comment} + +NAGIOS_PASSIVE_SERVICE_CHECK %{NAGIOS_TYPE_PASSIVE_SERVICE_CHECK:nagios_type}: %{DATA:nagios_hostname};%{DATA:nagios_service};%{DATA:nagios_state};%{GREEDYDATA:nagios_comment} +NAGIOS_PASSIVE_HOST_CHECK %{NAGIOS_TYPE_PASSIVE_HOST_CHECK:nagios_type}: %{DATA:nagios_hostname};%{DATA:nagios_state};%{GREEDYDATA:nagios_comment} + +NAGIOS_SERVICE_EVENT_HANDLER %{NAGIOS_TYPE_SERVICE_EVENT_HANDLER:nagios_type}: %{DATA:nagios_hostname};%{DATA:nagios_service};%{DATA:nagios_state};%{DATA:nagios_statelevel};%{DATA:nagios_event_handler_name} +NAGIOS_HOST_EVENT_HANDLER %{NAGIOS_TYPE_HOST_EVENT_HANDLER:nagios_type}: %{DATA:nagios_hostname};%{DATA:nagios_state};%{DATA:nagios_statelevel};%{DATA:nagios_event_handler_name} + +NAGIOS_TIMEPERIOD_TRANSITION %{NAGIOS_TYPE_TIMEPERIOD_TRANSITION:nagios_type}: %{DATA:nagios_service};%{DATA:nagios_unknown1};%{DATA:nagios_unknown2} + +#################### +#### External checks +#################### + +#Disable host & service check +NAGIOS_EC_LINE_DISABLE_SVC_CHECK %{NAGIOS_TYPE_EXTERNAL_COMMAND:nagios_type}: %{NAGIOS_EC_DISABLE_SVC_CHECK:nagios_command};%{DATA:nagios_hostname};%{DATA:nagios_service} +NAGIOS_EC_LINE_DISABLE_HOST_CHECK %{NAGIOS_TYPE_EXTERNAL_COMMAND:nagios_type}: %{NAGIOS_EC_DISABLE_HOST_CHECK:nagios_command};%{DATA:nagios_hostname} + +#Enable host & service check +NAGIOS_EC_LINE_ENABLE_SVC_CHECK %{NAGIOS_TYPE_EXTERNAL_COMMAND:nagios_type}: %{NAGIOS_EC_ENABLE_SVC_CHECK:nagios_command};%{DATA:nagios_hostname};%{DATA:nagios_service} +NAGIOS_EC_LINE_ENABLE_HOST_CHECK %{NAGIOS_TYPE_EXTERNAL_COMMAND:nagios_type}: %{NAGIOS_EC_ENABLE_HOST_CHECK:nagios_command};%{DATA:nagios_hostname} + +#Process host & service check +NAGIOS_EC_LINE_PROCESS_SERVICE_CHECK_RESULT %{NAGIOS_TYPE_EXTERNAL_COMMAND:nagios_type}: %{NAGIOS_EC_PROCESS_SERVICE_CHECK_RESULT:nagios_command};%{DATA:nagios_hostname};%{DATA:nagios_service};%{DATA:nagios_state};%{GREEDYDATA:nagios_check_result} +NAGIOS_EC_LINE_PROCESS_HOST_CHECK_RESULT %{NAGIOS_TYPE_EXTERNAL_COMMAND:nagios_type}: %{NAGIOS_EC_PROCESS_HOST_CHECK_RESULT:nagios_command};%{DATA:nagios_hostname};%{DATA:nagios_state};%{GREEDYDATA:nagios_check_result} + +#Disable host & service notifications +NAGIOS_EC_LINE_DISABLE_HOST_SVC_NOTIFICATIONS %{NAGIOS_TYPE_EXTERNAL_COMMAND:nagios_type}: %{NAGIOS_EC_DISABLE_HOST_SVC_NOTIFICATIONS:nagios_command};%{GREEDYDATA:nagios_hostname} +NAGIOS_EC_LINE_DISABLE_HOST_NOTIFICATIONS %{NAGIOS_TYPE_EXTERNAL_COMMAND:nagios_type}: %{NAGIOS_EC_DISABLE_HOST_NOTIFICATIONS:nagios_command};%{GREEDYDATA:nagios_hostname} +NAGIOS_EC_LINE_DISABLE_SVC_NOTIFICATIONS %{NAGIOS_TYPE_EXTERNAL_COMMAND:nagios_type}: %{NAGIOS_EC_DISABLE_SVC_NOTIFICATIONS:nagios_command};%{DATA:nagios_hostname};%{GREEDYDATA:nagios_service} + +#Enable host & service notifications +NAGIOS_EC_LINE_ENABLE_HOST_SVC_NOTIFICATIONS %{NAGIOS_TYPE_EXTERNAL_COMMAND:nagios_type}: %{NAGIOS_EC_ENABLE_HOST_SVC_NOTIFICATIONS:nagios_command};%{GREEDYDATA:nagios_hostname} +NAGIOS_EC_LINE_ENABLE_HOST_NOTIFICATIONS %{NAGIOS_TYPE_EXTERNAL_COMMAND:nagios_type}: %{NAGIOS_EC_ENABLE_HOST_NOTIFICATIONS:nagios_command};%{GREEDYDATA:nagios_hostname} +NAGIOS_EC_LINE_ENABLE_SVC_NOTIFICATIONS %{NAGIOS_TYPE_EXTERNAL_COMMAND:nagios_type}: %{NAGIOS_EC_ENABLE_SVC_NOTIFICATIONS:nagios_command};%{DATA:nagios_hostname};%{GREEDYDATA:nagios_service} + +#Schedule host & service downtime +NAGIOS_EC_LINE_SCHEDULE_HOST_DOWNTIME %{NAGIOS_TYPE_EXTERNAL_COMMAND:nagios_type}: %{NAGIOS_EC_SCHEDULE_HOST_DOWNTIME:nagios_command};%{DATA:nagios_hostname};%{NUMBER:nagios_start_time};%{NUMBER:nagios_end_time};%{NUMBER:nagios_fixed};%{NUMBER:nagios_trigger_id};%{NUMBER:nagios_duration};%{DATA:author};%{DATA:comment} + +#End matching line +NAGIOSLOGLINE %{NAGIOSTIME} (?:%{NAGIOS_WARNING}|%{NAGIOS_CURRENT_SERVICE_STATE}|%{NAGIOS_CURRENT_HOST_STATE}|%{NAGIOS_SERVICE_NOTIFICATION}|%{NAGIOS_HOST_NOTIFICATION}|%{NAGIOS_SERVICE_ALERT}|%{NAGIOS_HOST_ALERT}|%{NAGIOS_SERVICE_FLAPPING_ALERT}|%{NAGIOS_HOST_FLAPPING_ALERT}|%{NAGIOS_SERVICE_DOWNTIME_ALERT}|%{NAGIOS_HOST_DOWNTIME_ALERT}|%{NAGIOS_PASSIVE_SERVICE_CHECK}|%{NAGIOS_PASSIVE_HOST_CHECK}|%{NAGIOS_SERVICE_EVENT_HANDLER}|%{NAGIOS_HOST_EVENT_HANDLER}|%{NAGIOS_TIMEPERIOD_TRANSITION}|%{NAGIOS_EC_LINE_DISABLE_SVC_CHECK}|%{NAGIOS_EC_LINE_ENABLE_SVC_CHECK}|%{NAGIOS_EC_LINE_DISABLE_HOST_CHECK}|%{NAGIOS_EC_LINE_ENABLE_HOST_CHECK}|%{NAGIOS_EC_LINE_PROCESS_HOST_CHECK_RESULT}|%{NAGIOS_EC_LINE_PROCESS_SERVICE_CHECK_RESULT}|%{NAGIOS_EC_LINE_SCHEDULE_HOST_DOWNTIME}|%{NAGIOS_EC_LINE_DISABLE_HOST_SVC_NOTIFICATIONS}|%{NAGIOS_EC_LINE_ENABLE_HOST_SVC_NOTIFICATIONS}|%{NAGIOS_EC_LINE_DISABLE_HOST_NOTIFICATIONS}|%{NAGIOS_EC_LINE_ENABLE_HOST_NOTIFICATIONS}|%{NAGIOS_EC_LINE_DISABLE_SVC_NOTIFICATIONS}|%{NAGIOS_EC_LINE_ENABLE_SVC_NOTIFICATIONS}) diff --git a/modules/ingest-grok/src/main/resources/patterns/postgresql b/modules/ingest-grok/src/main/resources/patterns/postgresql new file mode 100644 index 00000000000..c5b3e90b725 --- /dev/null +++ b/modules/ingest-grok/src/main/resources/patterns/postgresql @@ -0,0 +1,3 @@ +# Default postgresql pg_log format pattern +POSTGRESQL %{DATESTAMP:timestamp} %{TZ} %{DATA:user_id} %{GREEDYDATA:connection_id} %{POSINT:pid} + diff --git a/modules/ingest-grok/src/main/resources/patterns/rails b/modules/ingest-grok/src/main/resources/patterns/rails new file mode 100644 index 00000000000..68a50c734cb --- /dev/null +++ b/modules/ingest-grok/src/main/resources/patterns/rails @@ -0,0 +1,13 @@ +RUUID \h{32} +# rails controller with action +RCONTROLLER (?[^#]+)#(?\w+) + +# this will often be the only line: +RAILS3HEAD (?m)Started %{WORD:verb} "%{URIPATHPARAM:request}" for %{IPORHOST:clientip} at (?%{YEAR}-%{MONTHNUM}-%{MONTHDAY} %{HOUR}:%{MINUTE}:%{SECOND} %{ISO8601_TIMEZONE}) +# for some a strange reason, params are stripped of {} - not sure that's a good idea. +RPROCESSING \W*Processing by %{RCONTROLLER} as (?\S+)(?:\W*Parameters: {%{DATA:params}}\W*)? +RAILS3FOOT Completed %{NUMBER:response}%{DATA} in %{NUMBER:totalms}ms %{RAILS3PROFILE}%{GREEDYDATA} +RAILS3PROFILE (?:\(Views: %{NUMBER:viewms}ms \| ActiveRecord: %{NUMBER:activerecordms}ms|\(ActiveRecord: %{NUMBER:activerecordms}ms)? + +# putting it all together +RAILS3 %{RAILS3HEAD}(?:%{RPROCESSING})?(?(?:%{DATA}\n)*)(?:%{RAILS3FOOT})? diff --git a/modules/ingest-grok/src/main/resources/patterns/redis b/modules/ingest-grok/src/main/resources/patterns/redis new file mode 100644 index 00000000000..8655c4f043e --- /dev/null +++ b/modules/ingest-grok/src/main/resources/patterns/redis @@ -0,0 +1,3 @@ +REDISTIMESTAMP %{MONTHDAY} %{MONTH} %{TIME} +REDISLOG \[%{POSINT:pid}\] %{REDISTIMESTAMP:timestamp} \* + diff --git a/modules/ingest-grok/src/main/resources/patterns/ruby b/modules/ingest-grok/src/main/resources/patterns/ruby new file mode 100644 index 00000000000..b1729cddcb0 --- /dev/null +++ b/modules/ingest-grok/src/main/resources/patterns/ruby @@ -0,0 +1,2 @@ +RUBY_LOGLEVEL (?:DEBUG|FATAL|ERROR|WARN|INFO) +RUBY_LOGGER [DFEWI], \[%{TIMESTAMP_ISO8601:timestamp} #%{POSINT:pid}\] *%{RUBY_LOGLEVEL:loglevel} -- +%{DATA:progname}: %{GREEDYDATA:message} diff --git a/modules/ingest-grok/src/test/java/org/elasticsearch/ingest/grok/GrokProcessorFactoryTests.java b/modules/ingest-grok/src/test/java/org/elasticsearch/ingest/grok/GrokProcessorFactoryTests.java new file mode 100644 index 00000000000..f6bed139552 --- /dev/null +++ b/modules/ingest-grok/src/test/java/org/elasticsearch/ingest/grok/GrokProcessorFactoryTests.java @@ -0,0 +1,60 @@ +/* + * 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.grok; + +import org.elasticsearch.ingest.core.AbstractProcessorFactory; +import org.elasticsearch.test.ESTestCase; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.notNullValue; + +public class GrokProcessorFactoryTests extends ESTestCase { + + public void testBuild() throws Exception { + GrokProcessor.Factory factory = new GrokProcessor.Factory(Collections.emptyMap()); + + Map config = new HashMap<>(); + config.put("field", "_field"); + config.put("pattern", "(?\\w+)"); + String processorTag = randomAsciiOfLength(10); + config.put(AbstractProcessorFactory.TAG_KEY, processorTag); + GrokProcessor processor = factory.create(config); + assertThat(processor.getTag(), equalTo(processorTag)); + assertThat(processor.getMatchField(), equalTo("_field")); + assertThat(processor.getGrok(), notNullValue()); + } + + public void testCreateWithCustomPatterns() throws Exception { + GrokProcessor.Factory factory = new GrokProcessor.Factory(Collections.emptyMap()); + + Map config = new HashMap<>(); + config.put("field", "_field"); + config.put("pattern", "%{MY_PATTERN:name}!"); + config.put("pattern_definitions", Collections.singletonMap("MY_PATTERN", "foo")); + GrokProcessor processor = factory.create(config); + assertThat(processor.getMatchField(), equalTo("_field")); + assertThat(processor.getGrok(), notNullValue()); + assertThat(processor.getGrok().match("foo!"), equalTo(true)); + } +} diff --git a/modules/ingest-grok/src/test/java/org/elasticsearch/ingest/grok/GrokProcessorTests.java b/modules/ingest-grok/src/test/java/org/elasticsearch/ingest/grok/GrokProcessorTests.java new file mode 100644 index 00000000000..840cf954022 --- /dev/null +++ b/modules/ingest-grok/src/test/java/org/elasticsearch/ingest/grok/GrokProcessorTests.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.ingest.grok; + +import org.elasticsearch.ingest.RandomDocumentPicks; +import org.elasticsearch.ingest.core.IngestDocument; +import org.elasticsearch.ingest.grok.Grok; +import org.elasticsearch.ingest.grok.GrokProcessor; +import org.elasticsearch.test.ESTestCase; + +import java.util.Collections; +import java.util.HashMap; + +import static org.hamcrest.Matchers.equalTo; + + +public class GrokProcessorTests extends ESTestCase { + + public void testMatch() throws Exception { + String fieldName = RandomDocumentPicks.randomFieldName(random()); + IngestDocument doc = RandomDocumentPicks.randomIngestDocument(random(), new HashMap<>()); + doc.setFieldValue(fieldName, "1"); + Grok grok = new Grok(Collections.singletonMap("ONE", "1"), "%{ONE:one}"); + GrokProcessor processor = new GrokProcessor(randomAsciiOfLength(10), grok, fieldName); + processor.execute(doc); + assertThat(doc.getFieldValue("one", String.class), equalTo("1")); + } + + public void testNoMatch() { + String fieldName = RandomDocumentPicks.randomFieldName(random()); + IngestDocument doc = RandomDocumentPicks.randomIngestDocument(random(), new HashMap<>()); + doc.setFieldValue(fieldName, "23"); + Grok grok = new Grok(Collections.singletonMap("ONE", "1"), "%{ONE:one}"); + GrokProcessor processor = new GrokProcessor(randomAsciiOfLength(10), grok, fieldName); + try { + processor.execute(doc); + fail(); + } catch (Exception e) { + assertThat(e.getMessage(), equalTo("Grok expression does not match field value: [23]")); + } + } + + public void testMatchWithoutCaptures() throws Exception { + String fieldName = "value"; + IngestDocument originalDoc = new IngestDocument(new HashMap<>(), new HashMap<>()); + originalDoc.setFieldValue(fieldName, fieldName); + IngestDocument doc = new IngestDocument(originalDoc); + Grok grok = new Grok(Collections.emptyMap(), fieldName); + GrokProcessor processor = new GrokProcessor(randomAsciiOfLength(10), grok, fieldName); + processor.execute(doc); + assertThat(doc, equalTo(originalDoc)); + } + + public void testNotStringField() { + String fieldName = RandomDocumentPicks.randomFieldName(random()); + IngestDocument doc = RandomDocumentPicks.randomIngestDocument(random(), new HashMap<>()); + doc.setFieldValue(fieldName, 1); + Grok grok = new Grok(Collections.singletonMap("ONE", "1"), "%{ONE:one}"); + GrokProcessor processor = new GrokProcessor(randomAsciiOfLength(10), grok, fieldName); + try { + processor.execute(doc); + fail(); + } catch (Exception e) { + assertThat(e.getMessage(), equalTo("field [" + fieldName + "] of type [java.lang.Integer] cannot be cast to [java.lang.String]")); + } + } + + public void testMissingField() { + String fieldName = "foo.bar"; + IngestDocument doc = RandomDocumentPicks.randomIngestDocument(random(), new HashMap<>()); + Grok grok = new Grok(Collections.singletonMap("ONE", "1"), "%{ONE:one}"); + GrokProcessor processor = new GrokProcessor(randomAsciiOfLength(10), grok, fieldName); + try { + processor.execute(doc); + fail(); + } catch (Exception e) { + assertThat(e.getMessage(), equalTo("field [foo] not present as part of path [foo.bar]")); + } + } +} diff --git a/modules/ingest-grok/src/test/java/org/elasticsearch/ingest/grok/GrokTests.java b/modules/ingest-grok/src/test/java/org/elasticsearch/ingest/grok/GrokTests.java new file mode 100644 index 00000000000..21ca17ad713 --- /dev/null +++ b/modules/ingest-grok/src/test/java/org/elasticsearch/ingest/grok/GrokTests.java @@ -0,0 +1,285 @@ +/* + * 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.grok; + +import org.elasticsearch.test.ESTestCase; +import org.junit.Before; + +import java.io.IOException; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.nullValue; + + +public class GrokTests extends ESTestCase { + private Map basePatterns; + + @Before + public void setup() throws IOException { + basePatterns = IngestGrokPlugin.loadBuiltinPatterns(); + } + + public void testMatchWithoutCaptures() { + String line = "value"; + Grok grok = new Grok(basePatterns, "value"); + Map matches = grok.captures(line); + assertEquals(0, matches.size()); + } + + public void testSimpleSyslogLine() { + String line = "Mar 16 00:01:25 evita postfix/smtpd[1713]: connect from camomile.cloud9.net[168.100.1.3]"; + Grok grok = new Grok(basePatterns, "%{SYSLOGLINE}"); + Map matches = grok.captures(line); + assertEquals("evita", matches.get("logsource")); + assertEquals("Mar 16 00:01:25", matches.get("timestamp")); + assertEquals("connect from camomile.cloud9.net[168.100.1.3]", matches.get("message")); + assertEquals("postfix/smtpd", matches.get("program")); + assertEquals("1713", matches.get("pid")); + } + + public void testSyslog5424Line() { + String line = "<191>1 2009-06-30T18:30:00+02:00 paxton.local grokdebug 4123 - [id1 foo=\\\"bar\\\"][id2 baz=\\\"something\\\"] Hello, syslog."; + Grok grok = new Grok(basePatterns, "%{SYSLOG5424LINE}"); + Map matches = grok.captures(line); + assertEquals("191", matches.get("syslog5424_pri")); + assertEquals("1", matches.get("syslog5424_ver")); + assertEquals("2009-06-30T18:30:00+02:00", matches.get("syslog5424_ts")); + assertEquals("paxton.local", matches.get("syslog5424_host")); + assertEquals("grokdebug", matches.get("syslog5424_app")); + assertEquals("4123", matches.get("syslog5424_proc")); + assertEquals(null, matches.get("syslog5424_msgid")); + assertEquals("[id1 foo=\\\"bar\\\"][id2 baz=\\\"something\\\"]", matches.get("syslog5424_sd")); + assertEquals("Hello, syslog.", matches.get("syslog5424_msg")); + } + + public void testDatePattern() { + String line = "fancy 12-12-12 12:12:12"; + Grok grok = new Grok(basePatterns, "(?%{DATE_EU} %{TIME})"); + Map matches = grok.captures(line); + assertEquals("12-12-12 12:12:12", matches.get("timestamp")); + } + + public void testNilCoercedValues() { + Grok grok = new Grok(basePatterns, "test (N/A|%{BASE10NUM:duration:float}ms)"); + Map matches = grok.captures("test 28.4ms"); + assertEquals(28.4f, matches.get("duration")); + matches = grok.captures("test N/A"); + assertEquals(null, matches.get("duration")); + } + + public void testNilWithNoCoercion() { + Grok grok = new Grok(basePatterns, "test (N/A|%{BASE10NUM:duration}ms)"); + Map matches = grok.captures("test 28.4ms"); + assertEquals("28.4", matches.get("duration")); + matches = grok.captures("test N/A"); + assertEquals(null, matches.get("duration")); + } + + public void testUnicodeSyslog() { + Grok grok = new Grok(basePatterns, "<%{POSINT:syslog_pri}>%{SPACE}%{SYSLOGTIMESTAMP:syslog_timestamp} %{SYSLOGHOST:syslog_hostname} %{PROG:syslog_program}(:?)(?:\\[%{GREEDYDATA:syslog_pid}\\])?(:?) %{GREEDYDATA:syslog_message}"); + Map matches = grok.captures("<22>Jan 4 07:50:46 mailmaster postfix/policy-spf[9454]: : SPF permerror (Junk encountered in record 'v=spf1 mx a:mail.domain.no ip4:192.168.0.4 �all'): Envelope-from: email@domain.no"); + assertThat(matches.get("syslog_pri"), equalTo("22")); + assertThat(matches.get("syslog_program"), equalTo("postfix/policy-spf")); + assertThat(matches.get("tags"), nullValue()); + } + + public void testNamedFieldsWithWholeTextMatch() { + Grok grok = new Grok(basePatterns, "%{DATE_EU:stimestamp}"); + Map matches = grok.captures("11/01/01"); + assertThat(matches.get("stimestamp"), equalTo("11/01/01")); + } + + public void testWithOniguramaNamedCaptures() { + Grok grok = new Grok(basePatterns, "(?\\w+)"); + Map matches = grok.captures("hello world"); + assertThat(matches.get("foo"), equalTo("hello")); + } + + public void testISO8601() { + Grok grok = new Grok(basePatterns, "^%{TIMESTAMP_ISO8601}$"); + List timeMessages = Arrays.asList( + "2001-01-01T00:00:00", + "1974-03-02T04:09:09", + "2010-05-03T08:18:18+00:00", + "2004-07-04T12:27:27-00:00", + "2001-09-05T16:36:36+0000", + "2001-11-06T20:45:45-0000", + "2001-12-07T23:54:54Z", + "2001-01-01T00:00:00.123456", + "1974-03-02T04:09:09.123456", + "2010-05-03T08:18:18.123456+00:00", + "2004-07-04T12:27:27.123456-00:00", + "2001-09-05T16:36:36.123456+0000", + "2001-11-06T20:45:45.123456-0000", + "2001-12-07T23:54:54.123456Z", + "2001-12-07T23:54:60.123456Z" // '60' second is a leap second. + ); + for (String msg : timeMessages) { + assertThat(grok.match(msg), is(true)); + } + } + + public void testNotISO8601() { + Grok grok = new Grok(basePatterns, "^%{TIMESTAMP_ISO8601}$"); + List timeMessages = Arrays.asList( + "2001-13-01T00:00:00", // invalid month + "2001-00-01T00:00:00", // invalid month + "2001-01-00T00:00:00", // invalid day + "2001-01-32T00:00:00", // invalid day + "2001-01-aT00:00:00", // invalid day + "2001-01-1aT00:00:00", // invalid day + "2001-01-01Ta0:00:00", // invalid hour + "2001-01-01T0:00:00", // invalid hour + "2001-01-01T25:00:00", // invalid hour + "2001-01-01T01:60:00", // invalid minute + "2001-01-01T00:aa:00", // invalid minute + "2001-01-01T00:00:aa", // invalid second + "2001-01-01T00:00:-1", // invalid second + "2001-01-01T00:00:61", // invalid second + "2001-01-01T00:00:00A", // invalid timezone + "2001-01-01T00:00:00+", // invalid timezone + "2001-01-01T00:00:00+25", // invalid timezone + "2001-01-01T00:00:00+2500", // invalid timezone + "2001-01-01T00:00:00+25:00", // invalid timezone + "2001-01-01T00:00:00-25", // invalid timezone + "2001-01-01T00:00:00-2500", // invalid timezone + "2001-01-01T00:00:00-00:61" // invalid timezone + ); + for (String msg : timeMessages) { + assertThat(grok.match(msg), is(false)); + } + } + + public void testNoNamedCaptures() { + Map bank = new HashMap<>(); + + bank.put("NAME", "Tal"); + bank.put("EXCITED_NAME", "!!!%{NAME:name}!!!"); + bank.put("TEST", "hello world"); + + String text = "wowza !!!Tal!!! - Tal"; + String pattern = "%{EXCITED_NAME} - %{NAME}"; + Grok g = new Grok(bank, pattern, false); + + assertEquals("(?!!!(?Tal)!!!) - (?Tal)", g.toRegex(pattern)); + assertEquals(true, g.match(text)); + + Object actual = g.captures(text); + Map expected = new HashMap<>(); + expected.put("EXCITED_NAME_0", "!!!Tal!!!"); + expected.put("NAME_21", "Tal"); + expected.put("NAME_22", "Tal"); + assertEquals(expected, actual); + } + + public void testNumericCapturesCoercion() { + Map bank = new HashMap<>(); + bank.put("BASE10NUM", "(?[+-]?(?:(?:[0-9]+(?:\\.[0-9]+)?)|(?:\\.[0-9]+)))"); + bank.put("NUMBER", "(?:%{BASE10NUM})"); + + String pattern = "%{NUMBER:bytes:float} %{NUMBER:status} %{NUMBER}"; + Grok g = new Grok(bank, pattern); + + String text = "12009.34 200 9032"; + Map expected = new HashMap<>(); + expected.put("bytes", 12009.34f); + expected.put("status", "200"); + Map actual = g.captures(text); + + assertEquals(expected, actual); + } + + public void testApacheLog() { + String logLine = "31.184.238.164 - - [24/Jul/2014:05:35:37 +0530] \"GET /logs/access.log HTTP/1.0\" 200 69849 \"http://8rursodiol.enjin.com\" \"Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/30.0.1599.12785 YaBrowser/13.12.1599.12785 Safari/537.36\" \"www.dlwindianrailways.com\""; + Grok grok = new Grok(basePatterns, "%{COMBINEDAPACHELOG}"); + Map matches = grok.captures(logLine); + + assertEquals("31.184.238.164", matches.get("clientip")); + assertEquals("-", matches.get("ident")); + assertEquals("-", matches.get("auth")); + assertEquals("24/Jul/2014:05:35:37 +0530", matches.get("timestamp")); + assertEquals("GET", matches.get("verb")); + assertEquals("/logs/access.log", matches.get("request")); + assertEquals("1.0", matches.get("httpversion")); + assertEquals("200", matches.get("response")); + assertEquals("69849", matches.get("bytes")); + assertEquals("\"http://8rursodiol.enjin.com\"", matches.get("referrer")); + assertEquals(null, matches.get("port")); + assertEquals("\"Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/30.0.1599.12785 YaBrowser/13.12.1599.12785 Safari/537.36\"", matches.get("agent")); + } + + public void testComplete() { + Map bank = new HashMap<>(); + bank.put("MONTHDAY", "(?:(?:0[1-9])|(?:[12][0-9])|(?:3[01])|[1-9])"); + bank.put("MONTH", "\\b(?:Jan(?:uary|uar)?|Feb(?:ruary|ruar)?|M(?:a|ä)?r(?:ch|z)?|Apr(?:il)?|Ma(?:y|i)?|Jun(?:e|i)?|Jul(?:y)?|Aug(?:ust)?|Sep(?:tember)?|O(?:c|k)?t(?:ober)?|Nov(?:ember)?|De(?:c|z)(?:ember)?)\\b"); + bank.put("MINUTE", "(?:[0-5][0-9])"); + bank.put("YEAR", "(?>\\d\\d){1,2}"); + bank.put("HOUR", "(?:2[0123]|[01]?[0-9])"); + bank.put("SECOND", "(?:(?:[0-5]?[0-9]|60)(?:[:.,][0-9]+)?)"); + bank.put("TIME", "(?!<[0-9])%{HOUR}:%{MINUTE}(?::%{SECOND})(?![0-9])"); + bank.put("INT", "(?:[+-]?(?:[0-9]+))"); + bank.put("HTTPDATE", "%{MONTHDAY}/%{MONTH}/%{YEAR}:%{TIME} %{INT}"); + bank.put("WORD", "\\b\\w+\\b"); + bank.put("BASE10NUM", "(?[+-]?(?:(?:[0-9]+(?:\\.[0-9]+)?)|(?:\\.[0-9]+)))"); + bank.put("NUMBER", "(?:%{BASE10NUM})"); + bank.put("IPV6", "((([0-9A-Fa-f]{1,4}:){7}([0-9A-Fa-f]{1,4}|:))|(([0-9A-Fa-f]{1,4}:){6}(:[0-9A-Fa-f]{1,4}|((25[0-5]|2[0-4]\\d|1\\d\\d|[1-9]?\\d)(\\.(25[0-5]|2[0-4]\\d|1\\d\\d|[1-9]?\\d)){3})|:))|(([0-9A-Fa-f]{1,4}:){5}(((:[0-9A-Fa-f]{1,4}){1,2})|:((25[0-5]|2[0-4]\\d|1\\d\\d|[1-9]?\\d)(\\.(25[0-5]|2[0-4]\\d|1\\d\\d|[1-9]?\\d)){3})|:))|(([0-9A-Fa-f]{1,4}:){4}(((:[0-9A-Fa-f]{1,4}){1,3})|((:[0-9A-Fa-f]{1,4})?:((25[0-5]|2[0-4]\\d|1\\d\\d|[1-9]?\\d)(\\.(25[0-5]|2[0-4]\\d|1\\d\\d|[1-9]?\\d)){3}))|:))|(([0-9A-Fa-f]{1,4}:){3}(((:[0-9A-Fa-f]{1,4}){1,4})|((:[0-9A-Fa-f]{1,4}){0,2}:((25[0-5]|2[0-4]\\d|1\\d\\d|[1-9]?\\d)(\\.(25[0-5]|2[0-4]\\d|1\\d\\d|[1-9]?\\d)){3}))|:))|(([0-9A-Fa-f]{1,4}:){2}(((:[0-9A-Fa-f]{1,4}){1,5})|((:[0-9A-Fa-f]{1,4}){0,3}:((25[0-5]|2[0-4]\\d|1\\d\\d|[1-9]?\\d)(\\.(25[0-5]|2[0-4]\\d|1\\d\\d|[1-9]?\\d)){3}))|:))|(([0-9A-Fa-f]{1,4}:){1}(((:[0-9A-Fa-f]{1,4}){1,6})|((:[0-9A-Fa-f]{1,4}){0,4}:((25[0-5]|2[0-4]\\d|1\\d\\d|[1-9]?\\d)(\\.(25[0-5]|2[0-4]\\d|1\\d\\d|[1-9]?\\d)){3}))|:))|(:(((:[0-9A-Fa-f]{1,4}){1,7})|((:[0-9A-Fa-f]{1,4}){0,5}:((25[0-5]|2[0-4]\\d|1\\d\\d|[1-9]?\\d)(\\.(25[0-5]|2[0-4]\\d|1\\d\\d|[1-9]?\\d)){3}))|:)))(%.+)?"); + bank.put("IPV4", "(?(?\"(?>\\\\.|[^\\\\\"]+)+\"|\"\"|(?>'(?>\\\\.|[^\\\\']+)+')|''|(?>`(?>\\\\.|[^\\\\`]+)+`)|``))"); + + String text = "83.149.9.216 - - [19/Jul/2015:08:13:42 +0000] \"GET /presentations/logstash-monitorama-2013/images/kibana-dashboard3.png HTTP/1.1\" 200 171717 \"http://semicomplete.com/presentations/logstash-monitorama-2013/\" \"Mozilla/5.0 (Macintosh; Intel Mac OS X 10_9_1) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/32.0.1700.77 Safari/537.36\""; + String pattern = "%{IPORHOST:clientip} %{USER:ident} %{USER:auth} \\[%{HTTPDATE:timestamp}\\] \"%{WORD:verb} %{DATA:request} HTTP/%{NUMBER:httpversion}\" %{NUMBER:response:int} (?:-|%{NUMBER:bytes:int}) %{QS:referrer} %{QS:agent}"; + + Grok grok = new Grok(bank, pattern); + + Map expected = new HashMap<>(); + expected.put("clientip", "83.149.9.216"); + expected.put("ident", "-"); + expected.put("auth", "-"); + expected.put("timestamp", "19/Jul/2015:08:13:42 +0000"); + expected.put("verb", "GET"); + expected.put("request", "/presentations/logstash-monitorama-2013/images/kibana-dashboard3.png"); + expected.put("httpversion", "1.1"); + expected.put("response", 200); + expected.put("bytes", 171717); + expected.put("referrer", "\"http://semicomplete.com/presentations/logstash-monitorama-2013/\""); + expected.put("agent", "\"Mozilla/5.0 (Macintosh; Intel Mac OS X 10_9_1) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/32.0.1700.77 Safari/537.36\""); + + Map actual = grok.captures(text); + + assertEquals(expected, actual); + } + + public void testNoMatch() { + Map bank = new HashMap<>(); + bank.put("MONTHDAY", "(?:(?:0[1-9])|(?:[12][0-9])|(?:3[01])|[1-9])"); + Grok grok = new Grok(bank, "%{MONTHDAY:greatday}"); + assertThat(grok.captures("nomatch"), nullValue()); + } +} diff --git a/modules/ingest-grok/src/test/java/org/elasticsearch/ingest/grok/IngestGrokRestIT.java b/modules/ingest-grok/src/test/java/org/elasticsearch/ingest/grok/IngestGrokRestIT.java new file mode 100644 index 00000000000..3f4bdf1f8b9 --- /dev/null +++ b/modules/ingest-grok/src/test/java/org/elasticsearch/ingest/grok/IngestGrokRestIT.java @@ -0,0 +1,44 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.ingest.grok; + +import com.carrotsearch.randomizedtesting.annotations.Name; +import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; +import org.elasticsearch.ingest.grok.IngestGrokPlugin; +import org.elasticsearch.plugins.Plugin; +import org.elasticsearch.test.rest.ESRestTestCase; +import org.elasticsearch.test.rest.RestTestCandidate; +import org.elasticsearch.test.rest.parser.RestTestParseException; + +import java.io.IOException; +import java.util.Collection; + +public class IngestGrokRestIT extends ESRestTestCase { + + public IngestGrokRestIT(@Name("yaml") RestTestCandidate testCandidate) { + super(testCandidate); + } + + @ParametersFactory + public static Iterable parameters() throws IOException, RestTestParseException { + return ESRestTestCase.createParameters(0, 1); + } +} + diff --git a/modules/ingest-grok/src/test/resources/rest-api-spec/test/ingest_grok/10_basic.yaml b/modules/ingest-grok/src/test/resources/rest-api-spec/test/ingest_grok/10_basic.yaml new file mode 100644 index 00000000000..5c0cca3772e --- /dev/null +++ b/modules/ingest-grok/src/test/resources/rest-api-spec/test/ingest_grok/10_basic.yaml @@ -0,0 +1,11 @@ +"Ingest grok installed": + - do: + cluster.state: {} + + # Get master node id + - set: { master_node: master } + + - do: + nodes.info: {} + + - match: { nodes.$master.modules.0.name: ingest-grok } diff --git a/modules/ingest-grok/src/test/resources/rest-api-spec/test/ingest_grok/20_grok.yaml b/modules/ingest-grok/src/test/resources/rest-api-spec/test/ingest_grok/20_grok.yaml new file mode 100644 index 00000000000..f88136d8a79 --- /dev/null +++ b/modules/ingest-grok/src/test/resources/rest-api-spec/test/ingest_grok/20_grok.yaml @@ -0,0 +1,109 @@ +--- +"Test Grok Pipeline": + - do: + ingest.put_pipeline: + id: "my_pipeline" + body: > + { + "description": "_description", + "processors": [ + { + "grok" : { + "field" : "field1", + "pattern" : "%{NUMBER:val:float} %{NUMBER:status:int} <%{WORD:msg}>" + } + } + ] + } + - match: { acknowledged: true } + + - do: + index: + index: test + type: test + id: 1 + pipeline: "my_pipeline" + body: {field1: "123.42 400 "} + + - do: + get: + index: test + type: test + id: 1 + - match: { _source.val: 123.42 } + - match: { _source.status: 400 } + - match: { _source.msg: "foo" } + +--- +"Test Grok Pipeline With Custom Pattern": + - do: + ingest.put_pipeline: + id: "my_pipeline" + body: > + { + "description": "_description", + "processors": [ + { + "grok" : { + "field" : "field1", + "pattern" : "<%{MY_PATTERN:msg}>", + "pattern_definitions" : { + "MY_PATTERN" : "foo" + } + } + } + ] + } + - match: { acknowledged: true } + + - do: + index: + index: test + type: test + id: 1 + pipeline: "my_pipeline" + body: {field1: ""} + + - do: + get: + index: test + type: test + id: 1 + - match: { _source.msg: "foo" } + +--- +"Test Grok Pipeline With Custom Pattern Sharing Same Name As Another": + - do: + ingest.put_pipeline: + id: "my_pipeline" + body: > + { + "description": "_description", + "processors": [ + { + "grok" : { + "field" : "field1", + "pattern" : "<%{NUMBER:msg}>", + "pattern_definitions" : { + "NUMBER" : "foo" + } + } + } + ] + } + - match: { acknowledged: true } + + - do: + index: + index: test + type: test + id: 1 + pipeline: "my_pipeline" + body: {field1: ""} + + - do: + get: + index: test + type: test + id: 1 + - match: { _source.msg: "foo" } diff --git a/plugins/ingest-geoip/build.gradle b/plugins/ingest-geoip/build.gradle new file mode 100644 index 00000000000..7eee668793c --- /dev/null +++ b/plugins/ingest-geoip/build.gradle @@ -0,0 +1,63 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +esplugin { + description 'Ingest processor that uses looksup geo data based on ip adresses using the Maxmind geo database' + classname 'org.elasticsearch.ingest.geoip.IngestGeoIpPlugin' +} + +dependencies { + compile ('com.maxmind.geoip2:geoip2:2.4.0') + // geoip2 dependencies: + compile('com.fasterxml.jackson.core:jackson-annotations:2.5.0') + compile('com.fasterxml.jackson.core:jackson-databind:2.5.3') + compile('com.maxmind.db:maxmind-db:1.0.1') + + testCompile 'org.elasticsearch:geolite2-databases:20151029' +} + +task copyDefaultGeoIp2DatabaseFiles(type: Copy) { + from { zipTree(configurations.testCompile.files.find { it.name.contains('geolite2-databases')}) } + into "${project.buildDir}/ingest-geoip" + include "*.mmdb" +} + +project.bundlePlugin.dependsOn(copyDefaultGeoIp2DatabaseFiles) + +compileJava.options.compilerArgs << "-Xlint:-rawtypes,-unchecked,-serial" +compileTestJava.options.compilerArgs << "-Xlint:-rawtypes,-unchecked" + +bundlePlugin { + from("${project.buildDir}/ingest-geoip") { + into 'config/' + } +} + +thirdPartyAudit.excludes = [ + // geoip WebServiceClient needs Google http client, but we're not using WebServiceClient: + 'com.google.api.client.http.HttpTransport', + 'com.google.api.client.http.GenericUrl', + 'com.google.api.client.http.HttpResponse', + 'com.google.api.client.http.HttpRequestFactory', + 'com.google.api.client.http.HttpRequest', + 'com.google.api.client.http.HttpHeaders', + 'com.google.api.client.http.HttpResponseException', + 'com.google.api.client.http.javanet.NetHttpTransport', + 'com.google.api.client.http.javanet.NetHttpTransport', +] \ No newline at end of file diff --git a/plugins/ingest-geoip/licenses/geoip2-2.4.0.jar.sha1 b/plugins/ingest-geoip/licenses/geoip2-2.4.0.jar.sha1 new file mode 100644 index 00000000000..485286f06a4 --- /dev/null +++ b/plugins/ingest-geoip/licenses/geoip2-2.4.0.jar.sha1 @@ -0,0 +1 @@ +ad40667ae87138e0aed075d2c15884497fa64acc \ No newline at end of file diff --git a/plugins/ingest-geoip/licenses/geoip2-LICENSE.txt b/plugins/ingest-geoip/licenses/geoip2-LICENSE.txt new file mode 100644 index 00000000000..7a4a3ea2424 --- /dev/null +++ b/plugins/ingest-geoip/licenses/geoip2-LICENSE.txt @@ -0,0 +1,202 @@ + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + Licensed 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. \ No newline at end of file diff --git a/plugins/ingest-geoip/licenses/geoip2-NOTICE.txt b/plugins/ingest-geoip/licenses/geoip2-NOTICE.txt new file mode 100644 index 00000000000..448b71d47d3 --- /dev/null +++ b/plugins/ingest-geoip/licenses/geoip2-NOTICE.txt @@ -0,0 +1,3 @@ +This software is Copyright (c) 2013 by MaxMind, Inc. + +This is free software, licensed under the Apache License, Version 2.0. \ No newline at end of file diff --git a/plugins/ingest-geoip/licenses/jackson-annotations-2.5.0.jar.sha1 b/plugins/ingest-geoip/licenses/jackson-annotations-2.5.0.jar.sha1 new file mode 100644 index 00000000000..862ac6f304f --- /dev/null +++ b/plugins/ingest-geoip/licenses/jackson-annotations-2.5.0.jar.sha1 @@ -0,0 +1 @@ +a2a55a3375bc1cef830ca426d68d2ea22961190e diff --git a/plugins/ingest-geoip/licenses/jackson-annotations-LICENSE b/plugins/ingest-geoip/licenses/jackson-annotations-LICENSE new file mode 100644 index 00000000000..f5f45d26a49 --- /dev/null +++ b/plugins/ingest-geoip/licenses/jackson-annotations-LICENSE @@ -0,0 +1,8 @@ +This copy of Jackson JSON processor streaming parser/generator is licensed under the +Apache (Software) License, version 2.0 ("the License"). +See the License for details about distribution rights, and the +specific rights regarding derivate works. + +You may obtain a copy of the License at: + +http://www.apache.org/licenses/LICENSE-2.0 diff --git a/plugins/ingest-geoip/licenses/jackson-annotations-NOTICE b/plugins/ingest-geoip/licenses/jackson-annotations-NOTICE new file mode 100644 index 00000000000..4c976b7b4cc --- /dev/null +++ b/plugins/ingest-geoip/licenses/jackson-annotations-NOTICE @@ -0,0 +1,20 @@ +# Jackson JSON processor + +Jackson is a high-performance, Free/Open Source JSON processing library. +It was originally written by Tatu Saloranta (tatu.saloranta@iki.fi), and has +been in development since 2007. +It is currently developed by a community of developers, as well as supported +commercially by FasterXML.com. + +## Licensing + +Jackson core and extension components may licensed under different licenses. +To find the details that apply to this artifact see the accompanying LICENSE file. +For more information, including possible other licensing options, contact +FasterXML.com (http://fasterxml.com). + +## Credits + +A list of contributors may be found from CREDITS file, which is included +in some artifacts (usually source distributions); but is always available +from the source code management (SCM) system project uses. diff --git a/plugins/ingest-geoip/licenses/jackson-databind-2.5.3.jar.sha1 b/plugins/ingest-geoip/licenses/jackson-databind-2.5.3.jar.sha1 new file mode 100644 index 00000000000..cdc66958059 --- /dev/null +++ b/plugins/ingest-geoip/licenses/jackson-databind-2.5.3.jar.sha1 @@ -0,0 +1 @@ +c37875ff66127d93e5f672708cb2dcc14c8232ab diff --git a/plugins/ingest-geoip/licenses/jackson-databind-LICENSE b/plugins/ingest-geoip/licenses/jackson-databind-LICENSE new file mode 100644 index 00000000000..f5f45d26a49 --- /dev/null +++ b/plugins/ingest-geoip/licenses/jackson-databind-LICENSE @@ -0,0 +1,8 @@ +This copy of Jackson JSON processor streaming parser/generator is licensed under the +Apache (Software) License, version 2.0 ("the License"). +See the License for details about distribution rights, and the +specific rights regarding derivate works. + +You may obtain a copy of the License at: + +http://www.apache.org/licenses/LICENSE-2.0 diff --git a/plugins/ingest-geoip/licenses/jackson-databind-NOTICE b/plugins/ingest-geoip/licenses/jackson-databind-NOTICE new file mode 100644 index 00000000000..4c976b7b4cc --- /dev/null +++ b/plugins/ingest-geoip/licenses/jackson-databind-NOTICE @@ -0,0 +1,20 @@ +# Jackson JSON processor + +Jackson is a high-performance, Free/Open Source JSON processing library. +It was originally written by Tatu Saloranta (tatu.saloranta@iki.fi), and has +been in development since 2007. +It is currently developed by a community of developers, as well as supported +commercially by FasterXML.com. + +## Licensing + +Jackson core and extension components may licensed under different licenses. +To find the details that apply to this artifact see the accompanying LICENSE file. +For more information, including possible other licensing options, contact +FasterXML.com (http://fasterxml.com). + +## Credits + +A list of contributors may be found from CREDITS file, which is included +in some artifacts (usually source distributions); but is always available +from the source code management (SCM) system project uses. diff --git a/plugins/ingest-geoip/licenses/maxmind-db-1.0.1.jar.sha1 b/plugins/ingest-geoip/licenses/maxmind-db-1.0.1.jar.sha1 new file mode 100644 index 00000000000..6cb749e35ae --- /dev/null +++ b/plugins/ingest-geoip/licenses/maxmind-db-1.0.1.jar.sha1 @@ -0,0 +1 @@ +305429b84dbcd1cc3d393686f412cdcaec9cdbe6 \ No newline at end of file diff --git a/plugins/ingest-geoip/licenses/maxmind-db-LICENSE.txt b/plugins/ingest-geoip/licenses/maxmind-db-LICENSE.txt new file mode 100644 index 00000000000..d6456956733 --- /dev/null +++ b/plugins/ingest-geoip/licenses/maxmind-db-LICENSE.txt @@ -0,0 +1,202 @@ + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + Licensed 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. diff --git a/plugins/ingest-geoip/licenses/maxmind-db-NOTICE.txt b/plugins/ingest-geoip/licenses/maxmind-db-NOTICE.txt new file mode 100644 index 00000000000..1ebe2b0826d --- /dev/null +++ b/plugins/ingest-geoip/licenses/maxmind-db-NOTICE.txt @@ -0,0 +1,3 @@ +This software is Copyright (c) 2014 by MaxMind, Inc. + +This is free software, licensed under the Apache License, Version 2.0. diff --git a/plugins/ingest-geoip/src/main/java/org/elasticsearch/ingest/geoip/GeoIpProcessor.java b/plugins/ingest-geoip/src/main/java/org/elasticsearch/ingest/geoip/GeoIpProcessor.java new file mode 100644 index 00000000000..b1c25f5a1ec --- /dev/null +++ b/plugins/ingest-geoip/src/main/java/org/elasticsearch/ingest/geoip/GeoIpProcessor.java @@ -0,0 +1,289 @@ +/* + * 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.geoip; + +import com.maxmind.geoip2.DatabaseReader; +import com.maxmind.geoip2.exception.AddressNotFoundException; +import com.maxmind.geoip2.model.CityResponse; +import com.maxmind.geoip2.model.CountryResponse; +import com.maxmind.geoip2.record.City; +import com.maxmind.geoip2.record.Continent; +import com.maxmind.geoip2.record.Country; +import com.maxmind.geoip2.record.Location; +import com.maxmind.geoip2.record.Subdivision; +import org.apache.lucene.util.IOUtils; +import org.elasticsearch.SpecialPermission; +import org.elasticsearch.common.network.InetAddresses; +import org.elasticsearch.common.network.NetworkAddress; +import org.elasticsearch.ingest.core.AbstractProcessor; +import org.elasticsearch.ingest.core.AbstractProcessorFactory; +import org.elasticsearch.ingest.core.IngestDocument; + +import java.io.Closeable; +import java.io.IOException; +import java.net.InetAddress; +import java.security.AccessController; +import java.security.PrivilegedAction; +import java.util.Arrays; +import java.util.Collections; +import java.util.EnumSet; +import java.util.HashMap; +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.Set; + +import static org.elasticsearch.ingest.core.ConfigurationUtils.readOptionalList; +import static org.elasticsearch.ingest.core.ConfigurationUtils.readStringProperty; + +public final class GeoIpProcessor extends AbstractProcessor { + + public static final String TYPE = "geoip"; + + private final String sourceField; + private final String targetField; + private final DatabaseReader dbReader; + private final Set fields; + + GeoIpProcessor(String tag, String sourceField, DatabaseReader dbReader, String targetField, Set fields) throws IOException { + super(tag); + this.sourceField = sourceField; + this.targetField = targetField; + this.dbReader = dbReader; + this.fields = fields; + } + + @Override + public void execute(IngestDocument ingestDocument) { + String ip = ingestDocument.getFieldValue(sourceField, String.class); + final InetAddress ipAddress = InetAddresses.forString(ip); + + Map geoData; + switch (dbReader.getMetadata().getDatabaseType()) { + case "GeoLite2-City": + try { + geoData = retrieveCityGeoData(ipAddress); + } catch (AddressNotFoundRuntimeException e) { + geoData = Collections.emptyMap(); + } + break; + case "GeoLite2-Country": + try { + geoData = retrieveCountryGeoData(ipAddress); + } catch (AddressNotFoundRuntimeException e) { + geoData = Collections.emptyMap(); + } + break; + default: + throw new IllegalStateException("Unsupported database type [" + dbReader.getMetadata().getDatabaseType() + "]"); + } + ingestDocument.setFieldValue(targetField, geoData); + } + + @Override + public String getType() { + return TYPE; + } + + String getSourceField() { + return sourceField; + } + + String getTargetField() { + return targetField; + } + + DatabaseReader getDbReader() { + return dbReader; + } + + Set getFields() { + return fields; + } + + private Map retrieveCityGeoData(InetAddress ipAddress) { + SecurityManager sm = System.getSecurityManager(); + if (sm != null) { + sm.checkPermission(new SpecialPermission()); + } + CityResponse response = AccessController.doPrivileged((PrivilegedAction) () -> { + try { + return dbReader.city(ipAddress); + } catch (AddressNotFoundException e) { + throw new AddressNotFoundRuntimeException(e); + } catch (Exception e) { + throw new RuntimeException(e); + } + }); + + Country country = response.getCountry(); + City city = response.getCity(); + Location location = response.getLocation(); + Continent continent = response.getContinent(); + Subdivision subdivision = response.getMostSpecificSubdivision(); + + Map geoData = new HashMap<>(); + for (Field field : fields) { + switch (field) { + case IP: + geoData.put("ip", NetworkAddress.formatAddress(ipAddress)); + break; + case COUNTRY_ISO_CODE: + geoData.put("country_iso_code", country.getIsoCode()); + break; + case COUNTRY_NAME: + geoData.put("country_name", country.getName()); + break; + case CONTINENT_NAME: + geoData.put("continent_name", continent.getName()); + break; + case REGION_NAME: + geoData.put("region_name", subdivision.getName()); + break; + case CITY_NAME: + geoData.put("city_name", city.getName()); + break; + case TIMEZONE: + geoData.put("timezone", location.getTimeZone()); + break; + case LOCATION: + Map locationObject = new HashMap<>(); + locationObject.put("lat", location.getLatitude()); + locationObject.put("lon", location.getLongitude()); + geoData.put("location", locationObject); + break; + } + } + return geoData; + } + + private Map retrieveCountryGeoData(InetAddress ipAddress) { + SecurityManager sm = System.getSecurityManager(); + if (sm != null) { + sm.checkPermission(new SpecialPermission()); + } + CountryResponse response = AccessController.doPrivileged((PrivilegedAction) () -> { + try { + return dbReader.country(ipAddress); + } catch (AddressNotFoundException e) { + throw new AddressNotFoundRuntimeException(e); + } catch (Exception e) { + throw new RuntimeException(e); + } + }); + + Country country = response.getCountry(); + Continent continent = response.getContinent(); + + Map geoData = new HashMap<>(); + for (Field field : fields) { + switch (field) { + case IP: + geoData.put("ip", NetworkAddress.formatAddress(ipAddress)); + break; + case COUNTRY_ISO_CODE: + geoData.put("country_iso_code", country.getIsoCode()); + break; + case COUNTRY_NAME: + geoData.put("country_name", country.getName()); + break; + case CONTINENT_NAME: + geoData.put("continent_name", continent.getName()); + break; + } + } + return geoData; + } + + public static final class Factory extends AbstractProcessorFactory implements Closeable { + + static final Set DEFAULT_FIELDS = EnumSet.of( + Field.CONTINENT_NAME, Field.COUNTRY_ISO_CODE, Field.REGION_NAME, Field.CITY_NAME, Field.LOCATION + ); + + private final Map databaseReaders; + + public Factory(Map databaseReaders) { + this.databaseReaders = databaseReaders; + } + + @Override + public GeoIpProcessor doCreate(String processorTag, Map config) throws Exception { + String ipField = readStringProperty(config, "source_field"); + String targetField = readStringProperty(config, "target_field", "geoip"); + String databaseFile = readStringProperty(config, "database_file", "GeoLite2-City.mmdb"); + List fieldNames = readOptionalList(config, "fields"); + + final Set fields; + if (fieldNames != null) { + fields = EnumSet.noneOf(Field.class); + for (String fieldName : fieldNames) { + try { + fields.add(Field.parse(fieldName)); + } catch (Exception e) { + throw new IllegalArgumentException("illegal field option [" + fieldName +"]. valid values are [" + Arrays.toString(Field.values()) +"]", e); + } + } + } else { + fields = DEFAULT_FIELDS; + } + + DatabaseReader databaseReader = databaseReaders.get(databaseFile); + if (databaseReader == null) { + throw new IllegalArgumentException("database file [" + databaseFile + "] doesn't exist"); + } + return new GeoIpProcessor(processorTag, ipField, databaseReader, targetField, fields); + } + + @Override + public void close() throws IOException { + IOUtils.close(databaseReaders.values()); + } + } + + // Geoip2's AddressNotFoundException is checked and due to the fact that we need run their code + // inside a PrivilegedAction code block, we are forced to catch any checked exception and rethrow + // it with an unchecked exception. + private final static class AddressNotFoundRuntimeException extends RuntimeException { + + public AddressNotFoundRuntimeException(Throwable cause) { + super(cause); + } + } + + public enum Field { + + IP, + COUNTRY_ISO_CODE, + COUNTRY_NAME, + CONTINENT_NAME, + REGION_NAME, + CITY_NAME, + TIMEZONE, + LATITUDE, + LONGITUDE, + LOCATION; + + public static Field parse(String value) { + return valueOf(value.toUpperCase(Locale.ROOT)); + } + } + +} diff --git a/plugins/ingest-geoip/src/main/java/org/elasticsearch/ingest/geoip/IngestGeoIpPlugin.java b/plugins/ingest-geoip/src/main/java/org/elasticsearch/ingest/geoip/IngestGeoIpPlugin.java new file mode 100644 index 00000000000..f92cb7b479f --- /dev/null +++ b/plugins/ingest-geoip/src/main/java/org/elasticsearch/ingest/geoip/IngestGeoIpPlugin.java @@ -0,0 +1,77 @@ +/* + * 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.geoip; + +import com.maxmind.geoip2.DatabaseReader; +import org.elasticsearch.node.NodeModule; +import org.elasticsearch.plugins.Plugin; + +import java.io.IOException; +import java.io.InputStream; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.PathMatcher; +import java.nio.file.StandardOpenOption; +import java.util.Collections; +import java.util.HashMap; +import java.util.Iterator; +import java.util.Map; +import java.util.stream.Stream; + +public class IngestGeoIpPlugin extends Plugin { + + @Override + public String name() { + return "ingest-geoip"; + } + + @Override + public String description() { + return "Ingest processor that adds information about the geographical location of ip addresses"; + } + + public void onModule(NodeModule nodeModule) throws IOException { + Path geoIpConfigDirectory = nodeModule.getNode().getEnvironment().configFile().resolve("ingest-geoip"); + Map databaseReaders = loadDatabaseReaders(geoIpConfigDirectory); + nodeModule.registerProcessor(GeoIpProcessor.TYPE, (templateService) -> new GeoIpProcessor.Factory(databaseReaders)); + } + + static Map loadDatabaseReaders(Path geoIpConfigDirectory) throws IOException { + if (Files.exists(geoIpConfigDirectory) == false && Files.isDirectory(geoIpConfigDirectory)) { + throw new IllegalStateException("the geoip directory [" + geoIpConfigDirectory + "] containing databases doesn't exist"); + } + + Map databaseReaders = new HashMap<>(); + try (Stream databaseFiles = Files.list(geoIpConfigDirectory)) { + PathMatcher pathMatcher = geoIpConfigDirectory.getFileSystem().getPathMatcher("glob:**.mmdb"); + // Use iterator instead of forEach otherwise IOException needs to be caught twice... + Iterator iterator = databaseFiles.iterator(); + while (iterator.hasNext()) { + Path databasePath = iterator.next(); + if (Files.isRegularFile(databasePath) && pathMatcher.matches(databasePath)) { + try (InputStream inputStream = Files.newInputStream(databasePath, StandardOpenOption.READ)) { + databaseReaders.put(databasePath.getFileName().toString(), new DatabaseReader.Builder(inputStream).build()); + } + } + } + } + return Collections.unmodifiableMap(databaseReaders); + } +} diff --git a/plugins/ingest-geoip/src/main/plugin-metadata/plugin-security.policy b/plugins/ingest-geoip/src/main/plugin-metadata/plugin-security.policy new file mode 100644 index 00000000000..f49d15d5521 --- /dev/null +++ b/plugins/ingest-geoip/src/main/plugin-metadata/plugin-security.policy @@ -0,0 +1,27 @@ +/* + * 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. + */ + +grant { + // needed because jackson-databind is using Class#getDeclaredConstructors(), Class#getDeclaredMethods() and + // Class#getDeclaredAnnotations() to find all public, private, protected, package protected and + // private constructors, methods or annotations. Just locating all public constructors, methods and annotations + // should be enough, so this permission wouldn't then be needed. Unfortunately this is not what jackson-databind does + // or can be configured to do. + permission java.lang.RuntimePermission "accessDeclaredMembers"; +}; diff --git a/plugins/ingest-geoip/src/test/java/org/elasticsearch/ingest/geoip/GeoIpProcessorFactoryTests.java b/plugins/ingest-geoip/src/test/java/org/elasticsearch/ingest/geoip/GeoIpProcessorFactoryTests.java new file mode 100644 index 00000000000..b59242ece84 --- /dev/null +++ b/plugins/ingest-geoip/src/test/java/org/elasticsearch/ingest/geoip/GeoIpProcessorFactoryTests.java @@ -0,0 +1,152 @@ +/* + * 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.geoip; + +import com.maxmind.geoip2.DatabaseReader; +import org.elasticsearch.ingest.core.AbstractProcessorFactory; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.test.StreamsUtils; +import org.junit.BeforeClass; + +import java.io.ByteArrayInputStream; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.ArrayList; +import java.util.Collections; +import java.util.EnumSet; +import java.util.HashMap; +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.Set; + +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.sameInstance; + +public class GeoIpProcessorFactoryTests extends ESTestCase { + + private static Map databaseReaders; + + @BeforeClass + public static void loadDatabaseReaders() throws IOException { + Path configDir = createTempDir(); + Path geoIpConfigDir = configDir.resolve("ingest-geoip"); + Files.createDirectories(geoIpConfigDir); + Files.copy(new ByteArrayInputStream(StreamsUtils.copyToBytesFromClasspath("/GeoLite2-City.mmdb")), geoIpConfigDir.resolve("GeoLite2-City.mmdb")); + Files.copy(new ByteArrayInputStream(StreamsUtils.copyToBytesFromClasspath("/GeoLite2-Country.mmdb")), geoIpConfigDir.resolve("GeoLite2-Country.mmdb")); + databaseReaders = IngestGeoIpPlugin.loadDatabaseReaders(geoIpConfigDir); + } + + public void testBuildDefaults() throws Exception { + GeoIpProcessor.Factory factory = new GeoIpProcessor.Factory(databaseReaders); + + Map config = new HashMap<>(); + config.put("source_field", "_field"); + + String processorTag = randomAsciiOfLength(10); + config.put(AbstractProcessorFactory.TAG_KEY, processorTag); + + GeoIpProcessor processor = factory.create(config); + assertThat(processor.getTag(), equalTo(processorTag)); + assertThat(processor.getSourceField(), equalTo("_field")); + assertThat(processor.getTargetField(), equalTo("geoip")); + assertThat(processor.getDbReader().getMetadata().getDatabaseType(), equalTo("GeoLite2-City")); + assertThat(processor.getFields(), sameInstance(GeoIpProcessor.Factory.DEFAULT_FIELDS)); + } + + public void testBuildTargetField() throws Exception { + GeoIpProcessor.Factory factory = new GeoIpProcessor.Factory(databaseReaders); + Map config = new HashMap<>(); + config.put("source_field", "_field"); + config.put("target_field", "_field"); + GeoIpProcessor processor = factory.create(config); + assertThat(processor.getSourceField(), equalTo("_field")); + assertThat(processor.getTargetField(), equalTo("_field")); + } + + public void testBuildDbFile() throws Exception { + GeoIpProcessor.Factory factory = new GeoIpProcessor.Factory(databaseReaders); + Map config = new HashMap<>(); + config.put("source_field", "_field"); + config.put("database_file", "GeoLite2-Country.mmdb"); + GeoIpProcessor processor = factory.create(config); + assertThat(processor.getSourceField(), equalTo("_field")); + assertThat(processor.getTargetField(), equalTo("geoip")); + assertThat(processor.getDbReader().getMetadata().getDatabaseType(), equalTo("GeoLite2-Country")); + } + + public void testBuildNonExistingDbFile() throws Exception { + GeoIpProcessor.Factory factory = new GeoIpProcessor.Factory(databaseReaders); + + Map config = new HashMap<>(); + config.put("source_field", "_field"); + config.put("database_file", "does-not-exist.mmdb"); + try { + factory.create(config); + fail("Exception expected"); + } catch (IllegalArgumentException e) { + assertThat(e.getMessage(), equalTo("database file [does-not-exist.mmdb] doesn't exist")); + } + } + + public void testBuildFields() throws Exception { + GeoIpProcessor.Factory factory = new GeoIpProcessor.Factory(databaseReaders); + + Set fields = EnumSet.noneOf(GeoIpProcessor.Field.class); + List fieldNames = new ArrayList<>(); + int numFields = scaledRandomIntBetween(1, GeoIpProcessor.Field.values().length); + for (int i = 0; i < numFields; i++) { + GeoIpProcessor.Field field = GeoIpProcessor.Field.values()[i]; + fields.add(field); + fieldNames.add(field.name().toLowerCase(Locale.ROOT)); + } + Map config = new HashMap<>(); + config.put("source_field", "_field"); + config.put("fields", fieldNames); + GeoIpProcessor processor = factory.create(config); + assertThat(processor.getSourceField(), equalTo("_field")); + assertThat(processor.getFields(), equalTo(fields)); + } + + public void testBuildIllegalFieldOption() throws Exception { + GeoIpProcessor.Factory factory = new GeoIpProcessor.Factory(databaseReaders); + + Map config = new HashMap<>(); + config.put("source_field", "_field"); + config.put("fields", Collections.singletonList("invalid")); + try { + factory.create(config); + fail("exception expected"); + } catch (IllegalArgumentException e) { + assertThat(e.getMessage(), equalTo("illegal field option [invalid]. valid values are [[IP, COUNTRY_ISO_CODE, COUNTRY_NAME, CONTINENT_NAME, REGION_NAME, CITY_NAME, TIMEZONE, LATITUDE, LONGITUDE, LOCATION]]")); + } + + config = new HashMap<>(); + config.put("source_field", "_field"); + config.put("fields", "invalid"); + try { + factory.create(config); + fail("exception expected"); + } catch (IllegalArgumentException e) { + assertThat(e.getMessage(), equalTo("property [fields] isn't a list, but of type [java.lang.String]")); + } + } +} diff --git a/plugins/ingest-geoip/src/test/java/org/elasticsearch/ingest/geoip/GeoIpProcessorTests.java b/plugins/ingest-geoip/src/test/java/org/elasticsearch/ingest/geoip/GeoIpProcessorTests.java new file mode 100644 index 00000000000..43517986c8e --- /dev/null +++ b/plugins/ingest-geoip/src/test/java/org/elasticsearch/ingest/geoip/GeoIpProcessorTests.java @@ -0,0 +1,112 @@ +/* + * 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.geoip; + +import com.maxmind.geoip2.DatabaseReader; +import org.elasticsearch.ingest.RandomDocumentPicks; +import org.elasticsearch.ingest.core.IngestDocument; +import org.elasticsearch.test.ESTestCase; + +import java.io.InputStream; +import java.util.EnumSet; +import java.util.HashMap; +import java.util.Map; + +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.equalTo; + +public class GeoIpProcessorTests extends ESTestCase { + + public void testCity() throws Exception { + InputStream database = GeoIpProcessor.class.getResourceAsStream("/GeoLite2-City.mmdb"); + GeoIpProcessor processor = new GeoIpProcessor(randomAsciiOfLength(10), "source_field", new DatabaseReader.Builder(database).build(), "target_field", EnumSet.allOf(GeoIpProcessor.Field.class)); + + Map document = new HashMap<>(); + document.put("source_field", "82.170.213.79"); + IngestDocument ingestDocument = RandomDocumentPicks.randomIngestDocument(random(), document); + processor.execute(ingestDocument); + + assertThat(ingestDocument.getSourceAndMetadata().get("source_field"), equalTo("82.170.213.79")); + @SuppressWarnings("unchecked") + Map geoData = (Map) ingestDocument.getSourceAndMetadata().get("target_field"); + assertThat(geoData.size(), equalTo(8)); + assertThat(geoData.get("ip"), equalTo("82.170.213.79")); + assertThat(geoData.get("country_iso_code"), equalTo("NL")); + assertThat(geoData.get("country_name"), equalTo("Netherlands")); + assertThat(geoData.get("continent_name"), equalTo("Europe")); + assertThat(geoData.get("region_name"), equalTo("North Holland")); + assertThat(geoData.get("city_name"), equalTo("Amsterdam")); + assertThat(geoData.get("timezone"), equalTo("Europe/Amsterdam")); + Map location = new HashMap<>(); + location.put("lat", 52.374d); + location.put("lon", 4.8897d); + assertThat(geoData.get("location"), equalTo(location)); + } + + public void testCountry() throws Exception { + InputStream database = GeoIpProcessor.class.getResourceAsStream("/GeoLite2-Country.mmdb"); + GeoIpProcessor processor = new GeoIpProcessor(randomAsciiOfLength(10), "source_field", new DatabaseReader.Builder(database).build(), "target_field", EnumSet.allOf(GeoIpProcessor.Field.class)); + + Map document = new HashMap<>(); + document.put("source_field", "82.170.213.79"); + IngestDocument ingestDocument = RandomDocumentPicks.randomIngestDocument(random(), document); + processor.execute(ingestDocument); + + assertThat(ingestDocument.getSourceAndMetadata().get("source_field"), equalTo("82.170.213.79")); + @SuppressWarnings("unchecked") + Map geoData = (Map) ingestDocument.getSourceAndMetadata().get("target_field"); + assertThat(geoData.size(), equalTo(4)); + assertThat(geoData.get("ip"), equalTo("82.170.213.79")); + assertThat(geoData.get("country_iso_code"), equalTo("NL")); + assertThat(geoData.get("country_name"), equalTo("Netherlands")); + assertThat(geoData.get("continent_name"), equalTo("Europe")); + } + + public void testAddressIsNotInTheDatabase() throws Exception { + InputStream database = GeoIpProcessor.class.getResourceAsStream("/GeoLite2-City.mmdb"); + GeoIpProcessor processor = new GeoIpProcessor(randomAsciiOfLength(10), "source_field", new DatabaseReader.Builder(database).build(), "target_field", EnumSet.allOf(GeoIpProcessor.Field.class)); + + Map document = new HashMap<>(); + document.put("source_field", "202.45.11.11"); + IngestDocument ingestDocument = RandomDocumentPicks.randomIngestDocument(random(), document); + processor.execute(ingestDocument); + @SuppressWarnings("unchecked") + Map geoData = (Map) ingestDocument.getSourceAndMetadata().get("target_field"); + assertThat(geoData.size(), equalTo(0)); + } + + /** Don't silently do DNS lookups or anything trappy on bogus data */ + public void testInvalid() throws Exception { + InputStream database = GeoIpProcessor.class.getResourceAsStream("/GeoLite2-City.mmdb"); + GeoIpProcessor processor = new GeoIpProcessor(randomAsciiOfLength(10), "source_field", new DatabaseReader.Builder(database).build(), "target_field", EnumSet.allOf(GeoIpProcessor.Field.class)); + + Map document = new HashMap<>(); + document.put("source_field", "www.google.com"); + IngestDocument ingestDocument = RandomDocumentPicks.randomIngestDocument(random(), document); + try { + processor.execute(ingestDocument); + fail("did not get expected exception"); + } catch (IllegalArgumentException expected) { + assertNotNull(expected.getMessage()); + assertThat(expected.getMessage(), containsString("not an IP string literal")); + } + } + +} diff --git a/plugins/ingest-geoip/src/test/java/org/elasticsearch/ingest/geoip/IngestGeoIpRestIT.java b/plugins/ingest-geoip/src/test/java/org/elasticsearch/ingest/geoip/IngestGeoIpRestIT.java new file mode 100644 index 00000000000..0e4d1ee4b2b --- /dev/null +++ b/plugins/ingest-geoip/src/test/java/org/elasticsearch/ingest/geoip/IngestGeoIpRestIT.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.ingest.geoip; + +import com.carrotsearch.randomizedtesting.annotations.Name; +import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; +import org.elasticsearch.plugins.Plugin; +import org.elasticsearch.test.rest.ESRestTestCase; +import org.elasticsearch.test.rest.RestTestCandidate; +import org.elasticsearch.test.rest.parser.RestTestParseException; + +import java.io.IOException; +import java.util.Collection; + +public class IngestGeoIpRestIT extends ESRestTestCase { + + public IngestGeoIpRestIT(@Name("yaml") RestTestCandidate testCandidate) { + super(testCandidate); + } + + @ParametersFactory + public static Iterable parameters() throws IOException, RestTestParseException { + return ESRestTestCase.createParameters(0, 1); + } +} + diff --git a/plugins/ingest-geoip/src/test/resources/rest-api-spec/test/ingest_geoip/10_basic.yaml b/plugins/ingest-geoip/src/test/resources/rest-api-spec/test/ingest_geoip/10_basic.yaml new file mode 100644 index 00000000000..b522cb77780 --- /dev/null +++ b/plugins/ingest-geoip/src/test/resources/rest-api-spec/test/ingest_geoip/10_basic.yaml @@ -0,0 +1,5 @@ +"Ingest plugin installed": + - do: + cluster.stats: {} + + - match: { nodes.plugins.0.name: ingest-geoip } diff --git a/plugins/ingest-geoip/src/test/resources/rest-api-spec/test/ingest_geoip/20_geoip_processor.yaml b/plugins/ingest-geoip/src/test/resources/rest-api-spec/test/ingest_geoip/20_geoip_processor.yaml new file mode 100644 index 00000000000..704f288646c --- /dev/null +++ b/plugins/ingest-geoip/src/test/resources/rest-api-spec/test/ingest_geoip/20_geoip_processor.yaml @@ -0,0 +1,124 @@ +--- +"Test geoip processor with defaults": + - do: + ingest.put_pipeline: + id: "my_pipeline" + body: > + { + "description": "_description", + "processors": [ + { + "geoip" : { + "source_field" : "field1" + } + } + ] + } + - match: { acknowledged: true } + + - do: + index: + index: test + type: test + id: 1 + pipeline: "my_pipeline" + body: {field1: "128.101.101.101"} + + - do: + get: + index: test + type: test + id: 1 + - match: { _source.field1: "128.101.101.101" } + - length: { _source.geoip: 5 } + - match: { _source.geoip.city_name: "Minneapolis" } + - match: { _source.geoip.country_iso_code: "US" } + - match: { _source.geoip.location.lon: -93.2166 } + - match: { _source.geoip.location.lat: 44.9759 } + - match: { _source.geoip.region_name: "Minnesota" } + - match: { _source.geoip.continent_name: "North America" } + +--- +"Test geoip processor with fields": + - do: + cluster.health: + wait_for_status: green + + - do: + ingest.put_pipeline: + id: "my_pipeline" + body: > + { + "description": "_description", + "processors": [ + { + "geoip" : { + "source_field" : "field1", + "fields" : ["city_name", "country_iso_code", "ip", "latitude", "longitude", "location", "timezone", "country_name", "region_name", "continent_name"] + } + } + ] + } + - match: { acknowledged: true } + + - do: + index: + index: test + type: test + id: 1 + pipeline: "my_pipeline" + body: {field1: "128.101.101.101"} + + - do: + get: + index: test + type: test + id: 1 + - match: { _source.field1: "128.101.101.101" } + - length: { _source.geoip: 8 } + - match: { _source.geoip.city_name: "Minneapolis" } + - match: { _source.geoip.country_iso_code: "US" } + - match: { _source.geoip.ip: "128.101.101.101" } + - match: { _source.geoip.location.lon: -93.2166 } + - match: { _source.geoip.location.lat: 44.9759 } + - match: { _source.geoip.timezone: "America/Chicago" } + - match: { _source.geoip.country_name: "United States" } + - match: { _source.geoip.region_name: "Minnesota" } + - match: { _source.geoip.continent_name: "North America" } + +--- +"Test geoip processor with different database file": + - do: + ingest.put_pipeline: + id: "my_pipeline" + body: > + { + "description": "_description", + "processors": [ + { + "geoip" : { + "source_field" : "field1", + "database_file" : "GeoLite2-Country.mmdb" + } + } + ] + } + - match: { acknowledged: true } + + - do: + index: + index: test + type: test + id: 1 + pipeline: "my_pipeline" + body: {field1: "128.101.101.101"} + + - do: + get: + index: test + type: test + id: 1 + - match: { _source.field1: "128.101.101.101" } + - length: { _source.geoip: 2 } + - match: { _source.geoip.country_iso_code: "US" } + - match: { _source.geoip.continent_name: "North America" } diff --git a/qa/ingest-disabled/build.gradle b/qa/ingest-disabled/build.gradle new file mode 100644 index 00000000000..ca71697a7b4 --- /dev/null +++ b/qa/ingest-disabled/build.gradle @@ -0,0 +1,26 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +apply plugin: 'elasticsearch.rest-test' + +integTest { + cluster { + systemProperty 'es.node.ingest', 'false' + } +} diff --git a/qa/ingest-disabled/src/test/java/org/elasticsearch/smoketest/IngestDisabledIT.java b/qa/ingest-disabled/src/test/java/org/elasticsearch/smoketest/IngestDisabledIT.java new file mode 100644 index 00000000000..e162807baca --- /dev/null +++ b/qa/ingest-disabled/src/test/java/org/elasticsearch/smoketest/IngestDisabledIT.java @@ -0,0 +1,41 @@ +/* + * 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.smoketest; + +import com.carrotsearch.randomizedtesting.annotations.Name; +import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; +import org.elasticsearch.test.rest.ESRestTestCase; +import org.elasticsearch.test.rest.RestTestCandidate; +import org.elasticsearch.test.rest.parser.RestTestParseException; + +import java.io.IOException; + +public class IngestDisabledIT extends ESRestTestCase { + + public IngestDisabledIT(@Name("yaml") RestTestCandidate testCandidate) { + super(testCandidate); + } + + @ParametersFactory + public static Iterable parameters() throws IOException, RestTestParseException { + return ESRestTestCase.createParameters(0, 1); + } + +} diff --git a/qa/ingest-disabled/src/test/resources/rest-api-spec/test/ingest_mustache/10_ingest_disabled.yaml b/qa/ingest-disabled/src/test/resources/rest-api-spec/test/ingest_mustache/10_ingest_disabled.yaml new file mode 100644 index 00000000000..01d674053ae --- /dev/null +++ b/qa/ingest-disabled/src/test/resources/rest-api-spec/test/ingest_mustache/10_ingest_disabled.yaml @@ -0,0 +1,122 @@ +--- +"Test ingest CRUD APIS work fine when node.ingest is set to false": + - do: + ingest.put_pipeline: + id: "my_pipeline" + body: > + { + "description": "_description", + "processors": [ + { + "set" : { + "field" : "field2", + "value": "_value" + } + } + ] + } + - match: { acknowledged: true } + + - do: + ingest.get_pipeline: + id: "my_pipeline" + - match: { pipelines.0.id: "my_pipeline" } + - match: { pipelines.0.config.description: "_description" } + + - do: + ingest.delete_pipeline: + id: "my_pipeline" + - match: { acknowledged: true } + +--- +"Test ingest simulate API works fine when node.ingest is set to false": + - do: + ingest.put_pipeline: + id: "my_pipeline" + body: > + { + "description": "_description", + "processors": [ + { + "set" : { + "field" : "field2", + "value" : "_value" + } + } + ] + } + - match: { acknowledged: true } + + - do: + ingest.simulate: + id: "my_pipeline" + body: > + { + "docs": [ + { + "_index": "index", + "_type": "type", + "_id": "id", + "_source": { + "foo": "bar" + } + } + ] + } + - length: { docs: 1 } + - match: { docs.0.doc._source.foo: "bar" } + - match: { docs.0.doc._source.field2: "_value" } + - length: { docs.0.doc._ingest: 1 } + - is_true: docs.0.doc._ingest.timestamp + +--- +"Test index api with pipeline id fails when node.ingest is set to false": + - do: + catch: /There are no ingest nodes in this cluster, unable to forward request to an ingest node./ + index: + index: test + type: test + id: 1 + pipeline: "my_pipeline_1" + body: { + field1: "1", + field2: "2", + field3: "3" + } + +--- +"Test bulk api with pipeline id fails when node.ingest is set to false": + - do: + catch: /There are no ingest nodes in this cluster, unable to forward request to an ingest node./ + bulk: + pipeline: "my_pipeline_1" + body: + - index: + _index: test_index + _type: test_type + _id: test_id + - f1: v1 + - index: + _index: test_index + _type: test_type + _id: test_id2 + - f1: v2 + +--- +"Test bulk api that contains a single index call with pipeline id fails when node.ingest is set to false": + - do: + catch: /There are no ingest nodes in this cluster, unable to forward request to an ingest node./ + bulk: + body: + - index: + _index: test_index + _type: test_type + _id: test_id + - f1: v1 + - index: + _index: test_index + _type: test_type + _id: test_id2 + pipeline: my_pipeline_1 + - f1: v2 + diff --git a/qa/ingest-with-mustache/build.gradle b/qa/ingest-with-mustache/build.gradle new file mode 100644 index 00000000000..e5ca482d85a --- /dev/null +++ b/qa/ingest-with-mustache/build.gradle @@ -0,0 +1,24 @@ +/* + * 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. + */ + +apply plugin: 'elasticsearch.rest-test' + +dependencies { + testCompile project(path: ':modules:lang-mustache', configuration: 'runtime') +} diff --git a/qa/ingest-with-mustache/src/test/java/org/elasticsearch/ingest/AbstractMustacheTests.java b/qa/ingest-with-mustache/src/test/java/org/elasticsearch/ingest/AbstractMustacheTests.java new file mode 100644 index 00000000000..57165e69fb6 --- /dev/null +++ b/qa/ingest-with-mustache/src/test/java/org/elasticsearch/ingest/AbstractMustacheTests.java @@ -0,0 +1,52 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.ingest; + +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.env.Environment; +import org.elasticsearch.ingest.InternalTemplateService; +import org.elasticsearch.ingest.core.TemplateService; +import org.elasticsearch.script.ScriptContextRegistry; +import org.elasticsearch.script.ScriptService; +import org.elasticsearch.script.mustache.MustacheScriptEngineService; +import org.elasticsearch.test.ESTestCase; +import org.junit.Before; + +import java.util.Collections; + +public abstract class AbstractMustacheTests extends ESTestCase { + + protected TemplateService templateService; + + @Before + public void init() throws Exception { + Settings settings = Settings.builder() + .put("path.home", createTempDir()) + .put(ScriptService.SCRIPT_AUTO_RELOAD_ENABLED_SETTING, false) + .build(); + MustacheScriptEngineService mustache = new MustacheScriptEngineService(settings); + ScriptContextRegistry registry = new ScriptContextRegistry(Collections.emptyList()); + ScriptService scriptService = new ScriptService( + settings, new Environment(settings), Collections.singleton(mustache), null, registry + ); + templateService = new InternalTemplateService(scriptService); + } + +} diff --git a/qa/ingest-with-mustache/src/test/java/org/elasticsearch/ingest/IngestDocumentMustacheIT.java b/qa/ingest-with-mustache/src/test/java/org/elasticsearch/ingest/IngestDocumentMustacheIT.java new file mode 100644 index 00000000000..f27a8e4c8d6 --- /dev/null +++ b/qa/ingest-with-mustache/src/test/java/org/elasticsearch/ingest/IngestDocumentMustacheIT.java @@ -0,0 +1,85 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.ingest; + +import org.elasticsearch.ingest.core.IngestDocument; +import org.elasticsearch.ingest.core.ValueSource; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.hamcrest.Matchers.equalTo; + +public class IngestDocumentMustacheIT extends AbstractMustacheTests { + + public void testAccessMetaDataViaTemplate() { + Map document = new HashMap<>(); + document.put("foo", "bar"); + IngestDocument ingestDocument = new IngestDocument("index", "type", "id", null, null, null, null, document); + ingestDocument.setFieldValue(templateService.compile("field1"), ValueSource.wrap("1 {{foo}}", templateService)); + assertThat(ingestDocument.getFieldValue("field1", String.class), equalTo("1 bar")); + + ingestDocument.setFieldValue(templateService.compile("field1"), ValueSource.wrap("2 {{_source.foo}}", templateService)); + assertThat(ingestDocument.getFieldValue("field1", String.class), equalTo("2 bar")); + } + + public void testAccessMapMetaDataViaTemplate() { + Map document = new HashMap<>(); + Map innerObject = new HashMap<>(); + innerObject.put("bar", "hello bar"); + innerObject.put("baz", "hello baz"); + innerObject.put("qux", Collections.singletonMap("fubar", "hello qux and fubar")); + document.put("foo", innerObject); + IngestDocument ingestDocument = new IngestDocument("index", "type", "id", null, null, null, null, document); + ingestDocument.setFieldValue(templateService.compile("field1"), ValueSource.wrap("1 {{foo.bar}} {{foo.baz}} {{foo.qux.fubar}}", templateService)); + assertThat(ingestDocument.getFieldValue("field1", String.class), equalTo("1 hello bar hello baz hello qux and fubar")); + + ingestDocument.setFieldValue(templateService.compile("field1"), ValueSource.wrap("2 {{_source.foo.bar}} {{_source.foo.baz}} {{_source.foo.qux.fubar}}", templateService)); + assertThat(ingestDocument.getFieldValue("field1", String.class), equalTo("2 hello bar hello baz hello qux and fubar")); + } + + public void testAccessListMetaDataViaTemplate() { + Map document = new HashMap<>(); + document.put("list1", Arrays.asList("foo", "bar", null)); + List> list = new ArrayList<>(); + Map value = new HashMap<>(); + value.put("field", "value"); + list.add(value); + list.add(null); + document.put("list2", list); + IngestDocument ingestDocument = new IngestDocument("index", "type", "id", null, null, null, null, document); + ingestDocument.setFieldValue(templateService.compile("field1"), ValueSource.wrap("1 {{list1.0}} {{list2.0}}", templateService)); + assertThat(ingestDocument.getFieldValue("field1", String.class), equalTo("1 foo {field=value}")); + } + + public void testAccessIngestMetadataViaTemplate() { + Map document = new HashMap<>(); + Map ingestMap = new HashMap<>(); + ingestMap.put("timestamp", "bogus_timestamp"); + document.put("_ingest", ingestMap); + IngestDocument ingestDocument = new IngestDocument("index", "type", "id", null, null, null, null, document); + ingestDocument.setFieldValue(templateService.compile("ingest_timestamp"), ValueSource.wrap("{{_ingest.timestamp}} and {{_source._ingest.timestamp}}", templateService)); + assertThat(ingestDocument.getFieldValue("ingest_timestamp", String.class), equalTo(ingestDocument.getIngestMetadata().get("timestamp") + " and bogus_timestamp")); + } +} diff --git a/qa/ingest-with-mustache/src/test/java/org/elasticsearch/ingest/IngestMustacheRemoveProcessorIT.java b/qa/ingest-with-mustache/src/test/java/org/elasticsearch/ingest/IngestMustacheRemoveProcessorIT.java new file mode 100644 index 00000000000..e94765a4aad --- /dev/null +++ b/qa/ingest-with-mustache/src/test/java/org/elasticsearch/ingest/IngestMustacheRemoveProcessorIT.java @@ -0,0 +1,38 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.ingest; + +import org.elasticsearch.ingest.processor.RemoveProcessor; +import org.hamcrest.CoreMatchers; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +public class IngestMustacheRemoveProcessorIT extends AbstractMustacheTests { + + public void testRemoveProcessorMustacheExpression() throws Exception { + RemoveProcessor.Factory factory = new RemoveProcessor.Factory(templateService); + Map config = new HashMap<>(); + config.put("field", "field{{var}}"); + RemoveProcessor processor = factory.create(config); + assertThat(processor.getField().execute(Collections.singletonMap("var", "_value")), CoreMatchers.equalTo("field_value")); + } +} diff --git a/qa/ingest-with-mustache/src/test/java/org/elasticsearch/ingest/IngestMustacheSetProcessorIT.java b/qa/ingest-with-mustache/src/test/java/org/elasticsearch/ingest/IngestMustacheSetProcessorIT.java new file mode 100644 index 00000000000..68466795b74 --- /dev/null +++ b/qa/ingest-with-mustache/src/test/java/org/elasticsearch/ingest/IngestMustacheSetProcessorIT.java @@ -0,0 +1,72 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.ingest; + + +import org.elasticsearch.ingest.core.IngestDocument; +import org.elasticsearch.ingest.core.ValueSource; +import org.elasticsearch.ingest.core.Processor; +import org.elasticsearch.ingest.processor.SetProcessor; +import org.hamcrest.Matchers; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.instanceOf; + +public class IngestMustacheSetProcessorIT extends AbstractMustacheTests { + + public void testExpression() throws Exception { + SetProcessor processor = createSetProcessor("_index", "text {{var}}"); + assertThat(processor.getValue(), instanceOf(ValueSource.TemplatedValue.class)); + assertThat(processor.getValue().copyAndResolve(Collections.singletonMap("var", "_value")), equalTo("text _value")); + } + + public void testSetMetadataWithTemplates() throws Exception { + IngestDocument.MetaData randomMetaData = randomFrom(IngestDocument.MetaData.values()); + Processor processor = createSetProcessor(randomMetaData.getFieldName(), "_value {{field}}"); + IngestDocument ingestDocument = createIngestDocument(Collections.singletonMap("field", "value")); + processor.execute(ingestDocument); + assertThat(ingestDocument.getFieldValue(randomMetaData.getFieldName(), String.class), Matchers.equalTo("_value value")); + } + + public void testSetWithTemplates() throws Exception { + IngestDocument.MetaData randomMetaData = randomFrom(IngestDocument.MetaData.INDEX, IngestDocument.MetaData.TYPE, IngestDocument.MetaData.ID); + Processor processor = createSetProcessor("field{{_type}}", "_value {{" + randomMetaData.getFieldName() + "}}"); + IngestDocument ingestDocument = createIngestDocument(new HashMap<>()); + processor.execute(ingestDocument); + assertThat(ingestDocument.getFieldValue("field_type", String.class), Matchers.equalTo("_value " + ingestDocument.getFieldValue(randomMetaData.getFieldName(), String.class))); + } + + private SetProcessor createSetProcessor(String fieldName, Object fieldValue) throws Exception { + SetProcessor.Factory factory = new SetProcessor.Factory(templateService); + Map config = new HashMap<>(); + config.put("field", fieldName); + config.put("value", fieldValue); + return factory.create(config); + } + + private IngestDocument createIngestDocument(Map source) { + return new IngestDocument("_index", "_type", "_id", null, null, null, null, source); + } + +} diff --git a/qa/ingest-with-mustache/src/test/java/org/elasticsearch/ingest/TemplateServiceIT.java b/qa/ingest-with-mustache/src/test/java/org/elasticsearch/ingest/TemplateServiceIT.java new file mode 100644 index 00000000000..1d1579fe66a --- /dev/null +++ b/qa/ingest-with-mustache/src/test/java/org/elasticsearch/ingest/TemplateServiceIT.java @@ -0,0 +1,56 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.ingest; + +import org.elasticsearch.ingest.core.TemplateService; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +import static org.hamcrest.Matchers.equalTo; + +public class TemplateServiceIT extends AbstractMustacheTests { + + public void testTemplates() { + Map model = new HashMap<>(); + model.put("fielda", "value1"); + model.put("fieldb", Collections.singletonMap("fieldc", "value3")); + + TemplateService.Template template = templateService.compile("{{fielda}}/{{fieldb}}/{{fieldb.fieldc}}"); + assertThat(template.execute(model), equalTo("value1/{fieldc=value3}/value3")); + } + + public void testWrongTemplateUsage() { + Map model = Collections.emptyMap(); + TemplateService.Template template = templateService.compile("value"); + assertThat(template.execute(model), equalTo("value")); + + template = templateService.compile("value {{"); + assertThat(template.execute(model), equalTo("value {{")); + template = templateService.compile("value {{abc"); + assertThat(template.execute(model), equalTo("value {{abc")); + template = templateService.compile("value }}"); + assertThat(template.execute(model), equalTo("value }}")); + template = templateService.compile("value }} {{"); + assertThat(template.execute(model), equalTo("value }} {{")); + } + +} diff --git a/qa/ingest-with-mustache/src/test/java/org/elasticsearch/ingest/ValueSourceMustacheIT.java b/qa/ingest-with-mustache/src/test/java/org/elasticsearch/ingest/ValueSourceMustacheIT.java new file mode 100644 index 00000000000..18085b94b04 --- /dev/null +++ b/qa/ingest-with-mustache/src/test/java/org/elasticsearch/ingest/ValueSourceMustacheIT.java @@ -0,0 +1,76 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.ingest; + +import org.elasticsearch.ingest.core.IngestDocument; +import org.elasticsearch.ingest.core.ValueSource; + +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.instanceOf; +import static org.hamcrest.Matchers.is; + +public class ValueSourceMustacheIT extends AbstractMustacheTests { + + public void testValueSourceWithTemplates() { + Map model = new HashMap<>(); + model.put("field1", "value1"); + model.put("field2", Collections.singletonMap("field3", "value3")); + + ValueSource valueSource = ValueSource.wrap("{{field1}}/{{field2}}/{{field2.field3}}", templateService); + assertThat(valueSource, instanceOf(ValueSource.TemplatedValue.class)); + assertThat(valueSource.copyAndResolve(model), equalTo("value1/{field3=value3}/value3")); + + valueSource = ValueSource.wrap(Arrays.asList("_value", "{{field1}}"), templateService); + assertThat(valueSource, instanceOf(ValueSource.ListValue.class)); + List result = (List) valueSource.copyAndResolve(model); + assertThat(result.size(), equalTo(2)); + assertThat(result.get(0), equalTo("_value")); + assertThat(result.get(1), equalTo("value1")); + + Map map = new HashMap<>(); + map.put("field1", "{{field1}}"); + map.put("field2", Collections.singletonMap("field3", "{{field2.field3}}")); + map.put("field4", "_value"); + valueSource = ValueSource.wrap(map, templateService); + assertThat(valueSource, instanceOf(ValueSource.MapValue.class)); + Map resultMap = (Map) valueSource.copyAndResolve(model); + assertThat(resultMap.size(), equalTo(3)); + assertThat(resultMap.get("field1"), equalTo("value1")); + assertThat(((Map) resultMap.get("field2")).size(), equalTo(1)); + assertThat(((Map) resultMap.get("field2")).get("field3"), equalTo("value3")); + assertThat(resultMap.get("field4"), equalTo("_value")); + } + + public void testAccessSourceViaTemplate() { + IngestDocument ingestDocument = new IngestDocument("marvel", "type", "id", null, null, null, null, new HashMap<>()); + assertThat(ingestDocument.hasField("marvel"), is(false)); + ingestDocument.setFieldValue(templateService.compile("{{_index}}"), ValueSource.wrap("{{_index}}", templateService)); + assertThat(ingestDocument.getFieldValue("marvel", String.class), equalTo("marvel")); + ingestDocument.removeField(templateService.compile("{{marvel}}")); + assertThat(ingestDocument.hasField("index"), is(false)); + } + +} diff --git a/qa/ingest-with-mustache/src/test/java/org/elasticsearch/smoketest/IngestWithMustacheIT.java b/qa/ingest-with-mustache/src/test/java/org/elasticsearch/smoketest/IngestWithMustacheIT.java new file mode 100644 index 00000000000..73f64d4433c --- /dev/null +++ b/qa/ingest-with-mustache/src/test/java/org/elasticsearch/smoketest/IngestWithMustacheIT.java @@ -0,0 +1,41 @@ +/* + * 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.smoketest; + +import com.carrotsearch.randomizedtesting.annotations.Name; +import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; +import org.elasticsearch.test.rest.ESRestTestCase; +import org.elasticsearch.test.rest.RestTestCandidate; +import org.elasticsearch.test.rest.parser.RestTestParseException; + +import java.io.IOException; + +public class IngestWithMustacheIT extends ESRestTestCase { + + public IngestWithMustacheIT(@Name("yaml") RestTestCandidate testCandidate) { + super(testCandidate); + } + + @ParametersFactory + public static Iterable parameters() throws IOException, RestTestParseException { + return ESRestTestCase.createParameters(0, 1); + } + +} diff --git a/qa/ingest-with-mustache/src/test/resources/rest-api-spec/test/ingest_mustache/10_pipeline_with_mustache_templates.yaml b/qa/ingest-with-mustache/src/test/resources/rest-api-spec/test/ingest_mustache/10_pipeline_with_mustache_templates.yaml new file mode 100644 index 00000000000..9e644773c6a --- /dev/null +++ b/qa/ingest-with-mustache/src/test/resources/rest-api-spec/test/ingest_mustache/10_pipeline_with_mustache_templates.yaml @@ -0,0 +1,220 @@ +--- +"Test metadata templating": + - do: + cluster.health: + wait_for_status: green + + - do: + ingest.put_pipeline: + id: "my_pipeline_1" + body: > + { + "description": "_description", + "processors": [ + { + "set" : { + "field" : "index_type_id", + "value": "{{_index}}/{{_type}}/{{_id}}" + } + }, + { + "append" : { + "field" : "metadata", + "value": ["{{_index}}", "{{_type}}", "{{_id}}"] + } + } + ] + } + - match: { acknowledged: true } + + - do: + index: + index: test + type: test + id: 1 + pipeline: "my_pipeline_1" + body: {} + + - do: + get: + index: test + type: test + id: 1 + - length: { _source: 2 } + - match: { _source.index_type_id: "test/test/1" } + - match: { _source.metadata: ["test", "test", "1"] } + +--- +"Test templating": + - do: + cluster.health: + wait_for_status: green + + - do: + ingest.put_pipeline: + id: "my_pipeline_1" + body: > + { + "description": "_description", + "processors": [ + { + "set" : { + "field" : "field4", + "value": "{{field1}}/{{field2}}/{{field3}}" + } + }, + { + "append" : { + "field" : "metadata", + "value": ["{{field1}}", "{{field2}}", "{{field3}}"] + } + } + + ] + } + - match: { acknowledged: true } + + - do: + ingest.put_pipeline: + id: "my_pipeline_2" + body: > + { + "description": "_description", + "processors": [ + { + "set" : { + "field" : "{{field1}}", + "value": "value" + } + } + ] + } + - match: { acknowledged: true } + + - do: + ingest.put_pipeline: + id: "my_pipeline_3" + body: > + { + "description": "_description", + "processors": [ + { + "remove" : { + "field" : "{{field_to_remove}}" + } + } + ] + } + - match: { acknowledged: true } + + - do: + index: + index: test + type: test + id: 1 + pipeline: "my_pipeline_1" + body: { + metadata: "0", + field1: "1", + field2: "2", + field3: "3" + } + + - do: + get: + index: test + type: test + id: 1 + - length: { _source: 5 } + - match: { _source.field1: "1" } + - match: { _source.field2: "2" } + - match: { _source.field3: "3" } + - match: { _source.field4: "1/2/3" } + - match: { _source.metadata: ["0","1","2","3"] } + + - do: + index: + index: test + type: test + id: 1 + pipeline: "my_pipeline_2" + body: { + field1: "field2" + } + + - do: + get: + index: test + type: test + id: 1 + - length: { _source: 2 } + - match: { _source.field1: "field2" } + - match: { _source.field2: "value" } + + - do: + index: + index: test + type: test + id: 1 + pipeline: "my_pipeline_3" + body: { + field_to_remove: "field2", + field2: "2", + } + + - do: + get: + index: test + type: test + id: 1 + - length: { _source: 1 } + - match: { _source.field_to_remove: "field2" } + +--- +"Test on_failure metadata context templating": + - do: + cluster.health: + wait_for_status: green + + - do: + ingest.put_pipeline: + id: "my_handled_pipeline" + body: > + { + "description": "_description", + "processors": [ + { + "remove" : { + "field" : "field_to_remove", + "on_failure" : [ + { + "set" : { + "field" : "error", + "value" : "processor [{{ _ingest.on_failure_processor }}]: {{ _ingest.on_failure_message }}" + } + } + ] + } + } + ] + } + - match: { acknowledged: true } + + - do: + index: + index: test + type: test + id: 1 + pipeline: "my_handled_pipeline" + body: { + do_nothing: "foo", + } + + - do: + get: + index: test + type: test + id: 1 + - length: { _source: 2 } + - match: { _source.do_nothing: "foo" } + - match: { _source.error: "processor [remove]: field [field_to_remove] not present as part of path [field_to_remove]" } diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/bulk.json b/rest-api-spec/src/main/resources/rest-api-spec/api/bulk.json index 577a03fd770..590054b04a4 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/bulk.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/bulk.json @@ -40,6 +40,10 @@ "fields": { "type": "list", "description" : "Default comma-separated list of fields to return in the response for updates" + }, + "pipeline" : { + "type" : "string", + "description" : "The pipeline id to preprocess incoming documents with" } } }, diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/index.json b/rest-api-spec/src/main/resources/rest-api-spec/api/index.json index 1b8f7140dcd..5c13f67c212 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/index.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/index.json @@ -65,6 +65,10 @@ "type" : "enum", "options" : ["internal", "external", "external_gte", "force"], "description" : "Specific version type" + }, + "pipeline" : { + "type" : "string", + "description" : "The pipeline id to preprocess incoming documents with" } } }, diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/ingest.delete_pipeline.json b/rest-api-spec/src/main/resources/rest-api-spec/api/ingest.delete_pipeline.json new file mode 100644 index 00000000000..1c515e45095 --- /dev/null +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/ingest.delete_pipeline.json @@ -0,0 +1,28 @@ +{ + "ingest.delete_pipeline": { + "documentation": "https://www.elastic.co/guide/en/elasticsearch/plugins/master/ingest.html", + "methods": [ "DELETE" ], + "url": { + "path": "/_ingest/pipeline/{id}", + "paths": [ "/_ingest/pipeline/{id}" ], + "parts": { + "id": { + "type" : "string", + "description" : "Pipeline ID", + "required" : true + } + }, + "params": { + "master_timeout": { + "type" : "time", + "description" : "Explicit operation timeout for connection to master node" + }, + "timeout": { + "type" : "time", + "description" : "Explicit operation timeout" + } + } + }, + "body": null + } +} diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/ingest.get_pipeline.json b/rest-api-spec/src/main/resources/rest-api-spec/api/ingest.get_pipeline.json new file mode 100644 index 00000000000..6c50657ae1a --- /dev/null +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/ingest.get_pipeline.json @@ -0,0 +1,24 @@ +{ + "ingest.get_pipeline": { + "documentation": "https://www.elastic.co/guide/en/elasticsearch/plugins/master/ingest.html", + "methods": [ "GET" ], + "url": { + "path": "/_ingest/pipeline/{id}", + "paths": [ "/_ingest/pipeline/{id}" ], + "parts": { + "id": { + "type" : "string", + "description" : "Comma separated list of pipeline ids. Wildcards supported", + "required" : true + } + }, + "params": { + "master_timeout": { + "type" : "time", + "description" : "Explicit operation timeout for connection to master node" + } + } + }, + "body": null + } +} diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/ingest.put_pipeline.json b/rest-api-spec/src/main/resources/rest-api-spec/api/ingest.put_pipeline.json new file mode 100644 index 00000000000..e4c3c2eb3f9 --- /dev/null +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/ingest.put_pipeline.json @@ -0,0 +1,31 @@ +{ + "ingest.put_pipeline": { + "documentation": "https://www.elastic.co/guide/en/elasticsearch/plugins/master/ingest.html", + "methods": [ "PUT" ], + "url": { + "path": "/_ingest/pipeline/{id}", + "paths": [ "/_ingest/pipeline/{id}" ], + "parts": { + "id": { + "type" : "string", + "description" : "Pipeline ID", + "required" : true + } + }, + "params": { + "master_timeout": { + "type" : "time", + "description" : "Explicit operation timeout for connection to master node" + }, + "timeout": { + "type" : "time", + "description" : "Explicit operation timeout" + } + } + }, + "body": { + "description" : "The ingest definition", + "required" : true + } + } +} diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/ingest.simulate.json b/rest-api-spec/src/main/resources/rest-api-spec/api/ingest.simulate.json new file mode 100644 index 00000000000..a4904cef80a --- /dev/null +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/ingest.simulate.json @@ -0,0 +1,28 @@ +{ + "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": { + "verbose": { + "type" : "boolean", + "description" : "Verbose mode. Display data output for each processor in executed pipeline", + "default" : false + } + } + }, + "body": { + "description" : "The simulate definition", + "required" : true + } + } +} diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/ingest/10_crud.yaml b/rest-api-spec/src/main/resources/rest-api-spec/test/ingest/10_crud.yaml new file mode 100644 index 00000000000..bf0817f2da1 --- /dev/null +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/ingest/10_crud.yaml @@ -0,0 +1,94 @@ +--- +"Test basic pipeline crud": + - do: + ingest.put_pipeline: + id: "my_pipeline" + body: > + { + "description": "_description", + "processors": [ + { + "set" : { + "field" : "field2", + "value": "_value" + } + } + ] + } + - match: { acknowledged: true } + + - do: + ingest.get_pipeline: + id: "my_pipeline" + - match: { pipelines.0.id: "my_pipeline" } + - match: { pipelines.0.config.description: "_description" } + + - do: + ingest.delete_pipeline: + id: "my_pipeline" + - match: { acknowledged: true } + + - do: + catch: missing + ingest.get_pipeline: + id: "my_pipeline" + +--- +"Test invalid config": + - do: + catch: param + ingest.put_pipeline: + id: "my_pipeline" + body: > + { + "description": "_description", + "processors": [ + { + "set" : { + } + } + ] + } + +--- +"Test basic pipeline with on_failure in processor": + - do: + ingest.put_pipeline: + id: "my_pipeline" + body: > + { + "description": "_description", + "processors": [ + { + "set" : { + "field" : "field2", + "value": "_value", + "on_failure": [ + { + "set" : { + "field" : "field2", + "value" : "_failed_value" + } + } + ] + } + } + ] + } + - match: { acknowledged: true } + + - do: + ingest.get_pipeline: + id: "my_pipeline" + - match: { pipelines.0.id: "my_pipeline" } + - match: { pipelines.0.config.description: "_description" } + + - do: + ingest.delete_pipeline: + id: "my_pipeline" + - match: { acknowledged: true } + + - do: + catch: missing + ingest.get_pipeline: + id: "my_pipeline" diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/ingest/20_date_processor.yaml b/rest-api-spec/src/main/resources/rest-api-spec/test/ingest/20_date_processor.yaml new file mode 100644 index 00000000000..71c5c4069b2 --- /dev/null +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/ingest/20_date_processor.yaml @@ -0,0 +1,37 @@ +--- +"Test date processor": + - do: + ingest.put_pipeline: + id: "my_pipeline" + body: > + { + "description": "_description", + "processors": [ + { + "date" : { + "match_field" : "date_source_field", + "target_field" : "date_target_field", + "match_formats" : ["dd/MM/yyyy"], + "timezone" : "Europe/Amsterdam" + } + } + ] + } + - match: { acknowledged: true } + + - do: + index: + index: test + type: test + id: 1 + pipeline: "my_pipeline" + body: {date_source_field: "12/06/2010"} + + - do: + get: + index: test + type: test + id: 1 + - match: { _source.date_source_field: "12/06/2010" } + - match: { _source.date_target_field: "2010-06-12T00:00:00.000+02:00" } + diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/ingest/30_mutate.yaml b/rest-api-spec/src/main/resources/rest-api-spec/test/ingest/30_mutate.yaml new file mode 100644 index 00000000000..1e7911e519a --- /dev/null +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/ingest/30_mutate.yaml @@ -0,0 +1,150 @@ +--- +"Test mutate processors": + - do: + ingest.put_pipeline: + id: "my_pipeline" + body: > + { + "description": "_description", + "processors": [ + { + "set" : { + "field" : "new_field", + "value": "new_value" + } + }, + { + "append" : { + "field" : "new_field", + "value": ["item2", "item3", "item4"] + } + }, + { + "rename" : { + "field" : "field_to_rename", + "to": "renamed_field" + } + }, + { + "remove" : { + "field" : "field_to_remove" + } + }, + { + "lowercase" : { + "field" : "field_to_lowercase" + } + }, + { + "uppercase" : { + "field" : "field_to_uppercase" + } + }, + { + "trim" : { + "field" : "field_to_trim" + } + }, + { + "split" : { + "field" : "field_to_split", + "separator": "-" + } + }, + { + "join" : { + "field" : "field_to_join", + "separator": "-" + } + }, + { + "convert" : { + "field" : "field_to_convert", + "type": "integer" + } + }, + { + "gsub" : { + "field": "field_to_gsub", + "pattern" : "-", + "replacement" : "." + } + } + ] + } + - match: { acknowledged: true } + + - do: + index: + index: test + type: test + id: 1 + pipeline: "my_pipeline" + body: { + field_to_rename: "value", + field_to_remove: "old_value", + field_to_lowercase: "LOWERCASE", + field_to_uppercase: "uppercase", + field_to_trim: " trimmed ", + field_to_split: "127-0-0-1", + field_to_join: ["127","0","0","1"], + field_to_convert: ["127","0","0","1"], + field_to_gsub: "127-0-0-1" + } + + - do: + get: + index: test + type: test + id: 1 + - is_false: _source.field_to_rename + - is_false: _source.field_to_remove + - match: { _source.new_field: ["new_value", "item2", "item3", "item4"] } + - match: { _source.renamed_field: "value" } + - match: { _source.field_to_lowercase: "lowercase" } + - match: { _source.field_to_uppercase: "UPPERCASE" } + - match: { _source.field_to_trim: "trimmed" } + - match: { _source.field_to_split: ["127","0","0","1"] } + - match: { _source.field_to_join: "127-0-0-1" } + - match: { _source.field_to_convert: [127,0,0,1] } + - match: { _source.field_to_gsub: "127.0.0.1" } + +--- +"Test metadata": + - do: + cluster.health: + wait_for_status: green + + - do: + ingest.put_pipeline: + id: "my_pipeline" + body: > + { + "description": "_description", + "processors": [ + { + "set" : { + "field" : "_index", + "value" : "surprise" + } + } + ] + } + - match: { acknowledged: true } + + - do: + index: + index: test + type: test + id: 1 + pipeline: "my_pipeline" + body: {field: "value"} + + - do: + get: + index: surprise + type: test + id: 1 + - length: { _source: 1 } + - match: { _source.field: "value" } + diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/ingest/40_simulate.yaml b/rest-api-spec/src/main/resources/rest-api-spec/test/ingest/40_simulate.yaml new file mode 100644 index 00000000000..3153ba85a59 --- /dev/null +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/ingest/40_simulate.yaml @@ -0,0 +1,421 @@ +--- +"Test simulate with stored ingest pipeline": + - do: + ingest.put_pipeline: + id: "my_pipeline" + body: > + { + "description": "_description", + "processors": [ + { + "set" : { + "field" : "field2", + "value" : "_value" + } + } + ] + } + - match: { acknowledged: true } + + - do: + ingest.simulate: + id: "my_pipeline" + body: > + { + "docs": [ + { + "_index": "index", + "_type": "type", + "_id": "id", + "_source": { + "foo": "bar" + } + } + ] + } + - length: { docs: 1 } + - match: { docs.0.doc._source.foo: "bar" } + - match: { docs.0.doc._source.field2: "_value" } + - length: { docs.0.doc._ingest: 1 } + - is_true: docs.0.doc._ingest.timestamp + +--- +"Test simulate with provided pipeline definition": + - do: + ingest.simulate: + body: > + { + "pipeline": { + "description": "_description", + "processors": [ + { + "set" : { + "field" : "field2", + "value" : "_value" + } + } + ] + }, + "docs": [ + { + "_index": "index", + "_type": "type", + "_id": "id", + "_source": { + "foo": "bar" + } + } + ] + } + - length: { docs: 1 } + +--- +"Test simulate with provided invalid pipeline definition": + - do: + catch: request + ingest.simulate: + body: > + { + "pipeline": { + "description": "_description", + "processors": [ + { + "set" : { + "value" : "_value" + } + } + ] + }, + "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 [field] is missing" } + +--- +"Test simulate without index type and id": + - do: + ingest.simulate: + body: > + { + "pipeline": { + "description": "_description", + "processors": [ + { + "set" : { + "field" : "field2", + "value" : "_value" + } + } + ] + }, + "docs": [ + { + "_source": { + "foo": "bar" + } + } + ] + } + - length: { docs: 1 } + +--- +"Test simulate with provided pipeline definition with on_failure block": + - do: + ingest.simulate: + body: > + { + "pipeline": { + "description": "_description", + "processors": [ + { + "rename" : { + "field" : "does_not_exist", + "to" : "field2", + "on_failure" : [ + { + "set" : { + "field" : "field2", + "value" : "_value" + } + } + ] + } + } + ] + }, + "docs": [ + { + "_index": "index", + "_type": "type", + "_id": "id", + "_source": { + "foo": "bar" + } + } + ] + } + - length: { docs: 1 } + - match: { docs.0.doc._source.foo: "bar" } + - match: { docs.0.doc._source.field2: "_value" } + - length: { docs.0.doc._ingest: 1 } + - is_true: docs.0.doc._ingest.timestamp + +--- +"Test simulate with no provided pipeline or pipeline_id": + - 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: + ingest.simulate: + verbose: true + body: > + { + "pipeline": { + "description": "_description", + "processors": [ + { + "set" : { + "tag" : "processor[set]-0", + "field" : "field2", + "value" : "_value" + } + }, + { + "set" : { + "field" : "field3", + "value" : "third_val" + } + } + ] + }, + "docs": [ + { + "_index": "index", + "_type": "type", + "_id": "id", + "_source": { + "foo": "bar" + } + } + ] + } + - length: { docs: 1 } + - length: { docs.0.processor_results: 2 } + - match: { docs.0.processor_results.0.tag: "processor[set]-0" } + - 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.0.doc._ingest: 1 } + - is_true: docs.0.processor_results.0.doc._ingest.timestamp + - 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" } + - length: { docs.0.processor_results.1.doc._ingest: 1 } + - is_true: docs.0.processor_results.1.doc._ingest.timestamp + +--- +"Test simulate with exception thrown": + - do: + ingest.simulate: + body: > + { + "pipeline": { + "description": "_description", + "processors": [ + { + "uppercase" : { + "field" : "foo" + } + } + ] + }, + "docs": [ + { + "_index": "index", + "_type": "type", + "_id": "id", + "_source": { + "not_foo": "bar" + } + }, + { + "_index": "index", + "_type": "type", + "_id": "id2", + "_source": { + "foo": "bar" + } + } + ] + } + - length: { docs: 2 } + - match: { docs.0.error.type: "illegal_argument_exception" } + - match: { docs.1.doc._source.foo: "BAR" } + - length: { docs.1.doc._ingest: 1 } + - is_true: docs.1.doc._ingest.timestamp + +--- +"Test verbose simulate with exception thrown": + - do: + ingest.simulate: + verbose: true + body: > + { + "pipeline": { + "description": "_description", + "processors": [ + { + "convert" : { + "field" : "foo", + "type" : "integer" + } + }, + { + "uppercase" : { + "field" : "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 } + - length: { docs.0.processor_results: 1 } + - match: { docs.0.processor_results.0.error.type: "illegal_argument_exception" } + - length: { docs.1.processor_results: 2 } + - match: { docs.1.processor_results.0.doc._index: "index" } + - match: { docs.1.processor_results.0.doc._source.foo: 5 } + - match: { docs.1.processor_results.0.doc._source.bar: "hello" } + - length: { docs.1.processor_results.0.doc._ingest: 1 } + - is_true: docs.1.processor_results.0.doc._ingest.timestamp + - match: { docs.1.processor_results.1.doc._source.foo: 5 } + - match: { docs.1.processor_results.1.doc._source.bar: "HELLO" } + - length: { docs.1.processor_results.1.doc._ingest: 1 } + - is_true: docs.1.processor_results.1.doc._ingest.timestamp + +--- +"Test verbose simulate with on_failure": + - do: + ingest.simulate: + verbose: true + body: > + { + "pipeline" : { + "description": "_description", + "processors": [ + { + "set" : { + "tag" : "setstatus-1", + "field" : "status", + "value" : 200 + } + }, + { + "rename" : { + "tag" : "rename-1", + "field" : "foofield", + "to" : "field1", + "on_failure" : [ + { + "set" : { + "tag" : "set on_failure rename", + "field" : "foofield", + "value" : "exists" + } + }, + { + "rename" : { + "field" : "foofield2", + "to" : "field1", + "on_failure" : [ + { + "set" : { + "field" : "foofield2", + "value" : "ran" + } + } + ] + } + } + ] + } + } + ] + }, + "docs": [ + { + "_index": "index", + "_type": "type", + "_id": "id", + "_source": { + "field1": "123.42 400 " + } + } + ] + } + - length: { docs: 1 } + - length: { docs.0.processor_results: 5 } + - match: { docs.0.processor_results.0.tag: "setstatus-1" } + - match: { docs.0.processor_results.0.doc._source.field1: "123.42 400 " } + - match: { docs.0.processor_results.0.doc._source.status: 200 } + - match: { docs.0.processor_results.1.tag: "rename-1" } + - match: { docs.0.processor_results.1.error.type: "illegal_argument_exception" } + - match: { docs.0.processor_results.1.error.reason: "field [foofield] doesn't exist" } + - match: { docs.0.processor_results.2.tag: "set on_failure rename" } + - is_false: docs.0.processor_results.3.tag + - is_false: docs.0.processor_results.4.tag + - match: { docs.0.processor_results.4.doc._source.foofield: "exists" } + - match: { docs.0.processor_results.4.doc._source.foofield2: "ran" } + - match: { docs.0.processor_results.4.doc._source.field1: "123.42 400 " } + - match: { docs.0.processor_results.4.doc._source.status: 200 } diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/ingest/50_on_failure.yaml b/rest-api-spec/src/main/resources/rest-api-spec/test/ingest/50_on_failure.yaml new file mode 100644 index 00000000000..7bce12d2ec5 --- /dev/null +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/ingest/50_on_failure.yaml @@ -0,0 +1,108 @@ +--- +"Test Pipeline With On Failure Block": + - do: + ingest.put_pipeline: + id: "my_pipeline" + body: > + { + "description": "_description", + "processors": [ + { + "set" : { + "field" : "_executed", + "value" : true + } + }, + { + "date" : { + "match_field" : "date", + "target_field" : "date", + "match_formats" : ["yyyy"] + } + } + ], + "on_failure" : [ + { + "set" : { + "field" : "_failed", + "value" : true + } + } + ] + } + - match: { acknowledged: true } + + - do: + index: + index: test + type: test + id: 1 + pipeline: "my_pipeline" + body: {field1: "value1"} + + - do: + get: + index: test + type: test + id: 1 + - match: { _source.field1: "value1" } + - match: { _source._executed: true } + - match: { _source._failed: true } + +--- +"Test Pipeline With Nested Processor On Failures": + - do: + ingest.put_pipeline: + id: "my_pipeline" + body: > + { + "description": "_description", + "processors": [ + { + "rename" : { + "field" : "foofield", + "to" : "field1", + "on_failure" : [ + { + "set" : { + "field" : "foofield", + "value" : "exists" + } + }, + { + "rename" : { + "field" : "foofield2", + "to" : "field1", + "on_failure" : [ + { + "set" : { + "field" : "foofield2", + "value" : "ran" + } + } + ] + } + } + ] + } + } + ] + } + - match: { acknowledged: true } + + - do: + index: + index: test + type: test + id: 1 + pipeline: "my_pipeline" + body: {field1: "value1"} + + - do: + get: + index: test + type: test + id: 1 + - match: { _source.field1: "value1" } + - match: { _source.foofield: "exists" } + - match: { _source.foofield2: "ran" } diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/ingest/60_fail.yaml b/rest-api-spec/src/main/resources/rest-api-spec/test/ingest/60_fail.yaml new file mode 100644 index 00000000000..019c229ae38 --- /dev/null +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/ingest/60_fail.yaml @@ -0,0 +1,68 @@ +--- +"Test Fail Processor": + - do: + ingest.put_pipeline: + id: "my_pipeline" + body: > + { + "description": "_description", + "processors": [ + { + "fail" : { + "message" : "error_message" + } + } + ] + } + - match: { acknowledged: true } + + - do: + catch: request + index: + index: test + type: test + id: 1 + pipeline: "my_pipeline" + body: {} + +--- +"Test fail with on_failure": + - do: + ingest.put_pipeline: + id: "my_pipeline" + body: > + { + "description": "_description", + "processors": [ + { + "fail" : { + "message" : "error", + "on_failure" : [ + { + "set" : { + "field" : "error_message", + "value" : "fail_processor_ran" + } + } + ] + } + } + ] + } + - match: { acknowledged: true } + + - do: + index: + index: test + type: test + id: 1 + pipeline: "my_pipeline" + body: {} + + - do: + get: + index: test + type: test + id: 1 + - match: { _source.error_message: "fail_processor_ran" } + diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/ingest/70_bulk.yaml b/rest-api-spec/src/main/resources/rest-api-spec/test/ingest/70_bulk.yaml new file mode 100644 index 00000000000..b70f05af67e --- /dev/null +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/ingest/70_bulk.yaml @@ -0,0 +1,105 @@ +setup: + - do: + ingest.put_pipeline: + id: "pipeline1" + body: > + { + "description": "_description", + "processors": [ + { + "set" : { + "field" : "field1", + "value": "value1" + } + } + ] + } + + - do: + ingest.put_pipeline: + id: "pipeline2" + body: > + { + "description": "_description", + "processors": [ + { + "set" : { + "field" : "field2", + "value": "value2" + } + } + ] + } + +--- +"Test bulk request without default pipeline": + + - do: + bulk: + body: + - index: + _index: test_index + _type: test_type + _id: test_id1 + pipeline: pipeline1 + - f1: v1 + - index: + _index: test_index + _type: test_type + _id: test_id2 + - f1: v2 + + - do: + get: + index: test_index + type: test_type + id: test_id1 + + - match: {_source.field1: value1} + - is_false: _source.field2 + + - do: + get: + index: test_index + type: test_type + id: test_id2 + + - is_false: _source.field1 + - is_false: _source.field2 + +--- +"Test bulk request with default pipeline": + + - do: + bulk: + pipeline: pipeline1 + body: + - index: + _index: test_index + _type: test_type + _id: test_id1 + - f1: v1 + - index: + _index: test_index + _type: test_type + _id: test_id2 + pipeline: pipeline2 + - f1: v2 + - do: + get: + index: test_index + type: test_type + id: test_id1 + + - match: {_source.field1: value1} + - is_false: _source.field2 + + - do: + get: + index: test_index + type: test_type + id: test_id2 + + - is_false: _source.field1 + - match: {_source.field2: value2} + diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/ingest/80_dedot_processor.yaml b/rest-api-spec/src/main/resources/rest-api-spec/test/ingest/80_dedot_processor.yaml new file mode 100644 index 00000000000..bdc64572a45 --- /dev/null +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/ingest/80_dedot_processor.yaml @@ -0,0 +1,64 @@ +--- +"Test De-Dot Processor With Provided Separator": + - do: + ingest.put_pipeline: + id: "my_pipeline" + body: > + { + "description": "_description", + "processors": [ + { + "dedot" : { + "separator" : "3" + } + } + ] + } + - match: { acknowledged: true } + + - do: + index: + index: test + type: test + id: 1 + pipeline: "my_pipeline" + body: {"a.b.c": "hello world"} + + - do: + get: + index: test + type: test + id: 1 + - match: { _source.a3b3c: "hello world" } + +--- +"Test De-Dot Processor With Default Separator": + - do: + ingest.put_pipeline: + id: "my_pipeline" + body: > + { + "description": "_description", + "processors": [ + { + "dedot" : { + } + } + ] + } + - match: { acknowledged: true } + + - do: + index: + index: test + type: test + id: 1 + pipeline: "my_pipeline" + body: {"a.b.c": "hello world"} + + - do: + get: + index: test + type: test + id: 1 + - match: { _source.a_b_c: "hello world" } diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/percolate/18_highligh_with_query.yaml b/rest-api-spec/src/main/resources/rest-api-spec/test/percolate/18_highligh_with_query.yaml index d7e1fbdcc68..97d652366da 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/percolate/18_highligh_with_query.yaml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/percolate/18_highligh_with_query.yaml @@ -27,7 +27,7 @@ - do: percolate: index: test_index - type: test_type + type: type_1 body: doc: foo: "bar foo" diff --git a/settings.gradle b/settings.gradle index 682bfec96d9..f6dab0f85b8 100644 --- a/settings.gradle +++ b/settings.gradle @@ -11,6 +11,7 @@ List projects = [ 'test:framework', 'test:fixtures:example-fixture', 'test:fixtures:hdfs-fixture', + 'modules:ingest-grok', 'modules:lang-expression', 'modules:lang-groovy', 'modules:lang-mustache', @@ -24,6 +25,7 @@ List projects = [ 'plugins:discovery-ec2', 'plugins:discovery-gce', 'plugins:discovery-multicast', + 'plugins:ingest-geoip', 'plugins:lang-javascript', 'plugins:lang-plan-a', 'plugins:lang-python', @@ -39,6 +41,8 @@ List projects = [ 'qa:smoke-test-client', 'qa:smoke-test-multinode', 'qa:smoke-test-plugins', + 'qa:ingest-with-mustache', + 'qa:ingest-disabled', 'qa:vagrant', ] diff --git a/test/framework/src/main/java/org/elasticsearch/ingest/RandomDocumentPicks.java b/test/framework/src/main/java/org/elasticsearch/ingest/RandomDocumentPicks.java new file mode 100644 index 00000000000..3f350cf425c --- /dev/null +++ b/test/framework/src/main/java/org/elasticsearch/ingest/RandomDocumentPicks.java @@ -0,0 +1,238 @@ +/* + * 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 com.carrotsearch.randomizedtesting.generators.RandomInts; +import com.carrotsearch.randomizedtesting.generators.RandomPicks; +import com.carrotsearch.randomizedtesting.generators.RandomStrings; +import org.elasticsearch.common.Strings; +import org.elasticsearch.ingest.core.IngestDocument; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Random; +import java.util.TreeMap; + +public final class RandomDocumentPicks { + + private RandomDocumentPicks() { + + } + + /** + * Returns a random field name. Can be a leaf field name or the + * path to refer to a field name using the dot notation. + */ + public static String randomFieldName(Random random) { + int numLevels = RandomInts.randomIntBetween(random, 1, 5); + String fieldName = ""; + for (int i = 0; i < numLevels; i++) { + if (i > 0) { + fieldName += "."; + } + fieldName += randomString(random); + } + return fieldName; + } + + /** + * Returns a random leaf field name. + */ + public static String randomLeafFieldName(Random random) { + String fieldName; + do { + fieldName = randomString(random); + } while (fieldName.contains(".")); + return fieldName; + } + + /** + * Returns a randomly selected existing field name out of the fields that are contained + * in the document provided as an argument. + */ + public static String randomExistingFieldName(Random random, IngestDocument ingestDocument) { + Map source = new TreeMap<>(ingestDocument.getSourceAndMetadata()); + Map.Entry randomEntry = RandomPicks.randomFrom(random, source.entrySet()); + String key = randomEntry.getKey(); + while (randomEntry.getValue() instanceof Map) { + @SuppressWarnings("unchecked") + Map map = (Map) randomEntry.getValue(); + Map treeMap = new TreeMap<>(map); + randomEntry = RandomPicks.randomFrom(random, treeMap.entrySet()); + key += "." + randomEntry.getKey(); + } + assert ingestDocument.getFieldValue(key, Object.class) != null; + return key; + } + + /** + * Adds a random non existing field to the provided document and associates it + * with the provided value. The field will be added at a random position within the document, + * not necessarily at the top level using a leaf field name. + */ + public static String addRandomField(Random random, IngestDocument ingestDocument, Object value) { + String fieldName; + do { + fieldName = randomFieldName(random); + } while (canAddField(fieldName, ingestDocument) == false); + ingestDocument.setFieldValue(fieldName, value); + return fieldName; + } + + /** + * Checks whether the provided field name can be safely added to the provided document. + * When the provided field name holds the path using the dot notation, we have to make sure + * that each node of the tree either doesn't exist or is a map, otherwise new fields cannot be added. + */ + public static boolean canAddField(String path, IngestDocument ingestDocument) { + String[] pathElements = Strings.splitStringToArray(path, '.'); + Map innerMap = ingestDocument.getSourceAndMetadata(); + if (pathElements.length > 1) { + for (int i = 0; i < pathElements.length - 1; i++) { + Object currentLevel = innerMap.get(pathElements[i]); + if (currentLevel == null) { + return true; + } + if (currentLevel instanceof Map == false) { + return false; + } + @SuppressWarnings("unchecked") + Map map = (Map) currentLevel; + innerMap = map; + } + } + String leafKey = pathElements[pathElements.length - 1]; + return innerMap.containsKey(leafKey) == false; + } + + /** + * Generates a random document and random metadata + */ + public static IngestDocument randomIngestDocument(Random random) { + return randomIngestDocument(random, randomSource(random)); + } + + /** + * Generates a document that holds random metadata and the document provided as a map argument + */ + public static IngestDocument randomIngestDocument(Random random, Map source) { + String index = randomString(random); + String type = randomString(random); + String id = randomString(random); + String routing = null; + if (random.nextBoolean()) { + routing = randomString(random); + } + String parent = null; + if (random.nextBoolean()) { + parent = randomString(random); + } + String timestamp = null; + if (random.nextBoolean()) { + timestamp = randomString(random); + } + String ttl = null; + if (random.nextBoolean()) { + ttl = randomString(random); + } + return new IngestDocument(index, type, id, routing, parent, timestamp, ttl, source); + } + + public static Map randomSource(Random random) { + Map document = new HashMap<>(); + addRandomFields(random, document, 0); + return document; + } + + /** + * Generates a random field value, can be a string, a number, a list of an object itself. + */ + public static Object randomFieldValue(Random random) { + return randomFieldValue(random, 0); + } + + private static Object randomFieldValue(Random random, int currentDepth) { + switch(RandomInts.randomIntBetween(random, 0, 8)) { + case 0: + return randomString(random); + case 1: + return random.nextInt(); + case 2: + return random.nextBoolean(); + case 3: + return random.nextDouble(); + case 4: + List stringList = new ArrayList<>(); + int numStringItems = RandomInts.randomIntBetween(random, 1, 10); + for (int j = 0; j < numStringItems; j++) { + stringList.add(randomString(random)); + } + return stringList; + case 5: + List intList = new ArrayList<>(); + int numIntItems = RandomInts.randomIntBetween(random, 1, 10); + for (int j = 0; j < numIntItems; j++) { + intList.add(random.nextInt()); + } + return intList; + case 6: + List booleanList = new ArrayList<>(); + int numBooleanItems = RandomInts.randomIntBetween(random, 1, 10); + for (int j = 0; j < numBooleanItems; j++) { + booleanList.add(random.nextBoolean()); + } + return booleanList; + case 7: + List doubleList = new ArrayList<>(); + int numDoubleItems = RandomInts.randomIntBetween(random, 1, 10); + for (int j = 0; j < numDoubleItems; j++) { + doubleList.add(random.nextDouble()); + } + return doubleList; + case 8: + Map newNode = new HashMap<>(); + addRandomFields(random, newNode, ++currentDepth); + return newNode; + default: + throw new UnsupportedOperationException(); + } + } + + public static String randomString(Random random) { + if (random.nextBoolean()) { + return RandomStrings.randomAsciiOfLengthBetween(random, 1, 10); + } + return RandomStrings.randomUnicodeOfCodepointLengthBetween(random, 1, 10); + } + + private static void addRandomFields(Random random, Map parentNode, int currentDepth) { + if (currentDepth > 5) { + return; + } + int numFields = RandomInts.randomIntBetween(random, 1, 10); + for (int i = 0; i < numFields; i++) { + String fieldName = randomLeafFieldName(random); + Object fieldValue = randomFieldValue(random, currentDepth); + parentNode.put(fieldName, fieldValue); + } + } +} diff --git a/test/framework/src/main/java/org/elasticsearch/ingest/TestProcessor.java b/test/framework/src/main/java/org/elasticsearch/ingest/TestProcessor.java new file mode 100644 index 00000000000..ae13174e7c1 --- /dev/null +++ b/test/framework/src/main/java/org/elasticsearch/ingest/TestProcessor.java @@ -0,0 +1,77 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.ingest; + +import org.elasticsearch.ingest.core.AbstractProcessorFactory; +import org.elasticsearch.ingest.core.IngestDocument; +import org.elasticsearch.ingest.core.Processor; + +import java.util.Map; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Consumer; + +/** + * Processor used for testing, keeps track of how many times it is invoked and + * accepts a {@link Consumer} of {@link IngestDocument} to be called when executed. + */ +public class TestProcessor implements Processor { + + private final String type; + private final String tag; + private final Consumer ingestDocumentConsumer; + private final AtomicInteger invokedCounter = new AtomicInteger(); + + public TestProcessor(Consumer ingestDocumentConsumer) { + this(null, "test-processor", ingestDocumentConsumer); + } + + public TestProcessor(String tag, String type, Consumer ingestDocumentConsumer) { + this.ingestDocumentConsumer = ingestDocumentConsumer; + this.type = type; + this.tag = tag; + } + + @Override + public void execute(IngestDocument ingestDocument) throws Exception { + invokedCounter.incrementAndGet(); + ingestDocumentConsumer.accept(ingestDocument); + } + + @Override + public String getType() { + return type; + } + + @Override + public String getTag() { + return tag; + } + + public int getInvokedCounter() { + return invokedCounter.get(); + } + + public static final class Factory extends AbstractProcessorFactory { + @Override + public TestProcessor doCreate(String processorTag, Map config) throws Exception { + return new TestProcessor(processorTag, "test-processor", ingestDocument -> {}); + } + } +} diff --git a/test/framework/src/main/java/org/elasticsearch/ingest/TestTemplateService.java b/test/framework/src/main/java/org/elasticsearch/ingest/TestTemplateService.java new file mode 100644 index 00000000000..9330db1bfcb --- /dev/null +++ b/test/framework/src/main/java/org/elasticsearch/ingest/TestTemplateService.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.ingest; + +import org.elasticsearch.ingest.core.TemplateService; + +import java.util.Map; + +public class TestTemplateService implements TemplateService { + + public static TemplateService instance() { + return new TestTemplateService(); + } + + private TestTemplateService() { + } + + @Override + public Template compile(String template) { + return new MockTemplate(template); + } + + public static class MockTemplate implements TemplateService.Template { + + private final String expected; + + public MockTemplate(String expected) { + this.expected = expected; + } + + @Override + public String execute(Map model) { + return expected; + } + + @Override + public String getKey() { + return expected; + } + } +}