From 04ebc63e340301f4d4c92c2475b448a55292a983 Mon Sep 17 00:00:00 2001 From: Christophe Bismuth Date: Mon, 26 Nov 2018 13:57:57 +0100 Subject: [PATCH] RoutingMissingException in more like this (#33974) More like this query allows to provide identifiers of documents to be retrieved as like/unlike items. It can happen that at retrieval time an error is thrown, for instance caused by missing routing value when `_routing` is set required in the mapping. Instead of ignoring such error and returning no documents for the query, the error should be re-thrown and returned to users. As part of this change also mget and mtermvectors are unified in the way they throw such exception like it happens in other places, so that a `RoutingMissingException` is raised. Closes #29678 --- .../action/get/TransportMultiGetAction.java | 11 +- .../TransportMultiTermVectorsAction.java | 18 +- .../index/query/MoreLikeThisQueryBuilder.java | 9 + .../get/TransportMultiGetActionTests.java | 228 +++++++++++++++++ .../TransportMultiTermVectorsActionTests.java | 229 ++++++++++++++++++ .../search/morelikethis/MoreLikeThisIT.java | 41 ++++ 6 files changed, 529 insertions(+), 7 deletions(-) create mode 100644 server/src/test/java/org/elasticsearch/action/get/TransportMultiGetActionTests.java create mode 100644 server/src/test/java/org/elasticsearch/action/termvectors/TransportMultiTermVectorsActionTests.java diff --git a/server/src/main/java/org/elasticsearch/action/get/TransportMultiGetAction.java b/server/src/main/java/org/elasticsearch/action/get/TransportMultiGetAction.java index 3763d88139a64..88a6b7d221a6c 100644 --- a/server/src/main/java/org/elasticsearch/action/get/TransportMultiGetAction.java +++ b/server/src/main/java/org/elasticsearch/action/get/TransportMultiGetAction.java @@ -20,6 +20,7 @@ package org.elasticsearch.action.get; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.RoutingMissingException; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.HandledTransportAction; import org.elasticsearch.cluster.ClusterState; @@ -69,8 +70,8 @@ protected void doExecute(Task task, final MultiGetRequest request, final ActionL item.routing(clusterState.metaData().resolveIndexRouting(item.routing(), item.index())); if ((item.routing() == null) && (clusterState.getMetaData().routingRequired(concreteSingleIndex, item.type()))) { - String message = "routing is required for [" + concreteSingleIndex + "]/[" + item.type() + "]/[" + item.id() + "]"; - responses.set(i, newItemFailure(concreteSingleIndex, item.type(), item.id(), new IllegalArgumentException(message))); + responses.set(i, newItemFailure(concreteSingleIndex, item.type(), item.id(), + new RoutingMissingException(concreteSingleIndex, item.type(), item.id()))); continue; } } catch (Exception e) { @@ -95,6 +96,12 @@ protected void doExecute(Task task, final MultiGetRequest request, final ActionL listener.onResponse(new MultiGetResponse(responses.toArray(new MultiGetItemResponse[responses.length()]))); } + executeShardAction(listener, responses, shardRequests); + } + + protected void executeShardAction(ActionListener listener, + AtomicArray responses, + Map shardRequests) { final AtomicInteger counter = new AtomicInteger(shardRequests.size()); for (final MultiGetShardRequest shardRequest : shardRequests.values()) { diff --git a/server/src/main/java/org/elasticsearch/action/termvectors/TransportMultiTermVectorsAction.java b/server/src/main/java/org/elasticsearch/action/termvectors/TransportMultiTermVectorsAction.java index 0b86ce32f3ff6..7f5f3152202ef 100644 --- a/server/src/main/java/org/elasticsearch/action/termvectors/TransportMultiTermVectorsAction.java +++ b/server/src/main/java/org/elasticsearch/action/termvectors/TransportMultiTermVectorsAction.java @@ -20,6 +20,7 @@ package org.elasticsearch.action.termvectors; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.RoutingMissingException; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.HandledTransportAction; import org.elasticsearch.cluster.ClusterState; @@ -67,17 +68,17 @@ protected void doExecute(Task task, final MultiTermVectorsRequest request, final termVectorsRequest.routing(clusterState.metaData().resolveIndexRouting(termVectorsRequest.routing(), termVectorsRequest.index())); if (!clusterState.metaData().hasConcreteIndex(termVectorsRequest.index())) { - responses.set(i, new MultiTermVectorsItemResponse(null, new MultiTermVectorsResponse.Failure(termVectorsRequest.index(), - termVectorsRequest.type(), termVectorsRequest.id(), new IndexNotFoundException(termVectorsRequest.index())))); + responses.set(i, new MultiTermVectorsItemResponse(null, + new MultiTermVectorsResponse.Failure(termVectorsRequest.index(), termVectorsRequest.type(), termVectorsRequest.id(), + new IndexNotFoundException(termVectorsRequest.index())))); continue; } String concreteSingleIndex = indexNameExpressionResolver.concreteSingleIndex(clusterState, termVectorsRequest).getName(); if (termVectorsRequest.routing() == null && - clusterState.getMetaData().routingRequired(concreteSingleIndex, termVectorsRequest.type())) { + clusterState.getMetaData().routingRequired(concreteSingleIndex, termVectorsRequest.type())) { responses.set(i, new MultiTermVectorsItemResponse(null, new MultiTermVectorsResponse.Failure(concreteSingleIndex, termVectorsRequest.type(), termVectorsRequest.id(), - new IllegalArgumentException("routing is required for [" + concreteSingleIndex + "]/[" + - termVectorsRequest.type() + "]/[" + termVectorsRequest.id() + "]")))); + new RoutingMissingException(concreteSingleIndex, termVectorsRequest.type(), termVectorsRequest.id())))); continue; } ShardId shardId = clusterService.operationRouting().shardId(clusterState, concreteSingleIndex, @@ -96,7 +97,14 @@ protected void doExecute(Task task, final MultiTermVectorsRequest request, final listener.onResponse(new MultiTermVectorsResponse(responses.toArray(new MultiTermVectorsItemResponse[responses.length()]))); } + executeShardAction(listener, responses, shardRequests); + } + + protected void executeShardAction(ActionListener listener, + AtomicArray responses, + Map shardRequests) { final AtomicInteger counter = new AtomicInteger(shardRequests.size()); + for (final MultiTermVectorsShardRequest shardRequest : shardRequests.values()) { shardAction.execute(shardRequest, new ActionListener() { @Override diff --git a/server/src/main/java/org/elasticsearch/index/query/MoreLikeThisQueryBuilder.java b/server/src/main/java/org/elasticsearch/index/query/MoreLikeThisQueryBuilder.java index 950c9e052adae..bdd11203b4581 100644 --- a/server/src/main/java/org/elasticsearch/index/query/MoreLikeThisQueryBuilder.java +++ b/server/src/main/java/org/elasticsearch/index/query/MoreLikeThisQueryBuilder.java @@ -26,6 +26,7 @@ import org.apache.lucene.search.Query; import org.elasticsearch.ElasticsearchParseException; import org.elasticsearch.ExceptionsHelper; +import org.elasticsearch.action.RoutingMissingException; import org.elasticsearch.action.termvectors.MultiTermVectorsItemResponse; import org.elasticsearch.action.termvectors.MultiTermVectorsRequest; import org.elasticsearch.action.termvectors.MultiTermVectorsResponse; @@ -1110,6 +1111,7 @@ private static Fields[] getFieldsFor(MultiTermVectorsResponse responses) throws for (MultiTermVectorsItemResponse response : responses) { if (response.isFailed()) { + checkRoutingMissingException(response); continue; } TermVectorsResponse getResponse = response.getResponse(); @@ -1121,6 +1123,13 @@ private static Fields[] getFieldsFor(MultiTermVectorsResponse responses) throws return likeFields.toArray(Fields.EMPTY_ARRAY); } + private static void checkRoutingMissingException(MultiTermVectorsItemResponse response) { + Throwable cause = ExceptionsHelper.unwrap(response.getFailure().getCause(), RoutingMissingException.class); + if (cause != null) { + throw ((RoutingMissingException) cause); + } + } + private static void handleExclude(BooleanQuery.Builder boolQuery, Item[] likeItems, QueryShardContext context) { MappedFieldType idField = context.fieldMapper(IdFieldMapper.NAME); if (idField == null) { diff --git a/server/src/test/java/org/elasticsearch/action/get/TransportMultiGetActionTests.java b/server/src/test/java/org/elasticsearch/action/get/TransportMultiGetActionTests.java new file mode 100644 index 0000000000000..f550c038e6016 --- /dev/null +++ b/server/src/test/java/org/elasticsearch/action/get/TransportMultiGetActionTests.java @@ -0,0 +1,228 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.get; + +import org.elasticsearch.Version; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.IndicesRequest; +import org.elasticsearch.action.RoutingMissingException; +import org.elasticsearch.action.support.ActionFilters; +import org.elasticsearch.client.node.NodeClient; +import org.elasticsearch.cluster.ClusterName; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.metadata.IndexMetaData; +import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; +import org.elasticsearch.cluster.metadata.MetaData; +import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.cluster.routing.OperationRouting; +import org.elasticsearch.cluster.routing.ShardIterator; +import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.concurrent.AtomicArray; +import org.elasticsearch.common.xcontent.XContentFactory; +import org.elasticsearch.common.xcontent.XContentHelper; +import org.elasticsearch.common.xcontent.XContentType; +import org.elasticsearch.index.Index; +import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.indices.IndicesService; +import org.elasticsearch.tasks.Task; +import org.elasticsearch.tasks.TaskId; +import org.elasticsearch.tasks.TaskManager; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.threadpool.TestThreadPool; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.Transport; +import org.elasticsearch.transport.TransportService; +import org.junit.AfterClass; +import org.junit.BeforeClass; + +import java.util.Map; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; + +import static java.util.Collections.emptyMap; +import static java.util.Collections.emptySet; +import static org.elasticsearch.common.UUIDs.randomBase64UUID; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.instanceOf; +import static org.mockito.Matchers.anyString; +import static org.mockito.Matchers.eq; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class TransportMultiGetActionTests extends ESTestCase { + + private static ThreadPool threadPool; + private static TransportService transportService; + private static ClusterService clusterService; + private static TransportMultiGetAction transportAction; + private static TransportShardMultiGetAction shardAction; + + @BeforeClass + public static void beforeClass() throws Exception { + threadPool = new TestThreadPool(TransportMultiGetActionTests.class.getSimpleName()); + + transportService = new TransportService(Settings.EMPTY, mock(Transport.class), threadPool, + TransportService.NOOP_TRANSPORT_INTERCEPTOR, + boundAddress -> DiscoveryNode.createLocal(Settings.builder().put("node.name", "node1").build(), + boundAddress.publishAddress(), randomBase64UUID()), null, emptySet()) { + @Override + public TaskManager getTaskManager() { + return taskManager; + } + }; + + final Index index1 = new Index("index1", randomBase64UUID()); + final ClusterState clusterState = ClusterState.builder(new ClusterName(TransportMultiGetActionTests.class.getSimpleName())) + .metaData(new MetaData.Builder() + .put(new IndexMetaData.Builder(index1.getName()) + .settings(Settings.builder().put("index.version.created", Version.CURRENT) + .put("index.number_of_shards", 1) + .put("index.number_of_replicas", 1) + .put(IndexMetaData.SETTING_INDEX_UUID, index1.getUUID())) + .putMapping("type1", + XContentHelper.convertToJson(BytesReference.bytes(XContentFactory.jsonBuilder() + .startObject() + .startObject("type1") + .startObject("_routing") + .field("required", false) + .endObject() + .endObject() + .endObject()), true, XContentType.JSON)) + .putMapping("type2", + XContentHelper.convertToJson(BytesReference.bytes(XContentFactory.jsonBuilder() + .startObject() + .startObject("type2") + .startObject("_routing") + .field("required", true) + .endObject() + .endObject() + .endObject()), true, XContentType.JSON)))).build(); + + final ShardIterator shardIterator = mock(ShardIterator.class); + when(shardIterator.shardId()).thenReturn(new ShardId(index1, randomInt())); + + final OperationRouting operationRouting = mock(OperationRouting.class); + when(operationRouting.getShards(eq(clusterState), eq(index1.getName()), anyString(), anyString(), anyString())) + .thenReturn(shardIterator); + when(operationRouting.shardId(eq(clusterState), eq(index1.getName()), anyString(), anyString())) + .thenReturn(new ShardId(index1, randomInt())); + + clusterService = mock(ClusterService.class); + when(clusterService.localNode()).thenReturn(transportService.getLocalNode()); + when(clusterService.state()).thenReturn(clusterState); + when(clusterService.operationRouting()).thenReturn(operationRouting); + + shardAction = new TransportShardMultiGetAction(clusterService, transportService, mock(IndicesService.class), threadPool, + new ActionFilters(emptySet()), new Resolver()) { + @Override + protected void doExecute(Task task, MultiGetShardRequest request, ActionListener listener) { + } + }; + } + + @AfterClass + public static void afterClass() { + ThreadPool.terminate(threadPool, 30, TimeUnit.SECONDS); + threadPool = null; + transportService = null; + clusterService = null; + transportAction = null; + shardAction = null; + } + + public void testTransportMultiGetAction() { + final Task task = createTask(); + final NodeClient client = new NodeClient(Settings.EMPTY, threadPool); + final MultiGetRequestBuilder request = new MultiGetRequestBuilder(client, MultiGetAction.INSTANCE); + request.add(new MultiGetRequest.Item("index1", "type1", "1")); + request.add(new MultiGetRequest.Item("index1", "type1", "2")); + + final AtomicBoolean shardActionInvoked = new AtomicBoolean(false); + transportAction = new TransportMultiGetAction(transportService, clusterService, shardAction, + new ActionFilters(emptySet()), new Resolver()) { + @Override + protected void executeShardAction(final ActionListener listener, + final AtomicArray responses, + final Map shardRequests) { + shardActionInvoked.set(true); + assertEquals(2, responses.length()); + assertNull(responses.get(0)); + assertNull(responses.get(1)); + } + }; + + transportAction.execute(task, request.request(), new ActionListenerAdapter()); + assertTrue(shardActionInvoked.get()); + } + + public void testTransportMultiGetAction_withMissingRouting() { + final Task task = createTask(); + final NodeClient client = new NodeClient(Settings.EMPTY, threadPool); + final MultiGetRequestBuilder request = new MultiGetRequestBuilder(client, MultiGetAction.INSTANCE); + request.add(new MultiGetRequest.Item("index1", "type2", "1").routing("1")); + request.add(new MultiGetRequest.Item("index1", "type2", "2")); + + final AtomicBoolean shardActionInvoked = new AtomicBoolean(false); + transportAction = new TransportMultiGetAction(transportService, clusterService, shardAction, + new ActionFilters(emptySet()), new Resolver()) { + @Override + protected void executeShardAction(final ActionListener listener, + final AtomicArray responses, + final Map shardRequests) { + shardActionInvoked.set(true); + assertEquals(2, responses.length()); + assertNull(responses.get(0)); + assertThat(responses.get(1).getFailure().getFailure(), instanceOf(RoutingMissingException.class)); + assertThat(responses.get(1).getFailure().getFailure().getMessage(), + equalTo("routing is required for [index1]/[type2]/[2]")); + } + }; + + transportAction.execute(task, request.request(), new ActionListenerAdapter()); + assertTrue(shardActionInvoked.get()); + + } + + private static Task createTask() { + return new Task(randomLong(), "transport", MultiGetAction.NAME, "description", + new TaskId(randomLong() + ":" + randomLong()), emptyMap()); + } + + static class Resolver extends IndexNameExpressionResolver { + + @Override + public Index concreteSingleIndex(ClusterState state, IndicesRequest request) { + return new Index("index1", randomBase64UUID()); + } + } + + static class ActionListenerAdapter implements ActionListener { + + @Override + public void onResponse(MultiGetResponse response) { + } + + @Override + public void onFailure(Exception e) { + } + } +} diff --git a/server/src/test/java/org/elasticsearch/action/termvectors/TransportMultiTermVectorsActionTests.java b/server/src/test/java/org/elasticsearch/action/termvectors/TransportMultiTermVectorsActionTests.java new file mode 100644 index 0000000000000..d2bae148ef596 --- /dev/null +++ b/server/src/test/java/org/elasticsearch/action/termvectors/TransportMultiTermVectorsActionTests.java @@ -0,0 +1,229 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.termvectors; + +import org.elasticsearch.Version; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.IndicesRequest; +import org.elasticsearch.action.RoutingMissingException; +import org.elasticsearch.action.get.TransportMultiGetActionTests; +import org.elasticsearch.action.support.ActionFilters; +import org.elasticsearch.client.node.NodeClient; +import org.elasticsearch.cluster.ClusterName; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.metadata.IndexMetaData; +import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; +import org.elasticsearch.cluster.metadata.MetaData; +import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.cluster.routing.OperationRouting; +import org.elasticsearch.cluster.routing.ShardIterator; +import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.concurrent.AtomicArray; +import org.elasticsearch.common.xcontent.XContentFactory; +import org.elasticsearch.common.xcontent.XContentHelper; +import org.elasticsearch.common.xcontent.XContentType; +import org.elasticsearch.index.Index; +import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.indices.IndicesService; +import org.elasticsearch.tasks.Task; +import org.elasticsearch.tasks.TaskId; +import org.elasticsearch.tasks.TaskManager; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.threadpool.TestThreadPool; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.Transport; +import org.elasticsearch.transport.TransportService; +import org.junit.AfterClass; +import org.junit.BeforeClass; + +import java.util.Map; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; + +import static java.util.Collections.emptyMap; +import static java.util.Collections.emptySet; +import static org.elasticsearch.common.UUIDs.randomBase64UUID; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.instanceOf; +import static org.mockito.Matchers.anyString; +import static org.mockito.Matchers.eq; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class TransportMultiTermVectorsActionTests extends ESTestCase { + + private static ThreadPool threadPool; + private static TransportService transportService; + private static ClusterService clusterService; + private static TransportMultiTermVectorsAction transportAction; + private static TransportShardMultiTermsVectorAction shardAction; + + @BeforeClass + public static void beforeClass() throws Exception { + threadPool = new TestThreadPool(TransportMultiGetActionTests.class.getSimpleName()); + + transportService = new TransportService(Settings.EMPTY, mock(Transport.class), threadPool, + TransportService.NOOP_TRANSPORT_INTERCEPTOR, + boundAddress -> DiscoveryNode.createLocal(Settings.builder().put("node.name", "node1").build(), + boundAddress.publishAddress(), randomBase64UUID()), null, emptySet()) { + @Override + public TaskManager getTaskManager() { + return taskManager; + } + }; + + final Index index1 = new Index("index1", randomBase64UUID()); + final ClusterState clusterState = ClusterState.builder(new ClusterName(TransportMultiGetActionTests.class.getSimpleName())) + .metaData(new MetaData.Builder() + .put(new IndexMetaData.Builder(index1.getName()) + .settings(Settings.builder().put("index.version.created", Version.CURRENT) + .put("index.number_of_shards", 1) + .put("index.number_of_replicas", 1) + .put(IndexMetaData.SETTING_INDEX_UUID, index1.getUUID())) + .putMapping("type1", + XContentHelper.convertToJson(BytesReference.bytes(XContentFactory.jsonBuilder() + .startObject() + .startObject("type1") + .startObject("_routing") + .field("required", false) + .endObject() + .endObject() + .endObject()), true, XContentType.JSON)) + .putMapping("type2", + XContentHelper.convertToJson(BytesReference.bytes(XContentFactory.jsonBuilder() + .startObject() + .startObject("type2") + .startObject("_routing") + .field("required", true) + .endObject() + .endObject() + .endObject()), true, XContentType.JSON)))).build(); + + final ShardIterator shardIterator = mock(ShardIterator.class); + when(shardIterator.shardId()).thenReturn(new ShardId(index1, randomInt())); + + final OperationRouting operationRouting = mock(OperationRouting.class); + when(operationRouting.getShards(eq(clusterState), eq(index1.getName()), anyString(), anyString(), anyString())) + .thenReturn(shardIterator); + when(operationRouting.shardId(eq(clusterState), eq(index1.getName()), anyString(), anyString())) + .thenReturn(new ShardId(index1, randomInt())); + + clusterService = mock(ClusterService.class); + when(clusterService.localNode()).thenReturn(transportService.getLocalNode()); + when(clusterService.state()).thenReturn(clusterState); + when(clusterService.operationRouting()).thenReturn(operationRouting); + + shardAction = new TransportShardMultiTermsVectorAction(clusterService, transportService, mock(IndicesService.class), threadPool, + new ActionFilters(emptySet()), new Resolver()) { + @Override + protected void doExecute(Task task, MultiTermVectorsShardRequest request, + ActionListener listener) { + } + }; + } + + @AfterClass + public static void afterClass() { + ThreadPool.terminate(threadPool, 30, TimeUnit.SECONDS); + threadPool = null; + transportService = null; + clusterService = null; + transportAction = null; + shardAction = null; + } + + public void testTransportMultiGetAction() { + final Task task = createTask(); + final NodeClient client = new NodeClient(Settings.EMPTY, threadPool); + final MultiTermVectorsRequestBuilder request = new MultiTermVectorsRequestBuilder(client, MultiTermVectorsAction.INSTANCE); + request.add(new TermVectorsRequest("index1", "type1", "1")); + request.add(new TermVectorsRequest("index1", "type1", "2")); + + final AtomicBoolean shardActionInvoked = new AtomicBoolean(false); + transportAction = new TransportMultiTermVectorsAction(transportService, clusterService, shardAction, + new ActionFilters(emptySet()), new Resolver()) { + @Override + protected void executeShardAction(final ActionListener listener, + final AtomicArray responses, + final Map shardRequests) { + shardActionInvoked.set(true); + assertEquals(2, responses.length()); + assertNull(responses.get(0)); + assertNull(responses.get(1)); + } + }; + + transportAction.execute(task, request.request(), new ActionListenerAdapter()); + assertTrue(shardActionInvoked.get()); + } + + public void testTransportMultiGetAction_withMissingRouting() { + final Task task = createTask(); + final NodeClient client = new NodeClient(Settings.EMPTY, threadPool); + final MultiTermVectorsRequestBuilder request = new MultiTermVectorsRequestBuilder(client, MultiTermVectorsAction.INSTANCE); + request.add(new TermVectorsRequest("index1", "type2", "1").routing("1")); + request.add(new TermVectorsRequest("index1", "type2", "2")); + + final AtomicBoolean shardActionInvoked = new AtomicBoolean(false); + transportAction = new TransportMultiTermVectorsAction(transportService, clusterService, shardAction, + new ActionFilters(emptySet()), new Resolver()) { + @Override + protected void executeShardAction(final ActionListener listener, + final AtomicArray responses, + final Map shardRequests) { + shardActionInvoked.set(true); + assertEquals(2, responses.length()); + assertNull(responses.get(0)); + assertThat(responses.get(1).getFailure().getCause(), instanceOf(RoutingMissingException.class)); + assertThat(responses.get(1).getFailure().getCause().getMessage(), + equalTo("routing is required for [index1]/[type2]/[2]")); + } + }; + + transportAction.execute(task, request.request(), new ActionListenerAdapter()); + assertTrue(shardActionInvoked.get()); + } + + private static Task createTask() { + return new Task(randomLong(), "transport", MultiTermVectorsAction.NAME, "description", + new TaskId(randomLong() + ":" + randomLong()), emptyMap()); + } + + static class Resolver extends IndexNameExpressionResolver { + + @Override + public Index concreteSingleIndex(ClusterState state, IndicesRequest request) { + return new Index("index1", randomBase64UUID()); + } + } + + static class ActionListenerAdapter implements ActionListener { + + @Override + public void onResponse(MultiTermVectorsResponse response) { + } + + @Override + public void onFailure(Exception e) { + } + } +} diff --git a/server/src/test/java/org/elasticsearch/search/morelikethis/MoreLikeThisIT.java b/server/src/test/java/org/elasticsearch/search/morelikethis/MoreLikeThisIT.java index 607133ea8f287..6180f4ec88f7d 100644 --- a/server/src/test/java/org/elasticsearch/search/morelikethis/MoreLikeThisIT.java +++ b/server/src/test/java/org/elasticsearch/search/morelikethis/MoreLikeThisIT.java @@ -19,6 +19,7 @@ package org.elasticsearch.search.morelikethis; +import org.elasticsearch.action.RoutingMissingException; import org.elasticsearch.action.admin.indices.create.CreateIndexRequestBuilder; import org.elasticsearch.action.index.IndexRequestBuilder; import org.elasticsearch.action.search.SearchPhaseExecutionException; @@ -60,6 +61,7 @@ import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertSearchResponse; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertThrows; import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.instanceOf; import static org.hamcrest.Matchers.notNullValue; public class MoreLikeThisIT extends ESIntegTestCase { @@ -673,4 +675,43 @@ public void testWithRouting() throws IOException { SearchResponse searchResponse = client().prepareSearch("index").setQuery(moreLikeThisQueryBuilder).get(); assertEquals(2, searchResponse.getHits().totalHits); } + + //Issue #29678 + public void testWithMissingRouting() throws IOException { + logger.info("Creating index test with routing required for type1"); + assertAcked(prepareCreate("test").addMapping("type1", + jsonBuilder().startObject().startObject("type1") + .startObject("properties").startObject("text").field("type", "text").endObject().endObject() + .startObject("_routing").field("required", true).endObject() + .endObject().endObject())); + + logger.info("Running Cluster Health"); + assertThat(ensureGreen(), equalTo(ClusterHealthStatus.GREEN)); + + { + logger.info("Running moreLikeThis with one item without routing attribute"); + SearchPhaseExecutionException exception = expectThrows(SearchPhaseExecutionException.class, () -> + client().prepareSearch().setQuery(new MoreLikeThisQueryBuilder(null, new Item[]{ + new Item("test", "type1", "1") + }).minTermFreq(1).minDocFreq(1)).get()); + + Throwable cause = exception.getCause(); + assertThat(cause, instanceOf(RoutingMissingException.class)); + assertThat(cause.getMessage(), equalTo("routing is required for [test]/[type1]/[1]")); + } + + { + logger.info("Running moreLikeThis with one item with routing attribute and two items without routing attribute"); + SearchPhaseExecutionException exception = expectThrows(SearchPhaseExecutionException.class, () -> + client().prepareSearch().setQuery(new MoreLikeThisQueryBuilder(null, new Item[]{ + new Item("test", "type1", "1").routing("1"), + new Item("test", "type1", "2"), + new Item("test", "type1", "3") + }).minTermFreq(1).minDocFreq(1)).get()); + + Throwable cause = exception.getCause(); + assertThat(cause, instanceOf(RoutingMissingException.class)); + assertThat(cause.getMessage(), equalTo("routing is required for [test]/[type1]/[2]")); + } + } }