Remove redudant children of BroadcastResponse (#104410)

A couple of children of `BroadCastResponse` are completely redundant,
adding no extra fields or separate serialization.
Removed them and replaced their use by the broadcast response itself.
This commit is contained in:
Armin Braun 2024-01-16 16:56:29 +01:00 committed by GitHub
parent af50962ec3
commit 51caf171bc
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
96 changed files with 293 additions and 655 deletions

View file

@ -13,7 +13,6 @@ import org.elasticsearch.action.DocWriteRequest;
import org.elasticsearch.action.IndicesRequest; import org.elasticsearch.action.IndicesRequest;
import org.elasticsearch.action.admin.cluster.settings.ClusterGetSettingsAction; import org.elasticsearch.action.admin.cluster.settings.ClusterGetSettingsAction;
import org.elasticsearch.action.admin.indices.flush.FlushRequest; import org.elasticsearch.action.admin.indices.flush.FlushRequest;
import org.elasticsearch.action.admin.indices.flush.FlushResponse;
import org.elasticsearch.action.admin.indices.forcemerge.ForceMergeAction; import org.elasticsearch.action.admin.indices.forcemerge.ForceMergeAction;
import org.elasticsearch.action.admin.indices.refresh.RefreshRequest; import org.elasticsearch.action.admin.indices.refresh.RefreshRequest;
import org.elasticsearch.action.admin.indices.rollover.RolloverRequest; import org.elasticsearch.action.admin.indices.rollover.RolloverRequest;
@ -30,6 +29,7 @@ import org.elasticsearch.action.datastreams.ModifyDataStreamsAction;
import org.elasticsearch.action.datastreams.lifecycle.ErrorEntry; import org.elasticsearch.action.datastreams.lifecycle.ErrorEntry;
import org.elasticsearch.action.datastreams.lifecycle.ExplainIndexDataStreamLifecycle; import org.elasticsearch.action.datastreams.lifecycle.ExplainIndexDataStreamLifecycle;
import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.index.IndexRequest;
import org.elasticsearch.action.support.broadcast.BroadcastResponse;
import org.elasticsearch.cluster.coordination.StableMasterHealthIndicatorService; import org.elasticsearch.cluster.coordination.StableMasterHealthIndicatorService;
import org.elasticsearch.cluster.metadata.ComposableIndexTemplate; import org.elasticsearch.cluster.metadata.ComposableIndexTemplate;
import org.elasticsearch.cluster.metadata.DataStream; import org.elasticsearch.cluster.metadata.DataStream;
@ -313,7 +313,7 @@ public class DataStreamLifecycleServiceIT extends ESIntegTestCase {
for (int i = 0; i < randomIntBetween(10, 50); i++) { for (int i = 0; i < randomIntBetween(10, 50); i++) {
indexDocs(dataStreamName, randomIntBetween(1, 300)); indexDocs(dataStreamName, randomIntBetween(1, 300));
// Make sure the segments get written: // Make sure the segments get written:
FlushResponse flushResponse = indicesAdmin().flush(new FlushRequest(toBeRolledOverIndex)).actionGet(); BroadcastResponse flushResponse = indicesAdmin().flush(new FlushRequest(toBeRolledOverIndex)).actionGet();
assertThat(flushResponse.getStatus(), equalTo(RestStatus.OK)); assertThat(flushResponse.getStatus(), equalTo(RestStatus.OK));
} }

View file

@ -19,7 +19,6 @@ import org.elasticsearch.action.admin.indices.delete.DeleteIndexRequest;
import org.elasticsearch.action.admin.indices.delete.TransportDeleteIndexAction; import org.elasticsearch.action.admin.indices.delete.TransportDeleteIndexAction;
import org.elasticsearch.action.admin.indices.forcemerge.ForceMergeAction; import org.elasticsearch.action.admin.indices.forcemerge.ForceMergeAction;
import org.elasticsearch.action.admin.indices.forcemerge.ForceMergeRequest; import org.elasticsearch.action.admin.indices.forcemerge.ForceMergeRequest;
import org.elasticsearch.action.admin.indices.forcemerge.ForceMergeResponse;
import org.elasticsearch.action.admin.indices.readonly.AddIndexBlockAction; import org.elasticsearch.action.admin.indices.readonly.AddIndexBlockAction;
import org.elasticsearch.action.admin.indices.readonly.AddIndexBlockRequest; import org.elasticsearch.action.admin.indices.readonly.AddIndexBlockRequest;
import org.elasticsearch.action.admin.indices.readonly.AddIndexBlockResponse; import org.elasticsearch.action.admin.indices.readonly.AddIndexBlockResponse;
@ -33,6 +32,7 @@ import org.elasticsearch.action.datastreams.lifecycle.ErrorEntry;
import org.elasticsearch.action.downsample.DownsampleAction; import org.elasticsearch.action.downsample.DownsampleAction;
import org.elasticsearch.action.downsample.DownsampleConfig; import org.elasticsearch.action.downsample.DownsampleConfig;
import org.elasticsearch.action.support.DefaultShardOperationFailedException; import org.elasticsearch.action.support.DefaultShardOperationFailedException;
import org.elasticsearch.action.support.broadcast.BroadcastResponse;
import org.elasticsearch.action.support.master.AcknowledgedResponse; import org.elasticsearch.action.support.master.AcknowledgedResponse;
import org.elasticsearch.client.internal.Client; import org.elasticsearch.client.internal.Client;
import org.elasticsearch.cluster.ClusterChangedEvent; import org.elasticsearch.cluster.ClusterChangedEvent;
@ -1168,7 +1168,7 @@ public class DataStreamLifecycleService implements ClusterStateListener, Closeab
logger.info("Data stream lifecycle is issuing a request to force merge index [{}]", targetIndex); logger.info("Data stream lifecycle is issuing a request to force merge index [{}]", targetIndex);
client.admin().indices().forceMerge(forceMergeRequest, new ActionListener<>() { client.admin().indices().forceMerge(forceMergeRequest, new ActionListener<>() {
@Override @Override
public void onResponse(ForceMergeResponse forceMergeResponse) { public void onResponse(BroadcastResponse forceMergeResponse) {
if (forceMergeResponse.getFailedShards() > 0) { if (forceMergeResponse.getFailedShards() > 0) {
DefaultShardOperationFailedException[] failures = forceMergeResponse.getShardFailures(); DefaultShardOperationFailedException[] failures = forceMergeResponse.getShardFailures();
String message = Strings.format( String message = Strings.format(

View file

@ -15,7 +15,6 @@ import org.elasticsearch.action.ActionResponse;
import org.elasticsearch.action.ActionType; import org.elasticsearch.action.ActionType;
import org.elasticsearch.action.admin.indices.delete.DeleteIndexRequest; import org.elasticsearch.action.admin.indices.delete.DeleteIndexRequest;
import org.elasticsearch.action.admin.indices.forcemerge.ForceMergeRequest; import org.elasticsearch.action.admin.indices.forcemerge.ForceMergeRequest;
import org.elasticsearch.action.admin.indices.forcemerge.ForceMergeResponse;
import org.elasticsearch.action.admin.indices.readonly.AddIndexBlockRequest; import org.elasticsearch.action.admin.indices.readonly.AddIndexBlockRequest;
import org.elasticsearch.action.admin.indices.rollover.MaxAgeCondition; import org.elasticsearch.action.admin.indices.rollover.MaxAgeCondition;
import org.elasticsearch.action.admin.indices.rollover.RolloverConditions; import org.elasticsearch.action.admin.indices.rollover.RolloverConditions;
@ -27,6 +26,7 @@ import org.elasticsearch.action.datastreams.lifecycle.ErrorEntry;
import org.elasticsearch.action.downsample.DownsampleAction; import org.elasticsearch.action.downsample.DownsampleAction;
import org.elasticsearch.action.downsample.DownsampleConfig; import org.elasticsearch.action.downsample.DownsampleConfig;
import org.elasticsearch.action.support.DefaultShardOperationFailedException; import org.elasticsearch.action.support.DefaultShardOperationFailedException;
import org.elasticsearch.action.support.broadcast.BroadcastResponse;
import org.elasticsearch.client.internal.Client; import org.elasticsearch.client.internal.Client;
import org.elasticsearch.cluster.ClusterName; import org.elasticsearch.cluster.ClusterName;
import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterState;
@ -578,7 +578,7 @@ public class DataStreamLifecycleServiceTests extends ESTestCase {
// We want this test method to get fake force merge responses, because this is what triggers a cluster state update // We want this test method to get fake force merge responses, because this is what triggers a cluster state update
clientDelegate = (action, request, listener) -> { clientDelegate = (action, request, listener) -> {
if (action.name().equals("indices:admin/forcemerge")) { if (action.name().equals("indices:admin/forcemerge")) {
listener.onResponse(new ForceMergeResponse(5, 5, 0, List.of())); listener.onResponse(new BroadcastResponse(5, 5, 0, List.of()));
} }
}; };
String dataStreamName = randomAlphaOfLength(10).toLowerCase(Locale.ROOT); String dataStreamName = randomAlphaOfLength(10).toLowerCase(Locale.ROOT);
@ -748,7 +748,7 @@ public class DataStreamLifecycleServiceTests extends ESTestCase {
clientDelegate = (action, request, listener) -> { clientDelegate = (action, request, listener) -> {
if (action.name().equals("indices:admin/forcemerge")) { if (action.name().equals("indices:admin/forcemerge")) {
listener.onResponse( listener.onResponse(
new ForceMergeResponse( new BroadcastResponse(
5, 5,
5, 5,
1, 1,
@ -779,7 +779,7 @@ public class DataStreamLifecycleServiceTests extends ESTestCase {
AtomicInteger forceMergeFailedCount = new AtomicInteger(0); AtomicInteger forceMergeFailedCount = new AtomicInteger(0);
clientDelegate = (action, request, listener) -> { clientDelegate = (action, request, listener) -> {
if (action.name().equals("indices:admin/forcemerge")) { if (action.name().equals("indices:admin/forcemerge")) {
listener.onResponse(new ForceMergeResponse(5, 4, 0, List.of())); listener.onResponse(new BroadcastResponse(5, 4, 0, List.of()));
forceMergeFailedCount.incrementAndGet(); forceMergeFailedCount.incrementAndGet();
} }
}; };
@ -800,7 +800,7 @@ public class DataStreamLifecycleServiceTests extends ESTestCase {
// For the final data stream lifecycle run, we let forcemerge run normally // For the final data stream lifecycle run, we let forcemerge run normally
clientDelegate = (action, request, listener) -> { clientDelegate = (action, request, listener) -> {
if (action.name().equals("indices:admin/forcemerge")) { if (action.name().equals("indices:admin/forcemerge")) {
listener.onResponse(new ForceMergeResponse(5, 5, 0, List.of())); listener.onResponse(new BroadcastResponse(5, 5, 0, List.of()));
} }
}; };
dataStreamLifecycleService.run(clusterService.state()); dataStreamLifecycleService.run(clusterService.state());
@ -900,7 +900,7 @@ public class DataStreamLifecycleServiceTests extends ESTestCase {
setState(clusterService, state); setState(clusterService, state);
clientDelegate = (action, request, listener) -> { clientDelegate = (action, request, listener) -> {
if (action.name().equals("indices:admin/forcemerge")) { if (action.name().equals("indices:admin/forcemerge")) {
listener.onResponse(new ForceMergeResponse(5, 5, 0, List.of())); listener.onResponse(new BroadcastResponse(5, 5, 0, List.of()));
} }
}; };
for (int i = 0; i < 100; i++) { for (int i = 0; i < 100; i++) {

View file

@ -17,13 +17,12 @@ import org.elasticsearch.action.DocWriteRequest.OpType;
import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.DocWriteResponse;
import org.elasticsearch.action.admin.indices.flush.FlushAction; import org.elasticsearch.action.admin.indices.flush.FlushAction;
import org.elasticsearch.action.admin.indices.flush.FlushRequest; import org.elasticsearch.action.admin.indices.flush.FlushRequest;
import org.elasticsearch.action.admin.indices.flush.FlushResponse;
import org.elasticsearch.action.admin.indices.refresh.RefreshAction; import org.elasticsearch.action.admin.indices.refresh.RefreshAction;
import org.elasticsearch.action.admin.indices.refresh.RefreshRequest; import org.elasticsearch.action.admin.indices.refresh.RefreshRequest;
import org.elasticsearch.action.admin.indices.refresh.RefreshResponse;
import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.index.IndexRequest;
import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.index.IndexResponse;
import org.elasticsearch.action.index.TransportIndexAction; import org.elasticsearch.action.index.TransportIndexAction;
import org.elasticsearch.action.support.broadcast.BroadcastResponse;
import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.block.ClusterBlockException; import org.elasticsearch.cluster.block.ClusterBlockException;
import org.elasticsearch.cluster.block.ClusterBlocks; import org.elasticsearch.cluster.block.ClusterBlocks;
@ -178,28 +177,34 @@ public class GeoIpDownloaderTests extends ESTestCase {
} }
public void testIndexChunksNoData() throws IOException { public void testIndexChunksNoData() throws IOException {
client.addHandler(FlushAction.INSTANCE, (FlushRequest request, ActionListener<FlushResponse> flushResponseActionListener) -> { client.addHandler(FlushAction.INSTANCE, (FlushRequest request, ActionListener<BroadcastResponse> flushResponseActionListener) -> {
assertArrayEquals(new String[] { GeoIpDownloader.DATABASES_INDEX }, request.indices()); assertArrayEquals(new String[] { GeoIpDownloader.DATABASES_INDEX }, request.indices());
flushResponseActionListener.onResponse(mock(FlushResponse.class)); flushResponseActionListener.onResponse(mock(BroadcastResponse.class));
}); });
client.addHandler(RefreshAction.INSTANCE, (RefreshRequest request, ActionListener<RefreshResponse> flushResponseActionListener) -> { client.addHandler(
RefreshAction.INSTANCE,
(RefreshRequest request, ActionListener<BroadcastResponse> flushResponseActionListener) -> {
assertArrayEquals(new String[] { GeoIpDownloader.DATABASES_INDEX }, request.indices()); assertArrayEquals(new String[] { GeoIpDownloader.DATABASES_INDEX }, request.indices());
flushResponseActionListener.onResponse(mock(RefreshResponse.class)); flushResponseActionListener.onResponse(mock(BroadcastResponse.class));
}); }
);
InputStream empty = new ByteArrayInputStream(new byte[0]); InputStream empty = new ByteArrayInputStream(new byte[0]);
assertEquals(0, geoIpDownloader.indexChunks("test", empty, 0, "d41d8cd98f00b204e9800998ecf8427e", 0)); assertEquals(0, geoIpDownloader.indexChunks("test", empty, 0, "d41d8cd98f00b204e9800998ecf8427e", 0));
} }
public void testIndexChunksMd5Mismatch() { public void testIndexChunksMd5Mismatch() {
client.addHandler(FlushAction.INSTANCE, (FlushRequest request, ActionListener<FlushResponse> flushResponseActionListener) -> { client.addHandler(FlushAction.INSTANCE, (FlushRequest request, ActionListener<BroadcastResponse> flushResponseActionListener) -> {
assertArrayEquals(new String[] { GeoIpDownloader.DATABASES_INDEX }, request.indices()); assertArrayEquals(new String[] { GeoIpDownloader.DATABASES_INDEX }, request.indices());
flushResponseActionListener.onResponse(mock(FlushResponse.class)); flushResponseActionListener.onResponse(mock(BroadcastResponse.class));
}); });
client.addHandler(RefreshAction.INSTANCE, (RefreshRequest request, ActionListener<RefreshResponse> flushResponseActionListener) -> { client.addHandler(
RefreshAction.INSTANCE,
(RefreshRequest request, ActionListener<BroadcastResponse> flushResponseActionListener) -> {
assertArrayEquals(new String[] { GeoIpDownloader.DATABASES_INDEX }, request.indices()); assertArrayEquals(new String[] { GeoIpDownloader.DATABASES_INDEX }, request.indices());
flushResponseActionListener.onResponse(mock(RefreshResponse.class)); flushResponseActionListener.onResponse(mock(BroadcastResponse.class));
}); }
);
IOException exception = expectThrows( IOException exception = expectThrows(
IOException.class, IOException.class,
@ -232,14 +237,17 @@ public class GeoIpDownloaderTests extends ESTestCase {
assertEquals(chunk + 15, source.get("chunk")); assertEquals(chunk + 15, source.get("chunk"));
listener.onResponse(mock(IndexResponse.class)); listener.onResponse(mock(IndexResponse.class));
}); });
client.addHandler(FlushAction.INSTANCE, (FlushRequest request, ActionListener<FlushResponse> flushResponseActionListener) -> { client.addHandler(FlushAction.INSTANCE, (FlushRequest request, ActionListener<BroadcastResponse> flushResponseActionListener) -> {
assertArrayEquals(new String[] { GeoIpDownloader.DATABASES_INDEX }, request.indices()); assertArrayEquals(new String[] { GeoIpDownloader.DATABASES_INDEX }, request.indices());
flushResponseActionListener.onResponse(mock(FlushResponse.class)); flushResponseActionListener.onResponse(mock(BroadcastResponse.class));
}); });
client.addHandler(RefreshAction.INSTANCE, (RefreshRequest request, ActionListener<RefreshResponse> flushResponseActionListener) -> { client.addHandler(
RefreshAction.INSTANCE,
(RefreshRequest request, ActionListener<BroadcastResponse> flushResponseActionListener) -> {
assertArrayEquals(new String[] { GeoIpDownloader.DATABASES_INDEX }, request.indices()); assertArrayEquals(new String[] { GeoIpDownloader.DATABASES_INDEX }, request.indices());
flushResponseActionListener.onResponse(mock(RefreshResponse.class)); flushResponseActionListener.onResponse(mock(BroadcastResponse.class));
}); }
);
InputStream big = new ByteArrayInputStream(bigArray); InputStream big = new ByteArrayInputStream(bigArray);
assertEquals(17, geoIpDownloader.indexChunks("test", big, 15, "a67563dfa8f3cba8b8cff61eb989a749", 0)); assertEquals(17, geoIpDownloader.indexChunks("test", big, 15, "a67563dfa8f3cba8b8cff61eb989a749", 0));

View file

@ -13,7 +13,6 @@ import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.DocWriteRequest; import org.elasticsearch.action.DocWriteRequest;
import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.DocWriteResponse;
import org.elasticsearch.action.admin.indices.refresh.RefreshRequest; import org.elasticsearch.action.admin.indices.refresh.RefreshRequest;
import org.elasticsearch.action.admin.indices.refresh.RefreshResponse;
import org.elasticsearch.action.bulk.BackoffPolicy; import org.elasticsearch.action.bulk.BackoffPolicy;
import org.elasticsearch.action.bulk.BulkItemResponse; import org.elasticsearch.action.bulk.BulkItemResponse;
import org.elasticsearch.action.bulk.BulkItemResponse.Failure; import org.elasticsearch.action.bulk.BulkItemResponse.Failure;
@ -24,6 +23,7 @@ import org.elasticsearch.action.delete.DeleteRequest;
import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.index.IndexRequest;
import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchRequest;
import org.elasticsearch.action.support.TransportAction; import org.elasticsearch.action.support.TransportAction;
import org.elasticsearch.action.support.broadcast.BroadcastResponse;
import org.elasticsearch.client.internal.ParentTaskAssigningClient; import org.elasticsearch.client.internal.ParentTaskAssigningClient;
import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.common.util.concurrent.AbstractRunnable; import org.elasticsearch.common.util.concurrent.AbstractRunnable;
@ -554,9 +554,9 @@ public abstract class AbstractAsyncBulkByScrollAction<
RefreshRequest refresh = new RefreshRequest(); RefreshRequest refresh = new RefreshRequest();
refresh.indices(destinationIndices.toArray(new String[destinationIndices.size()])); refresh.indices(destinationIndices.toArray(new String[destinationIndices.size()]));
logger.debug("[{}]: refreshing", task.getId()); logger.debug("[{}]: refreshing", task.getId());
bulkClient.admin().indices().refresh(refresh, new ActionListener<RefreshResponse>() { bulkClient.admin().indices().refresh(refresh, new ActionListener<>() {
@Override @Override
public void onResponse(RefreshResponse response) { public void onResponse(BroadcastResponse response) {
finishHim(null, indexingFailures, searchFailures, timedOut); finishHim(null, indexingFailures, searchFailures, timedOut);
} }

View file

@ -15,8 +15,8 @@ import com.sun.net.httpserver.HttpExchange;
import com.sun.net.httpserver.HttpHandler; import com.sun.net.httpserver.HttpHandler;
import org.elasticsearch.action.ActionRunnable; import org.elasticsearch.action.ActionRunnable;
import org.elasticsearch.action.admin.indices.forcemerge.ForceMergeResponse;
import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.action.support.PlainActionFuture;
import org.elasticsearch.action.support.broadcast.BroadcastResponse;
import org.elasticsearch.cluster.metadata.RepositoryMetadata; import org.elasticsearch.cluster.metadata.RepositoryMetadata;
import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.blobstore.BlobContainer; import org.elasticsearch.common.blobstore.BlobContainer;
@ -191,7 +191,7 @@ public class S3BlobStoreRepositoryTests extends ESMockAPIBasedRepositoryIntegTes
waitForDocs(nbDocs, indexer); waitForDocs(nbDocs, indexer);
} }
flushAndRefresh(index); flushAndRefresh(index);
ForceMergeResponse forceMerge = client().admin().indices().prepareForceMerge(index).setFlush(true).setMaxNumSegments(1).get(); BroadcastResponse forceMerge = client().admin().indices().prepareForceMerge(index).setFlush(true).setMaxNumSegments(1).get();
assertThat(forceMerge.getSuccessfulShards(), equalTo(1)); assertThat(forceMerge.getSuccessfulShards(), equalTo(1));
assertHitCount(prepareSearch(index).setSize(0).setTrackTotalHits(true), nbDocs); assertHitCount(prepareSearch(index).setSize(0).setTrackTotalHits(true), nbDocs);
@ -234,7 +234,7 @@ public class S3BlobStoreRepositoryTests extends ESMockAPIBasedRepositoryIntegTes
waitForDocs(nbDocs, indexer); waitForDocs(nbDocs, indexer);
} }
flushAndRefresh(index); flushAndRefresh(index);
ForceMergeResponse forceMerge = client().admin().indices().prepareForceMerge(index).setFlush(true).setMaxNumSegments(1).get(); BroadcastResponse forceMerge = client().admin().indices().prepareForceMerge(index).setFlush(true).setMaxNumSegments(1).get();
assertThat(forceMerge.getSuccessfulShards(), equalTo(1)); assertThat(forceMerge.getSuccessfulShards(), equalTo(1));
assertHitCount(prepareSearch(index).setSize(0).setTrackTotalHits(true), nbDocs); assertHitCount(prepareSearch(index).setSize(0).setTrackTotalHits(true), nbDocs);

View file

@ -17,8 +17,8 @@ import org.apache.http.nio.entity.NByteArrayEntity;
import org.apache.lucene.search.join.ScoreMode; import org.apache.lucene.search.join.ScoreMode;
import org.apache.lucene.tests.util.TimeUnits; import org.apache.lucene.tests.util.TimeUnits;
import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.BytesRef;
import org.elasticsearch.action.admin.indices.refresh.RefreshResponse;
import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchRequest;
import org.elasticsearch.action.support.broadcast.BroadcastResponse;
import org.elasticsearch.aggregations.pipeline.DerivativePipelineAggregationBuilder; import org.elasticsearch.aggregations.pipeline.DerivativePipelineAggregationBuilder;
import org.elasticsearch.client.Request; import org.elasticsearch.client.Request;
import org.elasticsearch.client.Response; import org.elasticsearch.client.Response;
@ -199,7 +199,7 @@ public class CCSDuelIT extends ESRestTestCase {
assertTrue(latch.await(30, TimeUnit.SECONDS)); assertTrue(latch.await(30, TimeUnit.SECONDS));
RefreshResponse refreshResponse = refresh(INDEX_NAME); BroadcastResponse refreshResponse = refresh(INDEX_NAME);
ElasticsearchAssertions.assertNoFailures(refreshResponse); ElasticsearchAssertions.assertNoFailures(refreshResponse);
} }

View file

@ -8,6 +8,7 @@
package org.elasticsearch.action.admin.indices.cache.clear; package org.elasticsearch.action.admin.indices.cache.clear;
import org.elasticsearch.action.support.broadcast.BroadcastResponse;
import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.ESIntegTestCase;
import org.elasticsearch.test.ESIntegTestCase.ClusterScope; import org.elasticsearch.test.ESIntegTestCase.ClusterScope;
@ -33,7 +34,7 @@ public class ClearIndicesCacheBlocksIT extends ESIntegTestCase {
for (String blockSetting : Arrays.asList(SETTING_BLOCKS_READ, SETTING_BLOCKS_WRITE)) { for (String blockSetting : Arrays.asList(SETTING_BLOCKS_READ, SETTING_BLOCKS_WRITE)) {
try { try {
enableIndexBlock("test", blockSetting); enableIndexBlock("test", blockSetting);
ClearIndicesCacheResponse clearIndicesCacheResponse = indicesAdmin().prepareClearCache("test") BroadcastResponse clearIndicesCacheResponse = indicesAdmin().prepareClearCache("test")
.setFieldDataCache(true) .setFieldDataCache(true)
.setQueryCache(true) .setQueryCache(true)
.setFieldDataCache(true) .setFieldDataCache(true)

View file

@ -8,6 +8,7 @@
package org.elasticsearch.action.admin.indices.flush; package org.elasticsearch.action.admin.indices.flush;
import org.elasticsearch.action.support.broadcast.BroadcastResponse;
import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.ESIntegTestCase;
import org.elasticsearch.test.ESIntegTestCase.ClusterScope; import org.elasticsearch.test.ESIntegTestCase.ClusterScope;
@ -44,7 +45,7 @@ public class FlushBlocksIT extends ESIntegTestCase {
)) { )) {
try { try {
enableIndexBlock("test", blockSetting); enableIndexBlock("test", blockSetting);
FlushResponse response = indicesAdmin().prepareFlush("test").get(); BroadcastResponse response = indicesAdmin().prepareFlush("test").get();
assertNoFailures(response); assertNoFailures(response);
assertThat(response.getSuccessfulShards(), equalTo(numShards.totalNumShards)); assertThat(response.getSuccessfulShards(), equalTo(numShards.totalNumShards));
} finally { } finally {

View file

@ -8,6 +8,7 @@
package org.elasticsearch.action.admin.indices.forcemerge; package org.elasticsearch.action.admin.indices.forcemerge;
import org.elasticsearch.action.support.broadcast.BaseBroadcastResponse;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.ESIntegTestCase;
import org.elasticsearch.test.ESIntegTestCase.ClusterScope; import org.elasticsearch.test.ESIntegTestCase.ClusterScope;
@ -50,7 +51,7 @@ public class ForceMergeBlocksIT extends ESIntegTestCase {
for (String blockSetting : Arrays.asList(SETTING_BLOCKS_READ, SETTING_BLOCKS_WRITE, SETTING_READ_ONLY_ALLOW_DELETE)) { for (String blockSetting : Arrays.asList(SETTING_BLOCKS_READ, SETTING_BLOCKS_WRITE, SETTING_READ_ONLY_ALLOW_DELETE)) {
try { try {
enableIndexBlock("test", blockSetting); enableIndexBlock("test", blockSetting);
ForceMergeResponse response = indicesAdmin().prepareForceMerge("test").get(); BaseBroadcastResponse response = indicesAdmin().prepareForceMerge("test").get();
assertNoFailures(response); assertNoFailures(response);
assertThat(response.getSuccessfulShards(), equalTo(numShards.totalNumShards)); assertThat(response.getSuccessfulShards(), equalTo(numShards.totalNumShards));
} finally { } finally {
@ -70,7 +71,7 @@ public class ForceMergeBlocksIT extends ESIntegTestCase {
// Merging all indices is blocked when the cluster is read-only // Merging all indices is blocked when the cluster is read-only
try { try {
ForceMergeResponse response = indicesAdmin().prepareForceMerge().get(); BaseBroadcastResponse response = indicesAdmin().prepareForceMerge().get();
assertNoFailures(response); assertNoFailures(response);
assertThat(response.getSuccessfulShards(), equalTo(numShards.totalNumShards)); assertThat(response.getSuccessfulShards(), equalTo(numShards.totalNumShards));

View file

@ -8,7 +8,7 @@
package org.elasticsearch.action.admin.indices.forcemerge; package org.elasticsearch.action.admin.indices.forcemerge;
import org.elasticsearch.action.admin.indices.flush.FlushResponse; import org.elasticsearch.action.support.broadcast.BroadcastResponse;
import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.routing.IndexRoutingTable; import org.elasticsearch.cluster.routing.IndexRoutingTable;
import org.elasticsearch.cluster.routing.IndexShardRoutingTable; import org.elasticsearch.cluster.routing.IndexShardRoutingTable;
@ -51,13 +51,13 @@ public class ForceMergeIT extends ESIntegTestCase {
assertThat(getForceMergeUUID(primary), nullValue()); assertThat(getForceMergeUUID(primary), nullValue());
assertThat(getForceMergeUUID(replica), nullValue()); assertThat(getForceMergeUUID(replica), nullValue());
final ForceMergeResponse forceMergeResponse = indicesAdmin().prepareForceMerge(index).setMaxNumSegments(1).get(); final BroadcastResponse forceMergeResponse = indicesAdmin().prepareForceMerge(index).setMaxNumSegments(1).get();
assertThat(forceMergeResponse.getFailedShards(), is(0)); assertThat(forceMergeResponse.getFailedShards(), is(0));
assertThat(forceMergeResponse.getSuccessfulShards(), is(2)); assertThat(forceMergeResponse.getSuccessfulShards(), is(2));
// Force flush to force a new commit that contains the force flush UUID // Force flush to force a new commit that contains the force flush UUID
final FlushResponse flushResponse = indicesAdmin().prepareFlush(index).setForce(true).get(); final BroadcastResponse flushResponse = indicesAdmin().prepareFlush(index).setForce(true).get();
assertThat(flushResponse.getFailedShards(), is(0)); assertThat(flushResponse.getFailedShards(), is(0));
assertThat(flushResponse.getSuccessfulShards(), is(2)); assertThat(flushResponse.getSuccessfulShards(), is(2));

View file

@ -8,6 +8,7 @@
package org.elasticsearch.action.admin.indices.refresh; package org.elasticsearch.action.admin.indices.refresh;
import org.elasticsearch.action.support.broadcast.BroadcastResponse;
import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.ESIntegTestCase;
import org.elasticsearch.test.ESIntegTestCase.ClusterScope; import org.elasticsearch.test.ESIntegTestCase.ClusterScope;
@ -39,7 +40,7 @@ public class RefreshBlocksIT extends ESIntegTestCase {
)) { )) {
try { try {
enableIndexBlock("test", blockSetting); enableIndexBlock("test", blockSetting);
RefreshResponse response = indicesAdmin().prepareRefresh("test").get(); BroadcastResponse response = indicesAdmin().prepareRefresh("test").get();
assertNoFailures(response); assertNoFailures(response);
assertThat(response.getSuccessfulShards(), equalTo(numShards.totalNumShards)); assertThat(response.getSuccessfulShards(), equalTo(numShards.totalNumShards));
} finally { } finally {

View file

@ -12,12 +12,12 @@ import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.Logger;
import org.elasticsearch.action.ActionResponse; import org.elasticsearch.action.ActionResponse;
import org.elasticsearch.action.admin.cluster.shards.ClusterSearchShardsGroup; import org.elasticsearch.action.admin.cluster.shards.ClusterSearchShardsGroup;
import org.elasticsearch.action.admin.indices.refresh.RefreshResponse;
import org.elasticsearch.action.admin.indices.refresh.TransportUnpromotableShardRefreshAction; import org.elasticsearch.action.admin.indices.refresh.TransportUnpromotableShardRefreshAction;
import org.elasticsearch.action.search.ClosePointInTimeRequest; import org.elasticsearch.action.search.ClosePointInTimeRequest;
import org.elasticsearch.action.search.OpenPointInTimeRequest; import org.elasticsearch.action.search.OpenPointInTimeRequest;
import org.elasticsearch.action.search.TransportClosePointInTimeAction; import org.elasticsearch.action.search.TransportClosePointInTimeAction;
import org.elasticsearch.action.search.TransportOpenPointInTimeAction; import org.elasticsearch.action.search.TransportOpenPointInTimeAction;
import org.elasticsearch.action.support.broadcast.BroadcastResponse;
import org.elasticsearch.cluster.ClusterChangedEvent; import org.elasticsearch.cluster.ClusterChangedEvent;
import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.ClusterStateListener; import org.elasticsearch.cluster.ClusterStateListener;
@ -699,7 +699,7 @@ public class ShardRoutingRoleIT extends ESIntegTestCase {
}); });
} }
RefreshResponse response = indicesAdmin().prepareRefresh(INDEX_NAME).get(); BroadcastResponse response = indicesAdmin().prepareRefresh(INDEX_NAME).get();
assertThat( assertThat(
"each unpromotable replica shard should be added to the shard failures", "each unpromotable replica shard should be added to the shard failures",
response.getFailedShards(), response.getFailedShards(),

View file

@ -10,17 +10,15 @@ package org.elasticsearch.document;
import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.DocWriteResponse;
import org.elasticsearch.action.admin.indices.cache.clear.ClearIndicesCacheRequest; 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;
import org.elasticsearch.action.admin.indices.forcemerge.ForceMergeResponse;
import org.elasticsearch.action.admin.indices.refresh.RefreshRequest; import org.elasticsearch.action.admin.indices.refresh.RefreshRequest;
import org.elasticsearch.action.admin.indices.refresh.RefreshResponse;
import org.elasticsearch.action.bulk.BulkResponse; import org.elasticsearch.action.bulk.BulkResponse;
import org.elasticsearch.action.delete.DeleteResponse; import org.elasticsearch.action.delete.DeleteResponse;
import org.elasticsearch.action.get.GetRequest; import org.elasticsearch.action.get.GetRequest;
import org.elasticsearch.action.get.GetResponse; import org.elasticsearch.action.get.GetResponse;
import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.index.IndexRequest;
import org.elasticsearch.action.support.WriteRequest.RefreshPolicy; import org.elasticsearch.action.support.WriteRequest.RefreshPolicy;
import org.elasticsearch.action.support.broadcast.BaseBroadcastResponse;
import org.elasticsearch.action.support.broadcast.BroadcastResponse;
import org.elasticsearch.cluster.health.ClusterHealthStatus; import org.elasticsearch.cluster.health.ClusterHealthStatus;
import org.elasticsearch.common.Strings; import org.elasticsearch.common.Strings;
import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.ESIntegTestCase;
@ -63,7 +61,7 @@ public class DocumentActionsIT extends ESIntegTestCase {
assertThat(indexResponse.getIndex(), equalTo(getConcreteIndexName())); assertThat(indexResponse.getIndex(), equalTo(getConcreteIndexName()));
assertThat(indexResponse.getId(), equalTo("1")); assertThat(indexResponse.getId(), equalTo("1"));
logger.info("Refreshing"); logger.info("Refreshing");
RefreshResponse refreshResponse = refresh(); BroadcastResponse refreshResponse = refresh();
assertThat(refreshResponse.getSuccessfulShards(), equalTo(numShards.totalNumShards)); assertThat(refreshResponse.getSuccessfulShards(), equalTo(numShards.totalNumShards));
logger.info("--> index exists?"); logger.info("--> index exists?");
@ -72,7 +70,7 @@ public class DocumentActionsIT extends ESIntegTestCase {
assertThat(indexExists("test1234565"), equalTo(false)); assertThat(indexExists("test1234565"), equalTo(false));
logger.info("Clearing cache"); logger.info("Clearing cache");
ClearIndicesCacheResponse clearIndicesCacheResponse = indicesAdmin().clearCache( BroadcastResponse clearIndicesCacheResponse = indicesAdmin().clearCache(
new ClearIndicesCacheRequest("test").fieldDataCache(true).queryCache(true) new ClearIndicesCacheRequest("test").fieldDataCache(true).queryCache(true)
).actionGet(); ).actionGet();
assertNoFailures(clearIndicesCacheResponse); assertNoFailures(clearIndicesCacheResponse);
@ -80,7 +78,7 @@ public class DocumentActionsIT extends ESIntegTestCase {
logger.info("Force Merging"); logger.info("Force Merging");
waitForRelocation(ClusterHealthStatus.GREEN); waitForRelocation(ClusterHealthStatus.GREEN);
ForceMergeResponse mergeResponse = forceMerge(); BaseBroadcastResponse mergeResponse = forceMerge();
assertThat(mergeResponse.getSuccessfulShards(), equalTo(numShards.totalNumShards)); assertThat(mergeResponse.getSuccessfulShards(), equalTo(numShards.totalNumShards));
GetResponse getResult; GetResponse getResult;
@ -130,7 +128,7 @@ public class DocumentActionsIT extends ESIntegTestCase {
client().index(new IndexRequest("test").id("2").source(source("2", "test2"))).actionGet(); client().index(new IndexRequest("test").id("2").source(source("2", "test2"))).actionGet();
logger.info("Flushing"); logger.info("Flushing");
FlushResponse flushResult = indicesAdmin().prepareFlush("test").get(); BroadcastResponse flushResult = indicesAdmin().prepareFlush("test").get();
assertThat(flushResult.getSuccessfulShards(), equalTo(numShards.totalNumShards)); assertThat(flushResult.getSuccessfulShards(), equalTo(numShards.totalNumShards));
assertThat(flushResult.getFailedShards(), equalTo(0)); assertThat(flushResult.getFailedShards(), equalTo(0));
logger.info("Refreshing"); logger.info("Refreshing");
@ -220,7 +218,7 @@ public class DocumentActionsIT extends ESIntegTestCase {
assertThat(bulkResponse.getItems()[5].getIndex(), equalTo(getConcreteIndexName())); assertThat(bulkResponse.getItems()[5].getIndex(), equalTo(getConcreteIndexName()));
waitForRelocation(ClusterHealthStatus.GREEN); waitForRelocation(ClusterHealthStatus.GREEN);
RefreshResponse refreshResponse = indicesAdmin().prepareRefresh("test").get(); BroadcastResponse refreshResponse = indicesAdmin().prepareRefresh("test").get();
assertNoFailures(refreshResponse); assertNoFailures(refreshResponse);
assertThat(refreshResponse.getSuccessfulShards(), equalTo(numShards.totalNumShards)); assertThat(refreshResponse.getSuccessfulShards(), equalTo(numShards.totalNumShards));

View file

@ -10,7 +10,6 @@ package org.elasticsearch.get;
import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.DocWriteResponse;
import org.elasticsearch.action.admin.indices.alias.Alias; import org.elasticsearch.action.admin.indices.alias.Alias;
import org.elasticsearch.action.admin.indices.flush.FlushResponse;
import org.elasticsearch.action.delete.DeleteResponse; import org.elasticsearch.action.delete.DeleteResponse;
import org.elasticsearch.action.get.GetRequestBuilder; import org.elasticsearch.action.get.GetRequestBuilder;
import org.elasticsearch.action.get.GetResponse; import org.elasticsearch.action.get.GetResponse;
@ -18,6 +17,7 @@ import org.elasticsearch.action.get.MultiGetRequest;
import org.elasticsearch.action.get.MultiGetRequestBuilder; import org.elasticsearch.action.get.MultiGetRequestBuilder;
import org.elasticsearch.action.get.MultiGetResponse; import org.elasticsearch.action.get.MultiGetResponse;
import org.elasticsearch.action.support.DefaultShardOperationFailedException; import org.elasticsearch.action.support.DefaultShardOperationFailedException;
import org.elasticsearch.action.support.broadcast.BroadcastResponse;
import org.elasticsearch.common.Strings; import org.elasticsearch.common.Strings;
import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.lucene.uid.Versions; import org.elasticsearch.common.lucene.uid.Versions;
@ -641,7 +641,7 @@ public class GetActionIT extends ESIntegTestCase {
ensureGreen(); ensureGreen();
logger.info("flushing"); logger.info("flushing");
FlushResponse flushResponse = indicesAdmin().prepareFlush("my-index").setForce(true).get(); BroadcastResponse flushResponse = indicesAdmin().prepareFlush("my-index").setForce(true).get();
if (flushResponse.getSuccessfulShards() == 0) { if (flushResponse.getSuccessfulShards() == 0) {
StringBuilder sb = new StringBuilder("failed to flush at least one shard. total shards [").append( StringBuilder sb = new StringBuilder("failed to flush at least one shard. total shards [").append(
flushResponse.getTotalShards() flushResponse.getTotalShards()

View file

@ -7,7 +7,7 @@
*/ */
package org.elasticsearch.indices; package org.elasticsearch.indices;
import org.elasticsearch.action.admin.indices.forcemerge.ForceMergeResponse; import org.elasticsearch.action.support.broadcast.BaseBroadcastResponse;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.ByteSizeUnit; import org.elasticsearch.common.unit.ByteSizeUnit;
import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.util.BigArrays;
@ -99,7 +99,7 @@ public class IndexingMemoryControllerIT extends ESSingleNodeTestCase {
prepareIndex("index").setId(Integer.toString(i)).setSource("field", "value").get(); prepareIndex("index").setId(Integer.toString(i)).setSource("field", "value").get();
} }
// Force merge so we know all merges are done before we start deleting: // Force merge so we know all merges are done before we start deleting:
ForceMergeResponse r = client().admin().indices().prepareForceMerge().setMaxNumSegments(1).get(); BaseBroadcastResponse r = client().admin().indices().prepareForceMerge().setMaxNumSegments(1).get();
assertNoFailures(r); assertNoFailures(r);
final RefreshStats refreshStats = shard.refreshStats(); final RefreshStats refreshStats = shard.refreshStats();
for (int i = 0; i < 100; i++) { for (int i = 0; i < 100; i++) {

View file

@ -9,8 +9,8 @@
package org.elasticsearch.indices; package org.elasticsearch.indices;
import org.elasticsearch.action.admin.indices.alias.Alias; import org.elasticsearch.action.admin.indices.alias.Alias;
import org.elasticsearch.action.admin.indices.forcemerge.ForceMergeResponse;
import org.elasticsearch.action.search.SearchType; import org.elasticsearch.action.search.SearchType;
import org.elasticsearch.action.support.broadcast.BroadcastResponse;
import org.elasticsearch.client.internal.Client; import org.elasticsearch.client.internal.Client;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.time.DateFormatter; import org.elasticsearch.common.time.DateFormatter;
@ -132,7 +132,7 @@ public class IndicesRequestCacheIT extends ESIntegTestCase {
assertCacheState(client, "index", 0, 0); assertCacheState(client, "index", 0, 0);
// Force merge the index to ensure there can be no background merges during the subsequent searches that would invalidate the cache // Force merge the index to ensure there can be no background merges during the subsequent searches that would invalidate the cache
ForceMergeResponse forceMergeResponse = indicesAdmin().prepareForceMerge("index").setFlush(true).get(); BroadcastResponse forceMergeResponse = indicesAdmin().prepareForceMerge("index").setFlush(true).get();
ElasticsearchAssertions.assertAllSuccessful(forceMergeResponse); ElasticsearchAssertions.assertAllSuccessful(forceMergeResponse);
refresh(); refresh();
ensureSearchable("index"); ensureSearchable("index");
@ -202,7 +202,7 @@ public class IndicesRequestCacheIT extends ESIntegTestCase {
assertCacheState(client, "index", 0, 0); assertCacheState(client, "index", 0, 0);
// Force merge the index to ensure there can be no background merges during the subsequent searches that would invalidate the cache // Force merge the index to ensure there can be no background merges during the subsequent searches that would invalidate the cache
ForceMergeResponse forceMergeResponse = indicesAdmin().prepareForceMerge("index").setFlush(true).get(); BroadcastResponse forceMergeResponse = indicesAdmin().prepareForceMerge("index").setFlush(true).get();
ElasticsearchAssertions.assertAllSuccessful(forceMergeResponse); ElasticsearchAssertions.assertAllSuccessful(forceMergeResponse);
refresh(); refresh();
ensureSearchable("index"); ensureSearchable("index");
@ -269,7 +269,7 @@ public class IndicesRequestCacheIT extends ESIntegTestCase {
assertCacheState(client, "index", 0, 0); assertCacheState(client, "index", 0, 0);
// Force merge the index to ensure there can be no background merges during the subsequent searches that would invalidate the cache // Force merge the index to ensure there can be no background merges during the subsequent searches that would invalidate the cache
ForceMergeResponse forceMergeResponse = indicesAdmin().prepareForceMerge("index").setFlush(true).get(); BroadcastResponse forceMergeResponse = indicesAdmin().prepareForceMerge("index").setFlush(true).get();
ElasticsearchAssertions.assertAllSuccessful(forceMergeResponse); ElasticsearchAssertions.assertAllSuccessful(forceMergeResponse);
refresh(); refresh();
ensureSearchable("index"); ensureSearchable("index");
@ -343,7 +343,7 @@ public class IndicesRequestCacheIT extends ESIntegTestCase {
assertCacheState(client, "index-3", 0, 0); assertCacheState(client, "index-3", 0, 0);
// Force merge the index to ensure there can be no background merges during the subsequent searches that would invalidate the cache // Force merge the index to ensure there can be no background merges during the subsequent searches that would invalidate the cache
ForceMergeResponse forceMergeResponse = client.admin() BroadcastResponse forceMergeResponse = client.admin()
.indices() .indices()
.prepareForceMerge("index-1", "index-2", "index-3") .prepareForceMerge("index-1", "index-2", "index-3")
.setFlush(true) .setFlush(true)
@ -424,7 +424,7 @@ public class IndicesRequestCacheIT extends ESIntegTestCase {
assertCacheState(client, "index", 0, 0); assertCacheState(client, "index", 0, 0);
// Force merge the index to ensure there can be no background merges during the subsequent searches that would invalidate the cache // Force merge the index to ensure there can be no background merges during the subsequent searches that would invalidate the cache
ForceMergeResponse forceMergeResponse = indicesAdmin().prepareForceMerge("index").setFlush(true).get(); BroadcastResponse forceMergeResponse = indicesAdmin().prepareForceMerge("index").setFlush(true).get();
ElasticsearchAssertions.assertAllSuccessful(forceMergeResponse); ElasticsearchAssertions.assertAllSuccessful(forceMergeResponse);
refresh(); refresh();
ensureSearchable("index"); ensureSearchable("index");
@ -529,7 +529,7 @@ public class IndicesRequestCacheIT extends ESIntegTestCase {
ZonedDateTime now = ZonedDateTime.now(ZoneOffset.UTC); ZonedDateTime now = ZonedDateTime.now(ZoneOffset.UTC);
client.prepareIndex("index").setId("1").setRouting("1").setSource("created_at", DateTimeFormatter.ISO_LOCAL_DATE.format(now)).get(); client.prepareIndex("index").setId("1").setRouting("1").setSource("created_at", DateTimeFormatter.ISO_LOCAL_DATE.format(now)).get();
// Force merge the index to ensure there can be no background merges during the subsequent searches that would invalidate the cache // Force merge the index to ensure there can be no background merges during the subsequent searches that would invalidate the cache
ForceMergeResponse forceMergeResponse = indicesAdmin().prepareForceMerge("index").setFlush(true).get(); BroadcastResponse forceMergeResponse = indicesAdmin().prepareForceMerge("index").setFlush(true).get();
ElasticsearchAssertions.assertAllSuccessful(forceMergeResponse); ElasticsearchAssertions.assertAllSuccessful(forceMergeResponse);
refresh(); refresh();

View file

@ -9,8 +9,8 @@ package org.elasticsearch.indices.flush;
import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.admin.indices.flush.FlushRequest; import org.elasticsearch.action.admin.indices.flush.FlushRequest;
import org.elasticsearch.action.admin.indices.flush.FlushResponse;
import org.elasticsearch.action.admin.indices.stats.ShardStats; import org.elasticsearch.action.admin.indices.stats.ShardStats;
import org.elasticsearch.action.support.broadcast.BroadcastResponse;
import org.elasticsearch.common.ValidationException; import org.elasticsearch.common.ValidationException;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.IndexService; import org.elasticsearch.index.IndexService;
@ -54,9 +54,9 @@ public class FlushIT extends ESIntegTestCase {
final CountDownLatch latch = new CountDownLatch(10); final CountDownLatch latch = new CountDownLatch(10);
final CopyOnWriteArrayList<Throwable> errors = new CopyOnWriteArrayList<>(); final CopyOnWriteArrayList<Throwable> errors = new CopyOnWriteArrayList<>();
for (int j = 0; j < 10; j++) { for (int j = 0; j < 10; j++) {
indicesAdmin().prepareFlush("test").execute(new ActionListener<FlushResponse>() { indicesAdmin().prepareFlush("test").execute(new ActionListener<>() {
@Override @Override
public void onResponse(FlushResponse flushResponse) { public void onResponse(BroadcastResponse flushResponse) {
try { try {
// don't use assertAllSuccessful it uses a randomized context that belongs to a different thread // don't use assertAllSuccessful it uses a randomized context that belongs to a different thread
assertThat( assertThat(

View file

@ -9,8 +9,8 @@
package org.elasticsearch.indices.mapping; package org.elasticsearch.indices.mapping;
import org.elasticsearch.action.admin.indices.mapping.get.GetMappingsResponse; import org.elasticsearch.action.admin.indices.mapping.get.GetMappingsResponse;
import org.elasticsearch.action.admin.indices.refresh.RefreshResponse;
import org.elasticsearch.action.index.IndexRequestBuilder; import org.elasticsearch.action.index.IndexRequestBuilder;
import org.elasticsearch.action.support.broadcast.BroadcastResponse;
import org.elasticsearch.action.support.master.AcknowledgedResponse; import org.elasticsearch.action.support.master.AcknowledgedResponse;
import org.elasticsearch.client.internal.Client; import org.elasticsearch.client.internal.Client;
import org.elasticsearch.cluster.action.index.MappingUpdatedAction; import org.elasticsearch.cluster.action.index.MappingUpdatedAction;
@ -82,7 +82,7 @@ public class UpdateMappingIntegrationIT extends ESIntegTestCase {
indexRandom(true, false, indexRequests); indexRandom(true, false, indexRequests);
logger.info("checking all the documents are there"); logger.info("checking all the documents are there");
RefreshResponse refreshResponse = indicesAdmin().prepareRefresh().get(); BroadcastResponse refreshResponse = indicesAdmin().prepareRefresh().get();
assertThat(refreshResponse.getFailedShards(), equalTo(0)); assertThat(refreshResponse.getFailedShards(), equalTo(0));
assertHitCount(prepareSearch("test").setSize(0), recCount); assertHitCount(prepareSearch("test").setSize(0), recCount);

View file

@ -15,9 +15,9 @@ import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.action.admin.cluster.node.stats.NodeStats; import org.elasticsearch.action.admin.cluster.node.stats.NodeStats;
import org.elasticsearch.action.admin.cluster.node.stats.NodesStatsResponse; import org.elasticsearch.action.admin.cluster.node.stats.NodesStatsResponse;
import org.elasticsearch.action.admin.indices.create.CreateIndexResponse; import org.elasticsearch.action.admin.indices.create.CreateIndexResponse;
import org.elasticsearch.action.admin.indices.refresh.RefreshResponse;
import org.elasticsearch.action.search.SearchPhaseExecutionException; import org.elasticsearch.action.search.SearchPhaseExecutionException;
import org.elasticsearch.action.search.SearchRequestBuilder; import org.elasticsearch.action.search.SearchRequestBuilder;
import org.elasticsearch.action.support.broadcast.BroadcastResponse;
import org.elasticsearch.common.Strings; import org.elasticsearch.common.Strings;
import org.elasticsearch.common.breaker.CircuitBreaker; import org.elasticsearch.common.breaker.CircuitBreaker;
import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Setting;
@ -134,7 +134,7 @@ public class RandomExceptionCircuitBreakerIT extends ESIntegTestCase {
} }
logger.info("Start Refresh"); logger.info("Start Refresh");
// don't assert on failures here // don't assert on failures here
RefreshResponse refreshResponse = indicesAdmin().prepareRefresh("test").execute().get(); BroadcastResponse refreshResponse = indicesAdmin().prepareRefresh("test").execute().get();
final boolean refreshFailed = refreshResponse.getShardFailures().length != 0 || refreshResponse.getFailedShards() != 0; final boolean refreshFailed = refreshResponse.getShardFailures().length != 0 || refreshResponse.getFailedShards() != 0;
logger.info( logger.info(
"Refresh failed: [{}] numShardsFailed: [{}], shardFailuresLength: [{}], successfulShards: [{}], totalShards: [{}] ", "Refresh failed: [{}] numShardsFailed: [{}], shardFailuresLength: [{}], successfulShards: [{}], totalShards: [{}] ",

View file

@ -13,7 +13,6 @@ import org.elasticsearch.action.ActionFuture;
import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.DocWriteResponse;
import org.elasticsearch.action.admin.cluster.node.stats.NodesStatsResponse; import org.elasticsearch.action.admin.cluster.node.stats.NodesStatsResponse;
import org.elasticsearch.action.admin.indices.create.CreateIndexRequest; import org.elasticsearch.action.admin.indices.create.CreateIndexRequest;
import org.elasticsearch.action.admin.indices.forcemerge.ForceMergeResponse;
import org.elasticsearch.action.admin.indices.stats.CommonStats; import org.elasticsearch.action.admin.indices.stats.CommonStats;
import org.elasticsearch.action.admin.indices.stats.CommonStatsFlags; import org.elasticsearch.action.admin.indices.stats.CommonStatsFlags;
import org.elasticsearch.action.admin.indices.stats.CommonStatsFlags.Flag; import org.elasticsearch.action.admin.indices.stats.CommonStatsFlags.Flag;
@ -31,6 +30,7 @@ import org.elasticsearch.action.index.IndexRequestBuilder;
import org.elasticsearch.action.search.SearchType; import org.elasticsearch.action.search.SearchType;
import org.elasticsearch.action.support.DefaultShardOperationFailedException; import org.elasticsearch.action.support.DefaultShardOperationFailedException;
import org.elasticsearch.action.support.WriteRequest; import org.elasticsearch.action.support.WriteRequest;
import org.elasticsearch.action.support.broadcast.BroadcastResponse;
import org.elasticsearch.cluster.metadata.IndexMetadata; import org.elasticsearch.cluster.metadata.IndexMetadata;
import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.common.io.stream.BytesStreamOutput;
@ -1138,7 +1138,7 @@ public class IndexStatsIT extends ESIntegTestCase {
}); });
flush("index"); flush("index");
logger.info("--> force merging to a single segment"); logger.info("--> force merging to a single segment");
ForceMergeResponse forceMergeResponse = indicesAdmin().prepareForceMerge("index").setFlush(true).setMaxNumSegments(1).get(); BroadcastResponse forceMergeResponse = indicesAdmin().prepareForceMerge("index").setFlush(true).setMaxNumSegments(1).get();
assertAllSuccessful(forceMergeResponse); assertAllSuccessful(forceMergeResponse);
logger.info("--> refreshing"); logger.info("--> refreshing");
refresh(); refresh();

View file

@ -8,11 +8,11 @@
package org.elasticsearch.recovery; package org.elasticsearch.recovery;
import org.elasticsearch.action.admin.indices.refresh.RefreshResponse;
import org.elasticsearch.action.admin.indices.stats.IndicesStatsResponse; import org.elasticsearch.action.admin.indices.stats.IndicesStatsResponse;
import org.elasticsearch.action.admin.indices.stats.ShardStats; import org.elasticsearch.action.admin.indices.stats.ShardStats;
import org.elasticsearch.action.get.GetResponse; import org.elasticsearch.action.get.GetResponse;
import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.support.broadcast.BroadcastResponse;
import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.routing.IndexShardRoutingTable; import org.elasticsearch.cluster.routing.IndexShardRoutingTable;
import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.routing.ShardRouting;
@ -405,7 +405,7 @@ public class RecoveryWhileUnderLoadIT extends ESIntegTestCase {
private void refreshAndAssert() throws Exception { private void refreshAndAssert() throws Exception {
assertBusy(() -> { assertBusy(() -> {
RefreshResponse actionGet = indicesAdmin().prepareRefresh().get(); BroadcastResponse actionGet = indicesAdmin().prepareRefresh().get();
assertAllSuccessful(actionGet); assertAllSuccessful(actionGet);
}, 5, TimeUnit.MINUTES); }, 5, TimeUnit.MINUTES);
} }

View file

@ -9,12 +9,11 @@
package org.elasticsearch.recovery; package org.elasticsearch.recovery;
import org.elasticsearch.action.admin.indices.flush.FlushRequest; import org.elasticsearch.action.admin.indices.flush.FlushRequest;
import org.elasticsearch.action.admin.indices.flush.FlushResponse;
import org.elasticsearch.action.admin.indices.refresh.RefreshRequest; import org.elasticsearch.action.admin.indices.refresh.RefreshRequest;
import org.elasticsearch.action.admin.indices.refresh.RefreshResponse;
import org.elasticsearch.action.get.GetRequest; import org.elasticsearch.action.get.GetRequest;
import org.elasticsearch.action.get.GetResponse; import org.elasticsearch.action.get.GetResponse;
import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.index.IndexRequest;
import org.elasticsearch.action.support.broadcast.BroadcastResponse;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.ESIntegTestCase;
import org.elasticsearch.xcontent.XContentType; import org.elasticsearch.xcontent.XContentType;
@ -43,12 +42,12 @@ public class SimpleRecoveryIT extends ESIntegTestCase {
NumShards numShards = getNumShards("test"); NumShards numShards = getNumShards("test");
client().index(new IndexRequest("test").id("1").source(source("1", "test"), XContentType.JSON)).actionGet(); client().index(new IndexRequest("test").id("1").source(source("1", "test"), XContentType.JSON)).actionGet();
FlushResponse flushResponse = indicesAdmin().flush(new FlushRequest("test")).actionGet(); BroadcastResponse flushResponse = indicesAdmin().flush(new FlushRequest("test")).actionGet();
assertThat(flushResponse.getTotalShards(), equalTo(numShards.totalNumShards)); assertThat(flushResponse.getTotalShards(), equalTo(numShards.totalNumShards));
assertThat(flushResponse.getSuccessfulShards(), equalTo(numShards.numPrimaries)); assertThat(flushResponse.getSuccessfulShards(), equalTo(numShards.numPrimaries));
assertThat(flushResponse.getFailedShards(), equalTo(0)); assertThat(flushResponse.getFailedShards(), equalTo(0));
client().index(new IndexRequest("test").id("2").source(source("2", "test"), XContentType.JSON)).actionGet(); client().index(new IndexRequest("test").id("2").source(source("2", "test"), XContentType.JSON)).actionGet();
RefreshResponse refreshResponse = indicesAdmin().refresh(new RefreshRequest("test")).actionGet(); BroadcastResponse refreshResponse = indicesAdmin().refresh(new RefreshRequest("test")).actionGet();
assertThat(refreshResponse.getTotalShards(), equalTo(numShards.totalNumShards)); assertThat(refreshResponse.getTotalShards(), equalTo(numShards.totalNumShards));
assertThat(refreshResponse.getSuccessfulShards(), equalTo(numShards.numPrimaries)); assertThat(refreshResponse.getSuccessfulShards(), equalTo(numShards.numPrimaries));
assertThat(refreshResponse.getFailedShards(), equalTo(0)); assertThat(refreshResponse.getFailedShards(), equalTo(0));

View file

@ -8,7 +8,7 @@
package org.elasticsearch.search.basic; package org.elasticsearch.search.basic;
import org.elasticsearch.action.admin.indices.refresh.RefreshResponse; import org.elasticsearch.action.support.broadcast.BroadcastResponse;
import org.elasticsearch.client.internal.Client; import org.elasticsearch.client.internal.Client;
import org.elasticsearch.cluster.health.ClusterHealthStatus; import org.elasticsearch.cluster.health.ClusterHealthStatus;
import org.elasticsearch.index.query.QueryBuilders; import org.elasticsearch.index.query.QueryBuilders;
@ -55,7 +55,7 @@ public class SearchWhileCreatingIndexIT extends ESIntegTestCase {
createIndex("test"); createIndex("test");
} }
prepareIndex("test").setId(id).setSource("field", "test").get(); prepareIndex("test").setId(id).setSource("field", "test").get();
RefreshResponse refreshResponse = indicesAdmin().prepareRefresh("test").get(); BroadcastResponse refreshResponse = indicesAdmin().prepareRefresh("test").get();
// at least one shard should be successful when refreshing // at least one shard should be successful when refreshing
assertThat(refreshResponse.getSuccessfulShards(), greaterThanOrEqualTo(1)); assertThat(refreshResponse.getSuccessfulShards(), greaterThanOrEqualTo(1));

View file

@ -14,8 +14,8 @@ import org.apache.lucene.index.LeafReader;
import org.apache.lucene.tests.util.English; import org.apache.lucene.tests.util.English;
import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.DocWriteResponse;
import org.elasticsearch.action.admin.indices.refresh.RefreshResponse;
import org.elasticsearch.action.search.SearchPhaseExecutionException; import org.elasticsearch.action.search.SearchPhaseExecutionException;
import org.elasticsearch.action.support.broadcast.BroadcastResponse;
import org.elasticsearch.common.Strings; import org.elasticsearch.common.Strings;
import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Setting.Property; import org.elasticsearch.common.settings.Setting.Property;
@ -111,7 +111,7 @@ public class SearchWithRandomExceptionsIT extends ESIntegTestCase {
} }
logger.info("Start Refresh"); logger.info("Start Refresh");
// don't assert on failures here // don't assert on failures here
RefreshResponse refreshResponse = indicesAdmin().prepareRefresh("test").execute().get(); BroadcastResponse refreshResponse = indicesAdmin().prepareRefresh("test").execute().get();
final boolean refreshFailed = refreshResponse.getShardFailures().length != 0 || refreshResponse.getFailedShards() != 0; final boolean refreshFailed = refreshResponse.getShardFailures().length != 0 || refreshResponse.getFailedShards() != 0;
logger.info( logger.info(
"Refresh failed [{}] numShardsFailed: [{}], shardFailuresLength: [{}], successfulShards: [{}], totalShards: [{}] ", "Refresh failed [{}] numShardsFailed: [{}], shardFailuresLength: [{}], successfulShards: [{}], totalShards: [{}] ",

View file

@ -13,8 +13,8 @@ import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.DocWriteResponse;
import org.elasticsearch.action.admin.cluster.health.ClusterHealthRequest; import org.elasticsearch.action.admin.cluster.health.ClusterHealthRequest;
import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse; import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse;
import org.elasticsearch.action.admin.indices.refresh.RefreshResponse;
import org.elasticsearch.action.search.SearchPhaseExecutionException; import org.elasticsearch.action.search.SearchPhaseExecutionException;
import org.elasticsearch.action.support.broadcast.BroadcastResponse;
import org.elasticsearch.common.Strings; import org.elasticsearch.common.Strings;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.core.TimeValue; import org.elasticsearch.core.TimeValue;
@ -135,7 +135,7 @@ public class SearchWithRandomIOExceptionsIT extends ESIntegTestCase {
ESIntegTestCase.NumShards numShards = getNumShards("test"); ESIntegTestCase.NumShards numShards = getNumShards("test");
logger.info("Start Refresh"); logger.info("Start Refresh");
// don't assert on failures here // don't assert on failures here
final RefreshResponse refreshResponse = indicesAdmin().prepareRefresh("test").execute().get(); final BroadcastResponse refreshResponse = indicesAdmin().prepareRefresh("test").execute().get();
final boolean refreshFailed = refreshResponse.getShardFailures().length != 0 || refreshResponse.getFailedShards() != 0; final boolean refreshFailed = refreshResponse.getShardFailures().length != 0 || refreshResponse.getFailedShards() != 0;
logger.info( logger.info(
"Refresh failed [{}] numShardsFailed: [{}], shardFailuresLength: [{}], successfulShards: [{}], totalShards: [{}] ", "Refresh failed [{}] numShardsFailed: [{}], shardFailuresLength: [{}], successfulShards: [{}], totalShards: [{}] ",

View file

@ -12,10 +12,10 @@ import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.action.admin.cluster.health.ClusterHealthRequest; import org.elasticsearch.action.admin.cluster.health.ClusterHealthRequest;
import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse; import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse;
import org.elasticsearch.action.admin.indices.refresh.RefreshRequest; import org.elasticsearch.action.admin.indices.refresh.RefreshRequest;
import org.elasticsearch.action.admin.indices.refresh.RefreshResponse;
import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.index.IndexRequest;
import org.elasticsearch.action.search.SearchPhaseExecutionException; import org.elasticsearch.action.search.SearchPhaseExecutionException;
import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchRequest;
import org.elasticsearch.action.support.broadcast.BroadcastResponse;
import org.elasticsearch.client.internal.Client; import org.elasticsearch.client.internal.Client;
import org.elasticsearch.cluster.health.ClusterHealthStatus; import org.elasticsearch.cluster.health.ClusterHealthStatus;
import org.elasticsearch.common.Priority; import org.elasticsearch.common.Priority;
@ -50,7 +50,7 @@ public class TransportSearchFailuresIT extends ESIntegTestCase {
for (int i = 0; i < 100; i++) { for (int i = 0; i < 100; i++) {
index(client(), Integer.toString(i), "test", i); index(client(), Integer.toString(i), "test", i);
} }
RefreshResponse refreshResponse = indicesAdmin().refresh(new RefreshRequest("test")).actionGet(); BroadcastResponse refreshResponse = indicesAdmin().refresh(new RefreshRequest("test")).actionGet();
assertThat(refreshResponse.getTotalShards(), equalTo(test.totalNumShards)); assertThat(refreshResponse.getTotalShards(), equalTo(test.totalNumShards));
assertThat(refreshResponse.getSuccessfulShards(), equalTo(test.numPrimaries)); assertThat(refreshResponse.getSuccessfulShards(), equalTo(test.numPrimaries));
assertThat(refreshResponse.getFailedShards(), equalTo(0)); assertThat(refreshResponse.getFailedShards(), equalTo(0));

View file

@ -12,13 +12,13 @@ import com.carrotsearch.randomizedtesting.generators.RandomStrings;
import org.apache.lucene.analysis.TokenStreamToAutomaton; import org.apache.lucene.analysis.TokenStreamToAutomaton;
import org.apache.lucene.search.suggest.document.ContextSuggestField; import org.apache.lucene.search.suggest.document.ContextSuggestField;
import org.apache.lucene.tests.util.LuceneTestCase.SuppressCodecs; import org.apache.lucene.tests.util.LuceneTestCase.SuppressCodecs;
import org.elasticsearch.action.admin.indices.forcemerge.ForceMergeResponse;
import org.elasticsearch.action.admin.indices.segments.IndexShardSegments; import org.elasticsearch.action.admin.indices.segments.IndexShardSegments;
import org.elasticsearch.action.admin.indices.segments.ShardSegments; import org.elasticsearch.action.admin.indices.segments.ShardSegments;
import org.elasticsearch.action.admin.indices.stats.IndicesStatsResponse; import org.elasticsearch.action.admin.indices.stats.IndicesStatsResponse;
import org.elasticsearch.action.index.IndexRequestBuilder; import org.elasticsearch.action.index.IndexRequestBuilder;
import org.elasticsearch.action.search.SearchPhaseExecutionException; import org.elasticsearch.action.search.SearchPhaseExecutionException;
import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.support.broadcast.BroadcastResponse;
import org.elasticsearch.action.support.master.AcknowledgedResponse; import org.elasticsearch.action.support.master.AcknowledgedResponse;
import org.elasticsearch.common.FieldMemoryStats; import org.elasticsearch.common.FieldMemoryStats;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
@ -1267,7 +1267,7 @@ public class CompletionSuggestSearchIT extends ESIntegTestCase {
.get(); .get();
// we have 2 docs in a segment... // we have 2 docs in a segment...
prepareIndex(INDEX).setId("2").setSource(jsonBuilder().startObject().field("somefield", "somevalue").endObject()).get(); prepareIndex(INDEX).setId("2").setSource(jsonBuilder().startObject().field("somefield", "somevalue").endObject()).get();
ForceMergeResponse actionGet = indicesAdmin().prepareForceMerge().setFlush(true).setMaxNumSegments(1).get(); BroadcastResponse actionGet = indicesAdmin().prepareForceMerge().setFlush(true).setMaxNumSegments(1).get();
assertAllSuccessful(actionGet); assertAllSuccessful(actionGet);
refresh(); refresh();
// update the first one and then merge.. the target segment will have no value in FIELD // update the first one and then merge.. the target segment will have no value in FIELD

View file

@ -11,7 +11,6 @@ import org.elasticsearch.action.DocWriteResponse;
import org.elasticsearch.action.admin.cluster.snapshots.restore.RestoreSnapshotResponse; import org.elasticsearch.action.admin.cluster.snapshots.restore.RestoreSnapshotResponse;
import org.elasticsearch.action.admin.cluster.snapshots.status.SnapshotStats; import org.elasticsearch.action.admin.cluster.snapshots.status.SnapshotStats;
import org.elasticsearch.action.admin.cluster.snapshots.status.SnapshotStatus; import org.elasticsearch.action.admin.cluster.snapshots.status.SnapshotStatus;
import org.elasticsearch.action.admin.indices.forcemerge.ForceMergeResponse;
import org.elasticsearch.action.admin.indices.stats.IndexStats; import org.elasticsearch.action.admin.indices.stats.IndexStats;
import org.elasticsearch.action.admin.indices.stats.IndicesStatsResponse; import org.elasticsearch.action.admin.indices.stats.IndicesStatsResponse;
import org.elasticsearch.action.bulk.BulkItemResponse; import org.elasticsearch.action.bulk.BulkItemResponse;
@ -20,6 +19,7 @@ import org.elasticsearch.action.bulk.BulkRequestBuilder;
import org.elasticsearch.action.bulk.BulkResponse; import org.elasticsearch.action.bulk.BulkResponse;
import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.index.IndexRequest;
import org.elasticsearch.action.support.WriteRequest; import org.elasticsearch.action.support.WriteRequest;
import org.elasticsearch.action.support.broadcast.BroadcastResponse;
import org.elasticsearch.cluster.metadata.IndexMetadata; import org.elasticsearch.cluster.metadata.IndexMetadata;
import org.elasticsearch.cluster.routing.UnassignedInfo; import org.elasticsearch.cluster.routing.UnassignedInfo;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
@ -159,7 +159,7 @@ public class BlobStoreIncrementalityIT extends AbstractSnapshotIntegTestCase {
clusterAdmin().prepareCreateSnapshot(repo, snapshot1).setIndices(indexName).setWaitForCompletion(true).get(); clusterAdmin().prepareCreateSnapshot(repo, snapshot1).setIndices(indexName).setWaitForCompletion(true).get();
logger.info("--> force merging down to a single segment"); logger.info("--> force merging down to a single segment");
final ForceMergeResponse forceMergeResponse = indicesAdmin().prepareForceMerge(indexName).setMaxNumSegments(1).setFlush(true).get(); final BroadcastResponse forceMergeResponse = indicesAdmin().prepareForceMerge(indexName).setMaxNumSegments(1).setFlush(true).get();
assertThat(forceMergeResponse.getFailedShards(), is(0)); assertThat(forceMergeResponse.getFailedShards(), is(0));
final String snapshot2 = "snap-2"; final String snapshot2 = "snap-2";

View file

@ -21,11 +21,11 @@ import org.elasticsearch.action.admin.cluster.snapshots.status.SnapshotIndexStat
import org.elasticsearch.action.admin.cluster.snapshots.status.SnapshotStatus; import org.elasticsearch.action.admin.cluster.snapshots.status.SnapshotStatus;
import org.elasticsearch.action.admin.cluster.snapshots.status.SnapshotsStatusResponse; import org.elasticsearch.action.admin.cluster.snapshots.status.SnapshotsStatusResponse;
import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse; import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse;
import org.elasticsearch.action.admin.indices.flush.FlushResponse;
import org.elasticsearch.action.admin.indices.stats.IndicesStatsResponse; import org.elasticsearch.action.admin.indices.stats.IndicesStatsResponse;
import org.elasticsearch.action.admin.indices.stats.ShardStats; import org.elasticsearch.action.admin.indices.stats.ShardStats;
import org.elasticsearch.action.index.IndexRequestBuilder; import org.elasticsearch.action.index.IndexRequestBuilder;
import org.elasticsearch.action.support.ActiveShardCount; import org.elasticsearch.action.support.ActiveShardCount;
import org.elasticsearch.action.support.broadcast.BroadcastResponse;
import org.elasticsearch.client.internal.Client; import org.elasticsearch.client.internal.Client;
import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.RestoreInProgress; import org.elasticsearch.cluster.RestoreInProgress;
@ -119,7 +119,7 @@ public class SharedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTestCas
createIndexWithRandomDocs("test-idx-2", 100); createIndexWithRandomDocs("test-idx-2", 100);
createIndexWithRandomDocs("test-idx-3", 100); createIndexWithRandomDocs("test-idx-3", 100);
ActionFuture<FlushResponse> flushResponseFuture = null; ActionFuture<BroadcastResponse> flushResponseFuture = null;
if (randomBoolean()) { if (randomBoolean()) {
ArrayList<String> indicesToFlush = new ArrayList<>(); ArrayList<String> indicesToFlush = new ArrayList<>();
for (int i = 1; i < 4; i++) { for (int i = 1; i < 4; i++) {

View file

@ -9,13 +9,14 @@
package org.elasticsearch.action.admin.indices.cache.clear; package org.elasticsearch.action.admin.indices.cache.clear;
import org.elasticsearch.action.ActionType; import org.elasticsearch.action.ActionType;
import org.elasticsearch.action.support.broadcast.BroadcastResponse;
public class ClearIndicesCacheAction extends ActionType<ClearIndicesCacheResponse> { public class ClearIndicesCacheAction extends ActionType<BroadcastResponse> {
public static final ClearIndicesCacheAction INSTANCE = new ClearIndicesCacheAction(); public static final ClearIndicesCacheAction INSTANCE = new ClearIndicesCacheAction();
public static final String NAME = "indices:admin/cache/clear"; public static final String NAME = "indices:admin/cache/clear";
private ClearIndicesCacheAction() { private ClearIndicesCacheAction() {
super(NAME, ClearIndicesCacheResponse::new); super(NAME, BroadcastResponse::new);
} }
} }

View file

@ -9,11 +9,12 @@
package org.elasticsearch.action.admin.indices.cache.clear; package org.elasticsearch.action.admin.indices.cache.clear;
import org.elasticsearch.action.support.broadcast.BroadcastOperationRequestBuilder; import org.elasticsearch.action.support.broadcast.BroadcastOperationRequestBuilder;
import org.elasticsearch.action.support.broadcast.BroadcastResponse;
import org.elasticsearch.client.internal.ElasticsearchClient; import org.elasticsearch.client.internal.ElasticsearchClient;
public class ClearIndicesCacheRequestBuilder extends BroadcastOperationRequestBuilder< public class ClearIndicesCacheRequestBuilder extends BroadcastOperationRequestBuilder<
ClearIndicesCacheRequest, ClearIndicesCacheRequest,
ClearIndicesCacheResponse, BroadcastResponse,
ClearIndicesCacheRequestBuilder> { ClearIndicesCacheRequestBuilder> {
public ClearIndicesCacheRequestBuilder(ElasticsearchClient client) { public ClearIndicesCacheRequestBuilder(ElasticsearchClient client) {

View file

@ -1,61 +0,0 @@
/*
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the Elastic License
* 2.0 and the Server Side Public License, v 1; you may not use this file except
* in compliance with, at your election, the Elastic License 2.0 or the Server
* Side Public License, v 1.
*/
package org.elasticsearch.action.admin.indices.cache.clear;
import org.elasticsearch.action.support.DefaultShardOperationFailedException;
import org.elasticsearch.action.support.broadcast.BaseBroadcastResponse;
import org.elasticsearch.action.support.broadcast.BroadcastResponse;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.xcontent.ConstructingObjectParser;
import org.elasticsearch.xcontent.XContentParser;
import java.io.IOException;
import java.util.Arrays;
import java.util.List;
/**
* The response of a clear cache action.
*/
public class ClearIndicesCacheResponse extends BroadcastResponse {
private static final ConstructingObjectParser<ClearIndicesCacheResponse, Void> PARSER = new ConstructingObjectParser<>(
"clear_cache",
true,
arg -> {
BaseBroadcastResponse response = (BaseBroadcastResponse) arg[0];
return new ClearIndicesCacheResponse(
response.getTotalShards(),
response.getSuccessfulShards(),
response.getFailedShards(),
Arrays.asList(response.getShardFailures())
);
}
);
static {
declareBroadcastFields(PARSER);
}
ClearIndicesCacheResponse(StreamInput in) throws IOException {
super(in);
}
ClearIndicesCacheResponse(
int totalShards,
int successfulShards,
int failedShards,
List<DefaultShardOperationFailedException> shardFailures
) {
super(totalShards, successfulShards, failedShards, shardFailures);
}
public static ClearIndicesCacheResponse fromXContent(XContentParser parser) {
return PARSER.apply(parser, null);
}
}

View file

@ -10,6 +10,7 @@ package org.elasticsearch.action.admin.indices.cache.clear;
import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.broadcast.BroadcastResponse;
import org.elasticsearch.action.support.broadcast.node.TransportBroadcastByNodeAction; import org.elasticsearch.action.support.broadcast.node.TransportBroadcastByNodeAction;
import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.block.ClusterBlockException; import org.elasticsearch.cluster.block.ClusterBlockException;
@ -32,7 +33,7 @@ import java.io.IOException;
*/ */
public class TransportClearIndicesCacheAction extends TransportBroadcastByNodeAction< public class TransportClearIndicesCacheAction extends TransportBroadcastByNodeAction<
ClearIndicesCacheRequest, ClearIndicesCacheRequest,
ClearIndicesCacheResponse, BroadcastResponse,
TransportBroadcastByNodeAction.EmptyResult> { TransportBroadcastByNodeAction.EmptyResult> {
private final IndicesService indicesService; private final IndicesService indicesService;
@ -64,11 +65,11 @@ public class TransportClearIndicesCacheAction extends TransportBroadcastByNodeAc
} }
@Override @Override
protected ResponseFactory<ClearIndicesCacheResponse, TransportBroadcastByNodeAction.EmptyResult> getResponseFactory( protected ResponseFactory<BroadcastResponse, TransportBroadcastByNodeAction.EmptyResult> getResponseFactory(
ClearIndicesCacheRequest request, ClearIndicesCacheRequest request,
ClusterState clusterState ClusterState clusterState
) { ) {
return (totalShards, successfulShards, failedShards, responses, shardFailures) -> new ClearIndicesCacheResponse( return (totalShards, successfulShards, failedShards, responses, shardFailures) -> new BroadcastResponse(
totalShards, totalShards,
successfulShards, successfulShards,
failedShards, failedShards,

View file

@ -9,13 +9,14 @@
package org.elasticsearch.action.admin.indices.flush; package org.elasticsearch.action.admin.indices.flush;
import org.elasticsearch.action.ActionType; import org.elasticsearch.action.ActionType;
import org.elasticsearch.action.support.broadcast.BroadcastResponse;
public class FlushAction extends ActionType<FlushResponse> { public class FlushAction extends ActionType<BroadcastResponse> {
public static final FlushAction INSTANCE = new FlushAction(); public static final FlushAction INSTANCE = new FlushAction();
public static final String NAME = "indices:admin/flush"; public static final String NAME = "indices:admin/flush";
private FlushAction() { private FlushAction() {
super(NAME, FlushResponse::new); super(NAME, BroadcastResponse::new);
} }
} }

View file

@ -23,7 +23,6 @@ import static org.elasticsearch.action.ValidateActions.addValidationError;
* memory heuristics in order to automatically trigger flush operations as required in order to clear memory. * memory heuristics in order to automatically trigger flush operations as required in order to clear memory.
* *
* @see org.elasticsearch.client.internal.IndicesAdminClient#flush(FlushRequest) * @see org.elasticsearch.client.internal.IndicesAdminClient#flush(FlushRequest)
* @see FlushResponse
*/ */
public class FlushRequest extends BroadcastRequest<FlushRequest> { public class FlushRequest extends BroadcastRequest<FlushRequest> {

View file

@ -9,9 +9,10 @@
package org.elasticsearch.action.admin.indices.flush; package org.elasticsearch.action.admin.indices.flush;
import org.elasticsearch.action.support.broadcast.BroadcastOperationRequestBuilder; import org.elasticsearch.action.support.broadcast.BroadcastOperationRequestBuilder;
import org.elasticsearch.action.support.broadcast.BroadcastResponse;
import org.elasticsearch.client.internal.ElasticsearchClient; import org.elasticsearch.client.internal.ElasticsearchClient;
public class FlushRequestBuilder extends BroadcastOperationRequestBuilder<FlushRequest, FlushResponse, FlushRequestBuilder> { public class FlushRequestBuilder extends BroadcastOperationRequestBuilder<FlushRequest, BroadcastResponse, FlushRequestBuilder> {
public FlushRequestBuilder(ElasticsearchClient client) { public FlushRequestBuilder(ElasticsearchClient client) {
super(client, FlushAction.INSTANCE, new FlushRequest()); super(client, FlushAction.INSTANCE, new FlushRequest());

View file

@ -1,52 +0,0 @@
/*
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the Elastic License
* 2.0 and the Server Side Public License, v 1; you may not use this file except
* in compliance with, at your election, the Elastic License 2.0 or the Server
* Side Public License, v 1.
*/
package org.elasticsearch.action.admin.indices.flush;
import org.elasticsearch.action.support.DefaultShardOperationFailedException;
import org.elasticsearch.action.support.broadcast.BaseBroadcastResponse;
import org.elasticsearch.action.support.broadcast.BroadcastResponse;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.xcontent.ConstructingObjectParser;
import org.elasticsearch.xcontent.XContentParser;
import java.io.IOException;
import java.util.Arrays;
import java.util.List;
/**
* A response to flush action.
*/
public class FlushResponse extends BroadcastResponse {
private static final ConstructingObjectParser<FlushResponse, Void> PARSER = new ConstructingObjectParser<>("flush", true, arg -> {
BaseBroadcastResponse response = (BaseBroadcastResponse) arg[0];
return new FlushResponse(
response.getTotalShards(),
response.getSuccessfulShards(),
response.getFailedShards(),
Arrays.asList(response.getShardFailures())
);
});
static {
declareBroadcastFields(PARSER);
}
FlushResponse(StreamInput in) throws IOException {
super(in);
}
FlushResponse(int totalShards, int successfulShards, int failedShards, List<DefaultShardOperationFailedException> shardFailures) {
super(totalShards, successfulShards, failedShards, shardFailures);
}
public static FlushResponse fromXContent(XContentParser parser) {
return PARSER.apply(parser, null);
}
}

View file

@ -11,6 +11,7 @@ package org.elasticsearch.action.admin.indices.flush;
import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.DefaultShardOperationFailedException; import org.elasticsearch.action.support.DefaultShardOperationFailedException;
import org.elasticsearch.action.support.broadcast.BroadcastResponse;
import org.elasticsearch.action.support.replication.ReplicationResponse; import org.elasticsearch.action.support.replication.ReplicationResponse;
import org.elasticsearch.action.support.replication.TransportBroadcastReplicationAction; import org.elasticsearch.action.support.replication.TransportBroadcastReplicationAction;
import org.elasticsearch.client.internal.node.NodeClient; import org.elasticsearch.client.internal.node.NodeClient;
@ -28,7 +29,7 @@ import java.util.List;
*/ */
public class TransportFlushAction extends TransportBroadcastReplicationAction< public class TransportFlushAction extends TransportBroadcastReplicationAction<
FlushRequest, FlushRequest,
FlushResponse, BroadcastResponse,
ShardFlushRequest, ShardFlushRequest,
ReplicationResponse> { ReplicationResponse> {
@ -59,12 +60,12 @@ public class TransportFlushAction extends TransportBroadcastReplicationAction<
} }
@Override @Override
protected FlushResponse newResponse( protected BroadcastResponse newResponse(
int successfulShards, int successfulShards,
int failedShards, int failedShards,
int totalNumCopies, int totalNumCopies,
List<DefaultShardOperationFailedException> shardFailures List<DefaultShardOperationFailedException> shardFailures
) { ) {
return new FlushResponse(totalNumCopies, successfulShards, failedShards, shardFailures); return new BroadcastResponse(totalNumCopies, successfulShards, failedShards, shardFailures);
} }
} }

View file

@ -9,13 +9,14 @@
package org.elasticsearch.action.admin.indices.forcemerge; package org.elasticsearch.action.admin.indices.forcemerge;
import org.elasticsearch.action.ActionType; import org.elasticsearch.action.ActionType;
import org.elasticsearch.action.support.broadcast.BroadcastResponse;
public class ForceMergeAction extends ActionType<ForceMergeResponse> { public class ForceMergeAction extends ActionType<BroadcastResponse> {
public static final ForceMergeAction INSTANCE = new ForceMergeAction(); public static final ForceMergeAction INSTANCE = new ForceMergeAction();
public static final String NAME = "indices:admin/forcemerge"; public static final String NAME = "indices:admin/forcemerge";
private ForceMergeAction() { private ForceMergeAction() {
super(NAME, ForceMergeResponse::new); super(NAME, BroadcastResponse::new);
} }
} }

View file

@ -30,7 +30,6 @@ import static org.elasticsearch.action.ValidateActions.addValidationError;
* to execute, and if so, executes it * to execute, and if so, executes it
* *
* @see org.elasticsearch.client.internal.IndicesAdminClient#forceMerge(ForceMergeRequest) * @see org.elasticsearch.client.internal.IndicesAdminClient#forceMerge(ForceMergeRequest)
* @see ForceMergeResponse
*/ */
public class ForceMergeRequest extends BroadcastRequest<ForceMergeRequest> { public class ForceMergeRequest extends BroadcastRequest<ForceMergeRequest> {

View file

@ -9,6 +9,7 @@
package org.elasticsearch.action.admin.indices.forcemerge; package org.elasticsearch.action.admin.indices.forcemerge;
import org.elasticsearch.action.support.broadcast.BroadcastOperationRequestBuilder; import org.elasticsearch.action.support.broadcast.BroadcastOperationRequestBuilder;
import org.elasticsearch.action.support.broadcast.BroadcastResponse;
import org.elasticsearch.client.internal.ElasticsearchClient; import org.elasticsearch.client.internal.ElasticsearchClient;
/** /**
@ -20,7 +21,7 @@ import org.elasticsearch.client.internal.ElasticsearchClient;
*/ */
public class ForceMergeRequestBuilder extends BroadcastOperationRequestBuilder< public class ForceMergeRequestBuilder extends BroadcastOperationRequestBuilder<
ForceMergeRequest, ForceMergeRequest,
ForceMergeResponse, BroadcastResponse,
ForceMergeRequestBuilder> { ForceMergeRequestBuilder> {
public ForceMergeRequestBuilder(ElasticsearchClient client) { public ForceMergeRequestBuilder(ElasticsearchClient client) {

View file

@ -1,61 +0,0 @@
/*
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the Elastic License
* 2.0 and the Server Side Public License, v 1; you may not use this file except
* in compliance with, at your election, the Elastic License 2.0 or the Server
* Side Public License, v 1.
*/
package org.elasticsearch.action.admin.indices.forcemerge;
import org.elasticsearch.action.support.DefaultShardOperationFailedException;
import org.elasticsearch.action.support.broadcast.BaseBroadcastResponse;
import org.elasticsearch.action.support.broadcast.BroadcastResponse;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.xcontent.ConstructingObjectParser;
import org.elasticsearch.xcontent.XContentParser;
import java.io.IOException;
import java.util.Arrays;
import java.util.List;
/**
* A response for force merge action.
*/
public class ForceMergeResponse extends BroadcastResponse {
private static final ConstructingObjectParser<ForceMergeResponse, Void> PARSER = new ConstructingObjectParser<>(
"force_merge",
true,
arg -> {
BaseBroadcastResponse response = (BaseBroadcastResponse) arg[0];
return new ForceMergeResponse(
response.getTotalShards(),
response.getSuccessfulShards(),
response.getFailedShards(),
Arrays.asList(response.getShardFailures())
);
}
);
static {
declareBroadcastFields(PARSER);
}
ForceMergeResponse(StreamInput in) throws IOException {
super(in);
}
public ForceMergeResponse(
int totalShards,
int successfulShards,
int failedShards,
List<DefaultShardOperationFailedException> shardFailures
) {
super(totalShards, successfulShards, failedShards, shardFailures);
}
public static ForceMergeResponse fromXContent(XContentParser parser) {
return PARSER.apply(parser, null);
}
}

View file

@ -11,6 +11,7 @@ package org.elasticsearch.action.admin.indices.forcemerge;
import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.ActionRunnable; import org.elasticsearch.action.ActionRunnable;
import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.broadcast.BroadcastResponse;
import org.elasticsearch.action.support.broadcast.node.TransportBroadcastByNodeAction; import org.elasticsearch.action.support.broadcast.node.TransportBroadcastByNodeAction;
import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.block.ClusterBlockException; import org.elasticsearch.cluster.block.ClusterBlockException;
@ -35,7 +36,7 @@ import java.io.IOException;
*/ */
public class TransportForceMergeAction extends TransportBroadcastByNodeAction< public class TransportForceMergeAction extends TransportBroadcastByNodeAction<
ForceMergeRequest, ForceMergeRequest,
ForceMergeResponse, BroadcastResponse,
TransportBroadcastByNodeAction.EmptyResult> { TransportBroadcastByNodeAction.EmptyResult> {
private final IndicesService indicesService; private final IndicesService indicesService;
@ -68,8 +69,8 @@ public class TransportForceMergeAction extends TransportBroadcastByNodeAction<
} }
@Override @Override
protected ResponseFactory<ForceMergeResponse, EmptyResult> getResponseFactory(ForceMergeRequest request, ClusterState clusterState) { protected ResponseFactory<BroadcastResponse, EmptyResult> getResponseFactory(ForceMergeRequest request, ClusterState clusterState) {
return (totalShards, successfulShards, failedShards, responses, shardFailures) -> new ForceMergeResponse( return (totalShards, successfulShards, failedShards, responses, shardFailures) -> new BroadcastResponse(
totalShards, totalShards,
successfulShards, successfulShards,
failedShards, failedShards,

View file

@ -9,13 +9,14 @@
package org.elasticsearch.action.admin.indices.refresh; package org.elasticsearch.action.admin.indices.refresh;
import org.elasticsearch.action.ActionType; import org.elasticsearch.action.ActionType;
import org.elasticsearch.action.support.broadcast.BroadcastResponse;
public class RefreshAction extends ActionType<RefreshResponse> { public class RefreshAction extends ActionType<BroadcastResponse> {
public static final RefreshAction INSTANCE = new RefreshAction(); public static final RefreshAction INSTANCE = new RefreshAction();
public static final String NAME = "indices:admin/refresh"; public static final String NAME = "indices:admin/refresh";
private RefreshAction() { private RefreshAction() {
super(NAME, RefreshResponse::new); super(NAME, BroadcastResponse::new);
} }
} }

View file

@ -19,7 +19,6 @@ import java.io.IOException;
* default a refresh is scheduled periodically. * default a refresh is scheduled periodically.
* *
* @see org.elasticsearch.client.internal.IndicesAdminClient#refresh(RefreshRequest) * @see org.elasticsearch.client.internal.IndicesAdminClient#refresh(RefreshRequest)
* @see RefreshResponse
*/ */
public class RefreshRequest extends BroadcastRequest<RefreshRequest> { public class RefreshRequest extends BroadcastRequest<RefreshRequest> {

View file

@ -9,6 +9,7 @@
package org.elasticsearch.action.admin.indices.refresh; package org.elasticsearch.action.admin.indices.refresh;
import org.elasticsearch.action.support.broadcast.BroadcastOperationRequestBuilder; import org.elasticsearch.action.support.broadcast.BroadcastOperationRequestBuilder;
import org.elasticsearch.action.support.broadcast.BroadcastResponse;
import org.elasticsearch.client.internal.ElasticsearchClient; import org.elasticsearch.client.internal.ElasticsearchClient;
/** /**
@ -16,7 +17,7 @@ import org.elasticsearch.client.internal.ElasticsearchClient;
* capabilities depends on the index engine used. For example, the internal one requires refresh to be called, but by * capabilities depends on the index engine used. For example, the internal one requires refresh to be called, but by
* default a refresh is scheduled periodically. * default a refresh is scheduled periodically.
*/ */
public class RefreshRequestBuilder extends BroadcastOperationRequestBuilder<RefreshRequest, RefreshResponse, RefreshRequestBuilder> { public class RefreshRequestBuilder extends BroadcastOperationRequestBuilder<RefreshRequest, BroadcastResponse, RefreshRequestBuilder> {
public RefreshRequestBuilder(ElasticsearchClient client) { public RefreshRequestBuilder(ElasticsearchClient client) {
super(client, RefreshAction.INSTANCE, new RefreshRequest()); super(client, RefreshAction.INSTANCE, new RefreshRequest());

View file

@ -1,57 +0,0 @@
/*
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the Elastic License
* 2.0 and the Server Side Public License, v 1; you may not use this file except
* in compliance with, at your election, the Elastic License 2.0 or the Server
* Side Public License, v 1.
*/
package org.elasticsearch.action.admin.indices.refresh;
import org.elasticsearch.action.support.DefaultShardOperationFailedException;
import org.elasticsearch.action.support.broadcast.BaseBroadcastResponse;
import org.elasticsearch.action.support.broadcast.BroadcastResponse;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.xcontent.ConstructingObjectParser;
import org.elasticsearch.xcontent.XContentParser;
import java.io.IOException;
import java.util.Arrays;
import java.util.List;
/**
* The response of a refresh action.
*/
public class RefreshResponse extends BroadcastResponse {
private static final ConstructingObjectParser<RefreshResponse, Void> PARSER = new ConstructingObjectParser<>("refresh", true, arg -> {
BaseBroadcastResponse response = (BaseBroadcastResponse) arg[0];
return new RefreshResponse(
response.getTotalShards(),
response.getSuccessfulShards(),
response.getFailedShards(),
Arrays.asList(response.getShardFailures())
);
});
static {
declareBroadcastFields(PARSER);
}
RefreshResponse(StreamInput in) throws IOException {
super(in);
}
public RefreshResponse(
int totalShards,
int successfulShards,
int failedShards,
List<DefaultShardOperationFailedException> shardFailures
) {
super(totalShards, successfulShards, failedShards, shardFailures);
}
public static RefreshResponse fromXContent(XContentParser parser) {
return PARSER.apply(parser, null);
}
}

View file

@ -11,6 +11,7 @@ package org.elasticsearch.action.admin.indices.refresh;
import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.ActiveShardCount; import org.elasticsearch.action.support.ActiveShardCount;
import org.elasticsearch.action.support.DefaultShardOperationFailedException; import org.elasticsearch.action.support.DefaultShardOperationFailedException;
import org.elasticsearch.action.support.broadcast.BroadcastResponse;
import org.elasticsearch.action.support.replication.BasicReplicationRequest; import org.elasticsearch.action.support.replication.BasicReplicationRequest;
import org.elasticsearch.action.support.replication.ReplicationResponse; import org.elasticsearch.action.support.replication.ReplicationResponse;
import org.elasticsearch.action.support.replication.TransportBroadcastReplicationAction; import org.elasticsearch.action.support.replication.TransportBroadcastReplicationAction;
@ -29,7 +30,7 @@ import java.util.List;
*/ */
public class TransportRefreshAction extends TransportBroadcastReplicationAction< public class TransportRefreshAction extends TransportBroadcastReplicationAction<
RefreshRequest, RefreshRequest,
RefreshResponse, BroadcastResponse,
BasicReplicationRequest, BasicReplicationRequest,
ReplicationResponse> { ReplicationResponse> {
@ -62,12 +63,12 @@ public class TransportRefreshAction extends TransportBroadcastReplicationAction<
} }
@Override @Override
protected RefreshResponse newResponse( protected BroadcastResponse newResponse(
int successfulShards, int successfulShards,
int failedShards, int failedShards,
int totalNumCopies, int totalNumCopies,
List<DefaultShardOperationFailedException> shardFailures List<DefaultShardOperationFailedException> shardFailures
) { ) {
return new RefreshResponse(totalNumCopies, successfulShards, failedShards, shardFailures); return new BroadcastResponse(totalNumCopies, successfulShards, failedShards, shardFailures);
} }
} }

View file

@ -42,7 +42,7 @@ public class BaseBroadcastResponse extends ActionResponse {
private final DefaultShardOperationFailedException[] shardFailures; private final DefaultShardOperationFailedException[] shardFailures;
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
protected static <T extends BaseBroadcastResponse> void declareBroadcastFields(ConstructingObjectParser<T, Void> PARSER) { public static <T extends BaseBroadcastResponse> void declareBroadcastFields(ConstructingObjectParser<T, Void> PARSER) {
ConstructingObjectParser<BaseBroadcastResponse, Void> shardsParser = new ConstructingObjectParser<>( ConstructingObjectParser<BaseBroadcastResponse, Void> shardsParser = new ConstructingObjectParser<>(
"_shards", "_shards",
true, true,

View file

@ -19,7 +19,6 @@ import org.elasticsearch.action.admin.indices.analyze.AnalyzeAction;
import org.elasticsearch.action.admin.indices.analyze.AnalyzeRequestBuilder; import org.elasticsearch.action.admin.indices.analyze.AnalyzeRequestBuilder;
import org.elasticsearch.action.admin.indices.cache.clear.ClearIndicesCacheRequest; import org.elasticsearch.action.admin.indices.cache.clear.ClearIndicesCacheRequest;
import org.elasticsearch.action.admin.indices.cache.clear.ClearIndicesCacheRequestBuilder; import org.elasticsearch.action.admin.indices.cache.clear.ClearIndicesCacheRequestBuilder;
import org.elasticsearch.action.admin.indices.cache.clear.ClearIndicesCacheResponse;
import org.elasticsearch.action.admin.indices.close.CloseIndexRequest; import org.elasticsearch.action.admin.indices.close.CloseIndexRequest;
import org.elasticsearch.action.admin.indices.close.CloseIndexRequestBuilder; import org.elasticsearch.action.admin.indices.close.CloseIndexRequestBuilder;
import org.elasticsearch.action.admin.indices.close.CloseIndexResponse; import org.elasticsearch.action.admin.indices.close.CloseIndexResponse;
@ -30,10 +29,8 @@ import org.elasticsearch.action.admin.indices.delete.DeleteIndexRequest;
import org.elasticsearch.action.admin.indices.delete.DeleteIndexRequestBuilder; import org.elasticsearch.action.admin.indices.delete.DeleteIndexRequestBuilder;
import org.elasticsearch.action.admin.indices.flush.FlushRequest; import org.elasticsearch.action.admin.indices.flush.FlushRequest;
import org.elasticsearch.action.admin.indices.flush.FlushRequestBuilder; import org.elasticsearch.action.admin.indices.flush.FlushRequestBuilder;
import org.elasticsearch.action.admin.indices.flush.FlushResponse;
import org.elasticsearch.action.admin.indices.forcemerge.ForceMergeRequest; import org.elasticsearch.action.admin.indices.forcemerge.ForceMergeRequest;
import org.elasticsearch.action.admin.indices.forcemerge.ForceMergeRequestBuilder; import org.elasticsearch.action.admin.indices.forcemerge.ForceMergeRequestBuilder;
import org.elasticsearch.action.admin.indices.forcemerge.ForceMergeResponse;
import org.elasticsearch.action.admin.indices.get.GetIndexRequest; import org.elasticsearch.action.admin.indices.get.GetIndexRequest;
import org.elasticsearch.action.admin.indices.get.GetIndexRequestBuilder; import org.elasticsearch.action.admin.indices.get.GetIndexRequestBuilder;
import org.elasticsearch.action.admin.indices.get.GetIndexResponse; import org.elasticsearch.action.admin.indices.get.GetIndexResponse;
@ -56,7 +53,6 @@ import org.elasticsearch.action.admin.indices.recovery.RecoveryRequestBuilder;
import org.elasticsearch.action.admin.indices.recovery.RecoveryResponse; import org.elasticsearch.action.admin.indices.recovery.RecoveryResponse;
import org.elasticsearch.action.admin.indices.refresh.RefreshRequest; import org.elasticsearch.action.admin.indices.refresh.RefreshRequest;
import org.elasticsearch.action.admin.indices.refresh.RefreshRequestBuilder; import org.elasticsearch.action.admin.indices.refresh.RefreshRequestBuilder;
import org.elasticsearch.action.admin.indices.refresh.RefreshResponse;
import org.elasticsearch.action.admin.indices.resolve.ResolveIndexAction; import org.elasticsearch.action.admin.indices.resolve.ResolveIndexAction;
import org.elasticsearch.action.admin.indices.rollover.RolloverRequest; import org.elasticsearch.action.admin.indices.rollover.RolloverRequest;
import org.elasticsearch.action.admin.indices.rollover.RolloverRequestBuilder; import org.elasticsearch.action.admin.indices.rollover.RolloverRequestBuilder;
@ -85,6 +81,7 @@ import org.elasticsearch.action.admin.indices.template.put.PutIndexTemplateReque
import org.elasticsearch.action.admin.indices.validate.query.ValidateQueryRequest; import org.elasticsearch.action.admin.indices.validate.query.ValidateQueryRequest;
import org.elasticsearch.action.admin.indices.validate.query.ValidateQueryRequestBuilder; import org.elasticsearch.action.admin.indices.validate.query.ValidateQueryRequestBuilder;
import org.elasticsearch.action.admin.indices.validate.query.ValidateQueryResponse; import org.elasticsearch.action.admin.indices.validate.query.ValidateQueryResponse;
import org.elasticsearch.action.support.broadcast.BroadcastResponse;
import org.elasticsearch.action.support.master.AcknowledgedResponse; import org.elasticsearch.action.support.master.AcknowledgedResponse;
import org.elasticsearch.cluster.metadata.IndexMetadata.APIBlock; import org.elasticsearch.cluster.metadata.IndexMetadata.APIBlock;
import org.elasticsearch.core.Nullable; import org.elasticsearch.core.Nullable;
@ -261,7 +258,7 @@ public interface IndicesAdminClient extends ElasticsearchClient {
* @param request The refresh request * @param request The refresh request
* @return The result future * @return The result future
*/ */
ActionFuture<RefreshResponse> refresh(RefreshRequest request); ActionFuture<BroadcastResponse> refresh(RefreshRequest request);
/** /**
* Explicitly refresh one or more indices (making the content indexed since the last refresh searchable). * Explicitly refresh one or more indices (making the content indexed since the last refresh searchable).
@ -269,7 +266,7 @@ public interface IndicesAdminClient extends ElasticsearchClient {
* @param request The refresh request * @param request The refresh request
* @param listener A listener to be notified with a result * @param listener A listener to be notified with a result
*/ */
void refresh(RefreshRequest request, ActionListener<RefreshResponse> listener); void refresh(RefreshRequest request, ActionListener<BroadcastResponse> listener);
/** /**
* Explicitly refresh one or more indices (making the content indexed since the last refresh searchable). * Explicitly refresh one or more indices (making the content indexed since the last refresh searchable).
@ -282,7 +279,7 @@ public interface IndicesAdminClient extends ElasticsearchClient {
* @param request The flush request * @param request The flush request
* @return A result future * @return A result future
*/ */
ActionFuture<FlushResponse> flush(FlushRequest request); ActionFuture<BroadcastResponse> flush(FlushRequest request);
/** /**
* Explicitly flush one or more indices (releasing memory from the node). * Explicitly flush one or more indices (releasing memory from the node).
@ -290,7 +287,7 @@ public interface IndicesAdminClient extends ElasticsearchClient {
* @param request The flush request * @param request The flush request
* @param listener A listener to be notified with a result * @param listener A listener to be notified with a result
*/ */
void flush(FlushRequest request, ActionListener<FlushResponse> listener); void flush(FlushRequest request, ActionListener<BroadcastResponse> listener);
/** /**
* Explicitly flush one or more indices (releasing memory from the node). * Explicitly flush one or more indices (releasing memory from the node).
@ -303,7 +300,7 @@ public interface IndicesAdminClient extends ElasticsearchClient {
* @param request The optimize request * @param request The optimize request
* @return A result future * @return A result future
*/ */
ActionFuture<ForceMergeResponse> forceMerge(ForceMergeRequest request); ActionFuture<BroadcastResponse> forceMerge(ForceMergeRequest request);
/** /**
* Explicitly force merge one or more indices into a the number of segments. * Explicitly force merge one or more indices into a the number of segments.
@ -311,7 +308,7 @@ public interface IndicesAdminClient extends ElasticsearchClient {
* @param request The force merge request * @param request The force merge request
* @param listener A listener to be notified with a result * @param listener A listener to be notified with a result
*/ */
void forceMerge(ForceMergeRequest request, ActionListener<ForceMergeResponse> listener); void forceMerge(ForceMergeRequest request, ActionListener<BroadcastResponse> listener);
/** /**
* Explicitly force merge one or more indices into a the number of segments. * Explicitly force merge one or more indices into a the number of segments.
@ -436,7 +433,7 @@ public interface IndicesAdminClient extends ElasticsearchClient {
* @param request The clear indices cache request * @param request The clear indices cache request
* @return The result future * @return The result future
*/ */
ActionFuture<ClearIndicesCacheResponse> clearCache(ClearIndicesCacheRequest request); ActionFuture<BroadcastResponse> clearCache(ClearIndicesCacheRequest request);
/** /**
* Clear indices cache. * Clear indices cache.
@ -444,7 +441,7 @@ public interface IndicesAdminClient extends ElasticsearchClient {
* @param request The clear indices cache request * @param request The clear indices cache request
* @param listener A listener to be notified with a result * @param listener A listener to be notified with a result
*/ */
void clearCache(ClearIndicesCacheRequest request, ActionListener<ClearIndicesCacheResponse> listener); void clearCache(ClearIndicesCacheRequest request, ActionListener<BroadcastResponse> listener);
/** /**
* Clear indices cache. * Clear indices cache.

View file

@ -129,7 +129,6 @@ import org.elasticsearch.action.admin.indices.analyze.AnalyzeRequestBuilder;
import org.elasticsearch.action.admin.indices.cache.clear.ClearIndicesCacheAction; import org.elasticsearch.action.admin.indices.cache.clear.ClearIndicesCacheAction;
import org.elasticsearch.action.admin.indices.cache.clear.ClearIndicesCacheRequest; import org.elasticsearch.action.admin.indices.cache.clear.ClearIndicesCacheRequest;
import org.elasticsearch.action.admin.indices.cache.clear.ClearIndicesCacheRequestBuilder; import org.elasticsearch.action.admin.indices.cache.clear.ClearIndicesCacheRequestBuilder;
import org.elasticsearch.action.admin.indices.cache.clear.ClearIndicesCacheResponse;
import org.elasticsearch.action.admin.indices.close.CloseIndexRequest; import org.elasticsearch.action.admin.indices.close.CloseIndexRequest;
import org.elasticsearch.action.admin.indices.close.CloseIndexRequestBuilder; import org.elasticsearch.action.admin.indices.close.CloseIndexRequestBuilder;
import org.elasticsearch.action.admin.indices.close.CloseIndexResponse; import org.elasticsearch.action.admin.indices.close.CloseIndexResponse;
@ -144,11 +143,9 @@ import org.elasticsearch.action.admin.indices.delete.TransportDeleteIndexAction;
import org.elasticsearch.action.admin.indices.flush.FlushAction; import org.elasticsearch.action.admin.indices.flush.FlushAction;
import org.elasticsearch.action.admin.indices.flush.FlushRequest; import org.elasticsearch.action.admin.indices.flush.FlushRequest;
import org.elasticsearch.action.admin.indices.flush.FlushRequestBuilder; import org.elasticsearch.action.admin.indices.flush.FlushRequestBuilder;
import org.elasticsearch.action.admin.indices.flush.FlushResponse;
import org.elasticsearch.action.admin.indices.forcemerge.ForceMergeAction; import org.elasticsearch.action.admin.indices.forcemerge.ForceMergeAction;
import org.elasticsearch.action.admin.indices.forcemerge.ForceMergeRequest; import org.elasticsearch.action.admin.indices.forcemerge.ForceMergeRequest;
import org.elasticsearch.action.admin.indices.forcemerge.ForceMergeRequestBuilder; import org.elasticsearch.action.admin.indices.forcemerge.ForceMergeRequestBuilder;
import org.elasticsearch.action.admin.indices.forcemerge.ForceMergeResponse;
import org.elasticsearch.action.admin.indices.get.GetIndexAction; import org.elasticsearch.action.admin.indices.get.GetIndexAction;
import org.elasticsearch.action.admin.indices.get.GetIndexRequest; import org.elasticsearch.action.admin.indices.get.GetIndexRequest;
import org.elasticsearch.action.admin.indices.get.GetIndexRequestBuilder; import org.elasticsearch.action.admin.indices.get.GetIndexRequestBuilder;
@ -179,7 +176,6 @@ import org.elasticsearch.action.admin.indices.recovery.RecoveryResponse;
import org.elasticsearch.action.admin.indices.refresh.RefreshAction; import org.elasticsearch.action.admin.indices.refresh.RefreshAction;
import org.elasticsearch.action.admin.indices.refresh.RefreshRequest; import org.elasticsearch.action.admin.indices.refresh.RefreshRequest;
import org.elasticsearch.action.admin.indices.refresh.RefreshRequestBuilder; import org.elasticsearch.action.admin.indices.refresh.RefreshRequestBuilder;
import org.elasticsearch.action.admin.indices.refresh.RefreshResponse;
import org.elasticsearch.action.admin.indices.resolve.ResolveIndexAction; import org.elasticsearch.action.admin.indices.resolve.ResolveIndexAction;
import org.elasticsearch.action.admin.indices.rollover.RolloverAction; import org.elasticsearch.action.admin.indices.rollover.RolloverAction;
import org.elasticsearch.action.admin.indices.rollover.RolloverRequest; import org.elasticsearch.action.admin.indices.rollover.RolloverRequest;
@ -275,6 +271,7 @@ import org.elasticsearch.action.search.TransportMultiSearchAction;
import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.action.search.TransportSearchAction;
import org.elasticsearch.action.search.TransportSearchScrollAction; import org.elasticsearch.action.search.TransportSearchScrollAction;
import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.action.support.PlainActionFuture;
import org.elasticsearch.action.support.broadcast.BroadcastResponse;
import org.elasticsearch.action.support.master.AcknowledgedResponse; import org.elasticsearch.action.support.master.AcknowledgedResponse;
import org.elasticsearch.action.termvectors.MultiTermVectorsAction; import org.elasticsearch.action.termvectors.MultiTermVectorsAction;
import org.elasticsearch.action.termvectors.MultiTermVectorsRequest; import org.elasticsearch.action.termvectors.MultiTermVectorsRequest;
@ -1118,7 +1115,7 @@ public abstract class AbstractClient implements Client {
} }
@Override @Override
public ActionFuture<ClearIndicesCacheResponse> clearCache(final ClearIndicesCacheRequest request) { public ActionFuture<BroadcastResponse> clearCache(final ClearIndicesCacheRequest request) {
return execute(ClearIndicesCacheAction.INSTANCE, request); return execute(ClearIndicesCacheAction.INSTANCE, request);
} }
@ -1138,7 +1135,7 @@ public abstract class AbstractClient implements Client {
} }
@Override @Override
public void clearCache(final ClearIndicesCacheRequest request, final ActionListener<ClearIndicesCacheResponse> listener) { public void clearCache(final ClearIndicesCacheRequest request, final ActionListener<BroadcastResponse> listener) {
execute(ClearIndicesCacheAction.INSTANCE, request, listener); execute(ClearIndicesCacheAction.INSTANCE, request, listener);
} }
@ -1218,12 +1215,12 @@ public abstract class AbstractClient implements Client {
} }
@Override @Override
public ActionFuture<FlushResponse> flush(final FlushRequest request) { public ActionFuture<BroadcastResponse> flush(final FlushRequest request) {
return execute(FlushAction.INSTANCE, request); return execute(FlushAction.INSTANCE, request);
} }
@Override @Override
public void flush(final FlushRequest request, final ActionListener<FlushResponse> listener) { public void flush(final FlushRequest request, final ActionListener<BroadcastResponse> listener) {
execute(FlushAction.INSTANCE, request, listener); execute(FlushAction.INSTANCE, request, listener);
} }
@ -1278,12 +1275,12 @@ public abstract class AbstractClient implements Client {
} }
@Override @Override
public ActionFuture<ForceMergeResponse> forceMerge(final ForceMergeRequest request) { public ActionFuture<BroadcastResponse> forceMerge(final ForceMergeRequest request) {
return execute(ForceMergeAction.INSTANCE, request); return execute(ForceMergeAction.INSTANCE, request);
} }
@Override @Override
public void forceMerge(final ForceMergeRequest request, final ActionListener<ForceMergeResponse> listener) { public void forceMerge(final ForceMergeRequest request, final ActionListener<BroadcastResponse> listener) {
execute(ForceMergeAction.INSTANCE, request, listener); execute(ForceMergeAction.INSTANCE, request, listener);
} }
@ -1293,12 +1290,12 @@ public abstract class AbstractClient implements Client {
} }
@Override @Override
public ActionFuture<RefreshResponse> refresh(final RefreshRequest request) { public ActionFuture<BroadcastResponse> refresh(final RefreshRequest request) {
return execute(RefreshAction.INSTANCE, request); return execute(RefreshAction.INSTANCE, request);
} }
@Override @Override
public void refresh(final RefreshRequest request, final ActionListener<RefreshResponse> listener) { public void refresh(final RefreshRequest request, final ActionListener<BroadcastResponse> listener) {
execute(RefreshAction.INSTANCE, request, listener); execute(RefreshAction.INSTANCE, request, listener);
} }

View file

@ -11,9 +11,9 @@ package org.elasticsearch.rest.action.admin.indices;
import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.ActionRequestValidationException;
import org.elasticsearch.action.admin.indices.forcemerge.ForceMergeAction; import org.elasticsearch.action.admin.indices.forcemerge.ForceMergeAction;
import org.elasticsearch.action.admin.indices.forcemerge.ForceMergeRequest; import org.elasticsearch.action.admin.indices.forcemerge.ForceMergeRequest;
import org.elasticsearch.action.admin.indices.forcemerge.ForceMergeResponse;
import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.action.support.SubscribableListener; import org.elasticsearch.action.support.SubscribableListener;
import org.elasticsearch.action.support.broadcast.BroadcastResponse;
import org.elasticsearch.client.internal.node.NodeClient; import org.elasticsearch.client.internal.node.NodeClient;
import org.elasticsearch.common.Strings; import org.elasticsearch.common.Strings;
import org.elasticsearch.rest.BaseRestHandler; import org.elasticsearch.rest.BaseRestHandler;
@ -65,7 +65,7 @@ public class RestForceMergeAction extends BaseRestHandler {
if (validationException != null) { if (validationException != null) {
throw validationException; throw validationException;
} }
final var responseListener = new SubscribableListener<ForceMergeResponse>(); final var responseListener = new SubscribableListener<BroadcastResponse>();
final var task = client.executeLocally(ForceMergeAction.INSTANCE, mergeRequest, responseListener); final var task = client.executeLocally(ForceMergeAction.INSTANCE, mergeRequest, responseListener);
responseListener.addListener(new LoggingTaskListener<>(task)); responseListener.addListener(new LoggingTaskListener<>(task));
return sendTask(client.getLocalNodeId(), task); return sendTask(client.getLocalNodeId(), task);

View file

@ -9,8 +9,8 @@
package org.elasticsearch.rest.action.admin.indices; package org.elasticsearch.rest.action.admin.indices;
import org.elasticsearch.action.admin.indices.flush.FlushRequest; import org.elasticsearch.action.admin.indices.flush.FlushRequest;
import org.elasticsearch.action.admin.indices.flush.FlushResponse;
import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.action.support.broadcast.BroadcastResponse;
import org.elasticsearch.client.internal.node.NodeClient; import org.elasticsearch.client.internal.node.NodeClient;
import org.elasticsearch.common.Strings; import org.elasticsearch.common.Strings;
import org.elasticsearch.core.RestApiVersion; import org.elasticsearch.core.RestApiVersion;
@ -55,14 +55,14 @@ public class RestSyncedFlushAction extends BaseRestHandler {
return channel -> client.admin().indices().flush(flushRequest, new SimulateSyncedFlushResponseListener(channel)); return channel -> client.admin().indices().flush(flushRequest, new SimulateSyncedFlushResponseListener(channel));
} }
static final class SimulateSyncedFlushResponseListener extends RestBuilderListener<FlushResponse> { static final class SimulateSyncedFlushResponseListener extends RestBuilderListener<BroadcastResponse> {
SimulateSyncedFlushResponseListener(RestChannel channel) { SimulateSyncedFlushResponseListener(RestChannel channel) {
super(channel); super(channel);
} }
@Override @Override
public RestResponse buildResponse(FlushResponse flushResponse, XContentBuilder builder) throws Exception { public RestResponse buildResponse(BroadcastResponse flushResponse, XContentBuilder builder) throws Exception {
builder.startObject(); builder.startObject();
buildSyncedFlushResponse(builder, flushResponse); buildSyncedFlushResponse(builder, flushResponse);
builder.endObject(); builder.endObject();
@ -70,7 +70,7 @@ public class RestSyncedFlushAction extends BaseRestHandler {
return new RestResponse(restStatus, builder); return new RestResponse(restStatus, builder);
} }
private static void buildSyncedFlushResponse(XContentBuilder builder, FlushResponse flushResponse) throws IOException { private static void buildSyncedFlushResponse(XContentBuilder builder, BroadcastResponse flushResponse) throws IOException {
builder.startObject("_shards"); builder.startObject("_shards");
builder.field("total", flushResponse.getTotalShards()); builder.field("total", flushResponse.getTotalShards());
builder.field("successful", flushResponse.getSuccessfulShards()); builder.field("successful", flushResponse.getSuccessfulShards());

View file

@ -1,33 +0,0 @@
/*
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the Elastic License
* 2.0 and the Server Side Public License, v 1; you may not use this file except
* in compliance with, at your election, the Elastic License 2.0 or the Server
* Side Public License, v 1.
*/
package org.elasticsearch.action.admin.indices.cache.clear;
import org.elasticsearch.action.support.DefaultShardOperationFailedException;
import org.elasticsearch.test.AbstractBroadcastResponseTestCase;
import org.elasticsearch.xcontent.XContentParser;
import java.util.List;
public class ClearIndicesCacheResponseTests extends AbstractBroadcastResponseTestCase<ClearIndicesCacheResponse> {
@Override
protected ClearIndicesCacheResponse createTestInstance(
int totalShards,
int successfulShards,
int failedShards,
List<DefaultShardOperationFailedException> failures
) {
return new ClearIndicesCacheResponse(totalShards, successfulShards, failedShards, failures);
}
@Override
protected ClearIndicesCacheResponse doParseInstance(XContentParser parser) {
return ClearIndicesCacheResponse.fromXContent(parser);
}
}

View file

@ -1,33 +0,0 @@
/*
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the Elastic License
* 2.0 and the Server Side Public License, v 1; you may not use this file except
* in compliance with, at your election, the Elastic License 2.0 or the Server
* Side Public License, v 1.
*/
package org.elasticsearch.action.admin.indices.flush;
import org.elasticsearch.action.support.DefaultShardOperationFailedException;
import org.elasticsearch.test.AbstractBroadcastResponseTestCase;
import org.elasticsearch.xcontent.XContentParser;
import java.util.List;
public class FlushResponseTests extends AbstractBroadcastResponseTestCase<FlushResponse> {
@Override
protected FlushResponse createTestInstance(
int totalShards,
int successfulShards,
int failedShards,
List<DefaultShardOperationFailedException> failures
) {
return new FlushResponse(totalShards, successfulShards, failedShards, failures);
}
@Override
protected FlushResponse doParseInstance(XContentParser parser) {
return FlushResponse.fromXContent(parser);
}
}

View file

@ -1,32 +0,0 @@
/*
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the Elastic License
* 2.0 and the Server Side Public License, v 1; you may not use this file except
* in compliance with, at your election, the Elastic License 2.0 or the Server
* Side Public License, v 1.
*/
package org.elasticsearch.action.admin.indices.forcemerge;
import org.elasticsearch.action.support.DefaultShardOperationFailedException;
import org.elasticsearch.test.AbstractBroadcastResponseTestCase;
import org.elasticsearch.xcontent.XContentParser;
import java.util.List;
public class ForceMergeResponseTests extends AbstractBroadcastResponseTestCase<ForceMergeResponse> {
@Override
protected ForceMergeResponse createTestInstance(
int totalShards,
int successfulShards,
int failedShards,
List<DefaultShardOperationFailedException> failures
) {
return new ForceMergeResponse(totalShards, successfulShards, failedShards, failures);
}
@Override
protected ForceMergeResponse doParseInstance(XContentParser parser) {
return ForceMergeResponse.fromXContent(parser);
}
}

View file

@ -1,33 +0,0 @@
/*
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the Elastic License
* 2.0 and the Server Side Public License, v 1; you may not use this file except
* in compliance with, at your election, the Elastic License 2.0 or the Server
* Side Public License, v 1.
*/
package org.elasticsearch.action.admin.indices.refresh;
import org.elasticsearch.action.support.DefaultShardOperationFailedException;
import org.elasticsearch.test.AbstractBroadcastResponseTestCase;
import org.elasticsearch.xcontent.XContentParser;
import java.util.List;
public class RefreshResponseTests extends AbstractBroadcastResponseTestCase<RefreshResponse> {
@Override
protected RefreshResponse createTestInstance(
int totalShards,
int successfulShards,
int failedShards,
List<DefaultShardOperationFailedException> failures
) {
return new RefreshResponse(totalShards, successfulShards, failedShards, failures);
}
@Override
protected RefreshResponse doParseInstance(XContentParser parser) {
return RefreshResponse.fromXContent(parser);
}
}

View file

@ -12,7 +12,6 @@ import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.NoShardAvailableActionException; import org.elasticsearch.action.NoShardAvailableActionException;
import org.elasticsearch.action.UnavailableShardsException; import org.elasticsearch.action.UnavailableShardsException;
import org.elasticsearch.action.admin.indices.flush.FlushRequest; import org.elasticsearch.action.admin.indices.flush.FlushRequest;
import org.elasticsearch.action.admin.indices.flush.FlushResponse;
import org.elasticsearch.action.admin.indices.flush.TransportFlushAction; import org.elasticsearch.action.admin.indices.flush.TransportFlushAction;
import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.ActionTestUtils; import org.elasticsearch.action.support.ActionTestUtils;
@ -20,6 +19,7 @@ import org.elasticsearch.action.support.DefaultShardOperationFailedException;
import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.action.support.PlainActionFuture;
import org.elasticsearch.action.support.broadcast.BaseBroadcastResponse; import org.elasticsearch.action.support.broadcast.BaseBroadcastResponse;
import org.elasticsearch.action.support.broadcast.BroadcastRequest; import org.elasticsearch.action.support.broadcast.BroadcastRequest;
import org.elasticsearch.action.support.broadcast.BroadcastResponse;
import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.cluster.routing.ShardRoutingState; import org.elasticsearch.cluster.routing.ShardRoutingState;
@ -286,9 +286,9 @@ public class BroadcastReplicationTests extends ESTestCase {
} }
} }
public FlushResponse assertImmediateResponse(String index, TransportFlushAction flushAction) { public BroadcastResponse assertImmediateResponse(String index, TransportFlushAction flushAction) {
Date beginDate = new Date(); Date beginDate = new Date();
FlushResponse flushResponse = ActionTestUtils.executeBlocking(flushAction, new FlushRequest(index)); BroadcastResponse flushResponse = ActionTestUtils.executeBlocking(flushAction, new FlushRequest(index));
Date endDate = new Date(); Date endDate = new Date();
long maxTime = 500; long maxTime = 500;
assertThat( assertThat(

View file

@ -9,8 +9,8 @@
package org.elasticsearch.index.fieldstats; package org.elasticsearch.index.fieldstats;
import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.DocWriteResponse;
import org.elasticsearch.action.admin.indices.refresh.RefreshResponse;
import org.elasticsearch.action.search.SearchType; import org.elasticsearch.action.search.SearchType;
import org.elasticsearch.action.support.broadcast.BroadcastResponse;
import org.elasticsearch.index.query.QueryBuilders; import org.elasticsearch.index.query.QueryBuilders;
import org.elasticsearch.indices.IndicesRequestCache; import org.elasticsearch.indices.IndicesRequestCache;
import org.elasticsearch.rest.RestStatus; import org.elasticsearch.rest.RestStatus;
@ -88,7 +88,7 @@ public class FieldStatsProviderRefreshTests extends ESSingleNodeTestCase {
} }
private void refreshIndex() { private void refreshIndex() {
RefreshResponse refreshResponse = indicesAdmin().prepareRefresh("index").get(); BroadcastResponse refreshResponse = indicesAdmin().prepareRefresh("index").get();
assertThat(refreshResponse.getSuccessfulShards(), equalTo(refreshResponse.getSuccessfulShards())); assertThat(refreshResponse.getSuccessfulShards(), equalTo(refreshResponse.getSuccessfulShards()));
} }

View file

@ -11,9 +11,9 @@ package org.elasticsearch.indices.recovery;
import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse; import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse;
import org.elasticsearch.action.admin.cluster.snapshots.create.CreateSnapshotResponse; import org.elasticsearch.action.admin.cluster.snapshots.create.CreateSnapshotResponse;
import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse; import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse;
import org.elasticsearch.action.admin.indices.forcemerge.ForceMergeResponse;
import org.elasticsearch.action.admin.indices.stats.ShardStats; import org.elasticsearch.action.admin.indices.stats.ShardStats;
import org.elasticsearch.action.index.IndexRequestBuilder; import org.elasticsearch.action.index.IndexRequestBuilder;
import org.elasticsearch.action.support.broadcast.BroadcastResponse;
import org.elasticsearch.cluster.NodeConnectionsService; import org.elasticsearch.cluster.NodeConnectionsService;
import org.elasticsearch.cluster.action.shard.ShardStateAction; import org.elasticsearch.cluster.action.shard.ShardStateAction;
import org.elasticsearch.cluster.metadata.IndexMetadata; import org.elasticsearch.cluster.metadata.IndexMetadata;
@ -466,7 +466,7 @@ public abstract class AbstractIndexRecoveryIntegTestCase extends ESIntegTestCase
}, 60, TimeUnit.SECONDS); }, 60, TimeUnit.SECONDS);
// Force merge to make sure that the resulting snapshot would contain the same index files as the safe commit // Force merge to make sure that the resulting snapshot would contain the same index files as the safe commit
ForceMergeResponse forceMergeResponse = client().admin().indices().prepareForceMerge(indexName).setFlush(randomBoolean()).get(); BroadcastResponse forceMergeResponse = client().admin().indices().prepareForceMerge(indexName).setFlush(randomBoolean()).get();
assertThat(forceMergeResponse.getTotalShards(), equalTo(forceMergeResponse.getSuccessfulShards())); assertThat(forceMergeResponse.getTotalShards(), equalTo(forceMergeResponse.getSuccessfulShards()));
// create repo // create repo

View file

@ -16,7 +16,7 @@ import org.apache.http.HttpStatus;
import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.Logger;
import org.apache.lucene.tests.util.LuceneTestCase; import org.apache.lucene.tests.util.LuceneTestCase;
import org.elasticsearch.action.admin.indices.forcemerge.ForceMergeResponse; import org.elasticsearch.action.support.broadcast.BroadcastResponse;
import org.elasticsearch.common.Strings; import org.elasticsearch.common.Strings;
import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.network.InetAddresses; import org.elasticsearch.common.network.InetAddresses;
@ -164,7 +164,7 @@ public abstract class ESMockAPIBasedRepositoryIntegTestCase extends ESBlobStoreR
} }
flushAndRefresh(index); flushAndRefresh(index);
ForceMergeResponse forceMerge = client().admin().indices().prepareForceMerge(index).setFlush(true).setMaxNumSegments(1).get(); BroadcastResponse forceMerge = client().admin().indices().prepareForceMerge(index).setFlush(true).setMaxNumSegments(1).get();
assertThat(forceMerge.getSuccessfulShards(), equalTo(1)); assertThat(forceMerge.getSuccessfulShards(), equalTo(1));
assertHitCount(prepareSearch(index).setSize(0).setTrackTotalHits(true), nbDocs); assertHitCount(prepareSearch(index).setSize(0).setTrackTotalHits(true), nbDocs);
@ -191,7 +191,7 @@ public abstract class ESMockAPIBasedRepositoryIntegTestCase extends ESBlobStoreR
} }
flushAndRefresh(index); flushAndRefresh(index);
ForceMergeResponse forceMerge = client().admin().indices().prepareForceMerge(index).setFlush(true).setMaxNumSegments(1).get(); BroadcastResponse forceMerge = client().admin().indices().prepareForceMerge(index).setFlush(true).setMaxNumSegments(1).get();
assertThat(forceMerge.getSuccessfulShards(), equalTo(1)); assertThat(forceMerge.getSuccessfulShards(), equalTo(1));
assertHitCount(prepareSearch(index).setSize(0).setTrackTotalHits(true), nbDocs); assertHitCount(prepareSearch(index).setSize(0).setTrackTotalHits(true), nbDocs);

View file

@ -34,10 +34,7 @@ import org.elasticsearch.action.admin.cluster.tasks.PendingClusterTasksRequest;
import org.elasticsearch.action.admin.cluster.tasks.PendingClusterTasksResponse; import org.elasticsearch.action.admin.cluster.tasks.PendingClusterTasksResponse;
import org.elasticsearch.action.admin.cluster.tasks.TransportPendingClusterTasksAction; import org.elasticsearch.action.admin.cluster.tasks.TransportPendingClusterTasksAction;
import org.elasticsearch.action.admin.indices.create.CreateIndexRequestBuilder; import org.elasticsearch.action.admin.indices.create.CreateIndexRequestBuilder;
import org.elasticsearch.action.admin.indices.flush.FlushResponse;
import org.elasticsearch.action.admin.indices.forcemerge.ForceMergeResponse;
import org.elasticsearch.action.admin.indices.get.GetIndexResponse; import org.elasticsearch.action.admin.indices.get.GetIndexResponse;
import org.elasticsearch.action.admin.indices.refresh.RefreshResponse;
import org.elasticsearch.action.admin.indices.segments.IndexSegments; import org.elasticsearch.action.admin.indices.segments.IndexSegments;
import org.elasticsearch.action.admin.indices.segments.IndexShardSegments; import org.elasticsearch.action.admin.indices.segments.IndexShardSegments;
import org.elasticsearch.action.admin.indices.segments.IndicesSegmentResponse; import org.elasticsearch.action.admin.indices.segments.IndicesSegmentResponse;
@ -57,6 +54,7 @@ import org.elasticsearch.action.support.DestructiveOperations;
import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.action.support.PlainActionFuture;
import org.elasticsearch.action.support.RefCountingListener; import org.elasticsearch.action.support.RefCountingListener;
import org.elasticsearch.action.support.broadcast.BroadcastResponse;
import org.elasticsearch.client.RestClient; import org.elasticsearch.client.RestClient;
import org.elasticsearch.client.RestClientBuilder; import org.elasticsearch.client.RestClientBuilder;
import org.elasticsearch.client.internal.AdminClient; import org.elasticsearch.client.internal.AdminClient;
@ -1478,9 +1476,9 @@ public abstract class ESIntegTestCase extends ESTestCase {
* *
* @see #waitForRelocation() * @see #waitForRelocation()
*/ */
protected final RefreshResponse refresh(String... indices) { protected final BroadcastResponse refresh(String... indices) {
waitForRelocation(); waitForRelocation();
RefreshResponse actionGet = indicesAdmin().prepareRefresh(indices) BroadcastResponse actionGet = indicesAdmin().prepareRefresh(indices)
.setIndicesOptions(IndicesOptions.STRICT_EXPAND_OPEN_HIDDEN_FORBID_CLOSED) .setIndicesOptions(IndicesOptions.STRICT_EXPAND_OPEN_HIDDEN_FORBID_CLOSED)
.get(); .get();
assertNoFailures(actionGet); assertNoFailures(actionGet);
@ -1498,9 +1496,9 @@ public abstract class ESIntegTestCase extends ESTestCase {
/** /**
* Flush some or all indices in the cluster. * Flush some or all indices in the cluster.
*/ */
protected final FlushResponse flush(String... indices) { protected final BroadcastResponse flush(String... indices) {
waitForRelocation(); waitForRelocation();
FlushResponse actionGet = indicesAdmin().prepareFlush(indices).get(); BroadcastResponse actionGet = indicesAdmin().prepareFlush(indices).get();
for (DefaultShardOperationFailedException failure : actionGet.getShardFailures()) { for (DefaultShardOperationFailedException failure : actionGet.getShardFailures()) {
assertThat("unexpected flush failure " + failure.reason(), failure.status(), equalTo(RestStatus.SERVICE_UNAVAILABLE)); assertThat("unexpected flush failure " + failure.reason(), failure.status(), equalTo(RestStatus.SERVICE_UNAVAILABLE));
} }
@ -1510,9 +1508,9 @@ public abstract class ESIntegTestCase extends ESTestCase {
/** /**
* Waits for all relocations and force merge all indices in the cluster to 1 segment. * Waits for all relocations and force merge all indices in the cluster to 1 segment.
*/ */
protected ForceMergeResponse forceMerge() { protected BroadcastResponse forceMerge() {
waitForRelocation(); waitForRelocation();
ForceMergeResponse actionGet = indicesAdmin().prepareForceMerge().setMaxNumSegments(1).get(); BroadcastResponse actionGet = indicesAdmin().prepareForceMerge().setMaxNumSegments(1).get();
assertNoFailures(actionGet); assertNoFailures(actionGet);
return actionGet; return actionGet;
} }

View file

@ -32,8 +32,9 @@ import org.elasticsearch.Version;
import org.elasticsearch.action.admin.cluster.node.tasks.list.TransportListTasksAction; import org.elasticsearch.action.admin.cluster.node.tasks.list.TransportListTasksAction;
import org.elasticsearch.action.admin.cluster.repositories.put.PutRepositoryRequest; import org.elasticsearch.action.admin.cluster.repositories.put.PutRepositoryRequest;
import org.elasticsearch.action.admin.indices.create.CreateIndexResponse; import org.elasticsearch.action.admin.indices.create.CreateIndexResponse;
import org.elasticsearch.action.admin.indices.refresh.RefreshResponse;
import org.elasticsearch.action.fieldcaps.FieldCapabilitiesResponse; import org.elasticsearch.action.fieldcaps.FieldCapabilitiesResponse;
import org.elasticsearch.action.support.broadcast.BaseBroadcastResponse;
import org.elasticsearch.action.support.broadcast.BroadcastResponse;
import org.elasticsearch.action.support.master.AcknowledgedResponse; import org.elasticsearch.action.support.master.AcknowledgedResponse;
import org.elasticsearch.client.Request; import org.elasticsearch.client.Request;
import org.elasticsearch.client.RequestOptions; import org.elasticsearch.client.RequestOptions;
@ -72,6 +73,7 @@ import org.elasticsearch.index.query.QueryBuilder;
import org.elasticsearch.index.seqno.ReplicationTracker; import org.elasticsearch.index.seqno.ReplicationTracker;
import org.elasticsearch.rest.RestStatus; import org.elasticsearch.rest.RestStatus;
import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.xcontent.ConstructingObjectParser;
import org.elasticsearch.xcontent.DeprecationHandler; import org.elasticsearch.xcontent.DeprecationHandler;
import org.elasticsearch.xcontent.NamedXContentRegistry; import org.elasticsearch.xcontent.NamedXContentRegistry;
import org.elasticsearch.xcontent.ToXContent; import org.elasticsearch.xcontent.ToXContent;
@ -1265,15 +1267,33 @@ public abstract class ESRestTestCase extends ESTestCase {
client().performRequest(refreshRequest); client().performRequest(refreshRequest);
} }
protected static RefreshResponse refresh(String index) throws IOException { protected static BroadcastResponse refresh(String index) throws IOException {
return refresh(client(), index); return refresh(client(), index);
} }
protected static RefreshResponse refresh(RestClient client, String index) throws IOException { private static final ConstructingObjectParser<BroadcastResponse, Void> BROADCAST_RESPONSE_PARSER = new ConstructingObjectParser<>(
"broadcast_response",
true,
arg -> {
BaseBroadcastResponse response = (BaseBroadcastResponse) arg[0];
return new BroadcastResponse(
response.getTotalShards(),
response.getSuccessfulShards(),
response.getFailedShards(),
Arrays.asList(response.getShardFailures())
);
}
);
static {
BaseBroadcastResponse.declareBroadcastFields(BROADCAST_RESPONSE_PARSER);
}
protected static BroadcastResponse refresh(RestClient client, String index) throws IOException {
Request refreshRequest = new Request("POST", "/" + index + "/_refresh"); Request refreshRequest = new Request("POST", "/" + index + "/_refresh");
Response response = client.performRequest(refreshRequest); Response response = client.performRequest(refreshRequest);
try (var parser = responseAsParser(response)) { try (var parser = responseAsParser(response)) {
return RefreshResponse.fromXContent(parser); return BROADCAST_RESPONSE_PARSER.apply(parser, null);
} }
} }

View file

@ -15,12 +15,12 @@ import org.elasticsearch.action.admin.cluster.node.tasks.list.ListTasksRequest;
import org.elasticsearch.action.admin.cluster.node.tasks.list.ListTasksResponse; import org.elasticsearch.action.admin.cluster.node.tasks.list.ListTasksResponse;
import org.elasticsearch.action.admin.cluster.snapshots.restore.RestoreSnapshotRequest; import org.elasticsearch.action.admin.cluster.snapshots.restore.RestoreSnapshotRequest;
import org.elasticsearch.action.admin.indices.get.GetIndexResponse; import org.elasticsearch.action.admin.indices.get.GetIndexResponse;
import org.elasticsearch.action.admin.indices.refresh.RefreshResponse;
import org.elasticsearch.action.admin.indices.settings.put.UpdateSettingsRequest; import org.elasticsearch.action.admin.indices.settings.put.UpdateSettingsRequest;
import org.elasticsearch.action.admin.indices.stats.ShardStats; import org.elasticsearch.action.admin.indices.stats.ShardStats;
import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchRequest;
import org.elasticsearch.action.support.ActiveShardCount; import org.elasticsearch.action.support.ActiveShardCount;
import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.action.support.PlainActionFuture;
import org.elasticsearch.action.support.broadcast.BroadcastResponse;
import org.elasticsearch.action.support.master.AcknowledgedRequest; import org.elasticsearch.action.support.master.AcknowledgedRequest;
import org.elasticsearch.analysis.common.CommonAnalysisPlugin; import org.elasticsearch.analysis.common.CommonAnalysisPlugin;
import org.elasticsearch.client.internal.Client; import org.elasticsearch.client.internal.Client;
@ -461,8 +461,8 @@ public abstract class CcrIntegTestCase extends ESTestCase {
return new Index(index, uuid); return new Index(index, uuid);
} }
protected final RefreshResponse refresh(Client client, String... indices) { protected final BroadcastResponse refresh(Client client, String... indices) {
RefreshResponse actionGet = client.admin().indices().prepareRefresh(indices).get(); BroadcastResponse actionGet = client.admin().indices().prepareRefresh(indices).get();
assertNoFailures(actionGet); assertNoFailures(actionGet);
return actionGet; return actionGet;
} }

View file

@ -10,9 +10,9 @@ import org.apache.lucene.util.SetOnce;
import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.admin.indices.forcemerge.ForceMergeRequest; import org.elasticsearch.action.admin.indices.forcemerge.ForceMergeRequest;
import org.elasticsearch.action.admin.indices.forcemerge.ForceMergeResponse;
import org.elasticsearch.action.support.DefaultShardOperationFailedException; import org.elasticsearch.action.support.DefaultShardOperationFailedException;
import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.action.support.PlainActionFuture;
import org.elasticsearch.action.support.broadcast.BroadcastResponse;
import org.elasticsearch.cluster.ClusterName; import org.elasticsearch.cluster.ClusterName;
import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.metadata.IndexMetadata; import org.elasticsearch.cluster.metadata.IndexMetadata;
@ -70,13 +70,13 @@ public class ForceMergeStepTests extends AbstractStepTestCase<ForceMergeStep> {
Step.StepKey stepKey = randomStepKey(); Step.StepKey stepKey = randomStepKey();
StepKey nextStepKey = randomStepKey(); StepKey nextStepKey = randomStepKey();
int maxNumSegments = randomIntBetween(1, 10); int maxNumSegments = randomIntBetween(1, 10);
ForceMergeResponse forceMergeResponse = Mockito.mock(ForceMergeResponse.class); BroadcastResponse forceMergeResponse = Mockito.mock(BroadcastResponse.class);
Mockito.when(forceMergeResponse.getStatus()).thenReturn(RestStatus.OK); Mockito.when(forceMergeResponse.getStatus()).thenReturn(RestStatus.OK);
Mockito.doAnswer(invocationOnMock -> { Mockito.doAnswer(invocationOnMock -> {
ForceMergeRequest request = (ForceMergeRequest) invocationOnMock.getArguments()[0]; ForceMergeRequest request = (ForceMergeRequest) invocationOnMock.getArguments()[0];
assertThat(request.maxNumSegments(), equalTo(maxNumSegments)); assertThat(request.maxNumSegments(), equalTo(maxNumSegments));
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
ActionListener<ForceMergeResponse> listener = (ActionListener<ForceMergeResponse>) invocationOnMock.getArguments()[1]; ActionListener<BroadcastResponse> listener = (ActionListener<BroadcastResponse>) invocationOnMock.getArguments()[1];
listener.onResponse(forceMergeResponse); listener.onResponse(forceMergeResponse);
return null; return null;
}).when(indicesClient).forceMerge(any(), any()); }).when(indicesClient).forceMerge(any(), any());
@ -95,7 +95,7 @@ public class ForceMergeStepTests extends AbstractStepTestCase<ForceMergeStep> {
Step.StepKey stepKey = randomStepKey(); Step.StepKey stepKey = randomStepKey();
StepKey nextStepKey = randomStepKey(); StepKey nextStepKey = randomStepKey();
int maxNumSegments = randomIntBetween(1, 10); int maxNumSegments = randomIntBetween(1, 10);
ForceMergeResponse forceMergeResponse = Mockito.mock(ForceMergeResponse.class); BroadcastResponse forceMergeResponse = Mockito.mock(BroadcastResponse.class);
Mockito.when(forceMergeResponse.getStatus()).thenReturn(RestStatus.OK); Mockito.when(forceMergeResponse.getStatus()).thenReturn(RestStatus.OK);
Mockito.doAnswer(invocationOnMock -> { Mockito.doAnswer(invocationOnMock -> {
ForceMergeRequest request = (ForceMergeRequest) invocationOnMock.getArguments()[0]; ForceMergeRequest request = (ForceMergeRequest) invocationOnMock.getArguments()[0];
@ -103,7 +103,7 @@ public class ForceMergeStepTests extends AbstractStepTestCase<ForceMergeStep> {
assertThat(request.indices()[0], equalTo(indexMetadata.getIndex().getName())); assertThat(request.indices()[0], equalTo(indexMetadata.getIndex().getName()));
assertThat(request.maxNumSegments(), equalTo(maxNumSegments)); assertThat(request.maxNumSegments(), equalTo(maxNumSegments));
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
ActionListener<ForceMergeResponse> listener = (ActionListener<ForceMergeResponse>) invocationOnMock.getArguments()[1]; ActionListener<BroadcastResponse> listener = (ActionListener<BroadcastResponse>) invocationOnMock.getArguments()[1];
listener.onFailure(exception); listener.onFailure(exception);
return null; return null;
}).when(indicesClient).forceMerge(any(), any()); }).when(indicesClient).forceMerge(any(), any());
@ -126,7 +126,7 @@ public class ForceMergeStepTests extends AbstractStepTestCase<ForceMergeStep> {
.numberOfReplicas(randomIntBetween(0, 5)) .numberOfReplicas(randomIntBetween(0, 5))
.build(); .build();
Index index = indexMetadata.getIndex(); Index index = indexMetadata.getIndex();
ForceMergeResponse forceMergeResponse = Mockito.mock(ForceMergeResponse.class); BroadcastResponse forceMergeResponse = Mockito.mock(BroadcastResponse.class);
Mockito.when(forceMergeResponse.getTotalShards()).thenReturn(numberOfShards); Mockito.when(forceMergeResponse.getTotalShards()).thenReturn(numberOfShards);
Mockito.when(forceMergeResponse.getFailedShards()).thenReturn(numberOfShards - 1); Mockito.when(forceMergeResponse.getFailedShards()).thenReturn(numberOfShards - 1);
Mockito.when(forceMergeResponse.getStatus()).thenReturn(RestStatus.BAD_REQUEST); Mockito.when(forceMergeResponse.getStatus()).thenReturn(RestStatus.BAD_REQUEST);
@ -143,7 +143,7 @@ public class ForceMergeStepTests extends AbstractStepTestCase<ForceMergeStep> {
Mockito.doAnswer(invocationOnMock -> { Mockito.doAnswer(invocationOnMock -> {
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
ActionListener<ForceMergeResponse> listener = (ActionListener<ForceMergeResponse>) invocationOnMock.getArguments()[1]; ActionListener<BroadcastResponse> listener = (ActionListener<BroadcastResponse>) invocationOnMock.getArguments()[1];
listener.onResponse(forceMergeResponse); listener.onResponse(forceMergeResponse);
return null; return null;
}).when(indicesClient).forceMerge(any(), any()); }).when(indicesClient).forceMerge(any(), any());

View file

@ -17,12 +17,12 @@ import org.elasticsearch.action.admin.indices.create.CreateIndexClusterStateUpda
import org.elasticsearch.action.admin.indices.forcemerge.ForceMergeRequest; import org.elasticsearch.action.admin.indices.forcemerge.ForceMergeRequest;
import org.elasticsearch.action.admin.indices.mapping.get.GetMappingsRequest; import org.elasticsearch.action.admin.indices.mapping.get.GetMappingsRequest;
import org.elasticsearch.action.admin.indices.refresh.RefreshRequest; import org.elasticsearch.action.admin.indices.refresh.RefreshRequest;
import org.elasticsearch.action.admin.indices.refresh.RefreshResponse;
import org.elasticsearch.action.admin.indices.settings.put.UpdateSettingsRequest; import org.elasticsearch.action.admin.indices.settings.put.UpdateSettingsRequest;
import org.elasticsearch.action.downsample.DownsampleAction; import org.elasticsearch.action.downsample.DownsampleAction;
import org.elasticsearch.action.downsample.DownsampleConfig; import org.elasticsearch.action.downsample.DownsampleConfig;
import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.ActiveShardCount; import org.elasticsearch.action.support.ActiveShardCount;
import org.elasticsearch.action.support.broadcast.BroadcastResponse;
import org.elasticsearch.action.support.master.AcknowledgedResponse; import org.elasticsearch.action.support.master.AcknowledgedResponse;
import org.elasticsearch.action.support.master.AcknowledgedTransportMasterNodeAction; import org.elasticsearch.action.support.master.AcknowledgedTransportMasterNodeAction;
import org.elasticsearch.client.internal.Client; import org.elasticsearch.client.internal.Client;
@ -848,7 +848,7 @@ public class TransportDownsampleAction extends AcknowledgedTransportMasterNodeAc
/** /**
* Updates the downsample target index metadata (task status) * Updates the downsample target index metadata (task status)
*/ */
class RefreshDownsampleIndexActionListener implements ActionListener<RefreshResponse> { class RefreshDownsampleIndexActionListener implements ActionListener<BroadcastResponse> {
private final ActionListener<AcknowledgedResponse> actionListener; private final ActionListener<AcknowledgedResponse> actionListener;
private final TaskId parentTask; private final TaskId parentTask;
@ -868,7 +868,7 @@ public class TransportDownsampleAction extends AcknowledgedTransportMasterNodeAc
} }
@Override @Override
public void onResponse(final RefreshResponse response) { public void onResponse(final BroadcastResponse response) {
if (response.getFailedShards() != 0) { if (response.getFailedShards() != 0) {
logger.info("Post refresh failed [{}],{}", downsampleIndexName, Strings.toString(response)); logger.info("Post refresh failed [{}],{}", downsampleIndexName, Strings.toString(response));
} }

View file

@ -9,7 +9,6 @@ package org.elasticsearch.xpack.downsample;
import org.elasticsearch.action.DocWriteRequest; import org.elasticsearch.action.DocWriteRequest;
import org.elasticsearch.action.admin.indices.refresh.RefreshRequest; import org.elasticsearch.action.admin.indices.refresh.RefreshRequest;
import org.elasticsearch.action.admin.indices.refresh.RefreshResponse;
import org.elasticsearch.action.admin.indices.rollover.RolloverRequest; import org.elasticsearch.action.admin.indices.rollover.RolloverRequest;
import org.elasticsearch.action.admin.indices.rollover.RolloverResponse; import org.elasticsearch.action.admin.indices.rollover.RolloverResponse;
import org.elasticsearch.action.admin.indices.settings.put.UpdateSettingsRequest; import org.elasticsearch.action.admin.indices.settings.put.UpdateSettingsRequest;
@ -24,6 +23,7 @@ import org.elasticsearch.action.downsample.DownsampleAction;
import org.elasticsearch.action.downsample.DownsampleConfig; import org.elasticsearch.action.downsample.DownsampleConfig;
import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.index.IndexRequest;
import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchRequest;
import org.elasticsearch.action.support.broadcast.BroadcastResponse;
import org.elasticsearch.action.support.master.AcknowledgedResponse; import org.elasticsearch.action.support.master.AcknowledgedResponse;
import org.elasticsearch.cluster.metadata.ComposableIndexTemplate; import org.elasticsearch.cluster.metadata.ComposableIndexTemplate;
import org.elasticsearch.cluster.metadata.DataStream; import org.elasticsearch.cluster.metadata.DataStream;
@ -240,7 +240,7 @@ public class DownsampleDataStreamTests extends ESSingleNodeTestCase {
final BulkItemResponse[] items = bulkResponse.getItems(); final BulkItemResponse[] items = bulkResponse.getItems();
assertThat(items.length, equalTo(numDocs)); assertThat(items.length, equalTo(numDocs));
assertThat(bulkResponse.hasFailures(), equalTo(false)); assertThat(bulkResponse.hasFailures(), equalTo(false));
final RefreshResponse refreshResponse = indicesAdmin().refresh(new RefreshRequest(dataStream)).actionGet(); final BroadcastResponse refreshResponse = indicesAdmin().refresh(new RefreshRequest(dataStream)).actionGet();
assertThat(refreshResponse.getStatus().getStatus(), equalTo(RestStatus.OK.getStatus())); assertThat(refreshResponse.getStatus().getStatus(), equalTo(RestStatus.OK.getStatus()));
} }
} }

View file

@ -8,9 +8,9 @@ package org.elasticsearch.xpack.graph.test;
import org.apache.lucene.search.BooleanQuery; import org.apache.lucene.search.BooleanQuery;
import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.ActionRequestValidationException;
import org.elasticsearch.action.admin.indices.forcemerge.ForceMergeResponse;
import org.elasticsearch.action.admin.indices.segments.IndexShardSegments; import org.elasticsearch.action.admin.indices.segments.IndexShardSegments;
import org.elasticsearch.action.admin.indices.segments.ShardSegments; import org.elasticsearch.action.admin.indices.segments.ShardSegments;
import org.elasticsearch.action.support.broadcast.BroadcastResponse;
import org.elasticsearch.common.Strings; import org.elasticsearch.common.Strings;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.settings.Settings.Builder; import org.elasticsearch.common.settings.Settings.Builder;
@ -97,7 +97,7 @@ public class GraphTests extends ESSingleNodeTestCase {
indicesAdmin().prepareRefresh("test").get(); indicesAdmin().prepareRefresh("test").get();
// Ensure single segment with no deletes. Hopefully solves test instability in // Ensure single segment with no deletes. Hopefully solves test instability in
// issue https://github.com/elastic/x-pack-elasticsearch/issues/918 // issue https://github.com/elastic/x-pack-elasticsearch/issues/918
ForceMergeResponse actionGet = indicesAdmin().prepareForceMerge("test").setFlush(true).setMaxNumSegments(1).get(); BroadcastResponse actionGet = indicesAdmin().prepareForceMerge("test").setFlush(true).setMaxNumSegments(1).get();
indicesAdmin().prepareRefresh("test").get(); indicesAdmin().prepareRefresh("test").get();
assertAllSuccessful(actionGet); assertAllSuccessful(actionGet);
for (IndexShardSegments seg : indicesAdmin().prepareSegments().get().getIndices().get("test")) { for (IndexShardSegments seg : indicesAdmin().prepareSegments().get().getIndices().get("test")) {

View file

@ -10,12 +10,12 @@ import org.elasticsearch.action.admin.cluster.snapshots.features.ResetFeatureSta
import org.elasticsearch.action.admin.cluster.snapshots.features.ResetFeatureStateRequest; import org.elasticsearch.action.admin.cluster.snapshots.features.ResetFeatureStateRequest;
import org.elasticsearch.action.admin.indices.refresh.RefreshAction; import org.elasticsearch.action.admin.indices.refresh.RefreshAction;
import org.elasticsearch.action.admin.indices.refresh.RefreshRequest; import org.elasticsearch.action.admin.indices.refresh.RefreshRequest;
import org.elasticsearch.action.admin.indices.refresh.RefreshResponse;
import org.elasticsearch.action.admin.indices.template.put.TransportPutComposableIndexTemplateAction; import org.elasticsearch.action.admin.indices.template.put.TransportPutComposableIndexTemplateAction;
import org.elasticsearch.action.datastreams.CreateDataStreamAction; import org.elasticsearch.action.datastreams.CreateDataStreamAction;
import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchRequest;
import org.elasticsearch.action.search.SearchRequestBuilder; import org.elasticsearch.action.search.SearchRequestBuilder;
import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.action.search.TransportSearchAction;
import org.elasticsearch.action.support.broadcast.BroadcastResponse;
import org.elasticsearch.action.support.master.AcknowledgedResponse; import org.elasticsearch.action.support.master.AcknowledgedResponse;
import org.elasticsearch.client.internal.Client; import org.elasticsearch.client.internal.Client;
import org.elasticsearch.cluster.ClusterModule; import org.elasticsearch.cluster.ClusterModule;
@ -308,7 +308,7 @@ abstract class MlNativeIntegTestCase extends ESIntegTestCase {
protected static List<String> fetchAllAuditMessages(String jobId) throws Exception { protected static List<String> fetchAllAuditMessages(String jobId) throws Exception {
RefreshRequest refreshRequest = new RefreshRequest(NotificationsIndex.NOTIFICATIONS_INDEX); RefreshRequest refreshRequest = new RefreshRequest(NotificationsIndex.NOTIFICATIONS_INDEX);
RefreshResponse refreshResponse = client().execute(RefreshAction.INSTANCE, refreshRequest).actionGet(); BroadcastResponse refreshResponse = client().execute(RefreshAction.INSTANCE, refreshRequest).actionGet();
assertThat(refreshResponse.getStatus().getStatus(), anyOf(equalTo(200), equalTo(201))); assertThat(refreshResponse.getStatus().getStatus(), anyOf(equalTo(200), equalTo(201)));
SearchRequest searchRequest = new SearchRequestBuilder(client()).setIndices(NotificationsIndex.NOTIFICATIONS_INDEX) SearchRequest searchRequest = new SearchRequestBuilder(client()).setIndices(NotificationsIndex.NOTIFICATIONS_INDEX)

View file

@ -7,12 +7,12 @@
package org.elasticsearch.xpack.ml.integration; package org.elasticsearch.xpack.ml.integration;
import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.DocWriteResponse;
import org.elasticsearch.action.admin.indices.refresh.RefreshResponse;
import org.elasticsearch.action.bulk.BulkRequestBuilder; import org.elasticsearch.action.bulk.BulkRequestBuilder;
import org.elasticsearch.action.bulk.BulkResponse; import org.elasticsearch.action.bulk.BulkResponse;
import org.elasticsearch.action.delete.DeleteRequest; import org.elasticsearch.action.delete.DeleteRequest;
import org.elasticsearch.action.index.IndexRequestBuilder; import org.elasticsearch.action.index.IndexRequestBuilder;
import org.elasticsearch.action.support.WriteRequest; import org.elasticsearch.action.support.WriteRequest;
import org.elasticsearch.action.support.broadcast.BroadcastResponse;
import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesArray;
import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.license.License; import org.elasticsearch.license.License;
@ -109,7 +109,7 @@ public class TrainedModelProviderIT extends MlSingleNodeTestCase {
); );
assertThat(exceptionHolder.get(), is(nullValue())); assertThat(exceptionHolder.get(), is(nullValue()));
AtomicReference<RefreshResponse> refreshResponseAtomicReference = new AtomicReference<>(); AtomicReference<BroadcastResponse> refreshResponseAtomicReference = new AtomicReference<>();
blockingCall( blockingCall(
listener -> trainedModelProvider.refreshInferenceIndex(listener), listener -> trainedModelProvider.refreshInferenceIndex(listener),
refreshResponseAtomicReference, refreshResponseAtomicReference,
@ -198,7 +198,7 @@ public class TrainedModelProviderIT extends MlSingleNodeTestCase {
); );
blockingCall( blockingCall(
listener -> trainedModelProvider.refreshInferenceIndex(listener), listener -> trainedModelProvider.refreshInferenceIndex(listener),
new AtomicReference<RefreshResponse>(), new AtomicReference<BroadcastResponse>(),
new AtomicReference<>() new AtomicReference<>()
); );

View file

@ -12,7 +12,7 @@ import org.apache.logging.log4j.Logger;
import org.apache.lucene.util.RamUsageEstimator; import org.apache.lucene.util.RamUsageEstimator;
import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.LatchedActionListener; import org.elasticsearch.action.LatchedActionListener;
import org.elasticsearch.action.admin.indices.refresh.RefreshResponse; import org.elasticsearch.action.support.broadcast.BroadcastResponse;
import org.elasticsearch.common.Strings; import org.elasticsearch.common.Strings;
import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.common.xcontent.XContentHelper;
import org.elasticsearch.license.License; import org.elasticsearch.license.License;
@ -157,7 +157,7 @@ public class ChunkedTrainedModelPersister {
CountDownLatch latch = new CountDownLatch(1); CountDownLatch latch = new CountDownLatch(1);
// Latch is attached to this action as it is the last one to execute. // Latch is attached to this action as it is the last one to execute.
ActionListener<RefreshResponse> refreshListener = new LatchedActionListener<>(ActionListener.wrap(refreshed -> { ActionListener<BroadcastResponse> refreshListener = new LatchedActionListener<>(ActionListener.wrap(refreshed -> {
if (refreshed != null) { if (refreshed != null) {
LOGGER.debug(() -> "[" + analytics.getId() + "] refreshed inference index after model store"); LOGGER.debug(() -> "[" + analytics.getId() + "] refreshed inference index after model store");
} }
@ -210,7 +210,7 @@ public class ChunkedTrainedModelPersister {
CountDownLatch latch = new CountDownLatch(1); CountDownLatch latch = new CountDownLatch(1);
// Latch is attached to this action as it is the last one to execute. // Latch is attached to this action as it is the last one to execute.
ActionListener<RefreshResponse> refreshListener = new LatchedActionListener<>(ActionListener.wrap(refreshed -> { ActionListener<BroadcastResponse> refreshListener = new LatchedActionListener<>(ActionListener.wrap(refreshed -> {
if (refreshed != null) { if (refreshed != null) {
LOGGER.debug(() -> "[" + analytics.getId() + "] refreshed inference index after model metadata store"); LOGGER.debug(() -> "[" + analytics.getId() + "] refreshed inference index after model metadata store");
} }

View file

@ -12,7 +12,7 @@ import org.apache.logging.log4j.Logger;
import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.admin.indices.refresh.RefreshAction; import org.elasticsearch.action.admin.indices.refresh.RefreshAction;
import org.elasticsearch.action.admin.indices.refresh.RefreshRequest; import org.elasticsearch.action.admin.indices.refresh.RefreshRequest;
import org.elasticsearch.action.admin.indices.refresh.RefreshResponse; import org.elasticsearch.action.support.broadcast.BroadcastResponse;
import org.elasticsearch.client.internal.ParentTaskAssigningClient; import org.elasticsearch.client.internal.ParentTaskAssigningClient;
import org.elasticsearch.client.internal.node.NodeClient; import org.elasticsearch.client.internal.node.NodeClient;
import org.elasticsearch.tasks.TaskId; import org.elasticsearch.tasks.TaskId;
@ -76,7 +76,7 @@ abstract class AbstractDataFrameAnalyticsStep implements DataFrameAnalyticsStep
protected abstract void doExecute(ActionListener<StepResponse> listener); protected abstract void doExecute(ActionListener<StepResponse> listener);
protected void refreshDestAsync(ActionListener<RefreshResponse> refreshListener) { protected void refreshDestAsync(ActionListener<BroadcastResponse> refreshListener) {
ParentTaskAssigningClient parentTaskClient = parentTaskClient(); ParentTaskAssigningClient parentTaskClient = parentTaskClient();
executeWithHeadersAsync( executeWithHeadersAsync(
config.getHeaders(), config.getHeaders(),

View file

@ -8,7 +8,7 @@
package org.elasticsearch.xpack.ml.dataframe.steps; package org.elasticsearch.xpack.ml.dataframe.steps;
import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.admin.indices.refresh.RefreshResponse; import org.elasticsearch.action.support.broadcast.BroadcastResponse;
import org.elasticsearch.client.internal.ParentTaskAssigningClient; import org.elasticsearch.client.internal.ParentTaskAssigningClient;
import org.elasticsearch.client.internal.node.NodeClient; import org.elasticsearch.client.internal.node.NodeClient;
import org.elasticsearch.core.TimeValue; import org.elasticsearch.core.TimeValue;
@ -63,7 +63,7 @@ public class AnalysisStep extends AbstractDataFrameAnalyticsStep {
listener::onFailure listener::onFailure
); );
ActionListener<RefreshResponse> refreshListener = ActionListener.wrap(refreshResponse -> { ActionListener<BroadcastResponse> refreshListener = ActionListener.wrap(refreshResponse -> {
// TODO This could fail with errors. In that case we get stuck with the copied index. // TODO This could fail with errors. In that case we get stuck with the copied index.
// We could delete the index in case of failure or we could try building the factory before reindexing // We could delete the index in case of failure or we could try building the factory before reindexing
// to catch the error early on. // to catch the error early on.

View file

@ -13,10 +13,10 @@ import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.DocWriteResponse;
import org.elasticsearch.action.admin.indices.refresh.RefreshAction; import org.elasticsearch.action.admin.indices.refresh.RefreshAction;
import org.elasticsearch.action.admin.indices.refresh.RefreshRequest; import org.elasticsearch.action.admin.indices.refresh.RefreshRequest;
import org.elasticsearch.action.admin.indices.refresh.RefreshResponse;
import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.index.IndexRequest;
import org.elasticsearch.action.index.TransportIndexAction; import org.elasticsearch.action.index.TransportIndexAction;
import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.action.support.broadcast.BroadcastResponse;
import org.elasticsearch.client.internal.node.NodeClient; import org.elasticsearch.client.internal.node.NodeClient;
import org.elasticsearch.core.TimeValue; import org.elasticsearch.core.TimeValue;
import org.elasticsearch.xcontent.ToXContent; import org.elasticsearch.xcontent.ToXContent;
@ -60,7 +60,7 @@ public class FinalStep extends AbstractDataFrameAnalyticsStep {
@Override @Override
protected void doExecute(ActionListener<StepResponse> listener) { protected void doExecute(ActionListener<StepResponse> listener) {
ActionListener<RefreshResponse> refreshListener = ActionListener.wrap( ActionListener<BroadcastResponse> refreshListener = ActionListener.wrap(
refreshResponse -> listener.onResponse(new StepResponse(false)), refreshResponse -> listener.onResponse(new StepResponse(false)),
listener::onFailure listener::onFailure
); );
@ -89,7 +89,7 @@ public class FinalStep extends AbstractDataFrameAnalyticsStep {
} }
} }
private void refreshIndices(ActionListener<RefreshResponse> listener) { private void refreshIndices(ActionListener<BroadcastResponse> listener) {
RefreshRequest refreshRequest = new RefreshRequest( RefreshRequest refreshRequest = new RefreshRequest(
AnomalyDetectorsIndex.jobStateIndexPattern(), AnomalyDetectorsIndex.jobStateIndexPattern(),
MlStatsIndex.indexPattern(), MlStatsIndex.indexPattern(),

View file

@ -11,9 +11,9 @@ import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.Logger;
import org.elasticsearch.ResourceNotFoundException; import org.elasticsearch.ResourceNotFoundException;
import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.admin.indices.refresh.RefreshResponse;
import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchRequest;
import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.action.search.TransportSearchAction;
import org.elasticsearch.action.support.broadcast.BroadcastResponse;
import org.elasticsearch.client.internal.node.NodeClient; import org.elasticsearch.client.internal.node.NodeClient;
import org.elasticsearch.core.TimeValue; import org.elasticsearch.core.TimeValue;
import org.elasticsearch.index.query.QueryBuilders; import org.elasticsearch.index.query.QueryBuilders;
@ -85,7 +85,7 @@ public class InferenceStep extends AbstractDataFrameAnalyticsStep {
} }
}, listener::onFailure); }, listener::onFailure);
ActionListener<RefreshResponse> refreshDestListener = ActionListener.wrap( ActionListener<BroadcastResponse> refreshDestListener = ActionListener.wrap(
refreshResponse -> searchIfTestDocsExist(testDocsExistListener), refreshResponse -> searchIfTestDocsExist(testDocsExistListener),
listener::onFailure listener::onFailure
); );

View file

@ -16,7 +16,6 @@ import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.DocWriteRequest; import org.elasticsearch.action.DocWriteRequest;
import org.elasticsearch.action.admin.indices.refresh.RefreshAction; import org.elasticsearch.action.admin.indices.refresh.RefreshAction;
import org.elasticsearch.action.admin.indices.refresh.RefreshRequest; import org.elasticsearch.action.admin.indices.refresh.RefreshRequest;
import org.elasticsearch.action.admin.indices.refresh.RefreshResponse;
import org.elasticsearch.action.bulk.BulkAction; import org.elasticsearch.action.bulk.BulkAction;
import org.elasticsearch.action.bulk.BulkItemResponse; import org.elasticsearch.action.bulk.BulkItemResponse;
import org.elasticsearch.action.bulk.BulkRequestBuilder; import org.elasticsearch.action.bulk.BulkRequestBuilder;
@ -30,6 +29,7 @@ import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.action.search.TransportSearchAction;
import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.action.support.WriteRequest; import org.elasticsearch.action.support.WriteRequest;
import org.elasticsearch.action.support.broadcast.BroadcastResponse;
import org.elasticsearch.client.internal.Client; import org.elasticsearch.client.internal.Client;
import org.elasticsearch.common.CheckedBiFunction; import org.elasticsearch.common.CheckedBiFunction;
import org.elasticsearch.common.Numbers; import org.elasticsearch.common.Numbers;
@ -419,7 +419,7 @@ public class TrainedModelProvider {
})); }));
} }
public void refreshInferenceIndex(ActionListener<RefreshResponse> listener) { public void refreshInferenceIndex(ActionListener<BroadcastResponse> listener) {
executeAsyncWithOrigin( executeAsyncWithOrigin(
client, client,
ML_ORIGIN, ML_ORIGIN,

View file

@ -15,7 +15,6 @@ import org.elasticsearch.action.admin.indices.alias.get.GetAliasesResponse;
import org.elasticsearch.action.admin.indices.delete.DeleteIndexRequest; import org.elasticsearch.action.admin.indices.delete.DeleteIndexRequest;
import org.elasticsearch.action.admin.indices.refresh.RefreshAction; import org.elasticsearch.action.admin.indices.refresh.RefreshAction;
import org.elasticsearch.action.admin.indices.refresh.RefreshRequest; import org.elasticsearch.action.admin.indices.refresh.RefreshRequest;
import org.elasticsearch.action.admin.indices.refresh.RefreshResponse;
import org.elasticsearch.action.bulk.BulkItemResponse; import org.elasticsearch.action.bulk.BulkItemResponse;
import org.elasticsearch.action.search.MultiSearchRequest; import org.elasticsearch.action.search.MultiSearchRequest;
import org.elasticsearch.action.search.MultiSearchResponse; import org.elasticsearch.action.search.MultiSearchResponse;
@ -23,6 +22,7 @@ import org.elasticsearch.action.search.SearchRequest;
import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.search.TransportMultiSearchAction; import org.elasticsearch.action.search.TransportMultiSearchAction;
import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.action.support.broadcast.BroadcastResponse;
import org.elasticsearch.action.support.master.AcknowledgedResponse; import org.elasticsearch.action.support.master.AcknowledgedResponse;
import org.elasticsearch.client.internal.Client; import org.elasticsearch.client.internal.Client;
import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterState;
@ -451,7 +451,7 @@ public class JobDataDeleter {
) { ) {
assert indices.length > 0; assert indices.length > 0;
ActionListener<RefreshResponse> refreshListener = ActionListener.wrap(refreshResponse -> { ActionListener<BroadcastResponse> refreshListener = ActionListener.wrap(refreshResponse -> {
logger.info("[{}] running delete by query on [{}]", jobId, String.join(", ", indices)); logger.info("[{}] running delete by query on [{}]", jobId, String.join(", ", indices));
ConstantScoreQueryBuilder query = new ConstantScoreQueryBuilder(new TermQueryBuilder(Job.ID.getPreferredName(), jobId)); ConstantScoreQueryBuilder query = new ConstantScoreQueryBuilder(new TermQueryBuilder(Job.ID.getPreferredName(), jobId));
DeleteByQueryRequest request = new DeleteByQueryRequest(indices).setQuery(query) DeleteByQueryRequest request = new DeleteByQueryRequest(indices).setQuery(query)

View file

@ -8,7 +8,7 @@
package org.elasticsearch.xpack.ml.dataframe.process; package org.elasticsearch.xpack.ml.dataframe.process;
import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.admin.indices.refresh.RefreshResponse; import org.elasticsearch.action.support.broadcast.BroadcastResponse;
import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.common.xcontent.XContentHelper;
import org.elasticsearch.license.License; import org.elasticsearch.license.License;
import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.ESTestCase;
@ -102,7 +102,7 @@ public class ChunkedTrainedModelPersisterTests extends ESTestCase {
}).when(trainedModelProvider).storeTrainedModelMetadata(any(TrainedModelMetadata.class), any(ActionListener.class)); }).when(trainedModelProvider).storeTrainedModelMetadata(any(TrainedModelMetadata.class), any(ActionListener.class));
doAnswer(invocationOnMock -> { doAnswer(invocationOnMock -> {
ActionListener<RefreshResponse> storeListener = (ActionListener<RefreshResponse>) invocationOnMock.getArguments()[0]; ActionListener<BroadcastResponse> storeListener = (ActionListener<BroadcastResponse>) invocationOnMock.getArguments()[0];
storeListener.onResponse(null); storeListener.onResponse(null);
return null; return null;
}).when(trainedModelProvider).refreshInferenceIndex(any(ActionListener.class)); }).when(trainedModelProvider).refreshInferenceIndex(any(ActionListener.class));

View file

@ -11,12 +11,12 @@ import org.apache.logging.log4j.Logger;
import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.admin.indices.refresh.RefreshRequest; import org.elasticsearch.action.admin.indices.refresh.RefreshRequest;
import org.elasticsearch.action.admin.indices.refresh.RefreshResponse;
import org.elasticsearch.action.bulk.BulkAction; import org.elasticsearch.action.bulk.BulkAction;
import org.elasticsearch.action.bulk.BulkRequest; import org.elasticsearch.action.bulk.BulkRequest;
import org.elasticsearch.action.bulk.BulkResponse; import org.elasticsearch.action.bulk.BulkResponse;
import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.action.search.TransportSearchAction;
import org.elasticsearch.action.support.broadcast.BroadcastResponse;
import org.elasticsearch.client.internal.Client; import org.elasticsearch.client.internal.Client;
import org.elasticsearch.client.internal.ParentTaskAssigningClient; import org.elasticsearch.client.internal.ParentTaskAssigningClient;
import org.elasticsearch.common.scheduler.SchedulerEngine; import org.elasticsearch.common.scheduler.SchedulerEngine;
@ -164,10 +164,10 @@ public class RollupJobTask extends AllocatedPersistentTask implements SchedulerE
@Override @Override
protected void onFinish(ActionListener<Void> listener) { protected void onFinish(ActionListener<Void> listener) {
final RollupJobConfig jobConfig = job.getConfig(); final RollupJobConfig jobConfig = job.getConfig();
final ActionListener<RefreshResponse> refreshResponseActionListener = new ActionListener<>() { final ActionListener<BroadcastResponse> refreshResponseActionListener = new ActionListener<>() {
@Override @Override
public void onResponse(RefreshResponse refreshResponse) { public void onResponse(BroadcastResponse refreshResponse) {
logger.trace("refreshing rollup index {} successful for job {}", jobConfig.getRollupIndex(), jobConfig.getId()); logger.trace("refreshing rollup index {} successful for job {}", jobConfig.getRollupIndex(), jobConfig.getId());
listener.onResponse(null); listener.onResponse(null);
} }

View file

@ -11,10 +11,10 @@ import org.elasticsearch.action.ActionRequest;
import org.elasticsearch.action.ActionResponse; import org.elasticsearch.action.ActionResponse;
import org.elasticsearch.action.ActionType; import org.elasticsearch.action.ActionType;
import org.elasticsearch.action.admin.indices.refresh.RefreshAction; import org.elasticsearch.action.admin.indices.refresh.RefreshAction;
import org.elasticsearch.action.admin.indices.refresh.RefreshResponse;
import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.search.ShardSearchFailure; import org.elasticsearch.action.search.ShardSearchFailure;
import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.action.search.TransportSearchAction;
import org.elasticsearch.action.support.broadcast.BroadcastResponse;
import org.elasticsearch.client.internal.Client; import org.elasticsearch.client.internal.Client;
import org.elasticsearch.common.scheduler.SchedulerEngine; import org.elasticsearch.common.scheduler.SchedulerEngine;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
@ -590,7 +590,7 @@ public class RollupJobTaskTests extends ESTestCase {
RollupJob job = new RollupJob(ConfigTestHelpers.randomRollupJobConfig(random()), Collections.emptyMap()); RollupJob job = new RollupJob(ConfigTestHelpers.randomRollupJobConfig(random()), Collections.emptyMap());
Client client = mock(Client.class); Client client = mock(Client.class);
doAnswer(invocationOnMock -> { doAnswer(invocationOnMock -> {
RefreshResponse r = new RefreshResponse(2, 2, 0, Collections.emptyList()); BroadcastResponse r = new BroadcastResponse(2, 2, 0, Collections.emptyList());
((ActionListener) invocationOnMock.getArguments()[2]).onResponse(r); ((ActionListener) invocationOnMock.getArguments()[2]).onResponse(r);
return null; return null;
}).when(client).execute(eq(RefreshAction.INSTANCE), any(), any()); }).when(client).execute(eq(RefreshAction.INSTANCE), any(), any());
@ -697,7 +697,7 @@ public class RollupJobTaskTests extends ESTestCase {
RollupJob job = new RollupJob(ConfigTestHelpers.randomRollupJobConfig(random()), headers); RollupJob job = new RollupJob(ConfigTestHelpers.randomRollupJobConfig(random()), headers);
Client client = mock(Client.class); Client client = mock(Client.class);
doAnswer(invocationOnMock -> { doAnswer(invocationOnMock -> {
RefreshResponse r = new RefreshResponse(2, 2, 0, Collections.emptyList()); BroadcastResponse r = new BroadcastResponse(2, 2, 0, Collections.emptyList());
((ActionListener) invocationOnMock.getArguments()[2]).onResponse(r); ((ActionListener) invocationOnMock.getArguments()[2]).onResponse(r);
return null; return null;
}).when(client).execute(eq(RefreshAction.INSTANCE), any(), any()); }).when(client).execute(eq(RefreshAction.INSTANCE), any(), any());
@ -806,7 +806,7 @@ public class RollupJobTaskTests extends ESTestCase {
RollupJob job = new RollupJob(ConfigTestHelpers.randomRollupJobConfig(random()), headers); RollupJob job = new RollupJob(ConfigTestHelpers.randomRollupJobConfig(random()), headers);
Client client = mock(Client.class); Client client = mock(Client.class);
doAnswer(invocationOnMock -> { doAnswer(invocationOnMock -> {
RefreshResponse r = new RefreshResponse(2, 2, 0, Collections.emptyList()); BroadcastResponse r = new BroadcastResponse(2, 2, 0, Collections.emptyList());
((ActionListener) invocationOnMock.getArguments()[2]).onResponse(r); ((ActionListener) invocationOnMock.getArguments()[2]).onResponse(r);
return null; return null;
}).when(client).execute(eq(RefreshAction.INSTANCE), any(), any()); }).when(client).execute(eq(RefreshAction.INSTANCE), any(), any());

View file

@ -12,6 +12,7 @@ import org.elasticsearch.action.ActionFuture;
import org.elasticsearch.action.FailedNodeException; import org.elasticsearch.action.FailedNodeException;
import org.elasticsearch.action.admin.cluster.snapshots.restore.RestoreSnapshotResponse; import org.elasticsearch.action.admin.cluster.snapshots.restore.RestoreSnapshotResponse;
import org.elasticsearch.action.support.DefaultShardOperationFailedException; import org.elasticsearch.action.support.DefaultShardOperationFailedException;
import org.elasticsearch.action.support.broadcast.BroadcastResponse;
import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.health.ClusterHealthStatus; import org.elasticsearch.cluster.health.ClusterHealthStatus;
import org.elasticsearch.cluster.metadata.Metadata; import org.elasticsearch.cluster.metadata.Metadata;
@ -31,7 +32,6 @@ import org.elasticsearch.xpack.core.searchablesnapshots.MountSearchableSnapshotA
import org.elasticsearch.xpack.core.searchablesnapshots.MountSearchableSnapshotRequest; import org.elasticsearch.xpack.core.searchablesnapshots.MountSearchableSnapshotRequest;
import org.elasticsearch.xpack.searchablesnapshots.action.ClearSearchableSnapshotsCacheAction; import org.elasticsearch.xpack.searchablesnapshots.action.ClearSearchableSnapshotsCacheAction;
import org.elasticsearch.xpack.searchablesnapshots.action.ClearSearchableSnapshotsCacheRequest; import org.elasticsearch.xpack.searchablesnapshots.action.ClearSearchableSnapshotsCacheRequest;
import org.elasticsearch.xpack.searchablesnapshots.action.ClearSearchableSnapshotsCacheResponse;
import org.elasticsearch.xpack.searchablesnapshots.action.SearchableSnapshotsStatsAction; import org.elasticsearch.xpack.searchablesnapshots.action.SearchableSnapshotsStatsAction;
import org.elasticsearch.xpack.searchablesnapshots.action.SearchableSnapshotsStatsRequest; import org.elasticsearch.xpack.searchablesnapshots.action.SearchableSnapshotsStatsRequest;
import org.elasticsearch.xpack.searchablesnapshots.action.SearchableSnapshotsStatsResponse; import org.elasticsearch.xpack.searchablesnapshots.action.SearchableSnapshotsStatsResponse;
@ -121,11 +121,11 @@ public class SearchableSnapshotsLicenseIntegTests extends BaseFrozenSearchableSn
} }
public void testClearCacheRequiresLicense() throws ExecutionException, InterruptedException { public void testClearCacheRequiresLicense() throws ExecutionException, InterruptedException {
final ActionFuture<ClearSearchableSnapshotsCacheResponse> future = client().execute( final ActionFuture<BroadcastResponse> future = client().execute(
ClearSearchableSnapshotsCacheAction.INSTANCE, ClearSearchableSnapshotsCacheAction.INSTANCE,
new ClearSearchableSnapshotsCacheRequest(indexName) new ClearSearchableSnapshotsCacheRequest(indexName)
); );
final ClearSearchableSnapshotsCacheResponse response = future.get(); final BroadcastResponse response = future.get();
assertThat(response.getTotalShards(), greaterThan(0)); assertThat(response.getTotalShards(), greaterThan(0));
assertThat(response.getSuccessfulShards(), equalTo(0)); assertThat(response.getSuccessfulShards(), equalTo(0));
for (DefaultShardOperationFailedException shardFailure : response.getShardFailures()) { for (DefaultShardOperationFailedException shardFailure : response.getShardFailures()) {

View file

@ -8,10 +8,9 @@
package org.elasticsearch.xpack.searchablesnapshots.cache.blob; package org.elasticsearch.xpack.searchablesnapshots.cache.blob;
import org.apache.lucene.store.AlreadyClosedException; import org.apache.lucene.store.AlreadyClosedException;
import org.elasticsearch.action.admin.indices.forcemerge.ForceMergeResponse;
import org.elasticsearch.action.admin.indices.refresh.RefreshResponse;
import org.elasticsearch.action.index.IndexRequestBuilder; import org.elasticsearch.action.index.IndexRequestBuilder;
import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.action.support.broadcast.BroadcastResponse;
import org.elasticsearch.blobcache.shared.SharedBlobCacheService; import org.elasticsearch.blobcache.shared.SharedBlobCacheService;
import org.elasticsearch.client.internal.Client; import org.elasticsearch.client.internal.Client;
import org.elasticsearch.client.internal.OriginSettingClient; import org.elasticsearch.client.internal.OriginSettingClient;
@ -139,7 +138,7 @@ public class SearchableSnapshotsBlobStoreCacheIntegTests extends BaseFrozenSearc
if (randomBoolean()) { if (randomBoolean()) {
logger.info("--> force-merging index before snapshotting"); logger.info("--> force-merging index before snapshotting");
final ForceMergeResponse forceMergeResponse = indicesAdmin().prepareForceMerge(indexName).setMaxNumSegments(1).get(); final BroadcastResponse forceMergeResponse = indicesAdmin().prepareForceMerge(indexName).setMaxNumSegments(1).get();
assertThat(forceMergeResponse.getSuccessfulShards(), equalTo(numberOfShards.totalNumShards)); assertThat(forceMergeResponse.getSuccessfulShards(), equalTo(numberOfShards.totalNumShards));
assertThat(forceMergeResponse.getFailedShards(), equalTo(0)); assertThat(forceMergeResponse.getFailedShards(), equalTo(0));
} }
@ -355,7 +354,7 @@ public class SearchableSnapshotsBlobStoreCacheIntegTests extends BaseFrozenSearc
private void refreshSystemIndex() { private void refreshSystemIndex() {
try { try {
final RefreshResponse refreshResponse = systemClient().admin().indices().prepareRefresh(SNAPSHOT_BLOB_CACHE_INDEX).get(); final BroadcastResponse refreshResponse = systemClient().admin().indices().prepareRefresh(SNAPSHOT_BLOB_CACHE_INDEX).get();
assertThat(refreshResponse.getSuccessfulShards(), greaterThan(0)); assertThat(refreshResponse.getSuccessfulShards(), greaterThan(0));
assertThat(refreshResponse.getFailedShards(), equalTo(0)); assertThat(refreshResponse.getFailedShards(), equalTo(0));
} catch (IndexNotFoundException indexNotFoundException) { } catch (IndexNotFoundException indexNotFoundException) {

View file

@ -10,9 +10,9 @@ package org.elasticsearch.xpack.searchablesnapshots.cache.blob;
import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.admin.cluster.snapshots.restore.RestoreSnapshotResponse; import org.elasticsearch.action.admin.cluster.snapshots.restore.RestoreSnapshotResponse;
import org.elasticsearch.action.admin.indices.refresh.RefreshRequest; import org.elasticsearch.action.admin.indices.refresh.RefreshRequest;
import org.elasticsearch.action.admin.indices.refresh.RefreshResponse;
import org.elasticsearch.action.index.IndexRequestBuilder; import org.elasticsearch.action.index.IndexRequestBuilder;
import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.action.support.broadcast.BroadcastResponse;
import org.elasticsearch.blobcache.common.ByteRange; import org.elasticsearch.blobcache.common.ByteRange;
import org.elasticsearch.client.internal.Client; import org.elasticsearch.client.internal.Client;
import org.elasticsearch.client.internal.OriginSettingClient; import org.elasticsearch.client.internal.OriginSettingClient;
@ -329,7 +329,7 @@ public class SearchableSnapshotsBlobStoreCacheMaintenanceIntegTests extends Base
private void refreshSystemIndex(boolean failIfNotExist) { private void refreshSystemIndex(boolean failIfNotExist) {
try { try {
final RefreshResponse refreshResponse = systemClient().admin() final BroadcastResponse refreshResponse = systemClient().admin()
.indices() .indices()
.prepareRefresh(SNAPSHOT_BLOB_CACHE_INDEX) .prepareRefresh(SNAPSHOT_BLOB_CACHE_INDEX)
.setIndicesOptions(failIfNotExist ? RefreshRequest.DEFAULT_INDICES_OPTIONS : IndicesOptions.LENIENT_EXPAND_OPEN) .setIndicesOptions(failIfNotExist ? RefreshRequest.DEFAULT_INDICES_OPTIONS : IndicesOptions.LENIENT_EXPAND_OPEN)

View file

@ -7,6 +7,7 @@
package org.elasticsearch.xpack.searchablesnapshots.cache.shared; package org.elasticsearch.xpack.searchablesnapshots.cache.shared;
import org.elasticsearch.action.support.broadcast.BroadcastResponse;
import org.elasticsearch.blobcache.shared.SharedBlobCacheService; import org.elasticsearch.blobcache.shared.SharedBlobCacheService;
import org.elasticsearch.cluster.metadata.IndexMetadata; import org.elasticsearch.cluster.metadata.IndexMetadata;
import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.routing.ShardRouting;
@ -22,7 +23,6 @@ import org.elasticsearch.xpack.searchablesnapshots.BaseFrozenSearchableSnapshots
import org.elasticsearch.xpack.searchablesnapshots.SearchableSnapshots; import org.elasticsearch.xpack.searchablesnapshots.SearchableSnapshots;
import org.elasticsearch.xpack.searchablesnapshots.action.ClearSearchableSnapshotsCacheAction; import org.elasticsearch.xpack.searchablesnapshots.action.ClearSearchableSnapshotsCacheAction;
import org.elasticsearch.xpack.searchablesnapshots.action.ClearSearchableSnapshotsCacheRequest; import org.elasticsearch.xpack.searchablesnapshots.action.ClearSearchableSnapshotsCacheRequest;
import org.elasticsearch.xpack.searchablesnapshots.action.ClearSearchableSnapshotsCacheResponse;
import org.elasticsearch.xpack.searchablesnapshots.action.cache.TransportSearchableSnapshotsNodeCachesStatsAction; import org.elasticsearch.xpack.searchablesnapshots.action.cache.TransportSearchableSnapshotsNodeCachesStatsAction;
import org.elasticsearch.xpack.searchablesnapshots.action.cache.TransportSearchableSnapshotsNodeCachesStatsAction.NodeCachesStatsResponse; import org.elasticsearch.xpack.searchablesnapshots.action.cache.TransportSearchableSnapshotsNodeCachesStatsAction.NodeCachesStatsResponse;
import org.elasticsearch.xpack.searchablesnapshots.action.cache.TransportSearchableSnapshotsNodeCachesStatsAction.NodesCachesStatsResponse; import org.elasticsearch.xpack.searchablesnapshots.action.cache.TransportSearchableSnapshotsNodeCachesStatsAction.NodesCachesStatsResponse;
@ -117,7 +117,7 @@ public class NodesCachesStatsIntegTests extends BaseFrozenSearchableSnapshotsInt
assertExecutorIsIdle(SearchableSnapshots.CACHE_FETCH_ASYNC_THREAD_POOL_NAME); assertExecutorIsIdle(SearchableSnapshots.CACHE_FETCH_ASYNC_THREAD_POOL_NAME);
final ClearSearchableSnapshotsCacheResponse clearCacheResponse = client().execute( final BroadcastResponse clearCacheResponse = client().execute(
ClearSearchableSnapshotsCacheAction.INSTANCE, ClearSearchableSnapshotsCacheAction.INSTANCE,
new ClearSearchableSnapshotsCacheRequest(mountedIndex) new ClearSearchableSnapshotsCacheRequest(mountedIndex)
).actionGet(); ).actionGet();

View file

@ -7,13 +7,14 @@
package org.elasticsearch.xpack.searchablesnapshots.action; package org.elasticsearch.xpack.searchablesnapshots.action;
import org.elasticsearch.action.ActionType; import org.elasticsearch.action.ActionType;
import org.elasticsearch.action.support.broadcast.BroadcastResponse;
public class ClearSearchableSnapshotsCacheAction extends ActionType<ClearSearchableSnapshotsCacheResponse> { public class ClearSearchableSnapshotsCacheAction extends ActionType<BroadcastResponse> {
public static final ClearSearchableSnapshotsCacheAction INSTANCE = new ClearSearchableSnapshotsCacheAction(); public static final ClearSearchableSnapshotsCacheAction INSTANCE = new ClearSearchableSnapshotsCacheAction();
static final String NAME = "cluster:admin/xpack/searchable_snapshots/cache/clear"; static final String NAME = "cluster:admin/xpack/searchable_snapshots/cache/clear";
private ClearSearchableSnapshotsCacheAction() { private ClearSearchableSnapshotsCacheAction() {
super(NAME, ClearSearchableSnapshotsCacheResponse::new); super(NAME, BroadcastResponse::new);
} }
} }

View file

@ -1,30 +0,0 @@
/*
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the Elastic License
* 2.0; you may not use this file except in compliance with the Elastic License
* 2.0.
*/
package org.elasticsearch.xpack.searchablesnapshots.action;
import org.elasticsearch.action.support.DefaultShardOperationFailedException;
import org.elasticsearch.action.support.broadcast.BroadcastResponse;
import org.elasticsearch.common.io.stream.StreamInput;
import java.io.IOException;
import java.util.List;
public class ClearSearchableSnapshotsCacheResponse extends BroadcastResponse {
ClearSearchableSnapshotsCacheResponse(StreamInput in) throws IOException {
super(in);
}
ClearSearchableSnapshotsCacheResponse(
int totalShards,
int successfulShards,
int failedShards,
List<DefaultShardOperationFailedException> shardFailures
) {
super(totalShards, successfulShards, failedShards, shardFailures);
}
}

View file

@ -7,6 +7,7 @@
package org.elasticsearch.xpack.searchablesnapshots.action; package org.elasticsearch.xpack.searchablesnapshots.action;
import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.broadcast.BroadcastResponse;
import org.elasticsearch.action.support.broadcast.node.TransportBroadcastByNodeAction.EmptyResult; import org.elasticsearch.action.support.broadcast.node.TransportBroadcastByNodeAction.EmptyResult;
import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
@ -24,7 +25,7 @@ import java.io.IOException;
public class TransportClearSearchableSnapshotsCacheAction extends AbstractTransportSearchableSnapshotsAction< public class TransportClearSearchableSnapshotsCacheAction extends AbstractTransportSearchableSnapshotsAction<
ClearSearchableSnapshotsCacheRequest, ClearSearchableSnapshotsCacheRequest,
ClearSearchableSnapshotsCacheResponse, BroadcastResponse,
EmptyResult> { EmptyResult> {
@Inject @Inject
@ -56,11 +57,11 @@ public class TransportClearSearchableSnapshotsCacheAction extends AbstractTransp
} }
@Override @Override
protected ResponseFactory<ClearSearchableSnapshotsCacheResponse, EmptyResult> getResponseFactory( protected ResponseFactory<BroadcastResponse, EmptyResult> getResponseFactory(
ClearSearchableSnapshotsCacheRequest request, ClearSearchableSnapshotsCacheRequest request,
ClusterState clusterState ClusterState clusterState
) { ) {
return (totalShards, successfulShards, failedShards, emptyResults, shardFailures) -> new ClearSearchableSnapshotsCacheResponse( return (totalShards, successfulShards, failedShards, emptyResults, shardFailures) -> new BroadcastResponse(
totalShards, totalShards,
successfulShards, successfulShards,
failedShards, failedShards,

View file

@ -9,9 +9,8 @@ package org.elasticsearch.integration;
import org.elasticsearch.ElasticsearchSecurityException; import org.elasticsearch.ElasticsearchSecurityException;
import org.elasticsearch.action.admin.indices.alias.Alias; import org.elasticsearch.action.admin.indices.alias.Alias;
import org.elasticsearch.action.admin.indices.forcemerge.ForceMergeResponse;
import org.elasticsearch.action.admin.indices.refresh.RefreshResponse;
import org.elasticsearch.action.search.SearchRequestBuilder; import org.elasticsearch.action.search.SearchRequestBuilder;
import org.elasticsearch.action.support.broadcast.BroadcastResponse;
import org.elasticsearch.client.internal.Client; import org.elasticsearch.client.internal.Client;
import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesArray;
import org.elasticsearch.common.settings.SecureString; import org.elasticsearch.common.settings.SecureString;
@ -377,14 +376,14 @@ public class DlsFlsRequestCacheTests extends SecuritySingleNodeTestCase {
assertCacheState(DLS_TEMPLATE_ROLE_QUERY_INDEX, 0, 0); assertCacheState(DLS_TEMPLATE_ROLE_QUERY_INDEX, 0, 0);
// Force merge the index to ensure there can be no background merges during the subsequent searches that would invalidate the cache // Force merge the index to ensure there can be no background merges during the subsequent searches that would invalidate the cache
final ForceMergeResponse forceMergeResponse = indicesAdmin().prepareForceMerge( final BroadcastResponse forceMergeResponse = indicesAdmin().prepareForceMerge(
DLS_INDEX, DLS_INDEX,
FLS_INDEX, FLS_INDEX,
INDEX, INDEX,
DLS_TEMPLATE_ROLE_QUERY_INDEX DLS_TEMPLATE_ROLE_QUERY_INDEX
).setFlush(true).get(); ).setFlush(true).get();
ElasticsearchAssertions.assertAllSuccessful(forceMergeResponse); ElasticsearchAssertions.assertAllSuccessful(forceMergeResponse);
final RefreshResponse refreshResponse = indicesAdmin().prepareRefresh(DLS_INDEX, FLS_INDEX, INDEX, DLS_TEMPLATE_ROLE_QUERY_INDEX) final BroadcastResponse refreshResponse = indicesAdmin().prepareRefresh(DLS_INDEX, FLS_INDEX, INDEX, DLS_TEMPLATE_ROLE_QUERY_INDEX)
.get(); .get();
assertThat(refreshResponse.getFailedShards(), equalTo(0)); assertThat(refreshResponse.getFailedShards(), equalTo(0));
ensureGreen(DLS_INDEX, FLS_INDEX, INDEX, DLS_TEMPLATE_ROLE_QUERY_INDEX); ensureGreen(DLS_INDEX, FLS_INDEX, INDEX, DLS_TEMPLATE_ROLE_QUERY_INDEX);

View file

@ -15,10 +15,10 @@ import org.elasticsearch.action.admin.cluster.node.info.NodeInfo;
import org.elasticsearch.action.admin.indices.close.CloseIndexRequest; import org.elasticsearch.action.admin.indices.close.CloseIndexRequest;
import org.elasticsearch.action.admin.indices.close.CloseIndexResponse; import org.elasticsearch.action.admin.indices.close.CloseIndexResponse;
import org.elasticsearch.action.admin.indices.refresh.RefreshRequestBuilder; import org.elasticsearch.action.admin.indices.refresh.RefreshRequestBuilder;
import org.elasticsearch.action.admin.indices.refresh.RefreshResponse;
import org.elasticsearch.action.get.GetRequest; import org.elasticsearch.action.get.GetRequest;
import org.elasticsearch.action.get.TransportGetAction; import org.elasticsearch.action.get.TransportGetAction;
import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.action.support.PlainActionFuture;
import org.elasticsearch.action.support.broadcast.BroadcastResponse;
import org.elasticsearch.action.update.UpdateResponse; import org.elasticsearch.action.update.UpdateResponse;
import org.elasticsearch.client.Request; import org.elasticsearch.client.Request;
import org.elasticsearch.client.RequestOptions; import org.elasticsearch.client.RequestOptions;
@ -810,7 +810,7 @@ public class ApiKeyIntegTests extends SecurityIntegTestCase {
private void refreshSecurityIndex() throws Exception { private void refreshSecurityIndex() throws Exception {
assertBusy(() -> { assertBusy(() -> {
final RefreshResponse refreshResponse = indicesAdmin().prepareRefresh(SECURITY_MAIN_ALIAS).get(); final BroadcastResponse refreshResponse = indicesAdmin().prepareRefresh(SECURITY_MAIN_ALIAS).get();
assertThat(refreshResponse.getFailedShards(), is(0)); assertThat(refreshResponse.getFailedShards(), is(0));
}); });
} }

View file

@ -16,7 +16,6 @@ import org.elasticsearch.action.DocWriteRequest;
import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.DocWriteResponse;
import org.elasticsearch.action.admin.indices.refresh.RefreshAction; import org.elasticsearch.action.admin.indices.refresh.RefreshAction;
import org.elasticsearch.action.admin.indices.refresh.RefreshRequest; import org.elasticsearch.action.admin.indices.refresh.RefreshRequest;
import org.elasticsearch.action.admin.indices.refresh.RefreshResponse;
import org.elasticsearch.action.bulk.BulkAction; import org.elasticsearch.action.bulk.BulkAction;
import org.elasticsearch.action.bulk.BulkItemResponse; import org.elasticsearch.action.bulk.BulkItemResponse;
import org.elasticsearch.action.bulk.BulkRequest; import org.elasticsearch.action.bulk.BulkRequest;
@ -34,6 +33,7 @@ import org.elasticsearch.action.search.TransportSearchAction;
import org.elasticsearch.action.search.TransportSearchScrollAction; import org.elasticsearch.action.search.TransportSearchScrollAction;
import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.action.support.PlainActionFuture;
import org.elasticsearch.action.support.broadcast.BroadcastResponse;
import org.elasticsearch.action.update.UpdateRequest; import org.elasticsearch.action.update.UpdateRequest;
import org.elasticsearch.action.update.UpdateResponse; import org.elasticsearch.action.update.UpdateResponse;
import org.elasticsearch.client.internal.Client; import org.elasticsearch.client.internal.Client;
@ -245,7 +245,7 @@ public class TransportSamlInvalidateSessionActionTests extends SamlTestCase {
listener.onResponse((Response) response); listener.onResponse((Response) response);
} else if (RefreshAction.NAME.equals(action.name())) { } else if (RefreshAction.NAME.equals(action.name())) {
assertThat(request, instanceOf(RefreshRequest.class)); assertThat(request, instanceOf(RefreshRequest.class));
listener.onResponse((Response) mock(RefreshResponse.class)); listener.onResponse((Response) mock(BroadcastResponse.class));
} else { } else {
super.doExecute(action, request, listener); super.doExecute(action, request, listener);
} }

View file

@ -18,7 +18,6 @@ import org.elasticsearch.action.DocWriteRequest;
import org.elasticsearch.action.admin.indices.delete.DeleteIndexRequest; import org.elasticsearch.action.admin.indices.delete.DeleteIndexRequest;
import org.elasticsearch.action.admin.indices.delete.TransportDeleteIndexAction; import org.elasticsearch.action.admin.indices.delete.TransportDeleteIndexAction;
import org.elasticsearch.action.admin.indices.refresh.RefreshRequest; import org.elasticsearch.action.admin.indices.refresh.RefreshRequest;
import org.elasticsearch.action.admin.indices.refresh.RefreshResponse;
import org.elasticsearch.action.bulk.BulkItemResponse; import org.elasticsearch.action.bulk.BulkItemResponse;
import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.index.IndexRequest;
import org.elasticsearch.action.index.TransportIndexAction; import org.elasticsearch.action.index.TransportIndexAction;
@ -27,6 +26,7 @@ import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.action.search.TransportSearchAction;
import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.action.support.WriteRequest; import org.elasticsearch.action.support.WriteRequest;
import org.elasticsearch.action.support.broadcast.BroadcastResponse;
import org.elasticsearch.client.internal.Client; import org.elasticsearch.client.internal.Client;
import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
@ -848,7 +848,7 @@ public class IndexBasedTransformConfigManager implements TransformConfigManager
client.threadPool().getThreadContext(), client.threadPool().getThreadContext(),
TRANSFORM_ORIGIN, TRANSFORM_ORIGIN,
new RefreshRequest(TransformInternalIndexConstants.LATEST_INDEX_NAME), new RefreshRequest(TransformInternalIndexConstants.LATEST_INDEX_NAME),
ActionListener.<RefreshResponse>wrap(r -> listener.onResponse(true), listener::onFailure), ActionListener.<BroadcastResponse>wrap(r -> listener.onResponse(true), listener::onFailure),
client.admin().indices()::refresh client.admin().indices()::refresh
); );
} }

View file

@ -7,7 +7,7 @@
package org.elasticsearch.xpack.watcher.test.integration; package org.elasticsearch.xpack.watcher.test.integration;
import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse; import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse;
import org.elasticsearch.action.admin.indices.refresh.RefreshResponse; import org.elasticsearch.action.support.broadcast.BroadcastResponse;
import org.elasticsearch.cluster.metadata.IndexMetadata; import org.elasticsearch.cluster.metadata.IndexMetadata;
import org.elasticsearch.protocol.xpack.watcher.PutWatchResponse; import org.elasticsearch.protocol.xpack.watcher.PutWatchResponse;
import org.elasticsearch.rest.RestStatus; import org.elasticsearch.rest.RestStatus;
@ -65,7 +65,7 @@ public class SingleNodeTests extends AbstractWatcherIntegrationTestCase {
ensureGreen(HistoryStoreField.DATA_STREAM); ensureGreen(HistoryStoreField.DATA_STREAM);
assertBusy(() -> { assertBusy(() -> {
RefreshResponse refreshResponse = indicesAdmin().prepareRefresh(".watcher-history*").get(); BroadcastResponse refreshResponse = indicesAdmin().prepareRefresh(".watcher-history*").get();
assertThat(refreshResponse.getStatus(), equalTo(RestStatus.OK)); assertThat(refreshResponse.getStatus(), equalTo(RestStatus.OK));
assertResponse( assertResponse(
prepareSearch(".watcher-history*").setSize(0), prepareSearch(".watcher-history*").setSize(0),

View file

@ -10,11 +10,11 @@ import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.Logger;
import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.action.admin.indices.refresh.RefreshRequest; import org.elasticsearch.action.admin.indices.refresh.RefreshRequest;
import org.elasticsearch.action.admin.indices.refresh.RefreshResponse;
import org.elasticsearch.action.search.ClearScrollRequest; import org.elasticsearch.action.search.ClearScrollRequest;
import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchRequest;
import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.search.SearchScrollRequest; import org.elasticsearch.action.search.SearchScrollRequest;
import org.elasticsearch.action.support.broadcast.BroadcastResponse;
import org.elasticsearch.client.internal.Client; import org.elasticsearch.client.internal.Client;
import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.metadata.IndexMetadata; import org.elasticsearch.cluster.metadata.IndexMetadata;
@ -404,7 +404,7 @@ public class WatcherService {
// Non private for unit testing purposes // Non private for unit testing purposes
void refreshWatches(IndexMetadata indexMetadata) { void refreshWatches(IndexMetadata indexMetadata) {
RefreshResponse refreshResponse = client.admin() BroadcastResponse refreshResponse = client.admin()
.indices() .indices()
.refresh(new RefreshRequest(INDEX)) .refresh(new RefreshRequest(INDEX))
.actionGet(TimeValue.timeValueSeconds(5)); .actionGet(TimeValue.timeValueSeconds(5));

View file

@ -11,7 +11,6 @@ import org.elasticsearch.ElasticsearchTimeoutException;
import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.admin.indices.refresh.RefreshAction; import org.elasticsearch.action.admin.indices.refresh.RefreshAction;
import org.elasticsearch.action.admin.indices.refresh.RefreshRequest; import org.elasticsearch.action.admin.indices.refresh.RefreshRequest;
import org.elasticsearch.action.admin.indices.refresh.RefreshResponse;
import org.elasticsearch.action.search.ClearScrollRequest; import org.elasticsearch.action.search.ClearScrollRequest;
import org.elasticsearch.action.search.ClearScrollResponse; import org.elasticsearch.action.search.ClearScrollResponse;
import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchRequest;
@ -21,6 +20,7 @@ import org.elasticsearch.action.search.ShardSearchFailure;
import org.elasticsearch.action.search.TransportClearScrollAction; import org.elasticsearch.action.search.TransportClearScrollAction;
import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.action.search.TransportSearchAction;
import org.elasticsearch.action.search.TransportSearchScrollAction; import org.elasticsearch.action.search.TransportSearchScrollAction;
import org.elasticsearch.action.support.broadcast.BroadcastResponse;
import org.elasticsearch.client.internal.Client; import org.elasticsearch.client.internal.Client;
import org.elasticsearch.cluster.ClusterName; import org.elasticsearch.cluster.ClusterName;
import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterState;
@ -162,12 +162,12 @@ public class WatcherServiceTests extends ESTestCase {
ClusterState clusterState = csBuilder.build(); ClusterState clusterState = csBuilder.build();
// response setup, successful refresh response // response setup, successful refresh response
RefreshResponse refreshResponse = mock(RefreshResponse.class); BroadcastResponse refreshResponse = mock(BroadcastResponse.class);
when(refreshResponse.getSuccessfulShards()).thenReturn( when(refreshResponse.getSuccessfulShards()).thenReturn(
clusterState.getMetadata().getIndices().get(Watch.INDEX).getNumberOfShards() clusterState.getMetadata().getIndices().get(Watch.INDEX).getNumberOfShards()
); );
doAnswer(invocation -> { doAnswer(invocation -> {
ActionListener<RefreshResponse> listener = (ActionListener<RefreshResponse>) invocation.getArguments()[2]; ActionListener<BroadcastResponse> listener = (ActionListener<BroadcastResponse>) invocation.getArguments()[2];
listener.onResponse(refreshResponse); listener.onResponse(refreshResponse);
return null; return null;
}).when(client).execute(eq(RefreshAction.INSTANCE), any(RefreshRequest.class), anyActionListener()); }).when(client).execute(eq(RefreshAction.INSTANCE), any(RefreshRequest.class), anyActionListener());

View file

@ -11,7 +11,6 @@ import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.DocWriteRequest; import org.elasticsearch.action.DocWriteRequest;
import org.elasticsearch.action.admin.indices.refresh.RefreshAction; import org.elasticsearch.action.admin.indices.refresh.RefreshAction;
import org.elasticsearch.action.admin.indices.refresh.RefreshResponse;
import org.elasticsearch.action.bulk.BulkAction; import org.elasticsearch.action.bulk.BulkAction;
import org.elasticsearch.action.bulk.BulkItemResponse; import org.elasticsearch.action.bulk.BulkItemResponse;
import org.elasticsearch.action.bulk.BulkProcessor2; import org.elasticsearch.action.bulk.BulkProcessor2;
@ -24,6 +23,7 @@ import org.elasticsearch.action.search.SearchScrollRequest;
import org.elasticsearch.action.search.TransportClearScrollAction; import org.elasticsearch.action.search.TransportClearScrollAction;
import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.action.search.TransportSearchAction;
import org.elasticsearch.action.search.TransportSearchScrollAction; import org.elasticsearch.action.search.TransportSearchScrollAction;
import org.elasticsearch.action.support.broadcast.BroadcastResponse;
import org.elasticsearch.client.internal.Client; import org.elasticsearch.client.internal.Client;
import org.elasticsearch.cluster.ClusterName; import org.elasticsearch.cluster.ClusterName;
import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterState;
@ -202,7 +202,7 @@ public class TriggeredWatchStoreTests extends ESTestCase {
doAnswer(invocation -> { doAnswer(invocation -> {
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
ActionListener<RefreshResponse> listener = (ActionListener<RefreshResponse>) invocation.getArguments()[2]; ActionListener<BroadcastResponse> listener = (ActionListener<BroadcastResponse>) invocation.getArguments()[2];
listener.onResponse(mockRefreshResponse(1, 1)); listener.onResponse(mockRefreshResponse(1, 1));
return null; return null;
}).when(client).execute(eq(RefreshAction.INSTANCE), any(), any()); }).when(client).execute(eq(RefreshAction.INSTANCE), any(), any());
@ -409,7 +409,7 @@ public class TriggeredWatchStoreTests extends ESTestCase {
doAnswer(invocation -> { doAnswer(invocation -> {
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
ActionListener<RefreshResponse> listener = (ActionListener<RefreshResponse>) invocation.getArguments()[2]; ActionListener<BroadcastResponse> listener = (ActionListener<BroadcastResponse>) invocation.getArguments()[2];
listener.onFailure(new IndexNotFoundException(TriggeredWatchStoreField.INDEX_NAME)); listener.onFailure(new IndexNotFoundException(TriggeredWatchStoreField.INDEX_NAME));
return null; return null;
}).when(client).execute(eq(RefreshAction.INSTANCE), any(), any()); }).when(client).execute(eq(RefreshAction.INSTANCE), any(), any());
@ -507,8 +507,8 @@ public class TriggeredWatchStoreTests extends ESTestCase {
assertThat(response.getItems().length, is(1)); assertThat(response.getItems().length, is(1));
} }
private RefreshResponse mockRefreshResponse(int total, int successful) { private BroadcastResponse mockRefreshResponse(int total, int successful) {
RefreshResponse refreshResponse = mock(RefreshResponse.class); BroadcastResponse refreshResponse = mock(BroadcastResponse.class);
when(refreshResponse.getTotalShards()).thenReturn(total); when(refreshResponse.getTotalShards()).thenReturn(total);
when(refreshResponse.getSuccessfulShards()).thenReturn(successful); when(refreshResponse.getSuccessfulShards()).thenReturn(successful);
return refreshResponse; return refreshResponse;