From eb9d280c80d2fdd0d7ac195a8a5092efc3d9fae4 Mon Sep 17 00:00:00 2001 From: Alan Woodward Date: Wed, 27 Sep 2023 15:56:00 +0100 Subject: [PATCH] IndexAction to return DocWriteResponse --- .../datastreams/DataStreamIT.java | 12 +++---- .../datastreams/DataStreamsSnapshotsIT.java | 8 +++-- .../SystemDataStreamSnapshotIT.java | 8 ++--- .../TimestampFieldMapperServiceTests.java | 8 ++--- .../ingest/common/IngestRestartIT.java | 5 ++- .../geoip/GeoIpProcessorNonIngestNodeIT.java | 4 +-- .../ingest/geoip/GeoIpDownloaderTests.java | 3 +- .../action/ListenerActionIT.java | 5 ++- .../admin/cluster/node/tasks/TasksIT.java | 4 +-- .../create/AutoCreateSystemIndexIT.java | 5 ++- .../action/bulk/BulkIntegrationIT.java | 4 +-- .../action/bulk/BulkWithUpdatesIT.java | 3 +- .../action/search/TransportSearchIT.java | 14 ++++---- .../action/support/AutoCreateIndexIT.java | 6 ++-- .../master/IndexingMasterFailoverIT.java | 3 +- .../elasticsearch/aliases/IndexAliasesIT.java | 6 ++-- .../aliases/NetNewSystemIndexAliasIT.java | 4 +-- .../elasticsearch/blocks/SimpleBlocksIT.java | 4 +-- .../cluster/SimpleDataNodesIT.java | 4 +-- .../coordination/RareClusterStateIT.java | 7 ++-- .../cluster/routing/PrimaryAllocationIT.java | 5 ++- .../discovery/ClusterDisruptionIT.java | 8 ++--- .../document/DocumentActionsIT.java | 4 +-- .../elasticsearch/document/ShardInfoIT.java | 4 +-- .../org/elasticsearch/get/GetActionIT.java | 6 ++-- .../elasticsearch/index/FinalPipelineIT.java | 16 ++++----- .../index/IndexingPressureIT.java | 4 +-- .../index/WaitUntilRefreshIT.java | 5 ++- .../index/engine/MaxDocsLimitIT.java | 4 +-- .../index/mapper/DynamicMappingIT.java | 5 ++- .../index/shard/SearchIdleIT.java | 29 +++++++-------- .../elasticsearch/indexing/IndexActionIT.java | 8 ++--- .../mapping/ConcurrentDynamicTemplateIT.java | 6 ++-- .../recovery/IndexPrimaryRelocationIT.java | 3 +- .../indices/recovery/IndexRecoveryIT.java | 4 +-- .../indices/state/OpenCloseIndexIT.java | 4 +-- .../indices/stats/IndexStatsIT.java | 7 ++-- .../elasticsearch/recovery/RelocationIT.java | 4 +-- .../basic/SearchWithRandomExceptionsIT.java | 3 +- .../basic/SearchWithRandomIOExceptionsIT.java | 3 +- .../search/nested/SimpleNestedIT.java | 5 ++- .../ContextCompletionSuggestSearchIT.java | 4 +-- .../support/TimeSeriesDimensionsLimitIT.java | 4 +-- .../org/elasticsearch/update/UpdateIT.java | 3 +- .../ConcurrentDocumentOperationIT.java | 6 ++-- .../ConcurrentSeqNoVersioningIT.java | 6 ++-- .../versioning/SimpleVersioningIT.java | 20 +++++------ .../action/index/IndexAction.java | 5 +-- .../action/index/IndexRequestBuilder.java | 5 +-- .../action/index/TransportIndexAction.java | 3 +- .../elasticsearch/client/internal/Client.java | 6 ++-- .../internal/support/AbstractClient.java | 6 ++-- .../tasks/TaskResultsService.java | 6 ++-- .../indices/stats/IndicesStatsTests.java | 4 +-- .../FieldStatsProviderRefreshTests.java | 4 +-- .../search/SearchServiceTests.java | 16 ++++----- .../elasticsearch/test/BackgroundIndexer.java | 6 ++-- .../elasticsearch/test/ESIntegTestCase.java | 13 ++++--- .../TransportSubmitAsyncSearchAction.java | 4 +-- .../search/AsyncSearchSingleNodeTests.java | 8 ++--- .../xpack/ccr/FollowerFailOverIT.java | 7 ++-- .../core/async/AsyncTaskIndexService.java | 6 ++-- .../common/notifications/AbstractAuditor.java | 4 +-- .../core/async/AsyncResultsServiceTests.java | 10 +++--- .../async/AsyncSearchIndexServiceTests.java | 11 +++--- .../core/async/AsyncTaskServiceTests.java | 6 ++-- .../xpack/enrich/EnrichPolicyRunnerTests.java | 36 +++++++++---------- .../rules/QueryRulesIndexService.java | 3 +- .../search/SearchApplicationIndexService.java | 5 ++- .../rules/QueryRulesIndexServiceTests.java | 16 ++++----- .../SearchApplicationIndexServiceTests.java | 22 ++++++------ .../esql/action/EsqlActionBreakerIT.java | 3 +- .../index/engine/frozen/FrozenIndexIT.java | 4 +-- .../index/engine/frozen/FrozenIndexTests.java | 6 ++-- .../integration/ModelSnapshotRetentionIT.java | 3 +- .../ml/integration/ModelSnapshotSearchIT.java | 3 +- .../ml/integration/AnnotationIndexIT.java | 4 +-- .../integration/DatafeedConfigProviderIT.java | 7 ++-- .../ml/integration/JobConfigProviderIT.java | 13 ++++--- .../integration/TrainedModelProviderIT.java | 4 +-- .../ml/action/TransportPutCalendarAction.java | 6 ++-- .../ml/action/TransportPutFilterAction.java | 6 ++-- .../action/TransportUpdateFilterAction.java | 6 ++-- .../persistence/DatafeedConfigProvider.java | 5 ++- .../ml/dataframe/DataFrameAnalyticsTask.java | 4 +-- .../xpack/ml/dataframe/steps/FinalStep.java | 6 ++-- .../xpack/ml/job/JobManager.java | 4 +-- .../ml/job/persistence/JobConfigProvider.java | 3 +- .../job/persistence/JobResultsPersister.java | 7 ++-- .../persistence/JobResultsPersisterTests.java | 3 +- .../xpack/profiling/ProfilingTestCase.java | 4 +-- .../rank/rrf/RRFRankShardCanMatchIT.java | 6 ++-- .../cache/blob/BlobStoreCacheService.java | 6 ++-- .../cache/common/TestUtils.java | 5 +-- .../DateMathExpressionIntegTests.java | 3 +- .../KibanaSystemRoleIntegTests.java | 3 +- .../MultipleIndicesPermissionsTests.java | 7 ++-- .../elasticsearch/license/LicensingTests.java | 3 +- .../authc/esnative/NativeUsersStore.java | 5 ++- .../mapper/NativeRoleMappingStore.java | 6 ++-- .../authz/store/NativePrivilegeStore.java | 7 ++-- .../authz/store/NativeRolesStore.java | 5 ++- .../integration/TransformInternalIndexIT.java | 3 +- .../integration/TransformOldTransformsIT.java | 3 +- .../persistence/SeqNoPrimaryTermAndIndex.java | 4 +-- .../test/integration/WatchAckTests.java | 3 +- .../action/activate/ActivateWatchTests.java | 4 +-- .../actions/index/ExecutableIndexAction.java | 5 ++- .../actions/TransportPutWatchAction.java | 3 +- .../actions/index/IndexActionTests.java | 5 +-- .../ldap/AbstractAdLdapRealmTestCase.java | 3 +- 111 files changed, 335 insertions(+), 364 deletions(-) diff --git a/modules/data-streams/src/internalClusterTest/java/org/elasticsearch/datastreams/DataStreamIT.java b/modules/data-streams/src/internalClusterTest/java/org/elasticsearch/datastreams/DataStreamIT.java index 384970bdc7ab9..61757ebebb5cc 100644 --- a/modules/data-streams/src/internalClusterTest/java/org/elasticsearch/datastreams/DataStreamIT.java +++ b/modules/data-streams/src/internalClusterTest/java/org/elasticsearch/datastreams/DataStreamIT.java @@ -13,6 +13,7 @@ import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.action.ActionRequestBuilder; import org.elasticsearch.action.DocWriteRequest; +import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.admin.cluster.state.ClusterStateRequest; import org.elasticsearch.action.admin.indices.alias.Alias; import org.elasticsearch.action.admin.indices.alias.IndicesAliasesRequest; @@ -48,7 +49,6 @@ import org.elasticsearch.action.datastreams.ModifyDataStreamsAction; import org.elasticsearch.action.delete.DeleteRequest; import org.elasticsearch.action.index.IndexRequest; -import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.search.MultiSearchRequestBuilder; import org.elasticsearch.action.search.MultiSearchResponse; import org.elasticsearch.action.search.SearchRequest; @@ -297,7 +297,7 @@ public void testOtherWriteOps() throws Exception { { IndexRequest indexRequest = new IndexRequest(dataStreamName).source("{\"@timestamp\": \"2020-12-12\"}", XContentType.JSON) .opType(DocWriteRequest.OpType.CREATE); - IndexResponse indexResponse = client().index(indexRequest).actionGet(); + DocWriteResponse indexResponse = client().index(indexRequest).actionGet(); assertThat(indexResponse.getIndex(), backingIndexEqualTo(dataStreamName, 1)); } { @@ -1176,7 +1176,7 @@ public void testIndexDocsWithCustomRoutingTargetingDataStreamIsNotAllowed() thro String dataStream = "logs-foobar"; IndexRequest indexRequest = new IndexRequest(dataStream).source("{\"@timestamp\": \"2020-12-12\"}", XContentType.JSON) .opType(DocWriteRequest.OpType.CREATE); - IndexResponse indexResponse = client().index(indexRequest).actionGet(); + DocWriteResponse indexResponse = client().index(indexRequest).actionGet(); assertThat(indexResponse.getIndex(), backingIndexEqualTo(dataStream, 1)); // Index doc with custom routing that targets the data stream @@ -1238,7 +1238,7 @@ public void testIndexDocsWithCustomRoutingAllowed() throws Exception { IndexRequest indexRequest = new IndexRequest(dataStream).source("{\"@timestamp\": \"2020-12-12\"}", XContentType.JSON) .opType(DocWriteRequest.OpType.CREATE) .routing("custom"); - IndexResponse indexResponse = client().index(indexRequest).actionGet(); + DocWriteResponse indexResponse = client().index(indexRequest).actionGet(); assertThat(indexResponse.getIndex(), backingIndexEqualTo(dataStream, 1)); // Index doc with custom routing that targets the data stream IndexRequest indexRequestWithRouting = new IndexRequest(dataStream).source("@timestamp", System.currentTimeMillis()) @@ -1266,7 +1266,7 @@ public void testIndexDocsWithCustomRoutingTargetingBackingIndex() throws Excepti // Index doc that triggers creation of a data stream IndexRequest indexRequest = new IndexRequest("logs-foobar").source("{\"@timestamp\": \"2020-12-12\"}", XContentType.JSON) .opType(DocWriteRequest.OpType.CREATE); - IndexResponse indexResponse = client().index(indexRequest).actionGet(); + DocWriteResponse indexResponse = client().index(indexRequest).actionGet(); assertThat(indexResponse.getIndex(), backingIndexEqualTo("logs-foobar", 1)); String backingIndex = indexResponse.getIndex(); @@ -1277,7 +1277,7 @@ public void testIndexDocsWithCustomRoutingTargetingBackingIndex() throws Excepti .id(indexResponse.getId()) .setIfPrimaryTerm(indexResponse.getPrimaryTerm()) .setIfSeqNo(indexResponse.getSeqNo()); - IndexResponse response = client().index(indexRequestWithRouting).actionGet(); + DocWriteResponse response = client().index(indexRequestWithRouting).actionGet(); assertThat(response.getIndex(), equalTo(backingIndex)); } diff --git a/modules/data-streams/src/internalClusterTest/java/org/elasticsearch/datastreams/DataStreamsSnapshotsIT.java b/modules/data-streams/src/internalClusterTest/java/org/elasticsearch/datastreams/DataStreamsSnapshotsIT.java index 9b07828e04225..715d2a7a4de2f 100644 --- a/modules/data-streams/src/internalClusterTest/java/org/elasticsearch/datastreams/DataStreamsSnapshotsIT.java +++ b/modules/data-streams/src/internalClusterTest/java/org/elasticsearch/datastreams/DataStreamsSnapshotsIT.java @@ -28,7 +28,6 @@ import org.elasticsearch.action.datastreams.CreateDataStreamAction; import org.elasticsearch.action.datastreams.DeleteDataStreamAction; import org.elasticsearch.action.datastreams.GetDataStreamAction; -import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.action.support.master.AcknowledgedResponse; import org.elasticsearch.client.internal.Client; @@ -116,7 +115,10 @@ public void setup() throws Exception { ds2BackingIndexName = dsBackingIndexName.replace("-ds-", "-ds2-"); otherDs2BackingIndexName = otherDsBackingIndexName.replace("-other-ds-", "-other-ds2-"); - IndexResponse indexResponse = client.prepareIndex("ds").setOpType(DocWriteRequest.OpType.CREATE).setSource(DOCUMENT_SOURCE).get(); + DocWriteResponse indexResponse = client.prepareIndex("ds") + .setOpType(DocWriteRequest.OpType.CREATE) + .setSource(DOCUMENT_SOURCE) + .get(); assertEquals(DocWriteResponse.Result.CREATED, indexResponse.getResult()); id = indexResponse.getId(); @@ -297,7 +299,7 @@ public void testSnapshotAndRestoreInPlace() { } public void testSnapshotAndRestoreAllIncludeSpecificDataStream() throws Exception { - IndexResponse indexResponse = client.prepareIndex("other-ds") + DocWriteResponse indexResponse = client.prepareIndex("other-ds") .setOpType(DocWriteRequest.OpType.CREATE) .setSource(DOCUMENT_SOURCE) .get(); diff --git a/modules/data-streams/src/internalClusterTest/java/org/elasticsearch/datastreams/SystemDataStreamSnapshotIT.java b/modules/data-streams/src/internalClusterTest/java/org/elasticsearch/datastreams/SystemDataStreamSnapshotIT.java index 865de8b41cc68..0f60cbba0a4ff 100644 --- a/modules/data-streams/src/internalClusterTest/java/org/elasticsearch/datastreams/SystemDataStreamSnapshotIT.java +++ b/modules/data-streams/src/internalClusterTest/java/org/elasticsearch/datastreams/SystemDataStreamSnapshotIT.java @@ -8,12 +8,12 @@ package org.elasticsearch.datastreams; import org.elasticsearch.action.DocWriteRequest; +import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.admin.cluster.snapshots.restore.RestoreSnapshotResponse; import org.elasticsearch.action.admin.indices.get.GetIndexResponse; import org.elasticsearch.action.datastreams.CreateDataStreamAction; import org.elasticsearch.action.datastreams.DeleteDataStreamAction; import org.elasticsearch.action.datastreams.GetDataStreamAction; -import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.support.master.AcknowledgedResponse; import org.elasticsearch.cluster.metadata.ComposableIndexTemplate; import org.elasticsearch.index.IndexNotFoundException; @@ -64,7 +64,7 @@ public void testSystemDataStreamInGlobalState() throws Exception { } // Index a doc so that a concrete backing index will be created - IndexResponse indexRepsonse = client().prepareIndex(SYSTEM_DATA_STREAM_NAME) + DocWriteResponse indexRepsonse = client().prepareIndex(SYSTEM_DATA_STREAM_NAME) .setId("42") .setSource("{ \"@timestamp\": \"2099-03-08T11:06:07.000Z\", \"name\": \"my-name\" }", XContentType.JSON) .setOpType(DocWriteRequest.OpType.CREATE) @@ -162,7 +162,7 @@ public void testSystemDataStreamInFeatureState() throws Exception { } // Index a doc so that a concrete backing index will be created - IndexResponse indexToDataStreamResponse = client().prepareIndex(SYSTEM_DATA_STREAM_NAME) + DocWriteResponse indexToDataStreamResponse = client().prepareIndex(SYSTEM_DATA_STREAM_NAME) .setId("42") .setSource("{ \"@timestamp\": \"2099-03-08T11:06:07.000Z\", \"name\": \"my-name\" }", XContentType.JSON) .setOpType(DocWriteRequest.OpType.CREATE) @@ -171,7 +171,7 @@ public void testSystemDataStreamInFeatureState() throws Exception { assertThat(indexToDataStreamResponse.status().getStatus(), oneOf(200, 201)); // Index a doc so that a concrete backing index will be created - IndexResponse indexResponse = client().prepareIndex("my-index") + DocWriteResponse indexResponse = client().prepareIndex("my-index") .setId("42") .setSource("{ \"name\": \"my-name\" }", XContentType.JSON) .setOpType(DocWriteRequest.OpType.CREATE) diff --git a/modules/data-streams/src/test/java/org/elasticsearch/datastreams/TimestampFieldMapperServiceTests.java b/modules/data-streams/src/test/java/org/elasticsearch/datastreams/TimestampFieldMapperServiceTests.java index 8617106d5cc28..803f5c8661f17 100644 --- a/modules/data-streams/src/test/java/org/elasticsearch/datastreams/TimestampFieldMapperServiceTests.java +++ b/modules/data-streams/src/test/java/org/elasticsearch/datastreams/TimestampFieldMapperServiceTests.java @@ -9,9 +9,9 @@ package org.elasticsearch.datastreams; import org.elasticsearch.action.DocWriteRequest; +import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.admin.indices.template.put.PutComposableIndexTemplateAction; import org.elasticsearch.action.index.IndexRequest; -import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.cluster.metadata.ComposableIndexTemplate; import org.elasticsearch.cluster.metadata.Template; import org.elasticsearch.common.compress.CompressedXContent; @@ -58,7 +58,7 @@ protected Collection> getPlugins() { public void testGetTimestampFieldTypeForTsdbDataStream() throws IOException { createTemplate(true); - IndexResponse indexResponse = indexDoc(); + DocWriteResponse indexResponse = indexDoc(); var indicesService = getInstanceFromNode(IndicesService.class); var result = indicesService.getTimestampFieldType(indexResponse.getShardId().getIndex()); @@ -67,14 +67,14 @@ public void testGetTimestampFieldTypeForTsdbDataStream() throws IOException { public void testGetTimestampFieldTypeForDataStream() throws IOException { createTemplate(false); - IndexResponse indexResponse = indexDoc(); + DocWriteResponse indexResponse = indexDoc(); var indicesService = getInstanceFromNode(IndicesService.class); var result = indicesService.getTimestampFieldType(indexResponse.getShardId().getIndex()); assertThat(result, nullValue()); } - private IndexResponse indexDoc() { + private DocWriteResponse indexDoc() { Instant time = Instant.now(); var indexRequest = new IndexRequest("k8s").opType(DocWriteRequest.OpType.CREATE); indexRequest.source(DOC.replace("$time", formatInstant(time)), XContentType.JSON); diff --git a/modules/ingest-common/src/internalClusterTest/java/org/elasticsearch/ingest/common/IngestRestartIT.java b/modules/ingest-common/src/internalClusterTest/java/org/elasticsearch/ingest/common/IngestRestartIT.java index f2cebfc2569d7..96ca77a5f65f9 100644 --- a/modules/ingest-common/src/internalClusterTest/java/org/elasticsearch/ingest/common/IngestRestartIT.java +++ b/modules/ingest-common/src/internalClusterTest/java/org/elasticsearch/ingest/common/IngestRestartIT.java @@ -9,7 +9,6 @@ import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.admin.cluster.node.stats.NodesStatsResponse; -import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.action.support.WriteRequest; import org.elasticsearch.cluster.block.ClusterBlockException; @@ -321,7 +320,7 @@ public boolean validateClusterForming() { ); // but this one should pass since it has a longer timeout - final PlainActionFuture future = new PlainActionFuture<>(); + final PlainActionFuture future = new PlainActionFuture<>(); client().prepareIndex("index") .setId("passes1") .setSource("x", 2) @@ -333,7 +332,7 @@ public boolean validateClusterForming() { internalCluster().startNode(Settings.builder().put(GatewayService.RECOVER_AFTER_DATA_NODES_SETTING.getKey(), "1")); ensureYellow("index"); - final IndexResponse indexResponse = future.actionGet(timeout); + final DocWriteResponse indexResponse = future.actionGet(timeout); assertThat(indexResponse.status(), equalTo(RestStatus.CREATED)); assertThat(indexResponse.getResult(), equalTo(DocWriteResponse.Result.CREATED)); diff --git a/modules/ingest-geoip/src/internalClusterTest/java/org/elasticsearch/ingest/geoip/GeoIpProcessorNonIngestNodeIT.java b/modules/ingest-geoip/src/internalClusterTest/java/org/elasticsearch/ingest/geoip/GeoIpProcessorNonIngestNodeIT.java index dbfde52fd62ea..8490b17d535c6 100644 --- a/modules/ingest-geoip/src/internalClusterTest/java/org/elasticsearch/ingest/geoip/GeoIpProcessorNonIngestNodeIT.java +++ b/modules/ingest-geoip/src/internalClusterTest/java/org/elasticsearch/ingest/geoip/GeoIpProcessorNonIngestNodeIT.java @@ -9,8 +9,8 @@ package org.elasticsearch.ingest.geoip; import org.apache.lucene.util.Constants; +import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.index.IndexRequest; -import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.ingest.PutPipelineRequest; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.settings.Settings; @@ -101,7 +101,7 @@ public void testLazyLoading() throws IOException { final IndexRequest indexRequest = new IndexRequest("index"); indexRequest.setPipeline("geoip"); indexRequest.source(Collections.singletonMap("ip", "1.1.1.1")); - final IndexResponse indexResponse = client(ingestNode).index(indexRequest).actionGet(); + final DocWriteResponse indexResponse = client(ingestNode).index(indexRequest).actionGet(); assertThat(indexResponse.status(), equalTo(RestStatus.CREATED)); // now the geo-IP database should be loaded on the ingest node assertDatabaseLoadStatus(ingestNode, true); diff --git a/modules/ingest-geoip/src/test/java/org/elasticsearch/ingest/geoip/GeoIpDownloaderTests.java b/modules/ingest-geoip/src/test/java/org/elasticsearch/ingest/geoip/GeoIpDownloaderTests.java index 9f3334a07d8f3..f5a57e68581f2 100644 --- a/modules/ingest-geoip/src/test/java/org/elasticsearch/ingest/geoip/GeoIpDownloaderTests.java +++ b/modules/ingest-geoip/src/test/java/org/elasticsearch/ingest/geoip/GeoIpDownloaderTests.java @@ -14,6 +14,7 @@ import org.elasticsearch.action.ActionResponse; import org.elasticsearch.action.ActionType; import org.elasticsearch.action.DocWriteRequest.OpType; +import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.admin.indices.flush.FlushAction; import org.elasticsearch.action.admin.indices.flush.FlushRequest; import org.elasticsearch.action.admin.indices.flush.FlushResponse; @@ -220,7 +221,7 @@ public void testIndexChunks() throws IOException { AtomicInteger chunkIndex = new AtomicInteger(); - client.addHandler(IndexAction.INSTANCE, (IndexRequest request, ActionListener listener) -> { + client.addHandler(IndexAction.INSTANCE, (IndexRequest request, ActionListener listener) -> { int chunk = chunkIndex.getAndIncrement(); assertEquals(OpType.CREATE, request.opType()); assertThat(request.id(), Matchers.startsWith("test_" + (chunk + 15) + "_")); diff --git a/server/src/internalClusterTest/java/org/elasticsearch/action/ListenerActionIT.java b/server/src/internalClusterTest/java/org/elasticsearch/action/ListenerActionIT.java index 617b78e4f1bb8..535b9b73c9dc2 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/action/ListenerActionIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/action/ListenerActionIT.java @@ -9,7 +9,6 @@ package org.elasticsearch.action; import org.elasticsearch.action.index.IndexRequest; -import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.client.internal.Client; import org.elasticsearch.client.internal.Requests; import org.elasticsearch.test.ESIntegTestCase; @@ -30,9 +29,9 @@ public void testThreadedListeners() throws Throwable { request.source(Requests.INDEX_CONTENT_TYPE, "field1", "value1"); } - client.index(request, new ActionListener() { + client.index(request, new ActionListener() { @Override - public void onResponse(IndexResponse indexResponse) { + public void onResponse(DocWriteResponse indexResponse) { threadName.set(Thread.currentThread().getName()); latch.countDown(); } diff --git a/server/src/internalClusterTest/java/org/elasticsearch/action/admin/cluster/node/tasks/TasksIT.java b/server/src/internalClusterTest/java/org/elasticsearch/action/admin/cluster/node/tasks/TasksIT.java index 86ff08b5de36d..54c10499b0b3a 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/action/admin/cluster/node/tasks/TasksIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/action/admin/cluster/node/tasks/TasksIT.java @@ -12,6 +12,7 @@ import org.elasticsearch.ResourceNotFoundException; import org.elasticsearch.action.ActionFuture; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.FailedNodeException; import org.elasticsearch.action.TaskOperationFailure; import org.elasticsearch.action.admin.cluster.health.ClusterHealthAction; @@ -25,7 +26,6 @@ import org.elasticsearch.action.admin.indices.validate.query.ValidateQueryAction; import org.elasticsearch.action.bulk.BulkAction; import org.elasticsearch.action.index.IndexAction; -import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.search.SearchAction; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchTransportService; @@ -454,7 +454,7 @@ public void waitForTaskCompletion(Task task) {} } // Need to run the task in a separate thread because node client's .execute() is blocked by our task listener index = new Thread(() -> { - IndexResponse indexResponse = client().prepareIndex("test").setSource("test", "test").get(); + DocWriteResponse indexResponse = client().prepareIndex("test").setSource("test", "test").get(); assertArrayEquals(ReplicationResponse.NO_FAILURES, indexResponse.getShardInfo().getFailures()); }); index.start(); diff --git a/server/src/internalClusterTest/java/org/elasticsearch/action/admin/indices/create/AutoCreateSystemIndexIT.java b/server/src/internalClusterTest/java/org/elasticsearch/action/admin/indices/create/AutoCreateSystemIndexIT.java index 2dd662f6782fa..127d399eab04a 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/action/admin/indices/create/AutoCreateSystemIndexIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/action/admin/indices/create/AutoCreateSystemIndexIT.java @@ -15,7 +15,6 @@ import org.elasticsearch.action.admin.indices.get.GetIndexResponse; import org.elasticsearch.action.admin.indices.template.delete.DeleteComposableIndexTemplateAction; import org.elasticsearch.action.admin.indices.template.put.PutComposableIndexTemplateAction; -import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.cluster.metadata.AliasMetadata; import org.elasticsearch.cluster.metadata.ComposableIndexTemplate; @@ -117,7 +116,7 @@ public void testWriteToAliasPrimaryAutoCreatedFirst() throws Exception { client().execute(AutoCreateAction.INSTANCE, request).get(); } - IndexResponse response = client().prepareIndex(INDEX_NAME).setSource("{\"foo\":\"bar\"}", XContentType.JSON).get(); + DocWriteResponse response = client().prepareIndex(INDEX_NAME).setSource("{\"foo\":\"bar\"}", XContentType.JSON).get(); assertThat(response.getResult(), equalTo(DocWriteResponse.Result.CREATED)); } @@ -136,7 +135,7 @@ public void testWriteToAliasSecondaryAutoCreatedFirst() throws Exception { client().execute(AutoCreateAction.INSTANCE, request).get(); } - IndexResponse response = client().prepareIndex(INDEX_NAME).setSource("{\"foo\":\"bar\"}", XContentType.JSON).get(); + DocWriteResponse response = client().prepareIndex(INDEX_NAME).setSource("{\"foo\":\"bar\"}", XContentType.JSON).get(); assertThat(response.getResult(), equalTo(DocWriteResponse.Result.CREATED)); } diff --git a/server/src/internalClusterTest/java/org/elasticsearch/action/bulk/BulkIntegrationIT.java b/server/src/internalClusterTest/java/org/elasticsearch/action/bulk/BulkIntegrationIT.java index 2a2238b8984fb..96e3939312870 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/action/bulk/BulkIntegrationIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/action/bulk/BulkIntegrationIT.java @@ -10,10 +10,10 @@ import org.elasticsearch.ElasticsearchException; import org.elasticsearch.action.ActionRequestValidationException; +import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.admin.indices.alias.Alias; import org.elasticsearch.action.admin.indices.mapping.get.GetMappingsResponse; import org.elasticsearch.action.index.IndexRequest; -import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.ingest.PutPipelineRequest; import org.elasticsearch.action.support.master.AcknowledgedResponse; import org.elasticsearch.action.support.replication.ReplicationRequest; @@ -164,7 +164,7 @@ public void testDeleteIndexWhileIndexing() throws Exception { while (stopped.get() == false && docID.get() < 5000) { String id = Integer.toString(docID.incrementAndGet()); try { - IndexResponse response = client().prepareIndex(index) + DocWriteResponse response = client().prepareIndex(index) .setId(id) .setSource(Map.of("f" + randomIntBetween(1, 10), randomNonNegativeLong()), XContentType.JSON) .get(); diff --git a/server/src/internalClusterTest/java/org/elasticsearch/action/bulk/BulkWithUpdatesIT.java b/server/src/internalClusterTest/java/org/elasticsearch/action/bulk/BulkWithUpdatesIT.java index efa96ba05182d..7365e39049430 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/action/bulk/BulkWithUpdatesIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/action/bulk/BulkWithUpdatesIT.java @@ -14,7 +14,6 @@ import org.elasticsearch.action.delete.DeleteRequest; import org.elasticsearch.action.get.GetResponse; import org.elasticsearch.action.index.IndexRequest; -import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.support.WriteRequest.RefreshPolicy; import org.elasticsearch.action.update.UpdateRequest; @@ -697,7 +696,7 @@ public void testNoopUpdate() { createIndex(indexName, Settings.builder().put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1).build()); internalCluster().ensureAtLeastNumDataNodes(2); ensureGreen(indexName); - IndexResponse doc = index(indexName, "1", Map.of("user", "xyz")); + DocWriteResponse doc = index(indexName, "1", Map.of("user", "xyz")); assertThat(doc.getShardInfo().getSuccessful(), equalTo(2)); final BulkResponse bulkResponse = client().prepareBulk() .add(new UpdateRequest().index(indexName).id("1").detectNoop(true).doc("user", "xyz")) // noop update diff --git a/server/src/internalClusterTest/java/org/elasticsearch/action/search/TransportSearchIT.java b/server/src/internalClusterTest/java/org/elasticsearch/action/search/TransportSearchIT.java index 655a13c154338..e44a5a6a48181 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/action/search/TransportSearchIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/action/search/TransportSearchIT.java @@ -13,10 +13,10 @@ import org.elasticsearch.TransportVersion; import org.elasticsearch.TransportVersions; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.admin.cluster.node.stats.NodeStats; import org.elasticsearch.action.admin.cluster.node.stats.NodesStatsResponse; import org.elasticsearch.action.index.IndexRequest; -import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.action.support.WriteRequest; import org.elasticsearch.client.internal.Client; @@ -125,7 +125,7 @@ public void testLocalClusterAlias() { indexRequest.id("1"); indexRequest.source("field", "value"); indexRequest.setRefreshPolicy(WriteRequest.RefreshPolicy.WAIT_UNTIL); - IndexResponse indexResponse = client().index(indexRequest).actionGet(); + DocWriteResponse indexResponse = client().index(indexRequest).actionGet(); assertEquals(RestStatus.CREATED, indexResponse.status()); TaskId parentTaskId = new TaskId("node", randomNonNegativeLong()); @@ -174,7 +174,7 @@ public void testAbsoluteStartMillis() { indexRequest.id("1"); indexRequest.source("date", "1970-01-01"); indexRequest.setRefreshPolicy(WriteRequest.RefreshPolicy.WAIT_UNTIL); - IndexResponse indexResponse = client().index(indexRequest).actionGet(); + DocWriteResponse indexResponse = client().index(indexRequest).actionGet(); assertEquals(RestStatus.CREATED, indexResponse.status()); } { @@ -182,7 +182,7 @@ public void testAbsoluteStartMillis() { indexRequest.id("1"); indexRequest.source("date", "1982-01-01"); indexRequest.setRefreshPolicy(WriteRequest.RefreshPolicy.WAIT_UNTIL); - IndexResponse indexResponse = client().index(indexRequest).actionGet(); + DocWriteResponse indexResponse = client().index(indexRequest).actionGet(); assertEquals(RestStatus.CREATED, indexResponse.status()); } { @@ -250,14 +250,14 @@ public void testFinalReduce() { IndexRequest indexRequest = new IndexRequest("test"); indexRequest.id("1"); indexRequest.source("price", 10); - IndexResponse indexResponse = client().index(indexRequest).actionGet(); + DocWriteResponse indexResponse = client().index(indexRequest).actionGet(); assertEquals(RestStatus.CREATED, indexResponse.status()); } { IndexRequest indexRequest = new IndexRequest("test"); indexRequest.id("2"); indexRequest.source("price", 100); - IndexResponse indexResponse = client().index(indexRequest).actionGet(); + DocWriteResponse indexResponse = client().index(indexRequest).actionGet(); assertEquals(RestStatus.CREATED, indexResponse.status()); } indicesAdmin().prepareRefresh("test").get(); @@ -552,7 +552,7 @@ private void indexSomeDocs(String indexName, int numberOfShards, int numberOfDoc createIndex(indexName, Settings.builder().put("index.number_of_shards", numberOfShards).build()); for (int i = 0; i < numberOfDocs; i++) { - IndexResponse indexResponse = client().prepareIndex(indexName).setSource("number", randomInt()).get(); + DocWriteResponse indexResponse = client().prepareIndex(indexName).setSource("number", randomInt()).get(); assertEquals(RestStatus.CREATED, indexResponse.status()); } indicesAdmin().prepareRefresh(indexName).get(); diff --git a/server/src/internalClusterTest/java/org/elasticsearch/action/support/AutoCreateIndexIT.java b/server/src/internalClusterTest/java/org/elasticsearch/action/support/AutoCreateIndexIT.java index 6ed680ecc7034..e4ef0fa7f2d4f 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/action/support/AutoCreateIndexIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/action/support/AutoCreateIndexIT.java @@ -9,7 +9,7 @@ package org.elasticsearch.action.support; import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.index.IndexResponse; +import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.Priority; import org.elasticsearch.test.ESIntegTestCase; @@ -41,7 +41,7 @@ public void testBatchingWithDeprecationWarnings() throws Exception { final var client = client(); client.prepareIndex("no-dot").setSource("{}", XContentType.JSON).execute(new ActionListener<>() { @Override - public void onResponse(IndexResponse indexResponse) { + public void onResponse(DocWriteResponse indexResponse) { try { final var warningHeaders = client.threadPool().getThreadContext().getResponseHeaders().get("Warning"); if (warningHeaders != null) { @@ -68,7 +68,7 @@ public void onFailure(Exception e) { client.prepareIndex(".has-dot").setSource("{}", XContentType.JSON).execute(new ActionListener<>() { @Override - public void onResponse(IndexResponse indexResponse) { + public void onResponse(DocWriteResponse indexResponse) { try { final var warningHeaders = client.threadPool().getThreadContext().getResponseHeaders().get("Warning"); assertNotNull(warningHeaders); diff --git a/server/src/internalClusterTest/java/org/elasticsearch/action/support/master/IndexingMasterFailoverIT.java b/server/src/internalClusterTest/java/org/elasticsearch/action/support/master/IndexingMasterFailoverIT.java index c6313a0a9b791..9e50d57f5eb99 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/action/support/master/IndexingMasterFailoverIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/action/support/master/IndexingMasterFailoverIT.java @@ -9,7 +9,6 @@ package org.elasticsearch.action.support.master; import org.elasticsearch.action.DocWriteResponse; -import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.ESIntegTestCase; @@ -70,7 +69,7 @@ public void run() { } for (int i = 0; i < 10; i++) { // index data with mapping changes - IndexResponse response = client(dataNode).prepareIndex("myindex").setSource("field_" + i, "val").get(); + DocWriteResponse response = client(dataNode).prepareIndex("myindex").setSource("field_" + i, "val").get(); assertEquals(DocWriteResponse.Result.CREATED, response.getResult()); } } diff --git a/server/src/internalClusterTest/java/org/elasticsearch/aliases/IndexAliasesIT.java b/server/src/internalClusterTest/java/org/elasticsearch/aliases/IndexAliasesIT.java index 13eed96075a19..53d28d98f9695 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/aliases/IndexAliasesIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/aliases/IndexAliasesIT.java @@ -8,6 +8,7 @@ package org.elasticsearch.aliases; +import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.admin.indices.alias.Alias; import org.elasticsearch.action.admin.indices.alias.IndicesAliasesRequest.AliasActions; import org.elasticsearch.action.admin.indices.alias.get.GetAliasesResponse; @@ -16,7 +17,6 @@ import org.elasticsearch.action.delete.DeleteRequest; import org.elasticsearch.action.delete.DeleteResponse; import org.elasticsearch.action.index.IndexRequest; -import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.action.support.WriteRequest.RefreshPolicy; @@ -107,7 +107,7 @@ public void testAliases() throws Exception { }); logger.info("--> indexing against [alias1], should work now"); - IndexResponse indexResponse = client().index(new IndexRequest("alias1").id("1").source(source("1", "test"), XContentType.JSON)) + DocWriteResponse indexResponse = client().index(new IndexRequest("alias1").id("1").source(source("1", "test"), XContentType.JSON)) .actionGet(); assertThat(indexResponse.getIndex(), equalTo("test")); @@ -1322,7 +1322,7 @@ public void testIndexingAndQueryingHiddenAliases() throws Exception { ensureGreen(); // Put a couple docs in each index directly - IndexResponse res = client().index(new IndexRequest(nonWriteIndex).id("1").source(source("1", "nonwrite"), XContentType.JSON)) + DocWriteResponse res = client().index(new IndexRequest(nonWriteIndex).id("1").source(source("1", "nonwrite"), XContentType.JSON)) .get(); assertThat(res.status().getStatus(), equalTo(201)); res = client().index(new IndexRequest(writeIndex).id("2").source(source("2", "writeindex"), XContentType.JSON)).get(); diff --git a/server/src/internalClusterTest/java/org/elasticsearch/aliases/NetNewSystemIndexAliasIT.java b/server/src/internalClusterTest/java/org/elasticsearch/aliases/NetNewSystemIndexAliasIT.java index 105e9f5ec91f0..2e2340294d058 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/aliases/NetNewSystemIndexAliasIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/aliases/NetNewSystemIndexAliasIT.java @@ -9,10 +9,10 @@ package org.elasticsearch.aliases; import org.elasticsearch.Version; +import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.admin.indices.alias.get.GetAliasesRequest; import org.elasticsearch.action.admin.indices.alias.get.GetAliasesResponse; import org.elasticsearch.action.index.IndexRequest; -import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.cluster.metadata.IndexMetadata; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.CollectionUtils; @@ -43,7 +43,7 @@ public void testGetAliasWithNetNewSystemIndices() throws Exception { { final IndexRequest request = new IndexRequest(SYSTEM_INDEX_NAME); request.source("some_field", "some_value"); - IndexResponse resp = client().index(request).get(); + DocWriteResponse resp = client().index(request).get(); assertThat(resp.status().getStatus(), is(201)); } ensureGreen(); diff --git a/server/src/internalClusterTest/java/org/elasticsearch/blocks/SimpleBlocksIT.java b/server/src/internalClusterTest/java/org/elasticsearch/blocks/SimpleBlocksIT.java index 8628f2542390c..992dfaace6284 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/blocks/SimpleBlocksIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/blocks/SimpleBlocksIT.java @@ -10,11 +10,11 @@ import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.action.ActionRequestValidationException; +import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.admin.indices.create.CreateIndexResponse; import org.elasticsearch.action.admin.indices.readonly.AddIndexBlockRequestBuilder; import org.elasticsearch.action.admin.indices.readonly.AddIndexBlockResponse; import org.elasticsearch.action.index.IndexRequestBuilder; -import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.support.ActiveShardCount; import org.elasticsearch.action.support.master.AcknowledgedResponse; import org.elasticsearch.cluster.ClusterState; @@ -127,7 +127,7 @@ private void canIndexDocument(String index) { try { IndexRequestBuilder builder = client().prepareIndex(index); builder.setSource("foo", "bar"); - IndexResponse r = builder.execute().actionGet(); + DocWriteResponse r = builder.execute().actionGet(); assertThat(r, notNullValue()); } catch (ClusterBlockException e) { fail(); diff --git a/server/src/internalClusterTest/java/org/elasticsearch/cluster/SimpleDataNodesIT.java b/server/src/internalClusterTest/java/org/elasticsearch/cluster/SimpleDataNodesIT.java index 9687fcd4a52e7..8618104fadc26 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/cluster/SimpleDataNodesIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/cluster/SimpleDataNodesIT.java @@ -8,11 +8,11 @@ package org.elasticsearch.cluster; +import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.UnavailableShardsException; import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse; import org.elasticsearch.action.admin.indices.create.CreateIndexRequest; import org.elasticsearch.action.index.IndexRequest; -import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.support.ActiveShardCount; import org.elasticsearch.cluster.health.ClusterHealthStatus; import org.elasticsearch.cluster.metadata.IndexMetadata; @@ -77,7 +77,7 @@ public void testIndexingBeforeAndAfterDataNodesStart() { equalTo(false) ); - IndexResponse indexResponse = client().index(new IndexRequest("test").id("1").source(SOURCE, XContentType.JSON)).actionGet(); + DocWriteResponse indexResponse = client().index(new IndexRequest("test").id("1").source(SOURCE, XContentType.JSON)).actionGet(); assertThat(indexResponse.getId(), equalTo("1")); } diff --git a/server/src/internalClusterTest/java/org/elasticsearch/cluster/coordination/RareClusterStateIT.java b/server/src/internalClusterTest/java/org/elasticsearch/cluster/coordination/RareClusterStateIT.java index f569727729841..8458d83e4e9be 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/cluster/coordination/RareClusterStateIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/cluster/coordination/RareClusterStateIT.java @@ -14,6 +14,7 @@ import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.ActionRequestBuilder; import org.elasticsearch.action.ActionResponse; +import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.action.support.master.AcknowledgedResponse; @@ -283,7 +284,7 @@ public void testDelayedMappingPropagationOnPrimary() throws Exception { // this request does not change the cluster state, because mapping is already created, // we don't await and cancel committed publication - ActionFuture docIndexResponse = client().prepareIndex("index").setId("1").setSource("field", 42).execute(); + ActionFuture docIndexResponse = client().prepareIndex("index").setId("1").setSource("field", 42).execute(); // Wait a bit to make sure that the reason why we did not get a response // is that cluster state processing is blocked and not just that it takes @@ -372,7 +373,7 @@ public void testDelayedMappingPropagationOnReplica() throws Exception { assertEquals(minVersion, maxVersion); }); - final ActionFuture docIndexResponse = client().prepareIndex("index").setId("1").setSource("field", 42).execute(); + final ActionFuture docIndexResponse = client().prepareIndex("index").setId("1").setSource("field", 42).execute(); assertBusy(() -> assertTrue(client().prepareGet("index", "1").get().isExists())); @@ -381,7 +382,7 @@ public void testDelayedMappingPropagationOnReplica() throws Exception { // if the dynamic mapping update is not applied on the replica yet. // this request does not change the cluster state, because the mapping is dynamic, // we need to await and cancel committed publication - ActionFuture dynamicMappingsFut = executeAndCancelCommittedPublication( + ActionFuture dynamicMappingsFut = executeAndCancelCommittedPublication( client().prepareIndex("index").setId("2").setSource("field2", 42) ); diff --git a/server/src/internalClusterTest/java/org/elasticsearch/cluster/routing/PrimaryAllocationIT.java b/server/src/internalClusterTest/java/org/elasticsearch/cluster/routing/PrimaryAllocationIT.java index 345dd07932901..8c63c5341bbc5 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/cluster/routing/PrimaryAllocationIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/cluster/routing/PrimaryAllocationIT.java @@ -13,7 +13,6 @@ import org.elasticsearch.action.admin.indices.shards.IndicesShardStoresResponse; import org.elasticsearch.action.admin.indices.stats.ShardStats; import org.elasticsearch.action.bulk.BulkResponse; -import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.support.ActiveShardCount; import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.cluster.ClusterState; @@ -520,14 +519,14 @@ public void testPrimaryReplicaResyncFailed() throws Exception { logger.info("--> Indexing with gap in seqno to ensure that some operations will be replayed in resync"); long numDocs = scaledRandomIntBetween(5, 50); for (int i = 0; i < numDocs; i++) { - IndexResponse indexResult = indexDoc("test", Long.toString(i)); + DocWriteResponse indexResult = indexDoc("test", Long.toString(i)); assertThat(indexResult.getShardInfo().getSuccessful(), equalTo(numberOfReplicas + 1)); } final IndexShard oldPrimaryShard = internalCluster().getInstance(IndicesService.class, oldPrimary).getShardOrNull(shardId); EngineTestCase.generateNewSeqNo(IndexShardTestCase.getEngine(oldPrimaryShard)); // Make gap in seqno. long moreDocs = scaledRandomIntBetween(1, 10); for (int i = 0; i < moreDocs; i++) { - IndexResponse indexResult = indexDoc("test", Long.toString(numDocs + i)); + DocWriteResponse indexResult = indexDoc("test", Long.toString(numDocs + i)); assertThat(indexResult.getShardInfo().getSuccessful(), equalTo(numberOfReplicas + 1)); } final Set replicasSide1 = Sets.newHashSet(randomSubsetOf(between(1, numberOfReplicas - 1), replicaNodes)); diff --git a/server/src/internalClusterTest/java/org/elasticsearch/discovery/ClusterDisruptionIT.java b/server/src/internalClusterTest/java/org/elasticsearch/discovery/ClusterDisruptionIT.java index ec5fa1647782f..ce9ec8b5fc75c 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/discovery/ClusterDisruptionIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/discovery/ClusterDisruptionIT.java @@ -11,10 +11,10 @@ import org.apache.lucene.index.CorruptIndexException; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.NoShardAvailableActionException; import org.elasticsearch.action.get.GetResponse; import org.elasticsearch.action.index.IndexRequestBuilder; -import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.client.internal.Client; import org.elasticsearch.cluster.ClusterState; @@ -170,7 +170,7 @@ public void testAckedIndexing() throws Exception { indexRequestBuilder.setCreate(true); } - IndexResponse response = indexRequestBuilder.get(timeout); + DocWriteResponse response = indexRequestBuilder.get(timeout); assertThat(response.getResult(), is(oneOf(CREATED, UPDATED))); ackedDocs.put(id, node); logger.trace("[{}] indexed id [{}] through node [{}], response [{}]", name, id, node, response); @@ -293,7 +293,7 @@ public void testRejoinDocumentExistsInAllShardCopies() throws Exception { ensureStableCluster(2, notIsolatedNode); assertFalse(client(notIsolatedNode).admin().cluster().prepareHealth("test").setWaitForYellowStatus().get().isTimedOut()); - IndexResponse indexResponse = internalCluster().client(notIsolatedNode).prepareIndex("test").setSource("field", "value").get(); + DocWriteResponse indexResponse = internalCluster().client(notIsolatedNode).prepareIndex("test").setSource("field", "value").get(); assertThat(indexResponse.getVersion(), equalTo(1L)); logger.info("Verifying if document exists via node[{}]", notIsolatedNode); @@ -483,7 +483,7 @@ public void testRestartNodeWhileIndexing() throws Exception { while (stopped.get() == false && docID.get() < 5000) { String id = Integer.toString(docID.incrementAndGet()); try { - IndexResponse response = client().prepareIndex(index) + DocWriteResponse response = client().prepareIndex(index) .setId(id) .setSource(Map.of("f" + randomIntBetween(1, 10), randomNonNegativeLong()), XContentType.JSON) .get(); diff --git a/server/src/internalClusterTest/java/org/elasticsearch/document/DocumentActionsIT.java b/server/src/internalClusterTest/java/org/elasticsearch/document/DocumentActionsIT.java index edb7c96d2b6eb..828ae839cdd7d 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/document/DocumentActionsIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/document/DocumentActionsIT.java @@ -8,6 +8,7 @@ package org.elasticsearch.document; +import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.admin.indices.cache.clear.ClearIndicesCacheRequest; import org.elasticsearch.action.admin.indices.cache.clear.ClearIndicesCacheResponse; import org.elasticsearch.action.admin.indices.flush.FlushResponse; @@ -19,7 +20,6 @@ import org.elasticsearch.action.get.GetRequest; import org.elasticsearch.action.get.GetResponse; import org.elasticsearch.action.index.IndexRequest; -import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.support.WriteRequest.RefreshPolicy; import org.elasticsearch.cluster.health.ClusterHealthStatus; @@ -56,7 +56,7 @@ public void testIndexActions() throws Exception { logger.info("Running Cluster Health"); ensureGreen(); logger.info("Indexing [type1/1]"); - IndexResponse indexResponse = client().prepareIndex() + DocWriteResponse indexResponse = client().prepareIndex() .setIndex("test") .setId("1") .setSource(source("1", "test")) diff --git a/server/src/internalClusterTest/java/org/elasticsearch/document/ShardInfoIT.java b/server/src/internalClusterTest/java/org/elasticsearch/document/ShardInfoIT.java index a99d97dfe18ff..75b818d082dff 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/document/ShardInfoIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/document/ShardInfoIT.java @@ -8,13 +8,13 @@ package org.elasticsearch.document; +import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse; import org.elasticsearch.action.admin.indices.recovery.RecoveryResponse; import org.elasticsearch.action.bulk.BulkItemResponse; import org.elasticsearch.action.bulk.BulkRequestBuilder; import org.elasticsearch.action.bulk.BulkResponse; import org.elasticsearch.action.delete.DeleteResponse; -import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.support.replication.ReplicationResponse; import org.elasticsearch.action.update.UpdateResponse; import org.elasticsearch.cluster.ClusterState; @@ -33,7 +33,7 @@ public class ShardInfoIT extends ESIntegTestCase { public void testIndexAndDelete() throws Exception { prepareIndex(1); - IndexResponse indexResponse = client().prepareIndex("idx").setSource("{}", XContentType.JSON).get(); + DocWriteResponse indexResponse = client().prepareIndex("idx").setSource("{}", XContentType.JSON).get(); assertShardInfo(indexResponse); DeleteResponse deleteResponse = client().prepareDelete("idx", indexResponse.getId()).get(); assertShardInfo(deleteResponse); diff --git a/server/src/internalClusterTest/java/org/elasticsearch/get/GetActionIT.java b/server/src/internalClusterTest/java/org/elasticsearch/get/GetActionIT.java index 91f17e90be24a..129b83f664927 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/get/GetActionIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/get/GetActionIT.java @@ -17,7 +17,6 @@ import org.elasticsearch.action.get.MultiGetRequest; import org.elasticsearch.action.get.MultiGetRequestBuilder; import org.elasticsearch.action.get.MultiGetResponse; -import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.support.DefaultShardOperationFailedException; import org.elasticsearch.common.Strings; import org.elasticsearch.common.bytes.BytesReference; @@ -203,7 +202,10 @@ public void testGetWithAliasPointingToMultipleIndices() { } else { indicesAdmin().prepareCreate("index3").addAlias(new Alias("alias1").indexRouting("1").writeIndex(true)).get(); } - IndexResponse indexResponse = client().prepareIndex("index1").setId("id").setSource(Collections.singletonMap("foo", "bar")).get(); + DocWriteResponse indexResponse = client().prepareIndex("index1") + .setId("id") + .setSource(Collections.singletonMap("foo", "bar")) + .get(); assertThat(indexResponse.status().getStatus(), equalTo(RestStatus.CREATED.getStatus())); IllegalArgumentException exception = expectThrows( diff --git a/server/src/internalClusterTest/java/org/elasticsearch/index/FinalPipelineIT.java b/server/src/internalClusterTest/java/org/elasticsearch/index/FinalPipelineIT.java index 165242128ca8a..ffa6799601b14 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/index/FinalPipelineIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/index/FinalPipelineIT.java @@ -8,10 +8,10 @@ package org.elasticsearch.index; +import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.get.GetRequestBuilder; import org.elasticsearch.action.get.GetResponse; import org.elasticsearch.action.index.IndexRequestBuilder; -import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.ingest.DeletePipelineRequest; import org.elasticsearch.action.ingest.GetPipelineResponse; import org.elasticsearch.action.ingest.PutPipelineRequest; @@ -133,7 +133,7 @@ public void testFinalPipelineOfOldDestinationIsNotInvoked() { {"processors": [{"final": {"exists":"no_such_field"}}]}"""); clusterAdmin().putPipeline(new PutPipelineRequest("final_pipeline", finalPipelineBody, XContentType.JSON)).actionGet(); - IndexResponse indexResponse = client().prepareIndex("index") + DocWriteResponse indexResponse = client().prepareIndex("index") .setId("1") .setSource(Map.of("field", "value")) .setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE) @@ -159,7 +159,7 @@ public void testFinalPipelineOfNewDestinationIsInvoked() { {"processors": [{"final": {}}]}"""); clusterAdmin().putPipeline(new PutPipelineRequest("final_pipeline", finalPipelineBody, XContentType.JSON)).actionGet(); - IndexResponse indexResponse = client().prepareIndex("index") + DocWriteResponse indexResponse = client().prepareIndex("index") .setId("1") .setSource(Map.of("field", "value")) .setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE) @@ -185,7 +185,7 @@ public void testDefaultPipelineOfNewDestinationIsNotInvoked() { {"processors": [{"final": {}}]}"""); clusterAdmin().putPipeline(new PutPipelineRequest("target_default_pipeline", targetPipeline, XContentType.JSON)).actionGet(); - IndexResponse indexResponse = client().prepareIndex("index") + DocWriteResponse indexResponse = client().prepareIndex("index") .setId("1") .setSource(Map.of("field", "value")) .setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE) @@ -211,7 +211,7 @@ public void testDefaultPipelineOfRerouteDestinationIsInvoked() { {"processors": [{"final": {}}]}"""); clusterAdmin().putPipeline(new PutPipelineRequest("target_default_pipeline", targetPipeline, XContentType.JSON)).actionGet(); - IndexResponse indexResponse = client().prepareIndex("index") + DocWriteResponse indexResponse = client().prepareIndex("index") .setId("1") .setSource(Map.of("field", "value")) .setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE) @@ -276,7 +276,7 @@ public void testRequestPipelineAndFinalPipeline() { index.setSource(Map.of("field", "value")); index.setPipeline("request_pipeline"); index.setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE); - final IndexResponse response = index.get(); + final DocWriteResponse response = index.get(); assertThat(response.status(), equalTo(RestStatus.CREATED)); final GetRequestBuilder get = client().prepareGet("index", "1"); final GetResponse getResponse = get.get(); @@ -303,7 +303,7 @@ public void testDefaultAndFinalPipeline() { final IndexRequestBuilder index = client().prepareIndex("index").setId("1"); index.setSource(Map.of("field", "value")); index.setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE); - final IndexResponse response = index.get(); + final DocWriteResponse response = index.get(); assertThat(response.status(), equalTo(RestStatus.CREATED)); final GetRequestBuilder get = client().prepareGet("index", "1"); final GetResponse getResponse = get.get(); @@ -350,7 +350,7 @@ public void testDefaultAndFinalPipelineFromTemplates() { final IndexRequestBuilder index = client().prepareIndex("index").setId("1"); index.setSource(Map.of("field", "value")); index.setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE); - final IndexResponse response = index.get(); + final DocWriteResponse response = index.get(); assertThat(response.status(), equalTo(RestStatus.CREATED)); final GetRequestBuilder get = client().prepareGet("index", "1"); final GetResponse getResponse = get.get(); diff --git a/server/src/internalClusterTest/java/org/elasticsearch/index/IndexingPressureIT.java b/server/src/internalClusterTest/java/org/elasticsearch/index/IndexingPressureIT.java index 6ff7470252122..206aa57bc84b3 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/index/IndexingPressureIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/index/IndexingPressureIT.java @@ -8,13 +8,13 @@ package org.elasticsearch.index; import org.elasticsearch.action.ActionFuture; +import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.admin.indices.stats.IndicesStatsResponse; import org.elasticsearch.action.admin.indices.stats.ShardStats; import org.elasticsearch.action.bulk.BulkRequest; import org.elasticsearch.action.bulk.BulkResponse; import org.elasticsearch.action.bulk.TransportShardBulkAction; import org.elasticsearch.action.index.IndexRequest; -import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.common.UUIDs; @@ -369,7 +369,7 @@ public void testWritesWillSucceedIfBelowThreshold() throws Exception { // The write limits is set to 1MB. We will send up to 800KB to stay below that threshold. int thresholdToStopSending = 800 * 1024; - ArrayList> responses = new ArrayList<>(); + ArrayList> responses = new ArrayList<>(); long totalRequestSize = 0; while (totalRequestSize < thresholdToStopSending) { IndexRequest request = new IndexRequest(INDEX_NAME).id(UUIDs.base64UUID()) diff --git a/server/src/internalClusterTest/java/org/elasticsearch/index/WaitUntilRefreshIT.java b/server/src/internalClusterTest/java/org/elasticsearch/index/WaitUntilRefreshIT.java index 4c7b5ee3e775e..302d6ce74d65f 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/index/WaitUntilRefreshIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/index/WaitUntilRefreshIT.java @@ -14,7 +14,6 @@ import org.elasticsearch.action.bulk.BulkRequestBuilder; import org.elasticsearch.action.bulk.BulkResponse; import org.elasticsearch.action.delete.DeleteResponse; -import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.support.WriteRequest.RefreshPolicy; import org.elasticsearch.action.update.UpdateResponse; import org.elasticsearch.client.internal.Requests; @@ -57,7 +56,7 @@ public void createTestIndex() { } public void testIndex() { - IndexResponse index = client().prepareIndex("test") + DocWriteResponse index = client().prepareIndex("test") .setId("1") .setSource("foo", "bar") .setRefreshPolicy(RefreshPolicy.WAIT_UNTIL) @@ -144,7 +143,7 @@ public void testBulk() { */ public void testNoRefreshInterval() throws InterruptedException, ExecutionException { updateIndexSettings(Settings.builder().put("index.refresh_interval", -1), "test"); - ActionFuture index = client().prepareIndex("test") + ActionFuture index = client().prepareIndex("test") .setId("1") .setSource("foo", "bar") .setRefreshPolicy(RefreshPolicy.WAIT_UNTIL) diff --git a/server/src/internalClusterTest/java/org/elasticsearch/index/engine/MaxDocsLimitIT.java b/server/src/internalClusterTest/java/org/elasticsearch/index/engine/MaxDocsLimitIT.java index f754d291c8010..56e1598bd7a15 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/index/engine/MaxDocsLimitIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/index/engine/MaxDocsLimitIT.java @@ -8,7 +8,7 @@ package org.elasticsearch.index.engine; -import org.elasticsearch.action.index.IndexResponse; +import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.cluster.metadata.IndexMetadata; import org.elasticsearch.common.settings.Settings; @@ -177,7 +177,7 @@ static IndexingResult indexDocs(int numRequests, int numThreads) throws Exceptio phaser.arriveAndAwaitAdvance(); while (completedRequests.incrementAndGet() <= numRequests) { try { - final IndexResponse resp = client().prepareIndex("test").setSource("{}", XContentType.JSON).get(); + final DocWriteResponse resp = client().prepareIndex("test").setSource("{}", XContentType.JSON).get(); numSuccess.incrementAndGet(); assertThat(resp.status(), equalTo(RestStatus.CREATED)); } catch (IllegalArgumentException e) { diff --git a/server/src/internalClusterTest/java/org/elasticsearch/index/mapper/DynamicMappingIT.java b/server/src/internalClusterTest/java/org/elasticsearch/index/mapper/DynamicMappingIT.java index 86b1cdf315f55..ce23c44cb96cc 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/index/mapper/DynamicMappingIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/index/mapper/DynamicMappingIT.java @@ -13,7 +13,6 @@ import org.elasticsearch.action.bulk.BulkResponse; import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.index.IndexRequestBuilder; -import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.support.WriteRequest; import org.elasticsearch.cluster.ClusterState; @@ -601,7 +600,7 @@ public void testSubobjectsFalseAtRoot() throws Exception { IndexRequest request = new IndexRequest("test").setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE) .source("host.name", "localhost", "host.id", 111, "time", 100, "time.max", 1000); - IndexResponse indexResponse = client().index(request).actionGet(); + DocWriteResponse indexResponse = client().index(request).actionGet(); assertEquals(RestStatus.CREATED, indexResponse.status()); assertBusy(() -> { @@ -646,7 +645,7 @@ public void testSubobjectsFalse() throws Exception { "foo.metrics.time.max", 1000 ); - IndexResponse indexResponse = client().index(request).actionGet(); + DocWriteResponse indexResponse = client().index(request).actionGet(); assertEquals(RestStatus.CREATED, indexResponse.status()); assertBusy(() -> { diff --git a/server/src/internalClusterTest/java/org/elasticsearch/index/shard/SearchIdleIT.java b/server/src/internalClusterTest/java/org/elasticsearch/index/shard/SearchIdleIT.java index ac25674456fde..38a878fee34c9 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/index/shard/SearchIdleIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/index/shard/SearchIdleIT.java @@ -9,12 +9,12 @@ package org.elasticsearch.index.shard; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.admin.indices.stats.IndicesStatsRequest; import org.elasticsearch.action.admin.indices.stats.IndicesStatsResponse; import org.elasticsearch.action.admin.indices.stats.ShardStats; import org.elasticsearch.action.get.GetRequest; import org.elasticsearch.action.get.MultiGetRequest; -import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.cluster.metadata.IndexMetadata; @@ -125,21 +125,18 @@ private void runTestAutomaticRefresh(final IntToLongFunction count) throws Inter started.await(); assertThat(count.applyAsLong(totalNumDocs.get()), equalTo(1L)); for (int i = 1; i < numDocs; i++) { - client().prepareIndex("test") - .setId("" + i) - .setSource("{\"foo\" : \"bar\"}", XContentType.JSON) - .execute(new ActionListener() { - @Override - public void onResponse(IndexResponse indexResponse) { - indexingDone.countDown(); - } - - @Override - public void onFailure(Exception e) { - indexingDone.countDown(); - throw new AssertionError(e); - } - }); + client().prepareIndex("test").setId("" + i).setSource("{\"foo\" : \"bar\"}", XContentType.JSON).execute(new ActionListener<>() { + @Override + public void onResponse(DocWriteResponse indexResponse) { + indexingDone.countDown(); + } + + @Override + public void onFailure(Exception e) { + indexingDone.countDown(); + throw new AssertionError(e); + } + }); } indexingDone.await(); t.join(); diff --git a/server/src/internalClusterTest/java/org/elasticsearch/indexing/IndexActionIT.java b/server/src/internalClusterTest/java/org/elasticsearch/indexing/IndexActionIT.java index eef2c21f432db..6c1b9a56f04cf 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/indexing/IndexActionIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/indexing/IndexActionIT.java @@ -103,7 +103,7 @@ public void testCreatedFlag() throws Exception { createIndex("test"); ensureGreen(); - IndexResponse indexResponse = client().prepareIndex("test").setId("1").setSource("field1", "value1_1").execute().actionGet(); + DocWriteResponse indexResponse = client().prepareIndex("test").setId("1").setSource("field1", "value1_1").execute().actionGet(); assertEquals(DocWriteResponse.Result.CREATED, indexResponse.getResult()); indexResponse = client().prepareIndex("test").setId("1").setSource("field1", "value1_2").execute().actionGet(); @@ -120,7 +120,7 @@ public void testCreatedFlagWithFlush() throws Exception { createIndex("test"); ensureGreen(); - IndexResponse indexResponse = client().prepareIndex("test").setId("1").setSource("field1", "value1_1").execute().actionGet(); + DocWriteResponse indexResponse = client().prepareIndex("test").setId("1").setSource("field1", "value1_1").execute().actionGet(); assertEquals(DocWriteResponse.Result.CREATED, indexResponse.getResult()); client().prepareDelete("test", "1").execute().actionGet(); @@ -148,7 +148,7 @@ public void testCreatedFlagParallelExecution() throws Exception { @Override public Void call() throws Exception { int docId = random.nextInt(docCount); - IndexResponse indexResponse = indexDoc("test", Integer.toString(docId), "field1", "value"); + DocWriteResponse indexResponse = indexDoc("test", Integer.toString(docId), "field1", "value"); if (indexResponse.getResult() == DocWriteResponse.Result.CREATED) { createdCounts.incrementAndGet(docId); } @@ -169,7 +169,7 @@ public void testCreatedFlagWithExternalVersioning() throws Exception { createIndex("test"); ensureGreen(); - IndexResponse indexResponse = client().prepareIndex("test") + DocWriteResponse indexResponse = client().prepareIndex("test") .setId("1") .setSource("field1", "value1_1") .setVersion(123) diff --git a/server/src/internalClusterTest/java/org/elasticsearch/indices/mapping/ConcurrentDynamicTemplateIT.java b/server/src/internalClusterTest/java/org/elasticsearch/indices/mapping/ConcurrentDynamicTemplateIT.java index 8649946308e86..64cd59fef36f1 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/indices/mapping/ConcurrentDynamicTemplateIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/indices/mapping/ConcurrentDynamicTemplateIT.java @@ -9,7 +9,7 @@ package org.elasticsearch.indices.mapping; import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.index.IndexResponse; +import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.core.Strings; import org.elasticsearch.index.query.QueryBuilders; import org.elasticsearch.test.ESIntegTestCase; @@ -61,9 +61,9 @@ public void testConcurrentDynamicMapping() throws Exception { client().prepareIndex("test") .setId(Integer.toString(currentID++)) .setSource(source) - .execute(new ActionListener() { + .execute(new ActionListener() { @Override - public void onResponse(IndexResponse response) { + public void onResponse(DocWriteResponse response) { latch.countDown(); } diff --git a/server/src/internalClusterTest/java/org/elasticsearch/indices/recovery/IndexPrimaryRelocationIT.java b/server/src/internalClusterTest/java/org/elasticsearch/indices/recovery/IndexPrimaryRelocationIT.java index b41b4727dc12b..08800587cab3c 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/indices/recovery/IndexPrimaryRelocationIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/indices/recovery/IndexPrimaryRelocationIT.java @@ -12,7 +12,6 @@ import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse; import org.elasticsearch.action.admin.cluster.node.hotthreads.NodeHotThreads; import org.elasticsearch.action.delete.DeleteResponse; -import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.routing.allocation.command.MoveAllocationCommand; @@ -41,7 +40,7 @@ public void testPrimaryRelocationWhileIndexing() throws Exception { @Override public void run() { while (finished.get() == false && numAutoGenDocs.get() < 10_000) { - IndexResponse indexResponse = client().prepareIndex("test").setId("id").setSource("field", "value").get(); + DocWriteResponse indexResponse = client().prepareIndex("test").setId("id").setSource("field", "value").get(); assertEquals(DocWriteResponse.Result.CREATED, indexResponse.getResult()); DeleteResponse deleteResponse = client().prepareDelete("test", "id").get(); assertEquals(DocWriteResponse.Result.DELETED, deleteResponse.getResult()); diff --git a/server/src/internalClusterTest/java/org/elasticsearch/indices/recovery/IndexRecoveryIT.java b/server/src/internalClusterTest/java/org/elasticsearch/indices/recovery/IndexRecoveryIT.java index 8ba0f1364bac8..fa07e8ffd9207 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/indices/recovery/IndexRecoveryIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/indices/recovery/IndexRecoveryIT.java @@ -25,6 +25,7 @@ import org.apache.lucene.util.SetOnce; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.admin.cluster.node.stats.NodeStats; import org.elasticsearch.action.admin.cluster.node.stats.NodesStatsResponse; import org.elasticsearch.action.admin.cluster.snapshots.create.CreateSnapshotResponse; @@ -35,7 +36,6 @@ import org.elasticsearch.action.admin.indices.stats.IndicesStatsResponse; import org.elasticsearch.action.admin.indices.stats.ShardStats; import org.elasticsearch.action.index.IndexRequestBuilder; -import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.support.ActiveShardCount; import org.elasticsearch.action.support.ChannelActionListener; import org.elasticsearch.action.support.PlainActionFuture; @@ -1506,7 +1506,7 @@ public void testPeerRecoveryTrimsLocalTranslog() throws Exception { indexers[i] = new Thread(() -> { while (stopped.get() == false) { try { - IndexResponse response = client().prepareIndex(indexName) + DocWriteResponse response = client().prepareIndex(indexName) .setSource(Map.of("f" + randomIntBetween(1, 10), randomNonNegativeLong()), XContentType.JSON) .get(); assertThat(response.getResult(), is(oneOf(CREATED, UPDATED))); diff --git a/server/src/internalClusterTest/java/org/elasticsearch/indices/state/OpenCloseIndexIT.java b/server/src/internalClusterTest/java/org/elasticsearch/indices/state/OpenCloseIndexIT.java index 2b628cacac01c..70ef73862016a 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/indices/state/OpenCloseIndexIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/indices/state/OpenCloseIndexIT.java @@ -9,11 +9,11 @@ package org.elasticsearch.indices.state; import org.elasticsearch.action.ActionRequestValidationException; +import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse; import org.elasticsearch.action.admin.indices.open.OpenIndexResponse; import org.elasticsearch.action.admin.indices.stats.IndicesStatsResponse; import org.elasticsearch.action.index.IndexRequestBuilder; -import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.action.support.master.AcknowledgedResponse; @@ -363,7 +363,7 @@ public void testTranslogStats() throws Exception { final int nbDocs = randomIntBetween(0, 50); int uncommittedOps = 0; for (long i = 0; i < nbDocs; i++) { - final IndexResponse indexResponse = client().prepareIndex(indexName).setId(Long.toString(i)).setSource("field", i).get(); + final DocWriteResponse indexResponse = client().prepareIndex(indexName).setId(Long.toString(i)).setSource("field", i).get(); assertThat(indexResponse.status(), is(RestStatus.CREATED)); if (rarely()) { diff --git a/server/src/internalClusterTest/java/org/elasticsearch/indices/stats/IndexStatsIT.java b/server/src/internalClusterTest/java/org/elasticsearch/indices/stats/IndexStatsIT.java index bd89812c2a185..53df5a1d3c834 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/indices/stats/IndexStatsIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/indices/stats/IndexStatsIT.java @@ -28,7 +28,6 @@ import org.elasticsearch.action.get.GetResponse; import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.index.IndexRequestBuilder; -import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.search.SearchType; import org.elasticsearch.action.support.DefaultShardOperationFailedException; import org.elasticsearch.action.support.WriteRequest; @@ -1320,7 +1319,7 @@ public void testConcurrentIndexingAndStatsRequests() throws BrokenBarrierExcepti } while (stop.get() == false) { final String id = Integer.toString(idGenerator.incrementAndGet()); - final IndexResponse response = client().prepareIndex("test").setId(id).setSource("{}", XContentType.JSON).get(); + final DocWriteResponse response = client().prepareIndex("test").setId(id).setSource("{}", XContentType.JSON).get(); assertThat(response.getResult(), equalTo(DocWriteResponse.Result.CREATED)); } }); @@ -1389,7 +1388,7 @@ public void testWriteLoadIsCaptured() throws Exception { final AtomicInteger idGenerator = new AtomicInteger(); assertBusy(() -> { final int numDocs = randomIntBetween(15, 25); - final List> indexRequestFutures = new ArrayList<>(numDocs); + final List> indexRequestFutures = new ArrayList<>(numDocs); for (int i = 0; i < numDocs; i++) { indexRequestFutures.add( client().prepareIndex(indexName) @@ -1399,7 +1398,7 @@ public void testWriteLoadIsCaptured() throws Exception { ); } - for (ActionFuture indexRequestFuture : indexRequestFutures) { + for (ActionFuture indexRequestFuture : indexRequestFutures) { assertThat(indexRequestFuture.get().getResult(), equalTo(DocWriteResponse.Result.CREATED)); } diff --git a/server/src/internalClusterTest/java/org/elasticsearch/recovery/RelocationIT.java b/server/src/internalClusterTest/java/org/elasticsearch/recovery/RelocationIT.java index ab852cf341514..936e9bf87c788 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/recovery/RelocationIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/recovery/RelocationIT.java @@ -11,11 +11,11 @@ import org.apache.lucene.index.IndexFileNames; import org.apache.lucene.tests.util.English; import org.elasticsearch.action.ActionFuture; +import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse; import org.elasticsearch.action.admin.cluster.reroute.ClusterRerouteResponse; import org.elasticsearch.action.admin.indices.stats.ShardStats; import org.elasticsearch.action.index.IndexRequestBuilder; -import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.support.WriteRequest; import org.elasticsearch.client.internal.Client; @@ -603,7 +603,7 @@ public void testRelocateWhileContinuouslyIndexingAndWaitingForRefresh() throws E logger.info("--> flush so we have an actual index"); indicesAdmin().prepareFlush().execute().actionGet(); logger.info("--> index more docs so we have something in the translog"); - final List> pendingIndexResponses = new ArrayList<>(); + final List> pendingIndexResponses = new ArrayList<>(); for (int i = 10; i < 20; i++) { pendingIndexResponses.add( client().prepareIndex("test") diff --git a/server/src/internalClusterTest/java/org/elasticsearch/search/basic/SearchWithRandomExceptionsIT.java b/server/src/internalClusterTest/java/org/elasticsearch/search/basic/SearchWithRandomExceptionsIT.java index dbfae5e9eea24..519e839c5d322 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/search/basic/SearchWithRandomExceptionsIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/search/basic/SearchWithRandomExceptionsIT.java @@ -15,7 +15,6 @@ import org.elasticsearch.ElasticsearchException; import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.admin.indices.refresh.RefreshResponse; -import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.search.SearchPhaseExecutionException; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.common.Strings; @@ -100,7 +99,7 @@ public void testRandomExceptions() throws IOException, InterruptedException, Exe boolean[] added = new boolean[numDocs]; for (int i = 0; i < numDocs; i++) { try { - IndexResponse indexResponse = client().prepareIndex("test") + DocWriteResponse indexResponse = client().prepareIndex("test") .setId("" + i) .setTimeout(TimeValue.timeValueSeconds(1)) .setSource("test", English.intToEnglish(i)) diff --git a/server/src/internalClusterTest/java/org/elasticsearch/search/basic/SearchWithRandomIOExceptionsIT.java b/server/src/internalClusterTest/java/org/elasticsearch/search/basic/SearchWithRandomIOExceptionsIT.java index e829c6da01b26..dba26d0560a14 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/search/basic/SearchWithRandomIOExceptionsIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/search/basic/SearchWithRandomIOExceptionsIT.java @@ -14,7 +14,6 @@ import org.elasticsearch.action.admin.cluster.health.ClusterHealthRequest; import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse; import org.elasticsearch.action.admin.indices.refresh.RefreshResponse; -import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.search.SearchPhaseExecutionException; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.common.Strings; @@ -124,7 +123,7 @@ public void testRandomDirectoryIOExceptions() throws IOException, InterruptedExc for (int i = 0; i < numDocs; i++) { added[i] = false; try { - IndexResponse indexResponse = client().prepareIndex("test") + DocWriteResponse indexResponse = client().prepareIndex("test") .setId(Integer.toString(i)) .setTimeout(TimeValue.timeValueSeconds(1)) .setSource("test", English.intToEnglish(i)) diff --git a/server/src/internalClusterTest/java/org/elasticsearch/search/nested/SimpleNestedIT.java b/server/src/internalClusterTest/java/org/elasticsearch/search/nested/SimpleNestedIT.java index 504f5265b4ba1..598c65b8c999d 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/search/nested/SimpleNestedIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/search/nested/SimpleNestedIT.java @@ -15,7 +15,6 @@ import org.elasticsearch.action.admin.indices.stats.IndicesStatsResponse; import org.elasticsearch.action.delete.DeleteResponse; import org.elasticsearch.action.get.GetResponse; -import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.search.SearchRequestBuilder; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchType; @@ -1415,7 +1414,7 @@ public void testNestedSortingWithNestedFilterAsFilter() throws Exception { ) ); - IndexResponse indexResponse1 = client().prepareIndex("test") + DocWriteResponse indexResponse1 = client().prepareIndex("test") .setId("1") .setSource( jsonBuilder().startObject() @@ -1469,7 +1468,7 @@ public void testNestedSortingWithNestedFilterAsFilter() throws Exception { .get(); assertTrue(indexResponse1.getShardInfo().getSuccessful() > 0); - IndexResponse indexResponse2 = client().prepareIndex("test") + DocWriteResponse indexResponse2 = client().prepareIndex("test") .setId("2") .setSource( jsonBuilder().startObject() diff --git a/server/src/internalClusterTest/java/org/elasticsearch/search/suggest/ContextCompletionSuggestSearchIT.java b/server/src/internalClusterTest/java/org/elasticsearch/search/suggest/ContextCompletionSuggestSearchIT.java index 1ce873a133f66..7b2ba7cb46770 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/search/suggest/ContextCompletionSuggestSearchIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/search/suggest/ContextCompletionSuggestSearchIT.java @@ -10,8 +10,8 @@ import com.carrotsearch.randomizedtesting.generators.RandomStrings; import org.apache.lucene.tests.util.LuceneTestCase.SuppressCodecs; +import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.index.IndexRequestBuilder; -import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.common.geo.GeoPoint; import org.elasticsearch.common.settings.Settings; @@ -168,7 +168,7 @@ public void testContextFilteringWorksWithUTF8Categories() throws Exception { LinkedHashMap> map = new LinkedHashMap<>(Collections.singletonMap("cat", contextMapping)); final CompletionMappingBuilder mapping = new CompletionMappingBuilder().context(map); createIndexAndMapping(mapping); - IndexResponse indexResponse = client().prepareIndex(INDEX) + DocWriteResponse indexResponse = client().prepareIndex(INDEX) .setId("1") .setSource( jsonBuilder().startObject() diff --git a/server/src/internalClusterTest/java/org/elasticsearch/timeseries/support/TimeSeriesDimensionsLimitIT.java b/server/src/internalClusterTest/java/org/elasticsearch/timeseries/support/TimeSeriesDimensionsLimitIT.java index 379539c3130c5..b3cb2e5f178ca 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/timeseries/support/TimeSeriesDimensionsLimitIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/timeseries/support/TimeSeriesDimensionsLimitIT.java @@ -8,7 +8,7 @@ package org.elasticsearch.timeseries.support; -import org.elasticsearch.action.index.IndexResponse; +import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.cluster.metadata.IndexMetadata; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.core.CheckedConsumer; @@ -141,7 +141,7 @@ public void testTotalDimensionFieldsSizeLuceneLimit() throws IOException { for (int i = 0; i < dimensionFieldLimit; i++) { source.put(dimensionFieldNames.get(i), randomAlphaOfLength(1024)); } - final IndexResponse indexResponse = client().prepareIndex("test").setSource(source).get(); + final DocWriteResponse indexResponse = client().prepareIndex("test").setSource(source).get(); assertEquals(RestStatus.CREATED.getStatus(), indexResponse.status().getStatus()); } diff --git a/server/src/internalClusterTest/java/org/elasticsearch/update/UpdateIT.java b/server/src/internalClusterTest/java/org/elasticsearch/update/UpdateIT.java index 98fe88c3b5327..4e97560284c67 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/update/UpdateIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/update/UpdateIT.java @@ -17,7 +17,6 @@ import org.elasticsearch.action.delete.DeleteRequest; import org.elasticsearch.action.delete.DeleteResponse; import org.elasticsearch.action.get.GetResponse; -import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.update.UpdateRequest; import org.elasticsearch.action.update.UpdateRequestBuilder; import org.elasticsearch.action.update.UpdateResponse; @@ -452,7 +451,7 @@ public void testUpdateWithIfSeqNo() throws Exception { createTestIndex(); ensureGreen(); - IndexResponse result = client().prepareIndex("test").setId("1").setSource("field", 1).get(); + DocWriteResponse result = client().prepareIndex("test").setId("1").setSource("field", 1).get(); expectThrows( VersionConflictEngineException.class, () -> client().prepareUpdate(indexOrAlias(), "1") diff --git a/server/src/internalClusterTest/java/org/elasticsearch/versioning/ConcurrentDocumentOperationIT.java b/server/src/internalClusterTest/java/org/elasticsearch/versioning/ConcurrentDocumentOperationIT.java index 9d84a1c4727be..b191eb0cf4fe3 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/versioning/ConcurrentDocumentOperationIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/versioning/ConcurrentDocumentOperationIT.java @@ -9,7 +9,7 @@ package org.elasticsearch.versioning; import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.index.IndexResponse; +import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.test.ESIntegTestCase; @@ -31,9 +31,9 @@ public void testConcurrentOperationOnSameDoc() throws Exception { final AtomicReference failure = new AtomicReference<>(); final CountDownLatch latch = new CountDownLatch(numberOfUpdates); for (int i = 0; i < numberOfUpdates; i++) { - client().prepareIndex("test").setId("1").setSource("field1", i).execute(new ActionListener() { + client().prepareIndex("test").setId("1").setSource("field1", i).execute(new ActionListener<>() { @Override - public void onResponse(IndexResponse response) { + public void onResponse(DocWriteResponse response) { latch.countDown(); } diff --git a/server/src/internalClusterTest/java/org/elasticsearch/versioning/ConcurrentSeqNoVersioningIT.java b/server/src/internalClusterTest/java/org/elasticsearch/versioning/ConcurrentSeqNoVersioningIT.java index 08cc5fa784fc1..7d9e1f3955089 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/versioning/ConcurrentSeqNoVersioningIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/versioning/ConcurrentSeqNoVersioningIT.java @@ -8,8 +8,8 @@ package org.elasticsearch.versioning; import org.elasticsearch.ExceptionsHelper; +import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.index.IndexRequest; -import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.cluster.coordination.LinearizabilityChecker; import org.elasticsearch.cluster.metadata.IndexMetadata; import org.elasticsearch.common.bytes.BytesReference; @@ -237,7 +237,7 @@ public void run() { Consumer historyResponse = partition.invoke(version); try { // we should be able to remove timeout or fail hard on timeouts - IndexResponse indexResponse = client().index(indexRequest).actionGet(timeout, TimeUnit.SECONDS); + DocWriteResponse indexResponse = client().index(indexRequest).actionGet(timeout, TimeUnit.SECONDS); IndexResponseHistoryOutput historyOutput = new IndexResponseHistoryOutput(indexResponse); historyResponse.accept(historyOutput); // validate version and seqNo strictly increasing for successful CAS to avoid that overhead during @@ -515,7 +515,7 @@ private interface HistoryOutput extends NamedWriteable { private static class IndexResponseHistoryOutput implements HistoryOutput { private final Version outputVersion; - private IndexResponseHistoryOutput(IndexResponse response) { + private IndexResponseHistoryOutput(DocWriteResponse response) { this(new Version(response.getPrimaryTerm(), response.getSeqNo())); } diff --git a/server/src/internalClusterTest/java/org/elasticsearch/versioning/SimpleVersioningIT.java b/server/src/internalClusterTest/java/org/elasticsearch/versioning/SimpleVersioningIT.java index c4a56770d186e..c6fbdc909e2e6 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/versioning/SimpleVersioningIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/versioning/SimpleVersioningIT.java @@ -65,7 +65,7 @@ public void testExternalVersioningInitialDelete() throws Exception { VersionConflictEngineException.class ); - IndexResponse indexResponse = client().prepareIndex("test") + DocWriteResponse indexResponse = client().prepareIndex("test") .setId("1") .setSource("field1", "value1_1") .setVersion(18) @@ -78,7 +78,7 @@ public void testExternalVersioningInitialDelete() throws Exception { public void testExternalGTE() throws Exception { createIndex("test"); - IndexResponse indexResponse = client().prepareIndex("test") + DocWriteResponse indexResponse = client().prepareIndex("test") .setId("1") .setSource("field1", "value1_1") .setVersion(12) @@ -151,7 +151,7 @@ public void testExternalVersioning() throws Exception { createIndex("test"); ensureGreen(); - IndexResponse indexResponse = client().prepareIndex("test") + DocWriteResponse indexResponse = client().prepareIndex("test") .setId("1") .setSource("field1", "value1_1") .setVersion(12) @@ -268,7 +268,7 @@ public void testCompareAndSetInitialDelete() throws Exception { VersionConflictEngineException.class ); - IndexResponse indexResponse = client().prepareIndex("test") + DocWriteResponse indexResponse = client().prepareIndex("test") .setId("1") .setSource("field1", "value1_1") .setCreate(true) @@ -281,7 +281,7 @@ public void testCompareAndSet() { createIndex("test"); ensureGreen(); - IndexResponse indexResponse = client().prepareIndex("test").setId("1").setSource("field1", "value1_1").execute().actionGet(); + DocWriteResponse indexResponse = client().prepareIndex("test").setId("1").setSource("field1", "value1_1").execute().actionGet(); assertThat(indexResponse.getSeqNo(), equalTo(0L)); assertThat(indexResponse.getPrimaryTerm(), equalTo(1L)); @@ -366,7 +366,7 @@ public void testSimpleVersioningWithFlush() throws Exception { createIndex("test"); ensureGreen(); - IndexResponse indexResponse = client().prepareIndex("test").setId("1").setSource("field1", "value1_1").get(); + DocWriteResponse indexResponse = client().prepareIndex("test").setId("1").setSource("field1", "value1_1").get(); assertThat(indexResponse.getSeqNo(), equalTo(0L)); client().admin().indices().prepareFlush().execute().actionGet(); @@ -834,7 +834,7 @@ public void testGCDeletesZero() throws Exception { public void testSpecialVersioning() { internalCluster().ensureAtLeastNumDataNodes(2); createIndex("test", Settings.builder().put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0).build()); - IndexResponse doc1 = client().prepareIndex("test") + DocWriteResponse doc1 = client().prepareIndex("test") .setId("1") .setSource("field", "value1") .setVersion(0) @@ -842,7 +842,7 @@ public void testSpecialVersioning() { .execute() .actionGet(); assertThat(doc1.getVersion(), equalTo(0L)); - IndexResponse doc2 = client().prepareIndex("test") + DocWriteResponse doc2 = client().prepareIndex("test") .setId("1") .setSource("field", "value2") .setVersion(Versions.MATCH_ANY) @@ -851,7 +851,7 @@ public void testSpecialVersioning() { .actionGet(); assertThat(doc2.getVersion(), equalTo(1L)); client().prepareDelete("test", "1").get(); // v2 - IndexResponse doc3 = client().prepareIndex("test") + DocWriteResponse doc3 = client().prepareIndex("test") .setId("1") .setSource("field", "value3") .setVersion(Versions.MATCH_DELETED) @@ -859,7 +859,7 @@ public void testSpecialVersioning() { .execute() .actionGet(); assertThat(doc3.getVersion(), equalTo(3L)); - IndexResponse doc4 = client().prepareIndex("test") + DocWriteResponse doc4 = client().prepareIndex("test") .setId("1") .setSource("field", "value4") .setVersion(4L) diff --git a/server/src/main/java/org/elasticsearch/action/index/IndexAction.java b/server/src/main/java/org/elasticsearch/action/index/IndexAction.java index 9ac379934c358..4553d2c9039b8 100644 --- a/server/src/main/java/org/elasticsearch/action/index/IndexAction.java +++ b/server/src/main/java/org/elasticsearch/action/index/IndexAction.java @@ -9,13 +9,14 @@ package org.elasticsearch.action.index; import org.elasticsearch.action.ActionType; +import org.elasticsearch.action.DocWriteResponse; -public class IndexAction extends ActionType { +public class IndexAction extends ActionType { public static final IndexAction INSTANCE = new IndexAction(); public static final String NAME = "indices:data/write/index"; private IndexAction() { - super(NAME, IndexResponse::new); + super(NAME, in -> { throw new UnsupportedOperationException(); }); } } diff --git a/server/src/main/java/org/elasticsearch/action/index/IndexRequestBuilder.java b/server/src/main/java/org/elasticsearch/action/index/IndexRequestBuilder.java index 3a190c4e314c0..360b470eb1ab4 100644 --- a/server/src/main/java/org/elasticsearch/action/index/IndexRequestBuilder.java +++ b/server/src/main/java/org/elasticsearch/action/index/IndexRequestBuilder.java @@ -9,6 +9,7 @@ package org.elasticsearch.action.index; import org.elasticsearch.action.DocWriteRequest; +import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.support.WriteRequestBuilder; import org.elasticsearch.action.support.replication.ReplicationRequestBuilder; import org.elasticsearch.client.internal.ElasticsearchClient; @@ -23,7 +24,7 @@ /** * An index document action request builder. */ -public class IndexRequestBuilder extends ReplicationRequestBuilder +public class IndexRequestBuilder extends ReplicationRequestBuilder implements WriteRequestBuilder { @@ -84,7 +85,7 @@ public IndexRequestBuilder setSource(Map source, XContentType content /** * Sets the document source to index. *

- * Note, its preferable to either set it using {@link #setSource(org.elasticsearch.common.xcontent.XContentBuilder)} + * Note, its preferable to either set it using {@link #setSource(org.elasticsearch.xcontent.XContentBuilder)} * or using the {@link #setSource(byte[], XContentType)}. */ public IndexRequestBuilder setSource(String source, XContentType xContentType) { diff --git a/server/src/main/java/org/elasticsearch/action/index/TransportIndexAction.java b/server/src/main/java/org/elasticsearch/action/index/TransportIndexAction.java index e837f59ce3ff3..6c75374d51012 100644 --- a/server/src/main/java/org/elasticsearch/action/index/TransportIndexAction.java +++ b/server/src/main/java/org/elasticsearch/action/index/TransportIndexAction.java @@ -8,6 +8,7 @@ package org.elasticsearch.action.index; +import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.bulk.TransportBulkAction; import org.elasticsearch.action.bulk.TransportSingleItemBulkWriteAction; import org.elasticsearch.action.support.ActionFilters; @@ -27,7 +28,7 @@ * Deprecated use TransportBulkAction with a single item instead */ @Deprecated -public class TransportIndexAction extends TransportSingleItemBulkWriteAction { +public class TransportIndexAction extends TransportSingleItemBulkWriteAction { @Inject public TransportIndexAction(ActionFilters actionFilters, TransportService transportService, TransportBulkAction bulkAction) { diff --git a/server/src/main/java/org/elasticsearch/client/internal/Client.java b/server/src/main/java/org/elasticsearch/client/internal/Client.java index 925c5af303470..1065efb857fe7 100644 --- a/server/src/main/java/org/elasticsearch/client/internal/Client.java +++ b/server/src/main/java/org/elasticsearch/client/internal/Client.java @@ -10,6 +10,7 @@ import org.elasticsearch.action.ActionFuture; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.bulk.BulkRequest; import org.elasticsearch.action.bulk.BulkRequestBuilder; import org.elasticsearch.action.bulk.BulkResponse; @@ -30,7 +31,6 @@ import org.elasticsearch.action.get.MultiGetResponse; import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.index.IndexRequestBuilder; -import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.search.ClearScrollRequest; import org.elasticsearch.action.search.ClearScrollRequestBuilder; import org.elasticsearch.action.search.ClearScrollResponse; @@ -93,7 +93,7 @@ public interface Client extends ElasticsearchClient, Releasable { * @param request The index request * @return The result future */ - ActionFuture index(IndexRequest request); + ActionFuture index(IndexRequest request); /** * Index a document associated with a given index. @@ -103,7 +103,7 @@ public interface Client extends ElasticsearchClient, Releasable { * @param request The index request * @param listener A listener to be notified with a result */ - void index(IndexRequest request, ActionListener listener); + void index(IndexRequest request, ActionListener listener); /** * Index a document associated with a given index. diff --git a/server/src/main/java/org/elasticsearch/client/internal/support/AbstractClient.java b/server/src/main/java/org/elasticsearch/client/internal/support/AbstractClient.java index 7a98c65d51247..5ac1ae6a76611 100644 --- a/server/src/main/java/org/elasticsearch/client/internal/support/AbstractClient.java +++ b/server/src/main/java/org/elasticsearch/client/internal/support/AbstractClient.java @@ -15,6 +15,7 @@ import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.ActionResponse; import org.elasticsearch.action.ActionType; +import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.admin.cluster.allocation.ClusterAllocationExplainAction; import org.elasticsearch.action.admin.cluster.allocation.ClusterAllocationExplainRequest; import org.elasticsearch.action.admin.cluster.allocation.ClusterAllocationExplainRequestBuilder; @@ -265,7 +266,6 @@ import org.elasticsearch.action.index.IndexAction; import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.index.IndexRequestBuilder; -import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.ingest.DeletePipelineAction; import org.elasticsearch.action.ingest.DeletePipelineRequest; import org.elasticsearch.action.ingest.DeletePipelineRequestBuilder; @@ -390,12 +390,12 @@ protected abstract index(final IndexRequest request) { + public ActionFuture index(final IndexRequest request) { return execute(IndexAction.INSTANCE, request); } @Override - public void index(final IndexRequest request, final ActionListener listener) { + public void index(final IndexRequest request, final ActionListener listener) { execute(IndexAction.INSTANCE, request, listener); } diff --git a/server/src/main/java/org/elasticsearch/tasks/TaskResultsService.java b/server/src/main/java/org/elasticsearch/tasks/TaskResultsService.java index b0da930cd17b6..a72ef9e83ccf2 100644 --- a/server/src/main/java/org/elasticsearch/tasks/TaskResultsService.java +++ b/server/src/main/java/org/elasticsearch/tasks/TaskResultsService.java @@ -12,9 +12,9 @@ import org.elasticsearch.ElasticsearchException; import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.bulk.BackoffPolicy; import org.elasticsearch.action.index.IndexRequestBuilder; -import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.client.internal.Client; import org.elasticsearch.client.internal.OriginSettingClient; import org.elasticsearch.client.internal.Requests; @@ -91,9 +91,9 @@ public void storeResult(TaskResult taskResult, ActionListener listener) { } private void doStoreResult(Iterator backoff, IndexRequestBuilder index, ActionListener listener) { - index.execute(new ActionListener() { + index.execute(new ActionListener() { @Override - public void onResponse(IndexResponse indexResponse) { + public void onResponse(DocWriteResponse indexResponse) { listener.onResponse(null); } diff --git a/server/src/test/java/org/elasticsearch/action/admin/indices/stats/IndicesStatsTests.java b/server/src/test/java/org/elasticsearch/action/admin/indices/stats/IndicesStatsTests.java index 31e973a9e9268..de28a7c7bd99d 100644 --- a/server/src/test/java/org/elasticsearch/action/admin/indices/stats/IndicesStatsTests.java +++ b/server/src/test/java/org/elasticsearch/action/admin/indices/stats/IndicesStatsTests.java @@ -9,8 +9,8 @@ package org.elasticsearch.action.admin.indices.stats; import org.elasticsearch.action.ActionFuture; +import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.admin.indices.close.CloseIndexRequest; -import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.support.DefaultShardOperationFailedException; import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.action.support.WriteRequest.RefreshPolicy; @@ -107,7 +107,7 @@ public void testRefreshListeners() throws Exception { createIndex("test", Settings.builder().put("refresh_interval", -1).build()); // Index a document asynchronously so the request will only return when document is refreshed - ActionFuture index = client().prepareIndex("test") + ActionFuture index = client().prepareIndex("test") .setId("test") .setSource("test", "test") .setRefreshPolicy(RefreshPolicy.WAIT_UNTIL) diff --git a/server/src/test/java/org/elasticsearch/index/fieldstats/FieldStatsProviderRefreshTests.java b/server/src/test/java/org/elasticsearch/index/fieldstats/FieldStatsProviderRefreshTests.java index 7e01a3d714a24..5c35733feedef 100644 --- a/server/src/test/java/org/elasticsearch/index/fieldstats/FieldStatsProviderRefreshTests.java +++ b/server/src/test/java/org/elasticsearch/index/fieldstats/FieldStatsProviderRefreshTests.java @@ -8,8 +8,8 @@ package org.elasticsearch.index.fieldstats; +import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.admin.indices.refresh.RefreshResponse; -import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchType; import org.elasticsearch.index.query.QueryBuilders; @@ -95,7 +95,7 @@ private void refreshIndex() { } private void indexDocument(String id, String sValue) { - IndexResponse response = client().prepareIndex("index").setId(id).setSource("s", sValue).get(); + DocWriteResponse response = client().prepareIndex("index").setId(id).setSource("s", sValue).get(); assertThat(response.status(), anyOf(equalTo(RestStatus.OK), equalTo(RestStatus.CREATED))); } } diff --git a/server/src/test/java/org/elasticsearch/search/SearchServiceTests.java b/server/src/test/java/org/elasticsearch/search/SearchServiceTests.java index 868832f61ef9c..279944c1b8347 100644 --- a/server/src/test/java/org/elasticsearch/search/SearchServiceTests.java +++ b/server/src/test/java/org/elasticsearch/search/SearchServiceTests.java @@ -21,9 +21,9 @@ import org.elasticsearch.TransportVersion; import org.elasticsearch.TransportVersions; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.OriginalIndices; import org.elasticsearch.action.admin.cluster.settings.ClusterUpdateSettingsResponse; -import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.search.ClearScrollRequest; import org.elasticsearch.action.search.ClosePointInTimeAction; import org.elasticsearch.action.search.ClosePointInTimeRequest; @@ -338,9 +338,9 @@ public void run() { client().prepareIndex("index") .setSource("field", "value") .setRefreshPolicy(randomFrom(WriteRequest.RefreshPolicy.values())) - .execute(new ActionListener() { + .execute(new ActionListener() { @Override - public void onResponse(IndexResponse indexResponse) { + public void onResponse(DocWriteResponse indexResponse) { semaphore.release(); } @@ -1012,7 +1012,7 @@ public void testCanMatch() throws Exception { ).canMatch() ); // the source can match and can be rewritten to a match_none, but not the alias filter - final IndexResponse response = client().prepareIndex("index").setSource("id", "1").get(); + final DocWriteResponse response = client().prepareIndex("index").setSource("id", "1").get(); assertEquals(RestStatus.CREATED, response.status()); searchRequest.indices("alias").source(new SearchSourceBuilder().query(new TermQueryBuilder("id", "1"))); assertFalse( @@ -1704,7 +1704,7 @@ public void testWaitOnRefresh() { searchRequest.setWaitForCheckpointsTimeout(TimeValue.timeValueSeconds(30)); searchRequest.setWaitForCheckpoints(Collections.singletonMap("index", new long[] { 0 })); - final IndexResponse response = client().prepareIndex("index").setSource("id", "1").get(); + final DocWriteResponse response = client().prepareIndex("index").setSource("id", "1").get(); assertEquals(RestStatus.CREATED, response.status()); SearchShardTask task = new SearchShardTask(123L, "", "", "", null, Collections.emptyMap()); @@ -1737,7 +1737,7 @@ public void testWaitOnRefreshFailsWithRefreshesDisabled() { searchRequest.setWaitForCheckpointsTimeout(TimeValue.timeValueSeconds(30)); searchRequest.setWaitForCheckpoints(Collections.singletonMap("index", new long[] { 0 })); - final IndexResponse response = client().prepareIndex("index").setSource("id", "1").get(); + final DocWriteResponse response = client().prepareIndex("index").setSource("id", "1").get(); assertEquals(RestStatus.CREATED, response.status()); SearchShardTask task = new SearchShardTask(123L, "", "", "", null, Collections.emptyMap()); @@ -1773,7 +1773,7 @@ public void testWaitOnRefreshFailsIfCheckpointNotIndexed() { searchRequest.setWaitForCheckpointsTimeout(TimeValue.timeValueMillis(randomIntBetween(10, 100))); searchRequest.setWaitForCheckpoints(Collections.singletonMap("index", new long[] { 1 })); - final IndexResponse response = client().prepareIndex("index").setSource("id", "1").get(); + final DocWriteResponse response = client().prepareIndex("index").setSource("id", "1").get(); assertEquals(RestStatus.CREATED, response.status()); SearchShardTask task = new SearchShardTask(123L, "", "", "", null, Collections.emptyMap()); @@ -1810,7 +1810,7 @@ public void testWaitOnRefreshTimeout() { searchRequest.setWaitForCheckpointsTimeout(TimeValue.timeValueMillis(randomIntBetween(10, 100))); searchRequest.setWaitForCheckpoints(Collections.singletonMap("index", new long[] { 0 })); - final IndexResponse response = client().prepareIndex("index").setSource("id", "1").get(); + final DocWriteResponse response = client().prepareIndex("index").setSource("id", "1").get(); assertEquals(RestStatus.CREATED, response.status()); SearchShardTask task = new SearchShardTask(123L, "", "", "", null, Collections.emptyMap()); diff --git a/test/framework/src/main/java/org/elasticsearch/test/BackgroundIndexer.java b/test/framework/src/main/java/org/elasticsearch/test/BackgroundIndexer.java index 9829e40088829..d23b79ed0cde2 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/BackgroundIndexer.java +++ b/test/framework/src/main/java/org/elasticsearch/test/BackgroundIndexer.java @@ -14,11 +14,11 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; +import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.bulk.BulkItemResponse; import org.elasticsearch.action.bulk.BulkRequestBuilder; import org.elasticsearch.action.bulk.BulkResponse; import org.elasticsearch.action.bulk.BulkShardRequest; -import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.client.internal.Client; import org.elasticsearch.common.util.concurrent.ConcurrentCollections; import org.elasticsearch.core.TimeValue; @@ -174,7 +174,7 @@ public void run() { id = idGenerator.incrementAndGet(); if (useAutoGeneratedIDs) { try { - IndexResponse indexResponse = client.prepareIndex(index) + DocWriteResponse indexResponse = client.prepareIndex(index) .setTimeout(timeout) .setSource(generateSource(id, threadRandom)) .get(); @@ -187,7 +187,7 @@ public void run() { } } else { try { - IndexResponse indexResponse = client.prepareIndex(index) + DocWriteResponse indexResponse = client.prepareIndex(index) .setId(Long.toString(id)) .setTimeout(timeout) .setSource(generateSource(id, threadRandom)) diff --git a/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java index 7711164eebf75..4956985c78a97 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java @@ -47,7 +47,6 @@ import org.elasticsearch.action.bulk.BulkRequestBuilder; import org.elasticsearch.action.bulk.BulkResponse; import org.elasticsearch.action.index.IndexRequestBuilder; -import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.search.ClearScrollResponse; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchResponse; @@ -1393,7 +1392,7 @@ protected void ensureFullyConnectedCluster() { * client().prepareIndex(index).setSource(source).execute().actionGet(); * */ - protected final IndexResponse index(String index, XContentBuilder source) { + protected final DocWriteResponse index(String index, XContentBuilder source) { return client().prepareIndex(index).setSource(source).execute().actionGet(); } @@ -1403,11 +1402,11 @@ protected final IndexResponse index(String index, XContentBuilder source) { * client().prepareIndex(index).setSource(source).execute().actionGet(); * */ - protected final IndexResponse index(String index, String id, Map source) { + protected final DocWriteResponse index(String index, String id, Map source) { return client().prepareIndex(index).setId(id).setSource(source).execute().actionGet(); } - protected final ActionFuture startIndex(String index, String id, BytesReference source, XContentType type) { + protected final ActionFuture startIndex(String index, String id, BytesReference source, XContentType type) { return client().prepareIndex(index).setId(id).setSource(source, type).execute(); } @@ -1417,7 +1416,7 @@ protected final ActionFuture startIndex(String index, String id, * return client().prepareIndex(index).setId(id).setSource(source).execute().actionGet(); * */ - protected final IndexResponse index(String index, String id, XContentBuilder source) { + protected final DocWriteResponse index(String index, String id, XContentBuilder source) { return client().prepareIndex(index).setId(id).setSource(source).execute().actionGet(); } @@ -1427,7 +1426,7 @@ protected final IndexResponse index(String index, String id, XContentBuilder sou * return client().prepareIndex(index).setId(id).setSource(source).execute().actionGet(); * */ - protected final IndexResponse indexDoc(String index, String id, Object... source) { + protected final DocWriteResponse indexDoc(String index, String id, Object... source) { return client().prepareIndex(index).setId(id).setSource(source).execute().actionGet(); } @@ -1439,7 +1438,7 @@ protected final IndexResponse indexDoc(String index, String id, Object... source *

* where source is a JSON String. */ - protected final IndexResponse index(String index, String id, String source) { + protected final DocWriteResponse index(String index, String id, String source) { return client().prepareIndex(index).setId(id).setSource(source, XContentType.JSON).execute().actionGet(); } diff --git a/x-pack/plugin/async-search/src/main/java/org/elasticsearch/xpack/search/TransportSubmitAsyncSearchAction.java b/x-pack/plugin/async-search/src/main/java/org/elasticsearch/xpack/search/TransportSubmitAsyncSearchAction.java index 60f477542dae5..0c3c339adb8bd 100644 --- a/x-pack/plugin/async-search/src/main/java/org/elasticsearch/xpack/search/TransportSubmitAsyncSearchAction.java +++ b/x-pack/plugin/async-search/src/main/java/org/elasticsearch/xpack/search/TransportSubmitAsyncSearchAction.java @@ -7,7 +7,7 @@ package org.elasticsearch.xpack.search; import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.index.IndexResponse; +import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.search.SearchAction; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.TransportSearchAction; @@ -102,7 +102,7 @@ public void onResponse(AsyncSearchResponse searchResponse) { AsyncSearchResponse initialResp = searchResponse.clone(searchResponse.getId()); store.createResponse(docId, searchTask.getOriginHeaders(), initialResp, new ActionListener<>() { @Override - public void onResponse(IndexResponse r) { + public void onResponse(DocWriteResponse r) { if (searchResponse.isRunning()) { try { // store the final response on completion unless the submit is cancelled diff --git a/x-pack/plugin/async-search/src/test/java/org/elasticsearch/xpack/search/AsyncSearchSingleNodeTests.java b/x-pack/plugin/async-search/src/test/java/org/elasticsearch/xpack/search/AsyncSearchSingleNodeTests.java index e2157345e025f..dae6e1e980eb4 100644 --- a/x-pack/plugin/async-search/src/test/java/org/elasticsearch/xpack/search/AsyncSearchSingleNodeTests.java +++ b/x-pack/plugin/async-search/src/test/java/org/elasticsearch/xpack/search/AsyncSearchSingleNodeTests.java @@ -9,9 +9,9 @@ import org.apache.lucene.index.LeafReaderContext; import org.elasticsearch.ElasticsearchStatusException; +import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.admin.indices.refresh.RefreshRequest; import org.elasticsearch.action.index.IndexRequest; -import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.search.SearchPhaseExecutionException; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.ShardSearchFailure; @@ -44,7 +44,7 @@ protected Collection> getPlugins() { public void testFetchFailuresAllShards() throws Exception { for (int i = 0; i < 10; i++) { - IndexResponse indexResponse = client().index(new IndexRequest("boom" + i).id("boom" + i).source("text", "value")).get(); + DocWriteResponse indexResponse = client().index(new IndexRequest("boom" + i).id("boom" + i).source("text", "value")).get(); assertEquals(RestStatus.CREATED, indexResponse.status()); } client().admin().indices().refresh(new RefreshRequest()).get(); @@ -83,11 +83,11 @@ public void testFetchFailuresAllShards() throws Exception { public void testFetchFailuresOnlySomeShards() throws Exception { for (int i = 0; i < 5; i++) { - IndexResponse indexResponse = client().index(new IndexRequest("boom" + i).id("boom" + i).source("text", "value")).get(); + DocWriteResponse indexResponse = client().index(new IndexRequest("boom" + i).id("boom" + i).source("text", "value")).get(); assertEquals(RestStatus.CREATED, indexResponse.status()); } for (int i = 0; i < 5; i++) { - IndexResponse indexResponse = client().index(new IndexRequest("index" + i).id("index" + i).source("text", "value")).get(); + DocWriteResponse indexResponse = client().index(new IndexRequest("index" + i).id("index" + i).source("text", "value")).get(); assertEquals(RestStatus.CREATED, indexResponse.status()); } client().admin().indices().refresh(new RefreshRequest()).get(); diff --git a/x-pack/plugin/ccr/src/internalClusterTest/java/org/elasticsearch/xpack/ccr/FollowerFailOverIT.java b/x-pack/plugin/ccr/src/internalClusterTest/java/org/elasticsearch/xpack/ccr/FollowerFailOverIT.java index ad66e34cd371e..845997872ed8d 100644 --- a/x-pack/plugin/ccr/src/internalClusterTest/java/org/elasticsearch/xpack/ccr/FollowerFailOverIT.java +++ b/x-pack/plugin/ccr/src/internalClusterTest/java/org/elasticsearch/xpack/ccr/FollowerFailOverIT.java @@ -9,7 +9,6 @@ import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.delete.DeleteResponse; -import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.metadata.IndexMetadata; import org.elasticsearch.cluster.node.DiscoveryNode; @@ -75,7 +74,7 @@ public void testFailOverOnFollower() throws Exception { } if (frequently()) { String id = Integer.toString(frequently() ? docID.incrementAndGet() : between(0, 10)); // sometimes update - IndexResponse indexResponse = leaderClient().prepareIndex(leaderIndex) + DocWriteResponse indexResponse = leaderClient().prepareIndex(leaderIndex) .setId(id) .setSource("{\"f\":" + id + "}", XContentType.JSON) .get(); @@ -142,7 +141,7 @@ public void testFollowIndexAndCloseNode() throws Exception { } Object[] args = new Object[] { counter++ }; final String source = Strings.format("{\"f\":%d}", args); - IndexResponse indexResp = leaderClient().prepareIndex("index1") + DocWriteResponse indexResp = leaderClient().prepareIndex("index1") .setSource(source, XContentType.JSON) .setTimeout(TimeValue.timeValueSeconds(1)) .get(); @@ -291,7 +290,7 @@ public void testReadRequestsReturnLatestMappingVersion() throws Exception { assertNotNull(mapper); assertNotNull(mapper.mappers().getMapper("balance")); }); - IndexResponse indexResp = leaderCluster.client() + DocWriteResponse indexResp = leaderCluster.client() .prepareIndex("leader-index") .setId("1") .setSource("{\"balance\": 100}", XContentType.JSON) diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/async/AsyncTaskIndexService.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/async/AsyncTaskIndexService.java index e1fa08f3c9bea..6f6cc6c259e34 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/async/AsyncTaskIndexService.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/async/AsyncTaskIndexService.java @@ -15,11 +15,11 @@ import org.elasticsearch.TransportVersions; import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.delete.DeleteRequest; import org.elasticsearch.action.delete.DeleteResponse; import org.elasticsearch.action.get.GetRequest; import org.elasticsearch.action.index.IndexRequest; -import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.update.UpdateRequest; import org.elasticsearch.action.update.UpdateResponse; import org.elasticsearch.client.internal.Client; @@ -214,7 +214,7 @@ public SecurityContext getSecurityContext() { * Currently for EQL we don't set limit for a stored async response * TODO: add limit for stored async response in EQL, and instead of this method use createResponse */ - public void createResponseForEQL(String docId, Map headers, R response, ActionListener listener) { + public void createResponseForEQL(String docId, Map headers, R response, ActionListener listener) { try { final ReleasableBytesStreamOutput buffer = new ReleasableBytesStreamOutput(0, bigArrays.withCircuitBreaking()); final XContentBuilder source = XContentFactory.jsonBuilder(buffer); @@ -239,7 +239,7 @@ public void createResponseForEQL(String docId, Map headers, R re * Stores the initial response with the original headers of the authenticated user * and the expected expiration time. */ - public void createResponse(String docId, Map headers, R response, ActionListener listener) + public void createResponse(String docId, Map headers, R response, ActionListener listener) throws IOException { try { final ReleasableBytesStreamOutput buffer = new ReleasableBytesStreamOutputWithLimit( diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/common/notifications/AbstractAuditor.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/common/notifications/AbstractAuditor.java index fe24c6f0d5cb2..f4c3704cd65c1 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/common/notifications/AbstractAuditor.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/common/notifications/AbstractAuditor.java @@ -10,10 +10,10 @@ import org.apache.logging.log4j.Logger; import org.elasticsearch.ElasticsearchParseException; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.admin.indices.template.put.PutComposableIndexTemplateAction; import org.elasticsearch.action.bulk.BulkRequest; import org.elasticsearch.action.index.IndexRequest; -import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.client.internal.OriginSettingClient; import org.elasticsearch.cluster.metadata.ComposableIndexTemplate; import org.elasticsearch.cluster.service.ClusterService; @@ -116,7 +116,7 @@ public void error(String resourceId, String message) { indexDoc(messageFactory.newMessage(resourceId, message, Level.ERROR, new Date(), nodeName)); } - private static void onIndexResponse(IndexResponse response) { + private static void onIndexResponse(DocWriteResponse response) { logger.trace("Successfully wrote audit message"); } diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/async/AsyncResultsServiceTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/async/AsyncResultsServiceTests.java index 1845e36ad34fd..a2f4239b4a3f9 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/async/AsyncResultsServiceTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/async/AsyncResultsServiceTests.java @@ -8,7 +8,7 @@ import org.elasticsearch.ResourceNotFoundException; import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.index.IndexResponse; +import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.action.support.master.AcknowledgedResponse; import org.elasticsearch.action.update.UpdateResponse; @@ -185,7 +185,7 @@ public void testRetrieveFromMemoryWithExpiration() throws Exception { if (updateInitialResultsInStore) { // we need to store initial result - PlainActionFuture future = new PlainActionFuture<>(); + PlainActionFuture future = new PlainActionFuture<>(); indexService.createResponse( task.getExecutionId().getDocId(), task.getOriginHeaders(), @@ -233,7 +233,7 @@ public void testAssertExpirationPropagation() throws Exception { if (updateInitialResultsInStore) { // we need to store initial result - PlainActionFuture future = new PlainActionFuture<>(); + PlainActionFuture future = new PlainActionFuture<>(); indexService.createResponse( task.getExecutionId().getDocId(), task.getOriginHeaders(), @@ -275,7 +275,7 @@ public void testRetrieveFromDisk() throws Exception { if (updateInitialResultsInStore) { // we need to store initial result - PlainActionFuture futureCreate = new PlainActionFuture<>(); + PlainActionFuture futureCreate = new PlainActionFuture<>(); indexService.createResponse( task.getExecutionId().getDocId(), task.getOriginHeaders(), @@ -293,7 +293,7 @@ public void testRetrieveFromDisk() throws Exception { ); futureUpdate.actionGet(TimeValue.timeValueSeconds(10)); } else { - PlainActionFuture futureCreate = new PlainActionFuture<>(); + PlainActionFuture futureCreate = new PlainActionFuture<>(); indexService.createResponse( task.getExecutionId().getDocId(), task.getOriginHeaders(), diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/async/AsyncSearchIndexServiceTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/async/AsyncSearchIndexServiceTests.java index 379530d72786d..61d48817f03a4 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/async/AsyncSearchIndexServiceTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/async/AsyncSearchIndexServiceTests.java @@ -8,7 +8,6 @@ import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.admin.cluster.settings.ClusterUpdateSettingsRequest; -import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.action.update.UpdateResponse; import org.elasticsearch.cluster.service.ClusterService; @@ -142,7 +141,7 @@ public void testEncodeSearchResponse() throws IOException { new TaskId(randomAlphaOfLength(10), randomNonNegativeLong()) ); - PlainActionFuture createFuture = new PlainActionFuture<>(); + PlainActionFuture createFuture = new PlainActionFuture<>(); indexService.createResponse(executionId.getDocId(), Map.of(), initialResponse, createFuture); assertThat(createFuture.actionGet().getResult(), equalTo(DocWriteResponse.Result.CREATED)); @@ -264,7 +263,7 @@ public CircuitBreakerStats stats(String name) { { circuitBreaker.adjustLimit(randomIntBetween(1, 64)); // small limit TestAsyncResponse initialResponse = new TestAsyncResponse(testMessage, expirationTime); - PlainActionFuture createFuture = new PlainActionFuture<>(); + PlainActionFuture createFuture = new PlainActionFuture<>(); indexService.createResponse(executionId.getDocId(), Map.of(), initialResponse, createFuture); CircuitBreakingException e = expectThrows(CircuitBreakingException.class, createFuture::actionGet); assertEquals(0, e.getSuppressed().length); // no other suppressed exceptions @@ -273,7 +272,7 @@ public CircuitBreakerStats stats(String name) { { circuitBreaker.adjustLimit(randomIntBetween(16 * 1024, 1024 * 1024)); // large enough TestAsyncResponse initialResponse = new TestAsyncResponse(testMessage, expirationTime); - PlainActionFuture createFuture = new PlainActionFuture<>(); + PlainActionFuture createFuture = new PlainActionFuture<>(); indexService.createResponse(executionId.getDocId(), Map.of(), initialResponse, createFuture); assertThat(createFuture.actionGet().getResult(), equalTo(DocWriteResponse.Result.CREATED)); assertThat(circuitBreaker.getUsed(), equalTo(0L)); @@ -337,7 +336,7 @@ public void testMaxAsyncSearchResponseSize() throws Exception { new TaskId(randomAlphaOfLength(10), randomNonNegativeLong()) ); TestAsyncResponse initialResponse = new TestAsyncResponse(randomAlphaOfLength(130), randomLong()); - PlainActionFuture createFuture1 = new PlainActionFuture<>(); + PlainActionFuture createFuture1 = new PlainActionFuture<>(); indexService.createResponse(executionId1.getDocId(), Map.of(), initialResponse, createFuture1); createFuture1.actionGet(); @@ -369,7 +368,7 @@ public void testMaxAsyncSearchResponseSize() throws Exception { Long.toString(randomNonNegativeLong()), new TaskId(randomAlphaOfLength(10), randomNonNegativeLong()) ); - PlainActionFuture createFuture = new PlainActionFuture<>(); + PlainActionFuture createFuture = new PlainActionFuture<>(); TestAsyncResponse initialResponse2 = new TestAsyncResponse(randomAlphaOfLength(130), randomLong()); indexService.createResponse(executionId2.getDocId(), Map.of(), initialResponse2, createFuture); IllegalArgumentException e2 = expectThrows(IllegalArgumentException.class, createFuture::actionGet); diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/async/AsyncTaskServiceTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/async/AsyncTaskServiceTests.java index 85689249486d1..d43295b2fe543 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/async/AsyncTaskServiceTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/async/AsyncTaskServiceTests.java @@ -6,10 +6,10 @@ */ package org.elasticsearch.xpack.core.async; +import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.admin.indices.get.GetIndexRequest; import org.elasticsearch.action.admin.indices.get.GetIndexResponse; import org.elasticsearch.action.delete.DeleteResponse; -import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.action.support.master.AcknowledgedResponse; import org.elasticsearch.action.update.UpdateResponse; @@ -217,7 +217,7 @@ public void testAutoCreateIndex() throws Exception { AsyncExecutionId id = new AsyncExecutionId("0", new TaskId("N/A", 0)); AsyncSearchResponse resp = new AsyncSearchResponse(id.getEncoded(), true, true, 0L, 0L); { - PlainActionFuture future = PlainActionFuture.newFuture(); + PlainActionFuture future = PlainActionFuture.newFuture(); indexService.createResponse(id.getDocId(), Collections.emptyMap(), resp, future); future.get(); assertSettings(); @@ -252,7 +252,7 @@ public void testAutoCreateIndex() throws Exception { // But the index is still auto-created { - PlainActionFuture future = PlainActionFuture.newFuture(); + PlainActionFuture future = PlainActionFuture.newFuture(); indexService.createResponse(id.getDocId(), Collections.emptyMap(), resp, future); future.get(); assertSettings(); diff --git a/x-pack/plugin/enrich/src/test/java/org/elasticsearch/xpack/enrich/EnrichPolicyRunnerTests.java b/x-pack/plugin/enrich/src/test/java/org/elasticsearch/xpack/enrich/EnrichPolicyRunnerTests.java index d8e582c9fb880..9e62ed225ccf0 100644 --- a/x-pack/plugin/enrich/src/test/java/org/elasticsearch/xpack/enrich/EnrichPolicyRunnerTests.java +++ b/x-pack/plugin/enrich/src/test/java/org/elasticsearch/xpack/enrich/EnrichPolicyRunnerTests.java @@ -11,6 +11,7 @@ import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.ActionResponse; import org.elasticsearch.action.ActionType; +import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.LatchedActionListener; import org.elasticsearch.action.admin.cluster.health.ClusterHealthAction; import org.elasticsearch.action.admin.indices.create.CreateIndexAction; @@ -29,7 +30,6 @@ import org.elasticsearch.action.admin.indices.segments.ShardSegments; import org.elasticsearch.action.admin.indices.settings.put.UpdateSettingsAction; import org.elasticsearch.action.index.IndexRequest; -import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.support.PlainActionFuture; @@ -112,7 +112,7 @@ public static void afterClass() { public void testRunner() throws Exception { final String sourceIndex = "source-index"; - IndexResponse indexRequest = client().index(new IndexRequest().index(sourceIndex).id("id").source(""" + DocWriteResponse indexRequest = client().index(new IndexRequest().index(sourceIndex).id("id").source(""" { "field1": "value1", "field2": 2, @@ -196,7 +196,7 @@ public void testRunner() throws Exception { public void testRunnerGeoMatchType() throws Exception { final String sourceIndex = "source-index"; - IndexResponse indexRequest = client().index(new IndexRequest().index(sourceIndex).id("id").source(""" + DocWriteResponse indexRequest = client().index(new IndexRequest().index(sourceIndex).id("id").source(""" {"location":"POINT(10.0 10.0)","zipcode":90210}""", XContentType.JSON).setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE)) .actionGet(); assertEquals(RestStatus.CREATED, indexRequest.status()); @@ -283,7 +283,7 @@ public void testRunnerDoubleRangeMatchType() throws Exception { private void testNumberRangeMatchType(String rangeType) throws Exception { final String sourceIndex = "source-index"; createIndex(sourceIndex, Settings.EMPTY, "_doc", "range", "type=" + rangeType + "_range"); - IndexResponse indexRequest = client().index(new IndexRequest().index(sourceIndex).id("id").source(""" + DocWriteResponse indexRequest = client().index(new IndexRequest().index(sourceIndex).id("id").source(""" {"range":{"gt":1,"lt":10},"zipcode":90210}""", XContentType.JSON).setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE)) .actionGet(); assertEquals(RestStatus.CREATED, indexRequest.status()); @@ -366,7 +366,7 @@ private GetIndexResponse getGetIndexResponseAndCheck(String createdEnrichIndex) public void testRunnerRangeTypeWithIpRange() throws Exception { final String sourceIndexName = "source-index"; createIndex(sourceIndexName, Settings.EMPTY, "_doc", "subnet", "type=ip_range"); - IndexResponse indexRequest = client().index(new IndexRequest().index(sourceIndexName).id("id").source(""" + DocWriteResponse indexRequest = client().index(new IndexRequest().index(sourceIndexName).id("id").source(""" {"subnet":"10.0.0.0/8","department":"research"}""", XContentType.JSON).setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE)) .actionGet(); assertEquals(RestStatus.CREATED, indexRequest.status()); @@ -450,7 +450,7 @@ public void testRunnerMultiSource() throws Exception { int numberOfSourceIndices = 3; for (int idx = 0; idx < numberOfSourceIndices; idx++) { final String sourceIndex = baseSourceName + idx; - IndexResponse indexRequest = client().index( + DocWriteResponse indexRequest = client().index( new IndexRequest().index(sourceIndex).id(randomAlphaOfLength(10)).source(Strings.format(""" { "idx": %s, @@ -554,7 +554,7 @@ public void testRunnerMultiSourceDocIdCollisions() throws Exception { String collidingDocId = randomAlphaOfLength(10); for (int idx = 0; idx < numberOfSourceIndices; idx++) { final String sourceIndex = baseSourceName + idx; - IndexResponse indexRequest = client().index( + DocWriteResponse indexRequest = client().index( new IndexRequest().index(sourceIndex).id(collidingDocId).routing(collidingDocId + idx).source(Strings.format(""" { "idx": %s, @@ -673,7 +673,7 @@ public void testRunnerMultiSourceEnrichKeyCollisions() throws Exception { int numberOfSourceIndices = 3; for (int idx = 0; idx < numberOfSourceIndices; idx++) { final String sourceIndex = baseSourceName + idx; - IndexResponse indexRequest = client().index( + DocWriteResponse indexRequest = client().index( new IndexRequest().index(sourceIndex).id(randomAlphaOfLength(10)).source(Strings.format(""" { "idx": %s, @@ -983,7 +983,7 @@ public void testRunnerObjectSourceMapping() throws Exception { CreateIndexResponse createResponse = indicesAdmin().create(new CreateIndexRequest(sourceIndex).mapping(mappingBuilder)).actionGet(); assertTrue(createResponse.isAcknowledged()); - IndexResponse indexRequest = client().index( + DocWriteResponse indexRequest = client().index( new IndexRequest().index(sourceIndex) .id("id") .source(""" @@ -1092,7 +1092,7 @@ public void testRunnerExplicitObjectSourceMapping() throws Exception { CreateIndexResponse createResponse = indicesAdmin().create(new CreateIndexRequest(sourceIndex).mapping(mappingBuilder)).actionGet(); assertTrue(createResponse.isAcknowledged()); - IndexResponse indexRequest = client().index( + DocWriteResponse indexRequest = client().index( new IndexRequest().index(sourceIndex) .id("id") .source(""" @@ -1201,7 +1201,7 @@ public void testRunnerExplicitObjectSourceMappingRangePolicy() throws Exception CreateIndexResponse createResponse = indicesAdmin().create(new CreateIndexRequest(sourceIndex).mapping(mappingBuilder)).actionGet(); assertTrue(createResponse.isAcknowledged()); - IndexResponse indexRequest = client().index(new IndexRequest().index(sourceIndex).id("id").source(""" + DocWriteResponse indexRequest = client().index(new IndexRequest().index(sourceIndex).id("id").source(""" { "data": { "subnet": "10.0.0.0/8", @@ -1316,7 +1316,7 @@ public void testRunnerTwoObjectLevelsSourceMapping() throws Exception { CreateIndexResponse createResponse = indicesAdmin().create(new CreateIndexRequest(sourceIndex).mapping(mappingBuilder)).actionGet(); assertTrue(createResponse.isAcknowledged()); - IndexResponse indexRequest = client().index(new IndexRequest().index(sourceIndex).id("id").source(""" + DocWriteResponse indexRequest = client().index(new IndexRequest().index(sourceIndex).id("id").source(""" { "data": { "fields": { @@ -1440,7 +1440,7 @@ public void testRunnerTwoObjectLevelsSourceMappingRangePolicy() throws Exception CreateIndexResponse createResponse = indicesAdmin().create(new CreateIndexRequest(sourceIndex).mapping(mappingBuilder)).actionGet(); assertTrue(createResponse.isAcknowledged()); - IndexResponse indexRequest = client().index(new IndexRequest().index(sourceIndex).id("id").source(""" + DocWriteResponse indexRequest = client().index(new IndexRequest().index(sourceIndex).id("id").source(""" { "data": { "fields": { @@ -1564,7 +1564,7 @@ public void testRunnerTwoObjectLevelsSourceMappingDateRangeWithFormat() throws E CreateIndexResponse createResponse = indicesAdmin().create(new CreateIndexRequest(sourceIndex).mapping(mappingBuilder)).actionGet(); assertTrue(createResponse.isAcknowledged()); - IndexResponse indexRequest = client().index(new IndexRequest().index(sourceIndex).id("id").source(""" + DocWriteResponse indexRequest = client().index(new IndexRequest().index(sourceIndex).id("id").source(""" { "data": { "fields": { @@ -1707,7 +1707,7 @@ public void testRunnerDottedKeyNameSourceMapping() throws Exception { CreateIndexResponse createResponse = indicesAdmin().create(new CreateIndexRequest(sourceIndex).mapping(mappingBuilder)).actionGet(); assertTrue(createResponse.isAcknowledged()); - IndexResponse indexRequest = client().index( + DocWriteResponse indexRequest = client().index( new IndexRequest().index(sourceIndex) .id("id") .source(""" @@ -1787,7 +1787,7 @@ public void testRunnerDottedKeyNameSourceMapping() throws Exception { public void testRunnerWithForceMergeRetry() throws Exception { final String sourceIndex = "source-index"; - IndexResponse indexRequest = client().index(new IndexRequest().index(sourceIndex).id("id").source(""" + DocWriteResponse indexRequest = client().index(new IndexRequest().index(sourceIndex).id("id").source(""" { "field1": "value1", "field2": 2, @@ -1885,7 +1885,7 @@ protected void ensureSingleSegment(String destinationIndexName, int attempt) { if (attempt == 1) { // Put and flush a document to increase the number of segments, simulating not // all segments were merged on the first try. - IndexResponse indexRequest = client().index( + DocWriteResponse indexRequest = client().index( new IndexRequest().index(createdEnrichIndex) .source(unmergedDocument) .setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE) @@ -1957,7 +1957,7 @@ protected void ensureSingleSegment(String destinationIndexName, int attempt) { public void testRunnerCancel() throws Exception { final String sourceIndex = "source-index"; - IndexResponse indexRequest = client().index(new IndexRequest().index(sourceIndex).id("id").source(""" + DocWriteResponse indexRequest = client().index(new IndexRequest().index(sourceIndex).id("id").source(""" { "field1": "value1", "field2": 2, diff --git a/x-pack/plugin/ent-search/src/main/java/org/elasticsearch/xpack/application/rules/QueryRulesIndexService.java b/x-pack/plugin/ent-search/src/main/java/org/elasticsearch/xpack/application/rules/QueryRulesIndexService.java index de823ab2a6ce2..01807fd986947 100644 --- a/x-pack/plugin/ent-search/src/main/java/org/elasticsearch/xpack/application/rules/QueryRulesIndexService.java +++ b/x-pack/plugin/ent-search/src/main/java/org/elasticsearch/xpack/application/rules/QueryRulesIndexService.java @@ -17,7 +17,6 @@ import org.elasticsearch.action.get.GetRequest; import org.elasticsearch.action.get.GetResponse; import org.elasticsearch.action.index.IndexRequest; -import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.support.WriteRequest; @@ -232,7 +231,7 @@ private List parseCriteria(List> rawCrite * @param queryRuleset The query ruleset object. * @param listener The action listener to invoke on response/failure. */ - public void putQueryRuleset(QueryRuleset queryRuleset, ActionListener listener) { + public void putQueryRuleset(QueryRuleset queryRuleset, ActionListener listener) { try { validateQueryRuleset(queryRuleset); final IndexRequest indexRequest = new IndexRequest(QUERY_RULES_ALIAS_NAME).opType(DocWriteRequest.OpType.INDEX) diff --git a/x-pack/plugin/ent-search/src/main/java/org/elasticsearch/xpack/application/search/SearchApplicationIndexService.java b/x-pack/plugin/ent-search/src/main/java/org/elasticsearch/xpack/application/search/SearchApplicationIndexService.java index 08b1e4f90c419..51b55a0a1c033 100644 --- a/x-pack/plugin/ent-search/src/main/java/org/elasticsearch/xpack/application/search/SearchApplicationIndexService.java +++ b/x-pack/plugin/ent-search/src/main/java/org/elasticsearch/xpack/application/search/SearchApplicationIndexService.java @@ -24,7 +24,6 @@ import org.elasticsearch.action.delete.DeleteResponse; import org.elasticsearch.action.get.GetRequest; import org.elasticsearch.action.index.IndexRequest; -import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.support.WriteRequest; @@ -226,7 +225,7 @@ private static String getSearchAliasName(SearchApplication app) { * @param create If true, the search application must not already exist * @param listener The action listener to invoke on response/failure. */ - public void putSearchApplication(SearchApplication app, boolean create, ActionListener listener) { + public void putSearchApplication(SearchApplication app, boolean create, ActionListener listener) { createOrUpdateAlias(app, new ActionListener<>() { @Override public void onResponse(AcknowledgedResponse acknowledgedResponse) { @@ -283,7 +282,7 @@ private IndicesAliasesRequestBuilder updateAliasIndices(Set currentAlias return aliasesRequestBuilder; } - private void updateSearchApplication(SearchApplication app, boolean create, ActionListener listener) { + private void updateSearchApplication(SearchApplication app, boolean create, ActionListener listener) { try (ReleasableBytesStreamOutput buffer = new ReleasableBytesStreamOutput(0, bigArrays.withCircuitBreaking())) { try (XContentBuilder source = XContentFactory.jsonBuilder(buffer)) { source.startObject() diff --git a/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/rules/QueryRulesIndexServiceTests.java b/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/rules/QueryRulesIndexServiceTests.java index 1e8fa5953606b..2c18a866d684a 100644 --- a/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/rules/QueryRulesIndexServiceTests.java +++ b/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/rules/QueryRulesIndexServiceTests.java @@ -9,8 +9,8 @@ import org.elasticsearch.ResourceNotFoundException; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.delete.DeleteResponse; -import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Settings; @@ -75,7 +75,7 @@ public void testUpdateQueryRuleset() throws Exception { Map.of("ids", List.of("id1", "id2")) ); final QueryRuleset myQueryRuleset = new QueryRuleset("my_ruleset", Collections.singletonList(myQueryRule1)); - IndexResponse resp = awaitPutQueryRuleset(myQueryRuleset); + DocWriteResponse resp = awaitPutQueryRuleset(myQueryRuleset); assertThat(resp.status(), anyOf(equalTo(RestStatus.CREATED), equalTo(RestStatus.OK))); assertThat(resp.getIndex(), equalTo(QUERY_RULES_CONCRETE_INDEX_NAME)); @@ -96,7 +96,7 @@ public void testUpdateQueryRuleset() throws Exception { Map.of("docs", List.of(Map.of("_index", "my_index1", "_id", "id3"), Map.of("_index", "my_index2", "_id", "id4"))) ); final QueryRuleset myQueryRuleset = new QueryRuleset("my_ruleset", List.of(myQueryRule1, myQueryRule2)); - IndexResponse newResp = awaitPutQueryRuleset(myQueryRuleset); + DocWriteResponse newResp = awaitPutQueryRuleset(myQueryRuleset); assertThat(newResp.status(), equalTo(RestStatus.OK)); assertThat(newResp.getIndex(), equalTo(QUERY_RULES_CONCRETE_INDEX_NAME)); QueryRuleset getQueryRuleset = awaitGetQueryRuleset(myQueryRuleset.id()); @@ -128,7 +128,7 @@ public void testListQueryRulesets() throws Exception { ); final QueryRuleset myQueryRuleset = new QueryRuleset("my_ruleset_" + i, rules); - IndexResponse resp = awaitPutQueryRuleset(myQueryRuleset); + DocWriteResponse resp = awaitPutQueryRuleset(myQueryRuleset); assertThat(resp.status(), equalTo(RestStatus.CREATED)); assertThat(resp.getIndex(), equalTo(QUERY_RULES_CONCRETE_INDEX_NAME)); } @@ -182,7 +182,7 @@ public void testDeleteQueryRuleset() throws Exception { Map.of("ids", List.of("id3", "id4")) ); final QueryRuleset myQueryRuleset = new QueryRuleset("my_ruleset", List.of(myQueryRule1, myQueryRule2)); - IndexResponse resp = awaitPutQueryRuleset(myQueryRuleset); + DocWriteResponse resp = awaitPutQueryRuleset(myQueryRuleset); assertThat(resp.status(), anyOf(equalTo(RestStatus.CREATED), equalTo(RestStatus.OK))); assertThat(resp.getIndex(), equalTo(QUERY_RULES_CONCRETE_INDEX_NAME)); @@ -195,13 +195,13 @@ public void testDeleteQueryRuleset() throws Exception { expectThrows(ResourceNotFoundException.class, () -> awaitGetQueryRuleset("my_ruleset")); } - private IndexResponse awaitPutQueryRuleset(QueryRuleset queryRuleset) throws Exception { + private DocWriteResponse awaitPutQueryRuleset(QueryRuleset queryRuleset) throws Exception { CountDownLatch latch = new CountDownLatch(1); - final AtomicReference resp = new AtomicReference<>(null); + final AtomicReference resp = new AtomicReference<>(null); final AtomicReference exc = new AtomicReference<>(null); queryRulesIndexService.putQueryRuleset(queryRuleset, new ActionListener<>() { @Override - public void onResponse(IndexResponse indexResponse) { + public void onResponse(DocWriteResponse indexResponse) { resp.set(indexResponse); latch.countDown(); } diff --git a/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/search/SearchApplicationIndexServiceTests.java b/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/search/SearchApplicationIndexServiceTests.java index 2bf35ef88ebc0..7891f5773d1a8 100644 --- a/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/search/SearchApplicationIndexServiceTests.java +++ b/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/search/SearchApplicationIndexServiceTests.java @@ -9,9 +9,9 @@ import org.elasticsearch.ResourceNotFoundException; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.admin.indices.alias.get.GetAliasesResponse; import org.elasticsearch.action.delete.DeleteResponse; -import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.cluster.metadata.Metadata; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.settings.Settings; @@ -78,7 +78,7 @@ public void testCreateSearchApplication() throws Exception { null ); - IndexResponse resp = awaitPutSearchApplication(searchApp, true); + DocWriteResponse resp = awaitPutSearchApplication(searchApp, true); assertThat(resp.status(), equalTo(RestStatus.CREATED)); assertThat(resp.getIndex(), equalTo(SEARCH_APPLICATION_CONCRETE_INDEX_NAME)); @@ -99,7 +99,7 @@ public void testCreateSearchApplication() throws Exception { SearchApplicationTemplate.DEFAULT_TEMPLATE ); - IndexResponse resp2 = awaitPutSearchApplication(searchApp2, true); + DocWriteResponse resp2 = awaitPutSearchApplication(searchApp2, true); assertThat(resp2.status(), equalTo(RestStatus.CREATED)); assertThat(resp2.getIndex(), equalTo(SEARCH_APPLICATION_CONCRETE_INDEX_NAME)); @@ -133,7 +133,7 @@ public void testUpdateSearchApplication() throws Exception { System.currentTimeMillis(), SearchApplicationTestUtils.getRandomSearchApplicationTemplate() ); - IndexResponse resp = awaitPutSearchApplication(searchApp, false); + DocWriteResponse resp = awaitPutSearchApplication(searchApp, false); assertThat(resp.status(), equalTo(RestStatus.CREATED)); assertThat(resp.getIndex(), equalTo(SEARCH_APPLICATION_CONCRETE_INDEX_NAME)); @@ -148,7 +148,7 @@ public void testUpdateSearchApplication() throws Exception { System.currentTimeMillis(), SearchApplicationTestUtils.getRandomSearchApplicationTemplate() ); - IndexResponse newResp = awaitPutSearchApplication(searchApp, false); + DocWriteResponse newResp = awaitPutSearchApplication(searchApp, false); assertThat(newResp.status(), equalTo(RestStatus.OK)); assertThat(newResp.getIndex(), equalTo(SEARCH_APPLICATION_CONCRETE_INDEX_NAME)); SearchApplication getNewSearchApp = awaitGetSearchApplication(searchApp.name()); @@ -166,7 +166,7 @@ public void testListSearchApplication() throws Exception { System.currentTimeMillis(), null ); - IndexResponse resp = awaitPutSearchApplication(searchApp, false); + DocWriteResponse resp = awaitPutSearchApplication(searchApp, false); assertThat(resp.status(), equalTo(RestStatus.CREATED)); assertThat(resp.getIndex(), equalTo(SEARCH_APPLICATION_CONCRETE_INDEX_NAME)); } @@ -208,7 +208,7 @@ public void testListSearchApplicationWithQuery() throws Exception { System.currentTimeMillis(), null ); - IndexResponse resp = awaitPutSearchApplication(app, false); + DocWriteResponse resp = awaitPutSearchApplication(app, false); assertThat(resp.status(), equalTo(RestStatus.CREATED)); assertThat(resp.getIndex(), equalTo(SEARCH_APPLICATION_CONCRETE_INDEX_NAME)); } @@ -234,7 +234,7 @@ public void testDeleteSearchApplication() throws Exception { System.currentTimeMillis(), null ); - IndexResponse resp = awaitPutSearchApplication(app, false); + DocWriteResponse resp = awaitPutSearchApplication(app, false); assertThat(resp.status(), equalTo(RestStatus.CREATED)); assertThat(resp.getIndex(), equalTo(SEARCH_APPLICATION_CONCRETE_INDEX_NAME)); @@ -262,13 +262,13 @@ public void testDeleteSearchApplication() throws Exception { } } - private IndexResponse awaitPutSearchApplication(SearchApplication app, boolean create) throws Exception { + private DocWriteResponse awaitPutSearchApplication(SearchApplication app, boolean create) throws Exception { CountDownLatch latch = new CountDownLatch(1); - final AtomicReference resp = new AtomicReference<>(null); + final AtomicReference resp = new AtomicReference<>(null); final AtomicReference exc = new AtomicReference<>(null); searchAppService.putSearchApplication(app, create, new ActionListener<>() { @Override - public void onResponse(IndexResponse indexResponse) { + public void onResponse(DocWriteResponse indexResponse) { resp.set(indexResponse); latch.countDown(); } diff --git a/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/EsqlActionBreakerIT.java b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/EsqlActionBreakerIT.java index 2bffd5c64cdaf..519938c8e3201 100644 --- a/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/EsqlActionBreakerIT.java +++ b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/EsqlActionBreakerIT.java @@ -9,7 +9,6 @@ import org.elasticsearch.ElasticsearchException; import org.elasticsearch.action.DocWriteResponse; -import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.common.breaker.CircuitBreakingException; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.compute.operator.exchange.ExchangeService; @@ -65,7 +64,7 @@ protected Settings nodeSettings(int nodeOrdinal, Settings otherSettings) { public void testBreaker() { for (int i = 0; i < 5000; i++) { - IndexResponse response = client().prepareIndex("test").setId(Integer.toString(i)).setSource("foo", i, "bar", i * 2).get(); + DocWriteResponse response = client().prepareIndex("test").setId(Integer.toString(i)).setSource("foo", i, "bar", i * 2).get(); if (response.getResult() != DocWriteResponse.Result.CREATED) { fail("failure: " + response); } diff --git a/x-pack/plugin/frozen-indices/src/internalClusterTest/java/org/elasticsearch/index/engine/frozen/FrozenIndexIT.java b/x-pack/plugin/frozen-indices/src/internalClusterTest/java/org/elasticsearch/index/engine/frozen/FrozenIndexIT.java index 30f24deac408b..f366a18c7393f 100644 --- a/x-pack/plugin/frozen-indices/src/internalClusterTest/java/org/elasticsearch/index/engine/frozen/FrozenIndexIT.java +++ b/x-pack/plugin/frozen-indices/src/internalClusterTest/java/org/elasticsearch/index/engine/frozen/FrozenIndexIT.java @@ -8,7 +8,7 @@ package org.elasticsearch.index.engine.frozen; import org.elasticsearch.ElasticsearchException; -import org.elasticsearch.action.index.IndexResponse; +import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.search.ClosePointInTimeAction; import org.elasticsearch.action.search.ClosePointInTimeRequest; import org.elasticsearch.action.search.OpenPointInTimeAction; @@ -73,7 +73,7 @@ public void testTimestampRangeRecalculatedOnStalePrimaryAllocation() throws IOEx createIndex("index", 1, 1); - final IndexResponse indexResponse = client().prepareIndex("index") + final DocWriteResponse indexResponse = client().prepareIndex("index") .setSource(DataStream.TIMESTAMP_FIELD_NAME, "2010-01-06T02:03:04.567Z") .get(); diff --git a/x-pack/plugin/frozen-indices/src/internalClusterTest/java/org/elasticsearch/index/engine/frozen/FrozenIndexTests.java b/x-pack/plugin/frozen-indices/src/internalClusterTest/java/org/elasticsearch/index/engine/frozen/FrozenIndexTests.java index 3a5f2c3027b08..baa2678985a33 100644 --- a/x-pack/plugin/frozen-indices/src/internalClusterTest/java/org/elasticsearch/index/engine/frozen/FrozenIndexTests.java +++ b/x-pack/plugin/frozen-indices/src/internalClusterTest/java/org/elasticsearch/index/engine/frozen/FrozenIndexTests.java @@ -7,11 +7,11 @@ package org.elasticsearch.index.engine.frozen; import org.elasticsearch.ResourceNotFoundException; +import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.OriginalIndices; import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse; import org.elasticsearch.action.admin.indices.stats.IndicesStatsResponse; import org.elasticsearch.action.delete.DeleteResponse; -import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.search.ClosePointInTimeAction; import org.elasticsearch.action.search.ClosePointInTimeRequest; import org.elasticsearch.action.search.OpenPointInTimeAction; @@ -549,7 +549,7 @@ public void testRecoveryState() { final long nbDocs = randomIntBetween(0, 50); for (long i = 0; i < nbDocs; i++) { - final IndexResponse indexResponse = client().prepareIndex(indexName).setId(Long.toString(i)).setSource("field", i).get(); + final DocWriteResponse indexResponse = client().prepareIndex(indexName).setId(Long.toString(i)).setSource("field", i).get(); assertThat(indexResponse.status(), is(RestStatus.CREATED)); } @@ -580,7 +580,7 @@ public void testTranslogStats() { final int nbDocs = randomIntBetween(0, 50); int uncommittedOps = 0; for (long i = 0; i < nbDocs; i++) { - final IndexResponse indexResponse = client().prepareIndex(indexName).setId(Long.toString(i)).setSource("field", i).get(); + final DocWriteResponse indexResponse = client().prepareIndex(indexName).setId(Long.toString(i)).setSource("field", i).get(); assertThat(indexResponse.status(), is(RestStatus.CREATED)); if (rarely()) { indicesAdmin().prepareFlush(indexName).get(); diff --git a/x-pack/plugin/ml/qa/native-multi-node-tests/src/javaRestTest/java/org/elasticsearch/xpack/ml/integration/ModelSnapshotRetentionIT.java b/x-pack/plugin/ml/qa/native-multi-node-tests/src/javaRestTest/java/org/elasticsearch/xpack/ml/integration/ModelSnapshotRetentionIT.java index 1b12eae165c29..027be542f00cb 100644 --- a/x-pack/plugin/ml/qa/native-multi-node-tests/src/javaRestTest/java/org/elasticsearch/xpack/ml/integration/ModelSnapshotRetentionIT.java +++ b/x-pack/plugin/ml/qa/native-multi-node-tests/src/javaRestTest/java/org/elasticsearch/xpack/ml/integration/ModelSnapshotRetentionIT.java @@ -13,7 +13,6 @@ 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.search.SearchAction; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchResponse; @@ -241,7 +240,7 @@ private void persistModelSnapshotDoc(String jobId, String snapshotId, Date times modelSnapshotBuilder.build().toXContent(xContentBuilder, ToXContent.EMPTY_PARAMS); indexRequest.source(xContentBuilder); - IndexResponse indexResponse = client().execute(IndexAction.INSTANCE, indexRequest).actionGet(); + DocWriteResponse indexResponse = client().execute(IndexAction.INSTANCE, indexRequest).actionGet(); assertThat(indexResponse.getResult(), is(DocWriteResponse.Result.CREATED)); } diff --git a/x-pack/plugin/ml/qa/native-multi-node-tests/src/javaRestTest/java/org/elasticsearch/xpack/ml/integration/ModelSnapshotSearchIT.java b/x-pack/plugin/ml/qa/native-multi-node-tests/src/javaRestTest/java/org/elasticsearch/xpack/ml/integration/ModelSnapshotSearchIT.java index 6dd463835c2d7..9852517ff0231 100644 --- a/x-pack/plugin/ml/qa/native-multi-node-tests/src/javaRestTest/java/org/elasticsearch/xpack/ml/integration/ModelSnapshotSearchIT.java +++ b/x-pack/plugin/ml/qa/native-multi-node-tests/src/javaRestTest/java/org/elasticsearch/xpack/ml/integration/ModelSnapshotSearchIT.java @@ -13,7 +13,6 @@ 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.PlainActionFuture; import org.elasticsearch.action.support.WriteRequest; import org.elasticsearch.action.support.master.MasterNodeRequest; @@ -127,7 +126,7 @@ private void persistModelSnapshotDoc(String jobId, String snapshotId, Date times modelSnapshotBuilder.build().toXContent(xContentBuilder, ToXContent.EMPTY_PARAMS); indexRequest.source(xContentBuilder); - IndexResponse indexResponse = client().execute(IndexAction.INSTANCE, indexRequest).actionGet(); + DocWriteResponse indexResponse = client().execute(IndexAction.INSTANCE, indexRequest).actionGet(); assertThat(indexResponse.getResult(), is(DocWriteResponse.Result.CREATED)); } diff --git a/x-pack/plugin/ml/src/internalClusterTest/java/org/elasticsearch/xpack/ml/integration/AnnotationIndexIT.java b/x-pack/plugin/ml/src/internalClusterTest/java/org/elasticsearch/xpack/ml/integration/AnnotationIndexIT.java index d4a0802b0c770..5980e04acbf9c 100644 --- a/x-pack/plugin/ml/src/internalClusterTest/java/org/elasticsearch/xpack/ml/integration/AnnotationIndexIT.java +++ b/x-pack/plugin/ml/src/internalClusterTest/java/org/elasticsearch/xpack/ml/integration/AnnotationIndexIT.java @@ -6,6 +6,7 @@ */ package org.elasticsearch.xpack.ml.integration; +import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.admin.indices.alias.Alias; import org.elasticsearch.action.admin.indices.alias.IndicesAliasesRequest; import org.elasticsearch.action.admin.indices.alias.IndicesAliasesRequestBuilder; @@ -13,7 +14,6 @@ import org.elasticsearch.action.admin.indices.create.CreateIndexRequest; import org.elasticsearch.action.admin.indices.get.GetIndexResponse; import org.elasticsearch.action.index.IndexRequest; -import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.search.SearchPhaseExecutionException; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchResponse; @@ -309,7 +309,7 @@ public void testNotCreatedWhenAfterOtherMlIndexAndResetInProgress() throws Excep IndexRequest stateDoc = new IndexRequest(".ml-state"); stateDoc.source(Collections.singletonMap("state", "blah")); - IndexResponse indexResponse = client().index(stateDoc).actionGet(); + DocWriteResponse indexResponse = client().index(stateDoc).actionGet(); assertEquals(RestStatus.CREATED, indexResponse.status()); // Creating the .ml-state index would normally cause .ml-annotations diff --git a/x-pack/plugin/ml/src/internalClusterTest/java/org/elasticsearch/xpack/ml/integration/DatafeedConfigProviderIT.java b/x-pack/plugin/ml/src/internalClusterTest/java/org/elasticsearch/xpack/ml/integration/DatafeedConfigProviderIT.java index 6d1155b1b8b64..1561520510c38 100644 --- a/x-pack/plugin/ml/src/internalClusterTest/java/org/elasticsearch/xpack/ml/integration/DatafeedConfigProviderIT.java +++ b/x-pack/plugin/ml/src/internalClusterTest/java/org/elasticsearch/xpack/ml/integration/DatafeedConfigProviderIT.java @@ -11,7 +11,6 @@ import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.delete.DeleteResponse; -import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.core.Tuple; import org.elasticsearch.persistent.PersistentTasksCustomMetadata; @@ -77,7 +76,7 @@ public void createComponents() throws Exception { public void testCrud() throws InterruptedException { String datafeedId = "df1"; - AtomicReference> responseHolder = new AtomicReference<>(); + AtomicReference> responseHolder = new AtomicReference<>(); AtomicReference exceptionHolder = new AtomicReference<>(); // Create datafeed config @@ -164,7 +163,7 @@ public void testGetDatafeedConfig_missing() throws InterruptedException { public void testMultipleCreateAndDeletes() throws InterruptedException { String datafeedId = "df2"; - AtomicReference> responseHolder = new AtomicReference<>(); + AtomicReference> responseHolder = new AtomicReference<>(); AtomicReference exceptionHolder = new AtomicReference<>(); // Create datafeed config @@ -576,7 +575,7 @@ private Map createSecurityHeader() { private DatafeedConfig putDatafeedConfig(DatafeedConfig.Builder builder, Map headers) throws Exception { builder.setHeaders(headers); DatafeedConfig config = builder.build(); - this.>blockingCall( + this.>blockingCall( actionListener -> datafeedConfigProvider.putDatafeedConfig(config, headers, actionListener) ); return config; diff --git a/x-pack/plugin/ml/src/internalClusterTest/java/org/elasticsearch/xpack/ml/integration/JobConfigProviderIT.java b/x-pack/plugin/ml/src/internalClusterTest/java/org/elasticsearch/xpack/ml/integration/JobConfigProviderIT.java index 2355904fb6910..bca437dbf676c 100644 --- a/x-pack/plugin/ml/src/internalClusterTest/java/org/elasticsearch/xpack/ml/integration/JobConfigProviderIT.java +++ b/x-pack/plugin/ml/src/internalClusterTest/java/org/elasticsearch/xpack/ml/integration/JobConfigProviderIT.java @@ -11,7 +11,6 @@ import org.elasticsearch.ResourceNotFoundException; import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.delete.DeleteResponse; -import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.persistent.PersistentTasksCustomMetadata; import org.elasticsearch.search.aggregations.AggregationBuilders; @@ -100,7 +99,7 @@ public void testCheckJobExists() throws InterruptedException { assertNull(exceptionHolder.get()); } - AtomicReference indexResponseHolder = new AtomicReference<>(); + AtomicReference indexResponseHolder = new AtomicReference<>(); // Create job Job job = createJob("existing-job", null).build(new Date()); @@ -120,7 +119,7 @@ public void testCheckJobExists() throws InterruptedException { public void testOverwriteNotAllowed() throws InterruptedException { final String jobId = "same-id"; - AtomicReference indexResponseHolder = new AtomicReference<>(); + AtomicReference indexResponseHolder = new AtomicReference<>(); AtomicReference exceptionHolder = new AtomicReference<>(); // Create job @@ -141,7 +140,7 @@ public void testOverwriteNotAllowed() throws InterruptedException { public void testCrud() throws InterruptedException { final String jobId = "crud-job"; - AtomicReference indexResponseHolder = new AtomicReference<>(); + AtomicReference indexResponseHolder = new AtomicReference<>(); AtomicReference exceptionHolder = new AtomicReference<>(); // Create job @@ -203,7 +202,7 @@ public void testCrud() throws InterruptedException { public void testUpdateWithAValidationError() throws Exception { final String jobId = "bad-update-job"; - AtomicReference indexResponseHolder = new AtomicReference<>(); + AtomicReference indexResponseHolder = new AtomicReference<>(); AtomicReference exceptionHolder = new AtomicReference<>(); // Create job @@ -233,7 +232,7 @@ public void testUpdateWithValidator() throws Exception { // Create job Job newJob = createJob(jobId, null).build(new Date()); - this.blockingCall(actionListener -> jobConfigProvider.putJob(newJob, actionListener)); + this.blockingCall(actionListener -> jobConfigProvider.putJob(newJob, actionListener)); JobUpdate jobUpdate = new JobUpdate.Builder(jobId).setDescription("This job has been updated").build(); @@ -664,7 +663,7 @@ private static Job.Builder addCustomRule(Job.Builder job, DetectionRule rule) { private Job putJob(Job.Builder job) throws Exception { Job builtJob = job.build(new Date()); - this.blockingCall(actionListener -> jobConfigProvider.putJob(builtJob, actionListener)); + this.blockingCall(actionListener -> jobConfigProvider.putJob(builtJob, actionListener)); return builtJob; } } diff --git a/x-pack/plugin/ml/src/internalClusterTest/java/org/elasticsearch/xpack/ml/integration/TrainedModelProviderIT.java b/x-pack/plugin/ml/src/internalClusterTest/java/org/elasticsearch/xpack/ml/integration/TrainedModelProviderIT.java index c659fb20aeaeb..1656970f17158 100644 --- a/x-pack/plugin/ml/src/internalClusterTest/java/org/elasticsearch/xpack/ml/integration/TrainedModelProviderIT.java +++ b/x-pack/plugin/ml/src/internalClusterTest/java/org/elasticsearch/xpack/ml/integration/TrainedModelProviderIT.java @@ -6,12 +6,12 @@ */ package org.elasticsearch.xpack.ml.integration; +import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.admin.indices.refresh.RefreshResponse; import org.elasticsearch.action.bulk.BulkRequestBuilder; import org.elasticsearch.action.bulk.BulkResponse; import org.elasticsearch.action.delete.DeleteRequest; import org.elasticsearch.action.index.IndexRequestBuilder; -import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.support.WriteRequest; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesReference; @@ -320,7 +320,7 @@ public void testGetTruncatedModelDeprecatedDefinition() throws Exception { new ToXContent.MapParams(Collections.singletonMap(FOR_INTERNAL_STORAGE, "true")) ) ) { - AtomicReference putDocHolder = new AtomicReference<>(); + AtomicReference putDocHolder = new AtomicReference<>(); blockingCall( listener -> client().prepareIndex(InferenceIndexConstants.LATEST_INDEX_NAME) .setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE) diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportPutCalendarAction.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportPutCalendarAction.java index 88350aa5cf3db..7b3fa5252306d 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportPutCalendarAction.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportPutCalendarAction.java @@ -8,9 +8,9 @@ import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.DocWriteRequest; +import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.index.IndexAction; import org.elasticsearch.action.index.IndexRequest; -import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.HandledTransportAction; import org.elasticsearch.action.support.WriteRequest; @@ -64,9 +64,9 @@ protected void doExecute(Task task, PutCalendarAction.Request request, ActionLis indexRequest.opType(DocWriteRequest.OpType.CREATE); indexRequest.setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE); - executeAsyncWithOrigin(client, ML_ORIGIN, IndexAction.INSTANCE, indexRequest, new ActionListener() { + executeAsyncWithOrigin(client, ML_ORIGIN, IndexAction.INSTANCE, indexRequest, new ActionListener<>() { @Override - public void onResponse(IndexResponse indexResponse) { + public void onResponse(DocWriteResponse indexResponse) { listener.onResponse(new PutCalendarAction.Response(calendar)); } diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportPutFilterAction.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportPutFilterAction.java index 851dcab1d6b5e..ea3404cd304c1 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportPutFilterAction.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportPutFilterAction.java @@ -9,9 +9,9 @@ import org.elasticsearch.ResourceAlreadyExistsException; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.DocWriteRequest; +import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.index.IndexAction; import org.elasticsearch.action.index.IndexRequest; -import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.HandledTransportAction; import org.elasticsearch.action.support.WriteRequest; @@ -58,9 +58,9 @@ protected void doExecute(Task task, PutFilterAction.Request request, ActionListe throw new IllegalStateException("Failed to serialise filter with id [" + filter.getId() + "]", e); } - executeAsyncWithOrigin(client, ML_ORIGIN, IndexAction.INSTANCE, indexRequest, new ActionListener() { + executeAsyncWithOrigin(client, ML_ORIGIN, IndexAction.INSTANCE, indexRequest, new ActionListener<>() { @Override - public void onResponse(IndexResponse indexResponse) { + public void onResponse(DocWriteResponse indexResponse) { listener.onResponse(new PutFilterAction.Response(filter)); } diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportUpdateFilterAction.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportUpdateFilterAction.java index 9b57f60e6b4ec..15de14d2de297 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportUpdateFilterAction.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportUpdateFilterAction.java @@ -8,12 +8,12 @@ import org.elasticsearch.ResourceNotFoundException; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.get.GetAction; import org.elasticsearch.action.get.GetRequest; import org.elasticsearch.action.get.GetResponse; import org.elasticsearch.action.index.IndexAction; import org.elasticsearch.action.index.IndexRequest; -import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.HandledTransportAction; import org.elasticsearch.action.support.WriteRequest; @@ -128,9 +128,9 @@ private void indexUpdatedFilter( throw new IllegalStateException("Failed to serialise filter with id [" + filter.getId() + "]", e); } - executeAsyncWithOrigin(client, ML_ORIGIN, IndexAction.INSTANCE, indexRequest, new ActionListener() { + executeAsyncWithOrigin(client, ML_ORIGIN, IndexAction.INSTANCE, indexRequest, new ActionListener<>() { @Override - public void onResponse(IndexResponse indexResponse) { + public void onResponse(DocWriteResponse indexResponse) { jobManager.notifyFilterChanged( filter, request.getAddItems(), diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/datafeed/persistence/DatafeedConfigProvider.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/datafeed/persistence/DatafeedConfigProvider.java index 2ed8685cadaea..316e5707a48dd 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/datafeed/persistence/DatafeedConfigProvider.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/datafeed/persistence/DatafeedConfigProvider.java @@ -21,7 +21,6 @@ import org.elasticsearch.action.get.GetResponse; import org.elasticsearch.action.index.IndexAction; import org.elasticsearch.action.index.IndexRequest; -import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.support.IndicesOptions; @@ -113,7 +112,7 @@ public DatafeedConfigProvider(Client client, NamedXContentRegistry xContentRegis public void putDatafeedConfig( DatafeedConfig config, Map headers, - ActionListener> listener + ActionListener> listener ) { DatafeedConfig finalConfig; @@ -353,7 +352,7 @@ public void onResponse(GetResponse getResponse) { }); } - private void indexUpdatedConfig(DatafeedConfig updatedConfig, long seqNo, long primaryTerm, ActionListener listener) { + private void indexUpdatedConfig(DatafeedConfig updatedConfig, long seqNo, long primaryTerm, ActionListener listener) { try (XContentBuilder builder = XContentFactory.jsonBuilder()) { XContentBuilder updatedSource = updatedConfig.toXContent(builder, new ToXContent.MapParams(TO_XCONTENT_PARAMS)); IndexRequest indexRequest = new IndexRequest(MlConfigIndex.indexName()).id(DatafeedConfig.documentId(updatedConfig.getId())) diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/dataframe/DataFrameAnalyticsTask.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/dataframe/DataFrameAnalyticsTask.java index 34ace0d0c9273..c6af1bcfa6f18 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/dataframe/DataFrameAnalyticsTask.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/dataframe/DataFrameAnalyticsTask.java @@ -10,9 +10,9 @@ import org.apache.logging.log4j.Logger; import org.apache.lucene.util.SetOnce; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.index.IndexAction; import org.elasticsearch.action.index.IndexRequest; -import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.search.SearchAction; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchResponse; @@ -215,7 +215,7 @@ void persistProgress(Client clientToUse, String jobId, Runnable runnable) { String progressDocId = StoredProgress.documentId(jobId); // Step 4: Run the runnable provided as the argument - ActionListener indexProgressDocListener = ActionListener.wrap(indexResponse -> { + ActionListener indexProgressDocListener = ActionListener.wrap(indexResponse -> { LOGGER.debug("[{}] Successfully indexed progress document: {}", jobId, storedProgress.get().get()); runnable.run(); }, indexError -> { diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/dataframe/steps/FinalStep.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/dataframe/steps/FinalStep.java index 4d7cb456d52d9..20791534b9801 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/dataframe/steps/FinalStep.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/dataframe/steps/FinalStep.java @@ -10,12 +10,12 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.admin.indices.refresh.RefreshAction; import org.elasticsearch.action.admin.indices.refresh.RefreshRequest; import org.elasticsearch.action.admin.indices.refresh.RefreshResponse; import org.elasticsearch.action.index.IndexAction; import org.elasticsearch.action.index.IndexRequest; -import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.client.internal.node.NodeClient; import org.elasticsearch.core.TimeValue; @@ -65,7 +65,7 @@ protected void doExecute(ActionListener listener) { listener::onFailure ); - ActionListener dataCountsIndexedListener = ActionListener.wrap( + ActionListener dataCountsIndexedListener = ActionListener.wrap( indexResponse -> refreshIndices(refreshListener), listener::onFailure ); @@ -73,7 +73,7 @@ protected void doExecute(ActionListener listener) { indexDataCounts(dataCountsIndexedListener); } - private void indexDataCounts(ActionListener listener) { + private void indexDataCounts(ActionListener listener) { DataCounts dataCounts = task.getStatsHolder().getDataCountsTracker().report(); try (XContentBuilder builder = XContentFactory.jsonBuilder()) { dataCounts.toXContent( diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/JobManager.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/JobManager.java index ba3a52df5cdb6..14a9474a0c787 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/JobManager.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/JobManager.java @@ -11,7 +11,7 @@ import org.elasticsearch.ResourceAlreadyExistsException; import org.elasticsearch.ResourceNotFoundException; import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.index.IndexResponse; +import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.support.WriteRequest; import org.elasticsearch.action.support.master.AcknowledgedResponse; import org.elasticsearch.client.internal.Client; @@ -666,7 +666,7 @@ public void revertSnapshot( // Step 3. After the model size stats is persisted, also persist the snapshot's quantiles and respond // ------- - CheckedConsumer modelSizeStatsResponseHandler = response -> { + CheckedConsumer modelSizeStatsResponseHandler = response -> { // In case we are reverting to the empty snapshot the quantiles will be null if (modelSnapshot.getQuantiles() == null) { actionListener.onResponse(new RevertModelSnapshotAction.Response(modelSnapshot)); diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobConfigProvider.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobConfigProvider.java index 5a00bb748fa4e..eababb5fab02c 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobConfigProvider.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobConfigProvider.java @@ -22,7 +22,6 @@ import org.elasticsearch.action.get.GetResponse; import org.elasticsearch.action.index.IndexAction; import org.elasticsearch.action.index.IndexRequest; -import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.support.IndicesOptions; @@ -120,7 +119,7 @@ public JobConfigProvider(Client client, NamedXContentRegistry xContentRegistry) * @param job The anomaly detector job configuration * @param listener Index response listener */ - public void putJob(Job job, ActionListener listener) { + public void putJob(Job job, ActionListener listener) { try (XContentBuilder builder = XContentFactory.jsonBuilder()) { XContentBuilder source = job.toXContent(builder, new ToXContent.MapParams(TO_XCONTENT_PARAMS)); IndexRequest indexRequest = new IndexRequest(MlConfigIndex.indexName()).id(Job.documentId(job.getId())) diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsPersister.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsPersister.java index c2661fc933ca9..e65e0abaca2e3 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsPersister.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsPersister.java @@ -10,6 +10,7 @@ import org.apache.logging.log4j.Logger; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.DocWriteRequest; +import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.DocWriteResponse.Result; import org.elasticsearch.action.admin.indices.refresh.RefreshRequest; import org.elasticsearch.action.bulk.BulkItemResponse; @@ -338,7 +339,7 @@ public void persistQuantiles(Quantiles quantiles, Supplier shouldRetry) /** * Persist the quantiles (async) */ - public void persistQuantiles(Quantiles quantiles, WriteRequest.RefreshPolicy refreshPolicy, ActionListener listener) { + public void persistQuantiles(Quantiles quantiles, WriteRequest.RefreshPolicy refreshPolicy, ActionListener listener) { String quantilesDocId = Quantiles.documentId(quantiles.getJobId()); // Step 2: Create or update the quantiles document: @@ -414,7 +415,7 @@ public void persistModelSizeStats(ModelSizeStats modelSizeStats, Supplier listener + ActionListener listener ) { String jobId = modelSizeStats.getJobId(); logger.trace("[{}] Persisting model size stats, for size {}", jobId, modelSizeStats.getModelBytes()); @@ -573,7 +574,7 @@ void persist(Supplier shouldRetry, boolean requireAlias, ActionListener } } - void persistWithoutRetries(ActionListener listener, boolean requireAlias) { + void persistWithoutRetries(ActionListener listener, boolean requireAlias) { logCall(); try (XContentBuilder content = toXContentBuilder(object, params)) { diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsPersisterTests.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsPersisterTests.java index 2dc8b77fc5834..fe82cfcb00d2c 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsPersisterTests.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsPersisterTests.java @@ -7,6 +7,7 @@ package org.elasticsearch.xpack.ml.job.persistence; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.bulk.BulkAction; import org.elasticsearch.action.bulk.BulkRequest; import org.elasticsearch.action.bulk.BulkResponse; @@ -375,7 +376,7 @@ private void testPersistQuantilesAsync(SearchHits searchHits, String expectedInd doAnswer(withResponse(indexResponse)).when(client).execute(eq(IndexAction.INSTANCE), any(), any()); Quantiles quantiles = new Quantiles("foo", new Date(), "bar"); - ActionListener indexResponseListener = mock(ActionListener.class); + ActionListener indexResponseListener = mock(ActionListener.class); persister.persistQuantiles(quantiles, WriteRequest.RefreshPolicy.IMMEDIATE, indexResponseListener); InOrder inOrder = inOrder(client, indexResponseListener); diff --git a/x-pack/plugin/profiling/src/internalClusterTest/java/org/elasticsearch/xpack/profiling/ProfilingTestCase.java b/x-pack/plugin/profiling/src/internalClusterTest/java/org/elasticsearch/xpack/profiling/ProfilingTestCase.java index f15925b7c891b..0fb3bf540a4ed 100644 --- a/x-pack/plugin/profiling/src/internalClusterTest/java/org/elasticsearch/xpack/profiling/ProfilingTestCase.java +++ b/x-pack/plugin/profiling/src/internalClusterTest/java/org/elasticsearch/xpack/profiling/ProfilingTestCase.java @@ -7,9 +7,9 @@ package org.elasticsearch.xpack.profiling; +import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.admin.cluster.settings.ClusterUpdateSettingsRequest; import org.elasticsearch.action.admin.cluster.settings.ClusterUpdateSettingsResponse; -import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.common.network.NetworkModule; import org.elasticsearch.common.settings.Settings; @@ -72,7 +72,7 @@ protected boolean ignoreExternalCluster() { } private void indexDoc(String index, String id, Map source) { - IndexResponse indexResponse = client().prepareIndex(index).setId(id).setSource(source).setCreate(true).get(); + DocWriteResponse indexResponse = client().prepareIndex(index).setId(id).setSource(source).setCreate(true).get(); assertEquals(RestStatus.CREATED, indexResponse.status()); } diff --git a/x-pack/plugin/rank-rrf/src/internalClusterTest/java/org/elasticsearch/xpack/rank/rrf/RRFRankShardCanMatchIT.java b/x-pack/plugin/rank-rrf/src/internalClusterTest/java/org/elasticsearch/xpack/rank/rrf/RRFRankShardCanMatchIT.java index eb360257dfff1..a7019d76cdcd1 100644 --- a/x-pack/plugin/rank-rrf/src/internalClusterTest/java/org/elasticsearch/xpack/rank/rrf/RRFRankShardCanMatchIT.java +++ b/x-pack/plugin/rank-rrf/src/internalClusterTest/java/org/elasticsearch/xpack/rank/rrf/RRFRankShardCanMatchIT.java @@ -8,7 +8,7 @@ package org.elasticsearch.xpack.rank.rrf; import org.apache.lucene.util.BytesRef; -import org.elasticsearch.action.index.IndexResponse; +import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchType; import org.elasticsearch.common.io.stream.StreamInput; @@ -129,13 +129,13 @@ public void testCanMatchShard() throws IOException { int shardB = -1; for (int i = 0; i < 10; i++) { - IndexResponse ir = client().prepareIndex("value_index").setSource("value", "" + i).setRouting("a").get(); + DocWriteResponse ir = client().prepareIndex("value_index").setSource("value", "" + i).setRouting("a").get(); int a = ir.getShardId().id(); assertTrue(shardA == a || shardA == -1); shardA = a; } for (int i = 10; i < 20; i++) { - IndexResponse ir = client().prepareIndex("value_index").setSource("value", "" + i).setRouting("b").get(); + DocWriteResponse ir = client().prepareIndex("value_index").setSource("value", "" + i).setRouting("b").get(); int b = ir.getShardId().id(); assertTrue(shardB == b || shardB == -1); shardB = b; diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/cache/blob/BlobStoreCacheService.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/cache/blob/BlobStoreCacheService.java index b8e527f14312e..ebf1296da9f55 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/cache/blob/BlobStoreCacheService.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/cache/blob/BlobStoreCacheService.java @@ -14,10 +14,10 @@ import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.get.GetRequest; import org.elasticsearch.action.get.GetResponse; import org.elasticsearch.action.index.IndexRequest; -import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.action.support.TransportActions; import org.elasticsearch.blobcache.common.ByteRange; @@ -274,7 +274,7 @@ public final void putAsync( final ActionListener wrappedListener = ActionListener.runAfter(listener, release); innerPut(request, new ActionListener<>() { @Override - public void onResponse(IndexResponse indexResponse) { + public void onResponse(DocWriteResponse indexResponse) { logger.trace("cache fill ({}): [{}]", indexResponse.status(), request.id()); wrappedListener.onResponse(null); } @@ -297,7 +297,7 @@ public void onFailure(Exception e) { } } - protected void innerPut(final IndexRequest request, final ActionListener listener) { + protected void innerPut(final IndexRequest request, final ActionListener listener) { client.index(request, listener); } diff --git a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/xpack/searchablesnapshots/cache/common/TestUtils.java b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/xpack/searchablesnapshots/cache/common/TestUtils.java index f3cac44d5b91b..e44b004a66b7b 100644 --- a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/xpack/searchablesnapshots/cache/common/TestUtils.java +++ b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/xpack/searchablesnapshots/cache/common/TestUtils.java @@ -7,6 +7,7 @@ package org.elasticsearch.xpack.searchablesnapshots.cache.common; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.get.GetRequest; import org.elasticsearch.action.get.GetResponse; import org.elasticsearch.action.index.IndexRequest; @@ -271,7 +272,7 @@ protected void innerGet(GetRequest request, ActionListener listener } @Override - protected void innerPut(IndexRequest request, ActionListener listener) { + protected void innerPut(IndexRequest request, ActionListener listener) { listener.onFailure(new IndexNotFoundException(request.index())); } @@ -315,7 +316,7 @@ protected void innerGet(GetRequest request, ActionListener listener } @Override - protected void innerPut(IndexRequest request, ActionListener listener) { + protected void innerPut(IndexRequest request, ActionListener listener) { final BytesArray bytesArray = blobs.put(request.id(), new BytesArray(request.source().toBytesRef(), true)); listener.onResponse( new IndexResponse( diff --git a/x-pack/plugin/security/src/internalClusterTest/java/org/elasticsearch/integration/DateMathExpressionIntegTests.java b/x-pack/plugin/security/src/internalClusterTest/java/org/elasticsearch/integration/DateMathExpressionIntegTests.java index 0612f2302404f..3a0fb370ac0d2 100644 --- a/x-pack/plugin/security/src/internalClusterTest/java/org/elasticsearch/integration/DateMathExpressionIntegTests.java +++ b/x-pack/plugin/security/src/internalClusterTest/java/org/elasticsearch/integration/DateMathExpressionIntegTests.java @@ -10,7 +10,6 @@ import org.elasticsearch.action.admin.indices.create.CreateIndexResponse; import org.elasticsearch.action.get.GetResponse; import org.elasticsearch.action.get.MultiGetResponse; -import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.search.MultiSearchResponse; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.support.master.AcknowledgedResponse; @@ -67,7 +66,7 @@ public void testDateMathExpressionsCanBeAuthorized() throws Exception { CreateIndexResponse response = client.admin().indices().prepareCreate(expression).get(); assertThat(response.isAcknowledged(), is(true)); } - IndexResponse response = client.prepareIndex(expression) + DocWriteResponse response = client.prepareIndex(expression) .setSource("foo", "bar") .setRefreshPolicy(refeshOnOperation ? IMMEDIATE : NONE) .get(); diff --git a/x-pack/plugin/security/src/internalClusterTest/java/org/elasticsearch/integration/KibanaSystemRoleIntegTests.java b/x-pack/plugin/security/src/internalClusterTest/java/org/elasticsearch/integration/KibanaSystemRoleIntegTests.java index e221bad6b2c1c..50b54e2c1fa51 100644 --- a/x-pack/plugin/security/src/internalClusterTest/java/org/elasticsearch/integration/KibanaSystemRoleIntegTests.java +++ b/x-pack/plugin/security/src/internalClusterTest/java/org/elasticsearch/integration/KibanaSystemRoleIntegTests.java @@ -9,7 +9,6 @@ import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.admin.indices.create.CreateIndexResponse; import org.elasticsearch.action.delete.DeleteResponse; -import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.common.settings.SecureString; import org.elasticsearch.test.SecurityIntegTestCase; import org.elasticsearch.test.SecuritySettingsSourceField; @@ -46,7 +45,7 @@ public void testCreateIndexDeleteInKibanaIndex() throws Exception { assertThat(createIndexResponse.isAcknowledged(), is(true)); } - IndexResponse response = client().filterWithHeader( + DocWriteResponse response = client().filterWithHeader( singletonMap("Authorization", UsernamePasswordToken.basicAuthHeaderValue("my_kibana_system", USERS_PASSWD)) ).prepareIndex().setIndex(index).setSource("foo", "bar").setRefreshPolicy(IMMEDIATE).get(); assertEquals(DocWriteResponse.Result.CREATED, response.getResult()); diff --git a/x-pack/plugin/security/src/internalClusterTest/java/org/elasticsearch/integration/MultipleIndicesPermissionsTests.java b/x-pack/plugin/security/src/internalClusterTest/java/org/elasticsearch/integration/MultipleIndicesPermissionsTests.java index 7493c95f8cebd..c9f987b1981a9 100644 --- a/x-pack/plugin/security/src/internalClusterTest/java/org/elasticsearch/integration/MultipleIndicesPermissionsTests.java +++ b/x-pack/plugin/security/src/internalClusterTest/java/org/elasticsearch/integration/MultipleIndicesPermissionsTests.java @@ -14,7 +14,6 @@ import org.elasticsearch.action.admin.indices.settings.get.GetSettingsResponse; import org.elasticsearch.action.admin.indices.shards.IndicesShardStoresResponse; import org.elasticsearch.action.admin.indices.stats.IndicesStatsResponse; -import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.search.MultiSearchResponse; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.client.Request; @@ -134,7 +133,7 @@ protected String configUsersRoles() { } public void testSingleRole() throws Exception { - IndexResponse indexResponse = index("test", jsonBuilder().startObject().field("name", "value").endObject()); + DocWriteResponse indexResponse = index("test", jsonBuilder().startObject().field("name", "value").endObject()); assertEquals(DocWriteResponse.Result.CREATED, indexResponse.getResult()); indexResponse = index("test1", jsonBuilder().startObject().field("name", "value1").endObject()); @@ -185,7 +184,7 @@ public void testSingleRole() throws Exception { public void testMonitorRestrictedWildcards() throws Exception { - IndexResponse indexResponse = index("foo", jsonBuilder().startObject().field("name", "value").endObject()); + DocWriteResponse indexResponse = index("foo", jsonBuilder().startObject().field("name", "value").endObject()); assertEquals(DocWriteResponse.Result.CREATED, indexResponse.getResult()); indexResponse = index("foobar", jsonBuilder().startObject().field("name", "value").endObject()); @@ -239,7 +238,7 @@ public void testMonitorRestrictedWildcards() throws Exception { } public void testMultipleRoles() throws Exception { - IndexResponse indexResponse = index("a", jsonBuilder().startObject().field("name", "value_a").endObject()); + DocWriteResponse indexResponse = index("a", jsonBuilder().startObject().field("name", "value_a").endObject()); assertEquals(DocWriteResponse.Result.CREATED, indexResponse.getResult()); indexResponse = index("b", jsonBuilder().startObject().field("name", "value_b").endObject()); diff --git a/x-pack/plugin/security/src/internalClusterTest/java/org/elasticsearch/license/LicensingTests.java b/x-pack/plugin/security/src/internalClusterTest/java/org/elasticsearch/license/LicensingTests.java index df35e705e8303..27b85888864c9 100644 --- a/x-pack/plugin/security/src/internalClusterTest/java/org/elasticsearch/license/LicensingTests.java +++ b/x-pack/plugin/security/src/internalClusterTest/java/org/elasticsearch/license/LicensingTests.java @@ -15,7 +15,6 @@ import org.elasticsearch.action.admin.cluster.stats.ClusterStatsIndices; import org.elasticsearch.action.admin.cluster.stats.ClusterStatsResponse; import org.elasticsearch.action.admin.indices.stats.IndicesStatsResponse; -import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.client.Request; import org.elasticsearch.client.RequestOptions; import org.elasticsearch.client.ResponseException; @@ -146,7 +145,7 @@ public void cleanupSecurityIndex() { } public void testEnableDisableBehaviour() throws Exception { - IndexResponse indexResponse = index("test", jsonBuilder().startObject().field("name", "value").endObject()); + DocWriteResponse indexResponse = index("test", jsonBuilder().startObject().field("name", "value").endObject()); assertEquals(DocWriteResponse.Result.CREATED, indexResponse.getResult()); indexResponse = index("test1", jsonBuilder().startObject().field("name", "value1").endObject()); diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/esnative/NativeUsersStore.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/esnative/NativeUsersStore.java index e05943210e8df..76029b779d8d9 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/esnative/NativeUsersStore.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/esnative/NativeUsersStore.java @@ -16,7 +16,6 @@ import org.elasticsearch.action.delete.DeleteRequest; import org.elasticsearch.action.delete.DeleteResponse; import org.elasticsearch.action.get.GetResponse; -import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.support.ContextPreservingActionListener; @@ -327,7 +326,7 @@ private void updateReservedUser( ) .setRefreshPolicy(refresh) .request(), - listener.delegateFailure((l, indexResponse) -> clearRealmCache(username, l, null)), + listener.delegateFailure((l, indexResponse) -> clearRealmCache(username, l, null)), client::index ); }); @@ -433,7 +432,7 @@ private void indexUser(final PutUserRequest putUserRequest, final ActionListener ) .setRefreshPolicy(putUserRequest.getRefreshPolicy()) .request(), - listener.delegateFailure( + listener.delegateFailure( (l, updateResponse) -> clearRealmCache( putUserRequest.username(), l, diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/support/mapper/NativeRoleMappingStore.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/support/mapper/NativeRoleMappingStore.java index 7fff5e646b2a4..f49558ad6875d 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/support/mapper/NativeRoleMappingStore.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/support/mapper/NativeRoleMappingStore.java @@ -9,8 +9,8 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.delete.DeleteResponse; -import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.support.ContextPreservingActionListener; import org.elasticsearch.client.internal.Client; @@ -226,9 +226,9 @@ private void innerPutMapping(PutRoleMappingRequest request, ActionListener() { + new ActionListener() { @Override - public void onResponse(IndexResponse indexResponse) { + public void onResponse(DocWriteResponse indexResponse) { boolean created = indexResponse.getResult() == CREATED; listener.onResponse(created); } diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/store/NativePrivilegeStore.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/store/NativePrivilegeStore.java index 592fa16b79ff7..10f5539b953b6 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/store/NativePrivilegeStore.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/store/NativePrivilegeStore.java @@ -12,7 +12,6 @@ import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.delete.DeleteResponse; -import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.support.ContextPreservingActionListener; import org.elasticsearch.action.support.GroupedActionListener; @@ -368,9 +367,9 @@ public void putPrivileges( ActionListener>> listener ) { securityIndexManager.prepareIndexIfNeededThenExecute(listener::onFailure, () -> { - ActionListener groupListener = new GroupedActionListener<>( + ActionListener groupListener = new GroupedActionListener<>( privileges.size(), - ActionListener.wrap((Collection responses) -> { + ActionListener.wrap((Collection responses) -> { final Map> createdNames = responses.stream() .filter(r -> r.getResult() == DocWriteResponse.Result.CREATED) .map(r -> r.getId()) @@ -392,7 +391,7 @@ public void putPrivileges( private void innerPutPrivilege( ApplicationPrivilegeDescriptor privilege, WriteRequest.RefreshPolicy refreshPolicy, - ActionListener listener + ActionListener listener ) { try { final String name = privilege.getName(); diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/store/NativeRolesStore.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/store/NativeRolesStore.java index 26a73c9201622..085863fdb5e31 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/store/NativeRolesStore.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/store/NativeRolesStore.java @@ -19,7 +19,6 @@ import org.elasticsearch.action.get.MultiGetRequest; import org.elasticsearch.action.get.MultiGetResponse; import org.elasticsearch.action.index.IndexRequest; -import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.search.MultiSearchResponse; import org.elasticsearch.action.search.MultiSearchResponse.Item; import org.elasticsearch.action.search.SearchRequest; @@ -289,9 +288,9 @@ void innerPutRole(final PutRoleRequest request, final RoleDescriptor role, final client.threadPool().getThreadContext(), SECURITY_ORIGIN, indexRequest, - new ActionListener() { + new ActionListener() { @Override - public void onResponse(IndexResponse indexResponse) { + public void onResponse(DocWriteResponse indexResponse) { final boolean created = indexResponse.getResult() == DocWriteResponse.Result.CREATED; logger.trace("Created role: [{}]", indexRequest); clearRoleCache(roleName, listener, created); diff --git a/x-pack/plugin/transform/src/internalClusterTest/java/org/elasticsearch/xpack/transform/integration/TransformInternalIndexIT.java b/x-pack/plugin/transform/src/internalClusterTest/java/org/elasticsearch/xpack/transform/integration/TransformInternalIndexIT.java index bb33353d7b714..74eea067f8811 100644 --- a/x-pack/plugin/transform/src/internalClusterTest/java/org/elasticsearch/xpack/transform/integration/TransformInternalIndexIT.java +++ b/x-pack/plugin/transform/src/internalClusterTest/java/org/elasticsearch/xpack/transform/integration/TransformInternalIndexIT.java @@ -12,7 +12,6 @@ import org.elasticsearch.action.get.GetRequest; import org.elasticsearch.action.get.GetResponse; import org.elasticsearch.action.index.IndexRequest; -import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.support.WriteRequest; import org.elasticsearch.action.support.master.AcknowledgedRequest; import org.elasticsearch.common.settings.Settings; @@ -97,7 +96,7 @@ public void testUpdateDeletesOldTransformConfig() throws Exception { IndexRequest indexRequest = new IndexRequest(OLD_INDEX).id(TransformConfig.documentId(transformId)) .source(config, XContentType.JSON) .setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE); - IndexResponse indexResponse = client().index(indexRequest).actionGet(); + DocWriteResponse indexResponse = client().index(indexRequest).actionGet(); assertThat(indexResponse.getResult(), is(DocWriteResponse.Result.CREATED)); GetTransformAction.Request getTransformRequest = new GetTransformAction.Request(transformId); diff --git a/x-pack/plugin/transform/src/internalClusterTest/java/org/elasticsearch/xpack/transform/integration/TransformOldTransformsIT.java b/x-pack/plugin/transform/src/internalClusterTest/java/org/elasticsearch/xpack/transform/integration/TransformOldTransformsIT.java index ca03f7eb9dd9c..346765c515d31 100644 --- a/x-pack/plugin/transform/src/internalClusterTest/java/org/elasticsearch/xpack/transform/integration/TransformOldTransformsIT.java +++ b/x-pack/plugin/transform/src/internalClusterTest/java/org/elasticsearch/xpack/transform/integration/TransformOldTransformsIT.java @@ -10,7 +10,6 @@ import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.admin.indices.create.CreateIndexRequest; import org.elasticsearch.action.index.IndexRequest; -import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.support.WriteRequest; import org.elasticsearch.action.support.master.AcknowledgedRequest; import org.elasticsearch.common.ValidationException; @@ -110,7 +109,7 @@ public void testStopThrowsForDeprecatedTransformConfig() throws Exception { IndexRequest indexRequest = new IndexRequest(OLD_INDEX).id(TransformConfig.documentId(transformId)) .source(config, XContentType.JSON) .setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE); - IndexResponse indexResponse = client().index(indexRequest).actionGet(); + DocWriteResponse indexResponse = client().index(indexRequest).actionGet(); assertThat(indexResponse.getResult(), is(DocWriteResponse.Result.CREATED)); GetTransformAction.Request getTransformRequest = new GetTransformAction.Request(transformId); diff --git a/x-pack/plugin/transform/src/main/java/org/elasticsearch/xpack/transform/persistence/SeqNoPrimaryTermAndIndex.java b/x-pack/plugin/transform/src/main/java/org/elasticsearch/xpack/transform/persistence/SeqNoPrimaryTermAndIndex.java index 1ecd300b243ba..3273ae810b77e 100644 --- a/x-pack/plugin/transform/src/main/java/org/elasticsearch/xpack/transform/persistence/SeqNoPrimaryTermAndIndex.java +++ b/x-pack/plugin/transform/src/main/java/org/elasticsearch/xpack/transform/persistence/SeqNoPrimaryTermAndIndex.java @@ -7,7 +7,7 @@ package org.elasticsearch.xpack.transform.persistence; -import org.elasticsearch.action.index.IndexResponse; +import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.search.SearchHit; import java.util.Objects; @@ -24,7 +24,7 @@ public static SeqNoPrimaryTermAndIndex fromSearchHit(SearchHit hit) { return new SeqNoPrimaryTermAndIndex(hit.getSeqNo(), hit.getPrimaryTerm(), hit.getIndex()); } - public static SeqNoPrimaryTermAndIndex fromIndexResponse(IndexResponse response) { + public static SeqNoPrimaryTermAndIndex fromIndexResponse(DocWriteResponse response) { return new SeqNoPrimaryTermAndIndex(response.getSeqNo(), response.getPrimaryTerm(), response.getIndex()); } diff --git a/x-pack/plugin/watcher/src/internalClusterTest/java/org/elasticsearch/xpack/watcher/test/integration/WatchAckTests.java b/x-pack/plugin/watcher/src/internalClusterTest/java/org/elasticsearch/xpack/watcher/test/integration/WatchAckTests.java index af8ada52b8057..82f302b6fb44d 100644 --- a/x-pack/plugin/watcher/src/internalClusterTest/java/org/elasticsearch/xpack/watcher/test/integration/WatchAckTests.java +++ b/x-pack/plugin/watcher/src/internalClusterTest/java/org/elasticsearch/xpack/watcher/test/integration/WatchAckTests.java @@ -10,7 +10,6 @@ import org.elasticsearch.action.delete.DeleteResponse; import org.elasticsearch.action.get.GetRequest; import org.elasticsearch.action.get.GetResponse; -import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.support.WriteRequest; import org.elasticsearch.core.TimeValue; import org.elasticsearch.protocol.xpack.watcher.PutWatchResponse; @@ -54,7 +53,7 @@ public class WatchAckTests extends AbstractWatcherIntegrationTestCase { @Before public void indexTestDocument() { - IndexResponse eventIndexResponse = client().prepareIndex() + DocWriteResponse eventIndexResponse = client().prepareIndex() .setIndex("events") .setId(id) .setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE) diff --git a/x-pack/plugin/watcher/src/internalClusterTest/java/org/elasticsearch/xpack/watcher/transport/action/activate/ActivateWatchTests.java b/x-pack/plugin/watcher/src/internalClusterTest/java/org/elasticsearch/xpack/watcher/transport/action/activate/ActivateWatchTests.java index ec03215435fae..d268c6df4b21b 100644 --- a/x-pack/plugin/watcher/src/internalClusterTest/java/org/elasticsearch/xpack/watcher/transport/action/activate/ActivateWatchTests.java +++ b/x-pack/plugin/watcher/src/internalClusterTest/java/org/elasticsearch/xpack/watcher/transport/action/activate/ActivateWatchTests.java @@ -7,8 +7,8 @@ package org.elasticsearch.xpack.watcher.transport.action.activate; +import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.get.GetResponse; -import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.common.util.set.Sets; @@ -143,7 +143,7 @@ public void testLoadWatchWithoutAState() throws Exception { source.toXContent(builder, ToXContent.EMPTY_PARAMS); // now that we filtered out the watch status state, lets put it back in - IndexResponse indexResponse = client().prepareIndex() + DocWriteResponse indexResponse = client().prepareIndex() .setIndex(".watches") .setId("_id") .setSource(BytesReference.bytes(builder), XContentType.JSON) diff --git a/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/actions/index/ExecutableIndexAction.java b/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/actions/index/ExecutableIndexAction.java index d919f79b0cb02..0a67129495cb5 100644 --- a/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/actions/index/ExecutableIndexAction.java +++ b/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/actions/index/ExecutableIndexAction.java @@ -13,7 +13,6 @@ 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.client.internal.Client; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.core.TimeValue; @@ -116,7 +115,7 @@ public Action.Result execute(String actionId, WatchExecutionContext ctx, Payload } ClientHelper.assertNoAuthorizationHeader(ctx.watch().status().getHeaders()); - IndexResponse response = ClientHelper.executeWithHeaders( + DocWriteResponse response = ClientHelper.executeWithHeaders( ctx.watch().status().getHeaders(), ClientHelper.WATCHER_ORIGIN, client, @@ -273,7 +272,7 @@ private static void itemResponseToXContent(XContentBuilder builder, BulkItemResp } } - static void indexResponseToXContent(XContentBuilder builder, IndexResponse response) throws IOException { + static void indexResponseToXContent(XContentBuilder builder, DocWriteResponse response) throws IOException { builder.startObject() .field("created", response.getResult() == DocWriteResponse.Result.CREATED) .field("result", response.getResult().getLowercase()) diff --git a/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/transport/actions/TransportPutWatchAction.java b/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/transport/actions/TransportPutWatchAction.java index 405fcac33ff9e..f19a1ecd0c0fc 100644 --- a/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/transport/actions/TransportPutWatchAction.java +++ b/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/transport/actions/TransportPutWatchAction.java @@ -9,7 +9,6 @@ import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.index.IndexRequest; -import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.WriteRequest; import org.elasticsearch.action.update.UpdateRequest; @@ -153,7 +152,7 @@ protected void doExecute(PutWatchRequest request, ActionListenerwrap(response -> { + ActionListener.wrap(response -> { boolean created = response.getResult() == DocWriteResponse.Result.CREATED; listener.onResponse( new PutWatchResponse( diff --git a/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/actions/index/IndexActionTests.java b/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/actions/index/IndexActionTests.java index 57582b9a9793b..c355ec3ebb0ed 100644 --- a/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/actions/index/IndexActionTests.java +++ b/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/actions/index/IndexActionTests.java @@ -9,6 +9,7 @@ import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ElasticsearchParseException; import org.elasticsearch.action.DocWriteRequest; +import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.bulk.BulkItemResponse; import org.elasticsearch.action.bulk.BulkRequest; import org.elasticsearch.action.bulk.BulkResponse; @@ -241,7 +242,7 @@ public void testThatIndexTypeIdDynamically() throws Exception { final WatchExecutionContext ctx = WatcherTestUtils.mockExecutionContext("_id", new Payload.Simple(Maps.ofEntries(entries))); ArgumentCaptor captor = ArgumentCaptor.forClass(IndexRequest.class); - PlainActionFuture listener = PlainActionFuture.newFuture(); + PlainActionFuture listener = PlainActionFuture.newFuture(); listener.onResponse(new IndexResponse(new ShardId(new Index("foo", "bar"), 0), "whatever", 1, 1, 1, true)); when(client.index(captor.capture())).thenReturn(listener); Action.Result result = executable.execute("_id", ctx, ctx.payload()); @@ -339,7 +340,7 @@ public void testIndexActionExecuteSingleDoc() throws Exception { WatchExecutionContext ctx = WatcherTestUtils.mockExecutionContext("_id", executionTime, payload); ArgumentCaptor captor = ArgumentCaptor.forClass(IndexRequest.class); - PlainActionFuture listener = PlainActionFuture.newFuture(); + PlainActionFuture listener = PlainActionFuture.newFuture(); listener.onResponse(new IndexResponse(new ShardId(new Index("test-index", "uuid"), 0), docId, 1, 1, 1, true)); when(client.index(captor.capture())).thenReturn(listener); diff --git a/x-pack/qa/third-party/active-directory/src/test/java/org/elasticsearch/xpack/security/authc/ldap/AbstractAdLdapRealmTestCase.java b/x-pack/qa/third-party/active-directory/src/test/java/org/elasticsearch/xpack/security/authc/ldap/AbstractAdLdapRealmTestCase.java index 72ccea479277f..0c0a35f227c8e 100644 --- a/x-pack/qa/third-party/active-directory/src/test/java/org/elasticsearch/xpack/security/authc/ldap/AbstractAdLdapRealmTestCase.java +++ b/x-pack/qa/third-party/active-directory/src/test/java/org/elasticsearch/xpack/security/authc/ldap/AbstractAdLdapRealmTestCase.java @@ -12,7 +12,6 @@ import org.elasticsearch.action.ActionFuture; import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.get.GetResponse; -import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.client.internal.Client; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.settings.SecureString; @@ -257,7 +256,7 @@ protected void assertAccessAllowed(String user, String index) throws IOException // We can safely re-try this if it fails, which makes it less likely that the index request will fail authenticateUser(client, user, 3); - IndexResponse indexResponse = client.prepareIndex(index) + DocWriteResponse indexResponse = client.prepareIndex(index) .setSource(jsonBuilder().startObject().field("name", "value").endObject()) .execute() .actionGet();