Add memory tracking to queued write operations (#58957)
Currently we do not track the memory consuming by in-process write operations. This commit adds a mechanism to track write operation memory usage.
This commit is contained in:
parent
a4e08acdd1
commit
1ef2cd7f1a
|
@ -0,0 +1,206 @@
|
||||||
|
/*
|
||||||
|
* Licensed to Elasticsearch under one or more contributor
|
||||||
|
* license agreements. See the NOTICE file distributed with
|
||||||
|
* this work for additional information regarding copyright
|
||||||
|
* ownership. Elasticsearch licenses this file to you under
|
||||||
|
* the Apache License, Version 2.0 (the "License"); you may
|
||||||
|
* not use this file except in compliance with the License.
|
||||||
|
* You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing,
|
||||||
|
* software distributed under the License is distributed on an
|
||||||
|
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||||
|
* KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations
|
||||||
|
* under the License.
|
||||||
|
*/
|
||||||
|
package org.elasticsearch.action.bulk;
|
||||||
|
|
||||||
|
import org.elasticsearch.action.ActionFuture;
|
||||||
|
import org.elasticsearch.action.admin.indices.stats.IndicesStatsResponse;
|
||||||
|
import org.elasticsearch.action.admin.indices.stats.ShardStats;
|
||||||
|
import org.elasticsearch.action.index.IndexRequest;
|
||||||
|
import org.elasticsearch.cluster.metadata.IndexMetadata;
|
||||||
|
import org.elasticsearch.cluster.routing.ShardRouting;
|
||||||
|
import org.elasticsearch.common.UUIDs;
|
||||||
|
import org.elasticsearch.common.settings.Settings;
|
||||||
|
import org.elasticsearch.plugins.Plugin;
|
||||||
|
import org.elasticsearch.test.ESIntegTestCase;
|
||||||
|
import org.elasticsearch.test.InternalSettingsPlugin;
|
||||||
|
import org.elasticsearch.test.transport.MockTransportService;
|
||||||
|
import org.elasticsearch.threadpool.ThreadPool;
|
||||||
|
import org.elasticsearch.transport.TransportService;
|
||||||
|
|
||||||
|
import java.util.Arrays;
|
||||||
|
import java.util.Collection;
|
||||||
|
import java.util.Collections;
|
||||||
|
import java.util.concurrent.CountDownLatch;
|
||||||
|
import java.util.stream.Stream;
|
||||||
|
|
||||||
|
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
|
||||||
|
import static org.hamcrest.Matchers.greaterThan;
|
||||||
|
|
||||||
|
@ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.SUITE, numDataNodes = 2)
|
||||||
|
public class WriteMemoryLimitsIT extends ESIntegTestCase {
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Settings nodeSettings(int nodeOrdinal) {
|
||||||
|
return Settings.builder()
|
||||||
|
.put(super.nodeSettings(nodeOrdinal))
|
||||||
|
// Need at least two threads because we are going to block one
|
||||||
|
.put("thread_pool.write.size", 2)
|
||||||
|
.build();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Collection<Class<? extends Plugin>> nodePlugins() {
|
||||||
|
return Arrays.asList(MockTransportService.TestPlugin.class, InternalSettingsPlugin.class);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected int numberOfReplicas() {
|
||||||
|
return 1;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected int numberOfShards() {
|
||||||
|
return 1;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void testWriteBytesAreIncremented() throws Exception {
|
||||||
|
final String index = "test";
|
||||||
|
assertAcked(prepareCreate(index, Settings.builder()
|
||||||
|
.put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1)
|
||||||
|
.put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1)));
|
||||||
|
ensureGreen(index);
|
||||||
|
|
||||||
|
IndicesStatsResponse response = client().admin().indices().prepareStats(index).get();
|
||||||
|
String primaryId = Stream.of(response.getShards())
|
||||||
|
.map(ShardStats::getShardRouting)
|
||||||
|
.filter(ShardRouting::primary)
|
||||||
|
.findAny()
|
||||||
|
.get()
|
||||||
|
.currentNodeId();
|
||||||
|
String replicaId = Stream.of(response.getShards())
|
||||||
|
.map(ShardStats::getShardRouting)
|
||||||
|
.filter(sr -> sr.primary() == false)
|
||||||
|
.findAny()
|
||||||
|
.get()
|
||||||
|
.currentNodeId();
|
||||||
|
String primaryName = client().admin().cluster().prepareState().get().getState().nodes().get(primaryId).getName();
|
||||||
|
String replicaName = client().admin().cluster().prepareState().get().getState().nodes().get(replicaId).getName();
|
||||||
|
|
||||||
|
final CountDownLatch replicationSendPointReached = new CountDownLatch(1);
|
||||||
|
final CountDownLatch latchBlockingReplicationSend = new CountDownLatch(1);
|
||||||
|
final CountDownLatch newActionsSendPointReached = new CountDownLatch(2);
|
||||||
|
final CountDownLatch latchBlockingReplication = new CountDownLatch(1);
|
||||||
|
|
||||||
|
TransportService primaryService = internalCluster().getInstance(TransportService.class, primaryName);
|
||||||
|
final MockTransportService primaryTransportService = (MockTransportService) primaryService;
|
||||||
|
TransportService replicaService = internalCluster().getInstance(TransportService.class, replicaName);
|
||||||
|
final MockTransportService replicaTransportService = (MockTransportService) replicaService;
|
||||||
|
|
||||||
|
primaryTransportService.addSendBehavior((connection, requestId, action, request, options) -> {
|
||||||
|
if (action.equals(TransportShardBulkAction.ACTION_NAME + "[r]")) {
|
||||||
|
try {
|
||||||
|
replicationSendPointReached.countDown();
|
||||||
|
latchBlockingReplicationSend.await();
|
||||||
|
} catch (InterruptedException e) {
|
||||||
|
throw new IllegalStateException(e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
connection.sendRequest(requestId, action, request, options);
|
||||||
|
});
|
||||||
|
|
||||||
|
final BulkRequest bulkRequest = new BulkRequest();
|
||||||
|
int totalRequestSize = 0;
|
||||||
|
for (int i = 0; i < 80; ++i) {
|
||||||
|
IndexRequest request = new IndexRequest(index).id(UUIDs.base64UUID())
|
||||||
|
.source(Collections.singletonMap("key", randomAlphaOfLength(50)));
|
||||||
|
totalRequestSize += request.ramBytesUsed();
|
||||||
|
assertTrue(request.ramBytesUsed() > request.source().length());
|
||||||
|
bulkRequest.add(request);
|
||||||
|
}
|
||||||
|
|
||||||
|
final long bulkRequestSize = bulkRequest.ramBytesUsed();
|
||||||
|
final long bulkShardRequestSize = totalRequestSize;
|
||||||
|
|
||||||
|
try {
|
||||||
|
final ActionFuture<BulkResponse> successFuture = client(replicaName).bulk(bulkRequest);
|
||||||
|
replicationSendPointReached.await();
|
||||||
|
|
||||||
|
WriteMemoryLimits primaryWriteLimits = internalCluster().getInstance(WriteMemoryLimits.class, primaryName);
|
||||||
|
WriteMemoryLimits replicaWriteLimits = internalCluster().getInstance(WriteMemoryLimits.class, replicaName);
|
||||||
|
|
||||||
|
assertThat(primaryWriteLimits.getCoordinatingBytes(), greaterThan(bulkShardRequestSize));
|
||||||
|
assertThat(primaryWriteLimits.getPrimaryBytes(), greaterThan(bulkShardRequestSize));
|
||||||
|
assertEquals(0, primaryWriteLimits.getReplicaBytes());
|
||||||
|
assertEquals(bulkRequestSize, replicaWriteLimits.getCoordinatingBytes());
|
||||||
|
assertEquals(0, replicaWriteLimits.getPrimaryBytes());
|
||||||
|
assertEquals(0, replicaWriteLimits.getReplicaBytes());
|
||||||
|
|
||||||
|
ThreadPool replicaThreadPool = replicaTransportService.getThreadPool();
|
||||||
|
// Block the replica Write thread pool
|
||||||
|
replicaThreadPool.executor(ThreadPool.Names.WRITE).execute(() -> {
|
||||||
|
try {
|
||||||
|
newActionsSendPointReached.countDown();
|
||||||
|
latchBlockingReplication.await();
|
||||||
|
} catch (InterruptedException e) {
|
||||||
|
throw new IllegalStateException(e);
|
||||||
|
}
|
||||||
|
});
|
||||||
|
replicaThreadPool.executor(ThreadPool.Names.WRITE).execute(() -> {
|
||||||
|
try {
|
||||||
|
newActionsSendPointReached.countDown();
|
||||||
|
latchBlockingReplication.await();
|
||||||
|
} catch (InterruptedException e) {
|
||||||
|
throw new IllegalStateException(e);
|
||||||
|
}
|
||||||
|
});
|
||||||
|
newActionsSendPointReached.await();
|
||||||
|
latchBlockingReplicationSend.countDown();
|
||||||
|
|
||||||
|
IndexRequest request = new IndexRequest(index).id(UUIDs.base64UUID())
|
||||||
|
.source(Collections.singletonMap("key", randomAlphaOfLength(50)));
|
||||||
|
final BulkRequest secondBulkRequest = new BulkRequest();
|
||||||
|
secondBulkRequest.add(request);
|
||||||
|
|
||||||
|
ActionFuture<BulkResponse> secondFuture = client(replicaName).bulk(secondBulkRequest);
|
||||||
|
|
||||||
|
final long secondBulkRequestSize = secondBulkRequest.ramBytesUsed();
|
||||||
|
final long secondBulkShardRequestSize = request.ramBytesUsed();
|
||||||
|
|
||||||
|
assertEquals(bulkRequestSize + secondBulkRequestSize, replicaWriteLimits.getCoordinatingBytes());
|
||||||
|
assertBusy(() -> assertThat(replicaWriteLimits.getReplicaBytes(),
|
||||||
|
greaterThan(bulkShardRequestSize + secondBulkShardRequestSize)));
|
||||||
|
|
||||||
|
latchBlockingReplication.countDown();
|
||||||
|
|
||||||
|
successFuture.actionGet();
|
||||||
|
secondFuture.actionGet();
|
||||||
|
|
||||||
|
assertEquals(0, primaryWriteLimits.getCoordinatingBytes());
|
||||||
|
assertEquals(0, primaryWriteLimits.getPrimaryBytes());
|
||||||
|
assertEquals(0, primaryWriteLimits.getReplicaBytes());
|
||||||
|
assertEquals(0, replicaWriteLimits.getCoordinatingBytes());
|
||||||
|
assertEquals(0, replicaWriteLimits.getPrimaryBytes());
|
||||||
|
assertEquals(0, replicaWriteLimits.getReplicaBytes());
|
||||||
|
} finally {
|
||||||
|
if (replicationSendPointReached.getCount() > 0) {
|
||||||
|
replicationSendPointReached.countDown();
|
||||||
|
}
|
||||||
|
while (newActionsSendPointReached.getCount() > 0) {
|
||||||
|
newActionsSendPointReached.countDown();
|
||||||
|
}
|
||||||
|
if (latchBlockingReplicationSend.getCount() > 0) {
|
||||||
|
latchBlockingReplicationSend.countDown();
|
||||||
|
}
|
||||||
|
if (latchBlockingReplication.getCount() > 0) {
|
||||||
|
latchBlockingReplication.countDown();
|
||||||
|
}
|
||||||
|
primaryTransportService.clearAllRules();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -124,8 +124,8 @@ public class TransportReplicationActionRetryOnClosedNodeIT extends ESIntegTestCa
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected ReplicaResult shardOperationOnReplica(Request shardRequest, IndexShard replica) {
|
protected void shardOperationOnReplica(Request shardRequest, IndexShard replica, ActionListener<ReplicaResult> listener) {
|
||||||
return new ReplicaResult();
|
listener.onResponse(new ReplicaResult());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -18,6 +18,7 @@
|
||||||
*/
|
*/
|
||||||
package org.elasticsearch.action;
|
package org.elasticsearch.action;
|
||||||
|
|
||||||
|
import org.apache.lucene.util.Accountable;
|
||||||
import org.elasticsearch.action.delete.DeleteRequest;
|
import org.elasticsearch.action.delete.DeleteRequest;
|
||||||
import org.elasticsearch.action.index.IndexRequest;
|
import org.elasticsearch.action.index.IndexRequest;
|
||||||
import org.elasticsearch.action.support.IndicesOptions;
|
import org.elasticsearch.action.support.IndicesOptions;
|
||||||
|
@ -40,7 +41,7 @@ import static org.elasticsearch.index.seqno.SequenceNumbers.UNASSIGNED_SEQ_NO;
|
||||||
* Generic interface to group ActionRequest, which perform writes to a single document
|
* Generic interface to group ActionRequest, which perform writes to a single document
|
||||||
* Action requests implementing this can be part of {@link org.elasticsearch.action.bulk.BulkRequest}
|
* Action requests implementing this can be part of {@link org.elasticsearch.action.bulk.BulkRequest}
|
||||||
*/
|
*/
|
||||||
public interface DocWriteRequest<T> extends IndicesRequest {
|
public interface DocWriteRequest<T> extends IndicesRequest, Accountable {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Set the index for this request
|
* Set the index for this request
|
||||||
|
|
|
@ -94,9 +94,11 @@ public class TransportVerifyShardBeforeCloseAction extends TransportReplicationA
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected ReplicaResult shardOperationOnReplica(final ShardRequest shardRequest, final IndexShard replica) throws IOException {
|
protected void shardOperationOnReplica(ShardRequest shardRequest, IndexShard replica, ActionListener<ReplicaResult> listener) {
|
||||||
executeShardOperation(shardRequest, replica);
|
ActionListener.completeWith(listener, () -> {
|
||||||
return new ReplicaResult();
|
executeShardOperation(shardRequest, replica);
|
||||||
|
return new ReplicaResult();
|
||||||
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
private void executeShardOperation(final ShardRequest request, final IndexShard indexShard) throws IOException {
|
private void executeShardOperation(final ShardRequest request, final IndexShard indexShard) throws IOException {
|
||||||
|
|
|
@ -64,9 +64,11 @@ public class TransportShardFlushAction
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected ReplicaResult shardOperationOnReplica(ShardFlushRequest request, IndexShard replica) {
|
protected void shardOperationOnReplica(ShardFlushRequest request, IndexShard replica, ActionListener<ReplicaResult> listener) {
|
||||||
replica.flush(request.getRequest());
|
ActionListener.completeWith(listener, () -> {
|
||||||
logger.trace("{} flush request executed on replica", replica.shardId());
|
replica.flush(request.getRequest());
|
||||||
return new ReplicaResult();
|
logger.trace("{} flush request executed on replica", replica.shardId());
|
||||||
|
return new ReplicaResult();
|
||||||
|
});
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -100,9 +100,11 @@ public class TransportVerifyShardIndexBlockAction extends TransportReplicationAc
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected ReplicaResult shardOperationOnReplica(ShardRequest shardRequest, IndexShard replica) {
|
protected void shardOperationOnReplica(ShardRequest shardRequest, IndexShard replica, ActionListener<ReplicaResult> listener) {
|
||||||
executeShardOperation(shardRequest, replica);
|
ActionListener.completeWith(listener, () -> {
|
||||||
return new ReplicaResult();
|
executeShardOperation(shardRequest, replica);
|
||||||
|
return new ReplicaResult();
|
||||||
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
private void executeShardOperation(final ShardRequest request, final IndexShard indexShard) {
|
private void executeShardOperation(final ShardRequest request, final IndexShard indexShard) {
|
||||||
|
|
|
@ -66,9 +66,12 @@ public class TransportShardRefreshAction
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected ReplicaResult shardOperationOnReplica(BasicReplicationRequest request, IndexShard replica) {
|
protected void shardOperationOnReplica(BasicReplicationRequest request, IndexShard replica,
|
||||||
replica.refresh("api");
|
ActionListener<ReplicaResult> listener) {
|
||||||
logger.trace("{} refresh request executed on replica", replica.shardId());
|
ActionListener.completeWith(listener, () -> {
|
||||||
return new ReplicaResult();
|
replica.refresh("api");
|
||||||
|
logger.trace("{} refresh request executed on replica", replica.shardId());
|
||||||
|
return new ReplicaResult();
|
||||||
|
});
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -19,6 +19,8 @@
|
||||||
|
|
||||||
package org.elasticsearch.action.bulk;
|
package org.elasticsearch.action.bulk;
|
||||||
|
|
||||||
|
import org.apache.lucene.util.Accountable;
|
||||||
|
import org.apache.lucene.util.RamUsageEstimator;
|
||||||
import org.elasticsearch.action.DocWriteRequest;
|
import org.elasticsearch.action.DocWriteRequest;
|
||||||
import org.elasticsearch.common.Nullable;
|
import org.elasticsearch.common.Nullable;
|
||||||
import org.elasticsearch.common.Strings;
|
import org.elasticsearch.common.Strings;
|
||||||
|
@ -30,7 +32,9 @@ import org.elasticsearch.index.shard.ShardId;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.Objects;
|
import java.util.Objects;
|
||||||
|
|
||||||
public class BulkItemRequest implements Writeable {
|
public class BulkItemRequest implements Writeable, Accountable {
|
||||||
|
|
||||||
|
private static final long SHALLOW_SIZE = RamUsageEstimator.shallowSizeOfInstance(BulkItemRequest.class);
|
||||||
|
|
||||||
private int id;
|
private int id;
|
||||||
private DocWriteRequest<?> request;
|
private DocWriteRequest<?> request;
|
||||||
|
@ -115,4 +119,9 @@ public class BulkItemRequest implements Writeable {
|
||||||
DocWriteRequest.writeDocumentRequestThin(out, request);
|
DocWriteRequest.writeDocumentRequestThin(out, request);
|
||||||
out.writeOptionalWriteable(primaryResponse == null ? null : primaryResponse::writeThin);
|
out.writeOptionalWriteable(primaryResponse == null ? null : primaryResponse::writeThin);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long ramBytesUsed() {
|
||||||
|
return SHALLOW_SIZE + request.ramBytesUsed();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -19,6 +19,8 @@
|
||||||
|
|
||||||
package org.elasticsearch.action.bulk;
|
package org.elasticsearch.action.bulk;
|
||||||
|
|
||||||
|
import org.apache.lucene.util.Accountable;
|
||||||
|
import org.apache.lucene.util.RamUsageEstimator;
|
||||||
import org.elasticsearch.action.ActionRequest;
|
import org.elasticsearch.action.ActionRequest;
|
||||||
import org.elasticsearch.action.ActionRequestValidationException;
|
import org.elasticsearch.action.ActionRequestValidationException;
|
||||||
import org.elasticsearch.action.CompositeIndicesRequest;
|
import org.elasticsearch.action.CompositeIndicesRequest;
|
||||||
|
@ -56,7 +58,9 @@ import static org.elasticsearch.action.ValidateActions.addValidationError;
|
||||||
* Note that we only support refresh on the bulk request not per item.
|
* Note that we only support refresh on the bulk request not per item.
|
||||||
* @see org.elasticsearch.client.Client#bulk(BulkRequest)
|
* @see org.elasticsearch.client.Client#bulk(BulkRequest)
|
||||||
*/
|
*/
|
||||||
public class BulkRequest extends ActionRequest implements CompositeIndicesRequest, WriteRequest<BulkRequest> {
|
public class BulkRequest extends ActionRequest implements CompositeIndicesRequest, WriteRequest<BulkRequest>, Accountable {
|
||||||
|
|
||||||
|
private static final long SHALLOW_SIZE = RamUsageEstimator.shallowSizeOfInstance(BulkRequest.class);
|
||||||
|
|
||||||
private static final int REQUEST_OVERHEAD = 50;
|
private static final int REQUEST_OVERHEAD = 50;
|
||||||
|
|
||||||
|
@ -429,4 +433,9 @@ public class BulkRequest extends ActionRequest implements CompositeIndicesReques
|
||||||
}
|
}
|
||||||
return value;
|
return value;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long ramBytesUsed() {
|
||||||
|
return SHALLOW_SIZE + requests.stream().mapToLong(Accountable::ramBytesUsed).sum();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -19,6 +19,8 @@
|
||||||
|
|
||||||
package org.elasticsearch.action.bulk;
|
package org.elasticsearch.action.bulk;
|
||||||
|
|
||||||
|
import org.apache.lucene.util.Accountable;
|
||||||
|
import org.apache.lucene.util.RamUsageEstimator;
|
||||||
import org.elasticsearch.Version;
|
import org.elasticsearch.Version;
|
||||||
import org.elasticsearch.action.support.replication.ReplicatedWriteRequest;
|
import org.elasticsearch.action.support.replication.ReplicatedWriteRequest;
|
||||||
import org.elasticsearch.action.support.replication.ReplicationRequest;
|
import org.elasticsearch.action.support.replication.ReplicationRequest;
|
||||||
|
@ -29,12 +31,14 @@ import org.elasticsearch.index.shard.ShardId;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.HashSet;
|
import java.util.HashSet;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
|
import java.util.stream.Stream;
|
||||||
|
|
||||||
public class BulkShardRequest extends ReplicatedWriteRequest<BulkShardRequest> {
|
public class BulkShardRequest extends ReplicatedWriteRequest<BulkShardRequest> implements Accountable {
|
||||||
|
|
||||||
public static final Version COMPACT_SHARD_ID_VERSION = Version.V_7_9_0;
|
public static final Version COMPACT_SHARD_ID_VERSION = Version.V_7_9_0;
|
||||||
|
private static final long SHALLOW_SIZE = RamUsageEstimator.shallowSizeOfInstance(BulkShardRequest.class);
|
||||||
|
|
||||||
private BulkItemRequest[] items;
|
private final BulkItemRequest[] items;
|
||||||
|
|
||||||
public BulkShardRequest(StreamInput in) throws IOException {
|
public BulkShardRequest(StreamInput in) throws IOException {
|
||||||
super(in);
|
super(in);
|
||||||
|
@ -143,4 +147,9 @@ public class BulkShardRequest extends ReplicatedWriteRequest<BulkShardRequest> {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long ramBytesUsed() {
|
||||||
|
return SHALLOW_SIZE + Stream.of(items).mapToLong(Accountable::ramBytesUsed).sum();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -59,6 +59,7 @@ import org.elasticsearch.cluster.metadata.Metadata;
|
||||||
import org.elasticsearch.cluster.metadata.MetadataIndexTemplateService;
|
import org.elasticsearch.cluster.metadata.MetadataIndexTemplateService;
|
||||||
import org.elasticsearch.cluster.service.ClusterService;
|
import org.elasticsearch.cluster.service.ClusterService;
|
||||||
import org.elasticsearch.common.inject.Inject;
|
import org.elasticsearch.common.inject.Inject;
|
||||||
|
import org.elasticsearch.common.lease.Releasable;
|
||||||
import org.elasticsearch.common.settings.Settings;
|
import org.elasticsearch.common.settings.Settings;
|
||||||
import org.elasticsearch.common.unit.TimeValue;
|
import org.elasticsearch.common.unit.TimeValue;
|
||||||
import org.elasticsearch.common.util.concurrent.AbstractRunnable;
|
import org.elasticsearch.common.util.concurrent.AbstractRunnable;
|
||||||
|
@ -112,23 +113,24 @@ public class TransportBulkAction extends HandledTransportAction<BulkRequest, Bul
|
||||||
private final NodeClient client;
|
private final NodeClient client;
|
||||||
private final IndexNameExpressionResolver indexNameExpressionResolver;
|
private final IndexNameExpressionResolver indexNameExpressionResolver;
|
||||||
private static final String DROPPED_ITEM_WITH_AUTO_GENERATED_ID = "auto-generated";
|
private static final String DROPPED_ITEM_WITH_AUTO_GENERATED_ID = "auto-generated";
|
||||||
|
private final WriteMemoryLimits writeMemoryLimits;
|
||||||
|
|
||||||
@Inject
|
@Inject
|
||||||
public TransportBulkAction(ThreadPool threadPool, TransportService transportService,
|
public TransportBulkAction(ThreadPool threadPool, TransportService transportService,
|
||||||
ClusterService clusterService, IngestService ingestService,
|
ClusterService clusterService, IngestService ingestService,
|
||||||
TransportShardBulkAction shardBulkAction, NodeClient client,
|
TransportShardBulkAction shardBulkAction, NodeClient client,
|
||||||
ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver,
|
ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver,
|
||||||
AutoCreateIndex autoCreateIndex) {
|
AutoCreateIndex autoCreateIndex, WriteMemoryLimits writeMemoryLimits) {
|
||||||
this(threadPool, transportService, clusterService, ingestService, shardBulkAction, client, actionFilters,
|
this(threadPool, transportService, clusterService, ingestService, shardBulkAction, client, actionFilters,
|
||||||
indexNameExpressionResolver, autoCreateIndex, System::nanoTime);
|
indexNameExpressionResolver, autoCreateIndex, writeMemoryLimits, System::nanoTime);
|
||||||
}
|
}
|
||||||
|
|
||||||
public TransportBulkAction(ThreadPool threadPool, TransportService transportService,
|
public TransportBulkAction(ThreadPool threadPool, TransportService transportService,
|
||||||
ClusterService clusterService, IngestService ingestService,
|
ClusterService clusterService, IngestService ingestService,
|
||||||
TransportShardBulkAction shardBulkAction, NodeClient client,
|
TransportShardBulkAction shardBulkAction, NodeClient client,
|
||||||
ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver,
|
ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver,
|
||||||
AutoCreateIndex autoCreateIndex, LongSupplier relativeTimeProvider) {
|
AutoCreateIndex autoCreateIndex, WriteMemoryLimits writeMemoryLimits, LongSupplier relativeTimeProvider) {
|
||||||
super(BulkAction.NAME, transportService, actionFilters, BulkRequest::new, ThreadPool.Names.WRITE);
|
super(BulkAction.NAME, transportService, actionFilters, BulkRequest::new, ThreadPool.Names.SAME);
|
||||||
Objects.requireNonNull(relativeTimeProvider);
|
Objects.requireNonNull(relativeTimeProvider);
|
||||||
this.threadPool = threadPool;
|
this.threadPool = threadPool;
|
||||||
this.clusterService = clusterService;
|
this.clusterService = clusterService;
|
||||||
|
@ -139,6 +141,7 @@ public class TransportBulkAction extends HandledTransportAction<BulkRequest, Bul
|
||||||
this.ingestForwarder = new IngestActionForwarder(transportService);
|
this.ingestForwarder = new IngestActionForwarder(transportService);
|
||||||
this.client = client;
|
this.client = client;
|
||||||
this.indexNameExpressionResolver = indexNameExpressionResolver;
|
this.indexNameExpressionResolver = indexNameExpressionResolver;
|
||||||
|
this.writeMemoryLimits = writeMemoryLimits;
|
||||||
clusterService.addStateApplier(this.ingestForwarder);
|
clusterService.addStateApplier(this.ingestForwarder);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -149,7 +152,7 @@ public class TransportBulkAction extends HandledTransportAction<BulkRequest, Bul
|
||||||
* @param docWriteRequest The request to find the {@link IndexRequest}
|
* @param docWriteRequest The request to find the {@link IndexRequest}
|
||||||
* @return the found {@link IndexRequest} or {@code null} if one can not be found.
|
* @return the found {@link IndexRequest} or {@code null} if one can not be found.
|
||||||
*/
|
*/
|
||||||
public static IndexRequest getIndexWriteRequest(DocWriteRequest docWriteRequest) {
|
public static IndexRequest getIndexWriteRequest(DocWriteRequest<?> docWriteRequest) {
|
||||||
IndexRequest indexRequest = null;
|
IndexRequest indexRequest = null;
|
||||||
if (docWriteRequest instanceof IndexRequest) {
|
if (docWriteRequest instanceof IndexRequest) {
|
||||||
indexRequest = (IndexRequest) docWriteRequest;
|
indexRequest = (IndexRequest) docWriteRequest;
|
||||||
|
@ -162,6 +165,17 @@ public class TransportBulkAction extends HandledTransportAction<BulkRequest, Bul
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected void doExecute(Task task, BulkRequest bulkRequest, ActionListener<BulkResponse> listener) {
|
protected void doExecute(Task task, BulkRequest bulkRequest, ActionListener<BulkResponse> listener) {
|
||||||
|
long indexingBytes = bulkRequest.ramBytesUsed();
|
||||||
|
final Releasable releasable = writeMemoryLimits.markCoordinatingOperationStarted(indexingBytes);
|
||||||
|
final ActionListener<BulkResponse> releasingListener = ActionListener.runBefore(listener, releasable::close);
|
||||||
|
try {
|
||||||
|
doInternalExecute(task, bulkRequest, releasingListener);
|
||||||
|
} catch (Exception e) {
|
||||||
|
releasingListener.onFailure(e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
protected void doInternalExecute(Task task, BulkRequest bulkRequest, ActionListener<BulkResponse> listener) {
|
||||||
final long startTime = relativeTime();
|
final long startTime = relativeTime();
|
||||||
final AtomicArray<BulkItemResponse> responses = new AtomicArray<>(bulkRequest.requests.size());
|
final AtomicArray<BulkItemResponse> responses = new AtomicArray<>(bulkRequest.requests.size());
|
||||||
|
|
||||||
|
@ -749,7 +763,7 @@ public class TransportBulkAction extends HandledTransportAction<BulkRequest, Bul
|
||||||
// before we continue the bulk request we should fork back on a write thread:
|
// before we continue the bulk request we should fork back on a write thread:
|
||||||
if (originalThread == Thread.currentThread()) {
|
if (originalThread == Thread.currentThread()) {
|
||||||
assert Thread.currentThread().getName().contains(ThreadPool.Names.WRITE);
|
assert Thread.currentThread().getName().contains(ThreadPool.Names.WRITE);
|
||||||
doExecute(task, bulkRequest, actionListener);
|
doInternalExecute(task, bulkRequest, actionListener);
|
||||||
} else {
|
} else {
|
||||||
threadPool.executor(ThreadPool.Names.WRITE).execute(new AbstractRunnable() {
|
threadPool.executor(ThreadPool.Names.WRITE).execute(new AbstractRunnable() {
|
||||||
@Override
|
@Override
|
||||||
|
@ -759,7 +773,7 @@ public class TransportBulkAction extends HandledTransportAction<BulkRequest, Bul
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected void doRun() throws Exception {
|
protected void doRun() throws Exception {
|
||||||
doExecute(task, bulkRequest, actionListener);
|
doInternalExecute(task, bulkRequest, actionListener);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -90,9 +90,10 @@ public class TransportShardBulkAction extends TransportWriteAction<BulkShardRequ
|
||||||
@Inject
|
@Inject
|
||||||
public TransportShardBulkAction(Settings settings, TransportService transportService, ClusterService clusterService,
|
public TransportShardBulkAction(Settings settings, TransportService transportService, ClusterService clusterService,
|
||||||
IndicesService indicesService, ThreadPool threadPool, ShardStateAction shardStateAction,
|
IndicesService indicesService, ThreadPool threadPool, ShardStateAction shardStateAction,
|
||||||
MappingUpdatedAction mappingUpdatedAction, UpdateHelper updateHelper, ActionFilters actionFilters) {
|
MappingUpdatedAction mappingUpdatedAction, UpdateHelper updateHelper, ActionFilters actionFilters,
|
||||||
|
WriteMemoryLimits writeMemoryLimits) {
|
||||||
super(settings, ACTION_NAME, transportService, clusterService, indicesService, threadPool, shardStateAction, actionFilters,
|
super(settings, ACTION_NAME, transportService, clusterService, indicesService, threadPool, shardStateAction, actionFilters,
|
||||||
BulkShardRequest::new, BulkShardRequest::new, ThreadPool.Names.WRITE, false);
|
BulkShardRequest::new, BulkShardRequest::new, ThreadPool.Names.WRITE, false, writeMemoryLimits);
|
||||||
this.updateHelper = updateHelper;
|
this.updateHelper = updateHelper;
|
||||||
this.mappingUpdatedAction = mappingUpdatedAction;
|
this.mappingUpdatedAction = mappingUpdatedAction;
|
||||||
}
|
}
|
||||||
|
@ -108,7 +109,7 @@ public class TransportShardBulkAction extends TransportWriteAction<BulkShardRequ
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected void shardOperationOnPrimary(BulkShardRequest request, IndexShard primary,
|
protected void dispatchedShardOperationOnPrimary(BulkShardRequest request, IndexShard primary,
|
||||||
ActionListener<PrimaryResult<BulkShardRequest, BulkShardResponse>> listener) {
|
ActionListener<PrimaryResult<BulkShardRequest, BulkShardResponse>> listener) {
|
||||||
ClusterStateObserver observer = new ClusterStateObserver(clusterService, request.timeout(), logger, threadPool.getThreadContext());
|
ClusterStateObserver observer = new ClusterStateObserver(clusterService, request.timeout(), logger, threadPool.getThreadContext());
|
||||||
performOnPrimary(request, primary, updateHelper, threadPool::absoluteTimeInMillis,
|
performOnPrimary(request, primary, updateHelper, threadPool::absoluteTimeInMillis,
|
||||||
|
@ -136,6 +137,11 @@ public class TransportShardBulkAction extends TransportWriteAction<BulkShardRequ
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected long primaryOperationSize(BulkShardRequest request) {
|
||||||
|
return request.ramBytesUsed();
|
||||||
|
}
|
||||||
|
|
||||||
public static void performOnPrimary(
|
public static void performOnPrimary(
|
||||||
BulkShardRequest request,
|
BulkShardRequest request,
|
||||||
IndexShard primary,
|
IndexShard primary,
|
||||||
|
@ -404,9 +410,16 @@ public class TransportShardBulkAction extends TransportWriteAction<BulkShardRequ
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public WriteReplicaResult<BulkShardRequest> shardOperationOnReplica(BulkShardRequest request, IndexShard replica) throws Exception {
|
protected void dispatchedShardOperationOnReplica(BulkShardRequest request, IndexShard replica, ActionListener<ReplicaResult> listener) {
|
||||||
final Translog.Location location = performOnReplica(request, replica);
|
ActionListener.completeWith(listener, () -> {
|
||||||
return new WriteReplicaResult<>(request, location, null, replica, logger);
|
final Translog.Location location = performOnReplica(request, replica);
|
||||||
|
return new WriteReplicaResult<>(request, location, null, replica, logger);
|
||||||
|
});
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected long replicaOperationSize(BulkShardRequest request) {
|
||||||
|
return request.ramBytesUsed();
|
||||||
}
|
}
|
||||||
|
|
||||||
public static Translog.Location performOnReplica(BulkShardRequest request, IndexShard replica) throws Exception {
|
public static Translog.Location performOnReplica(BulkShardRequest request, IndexShard replica) throws Exception {
|
||||||
|
|
|
@ -0,0 +1,58 @@
|
||||||
|
/*
|
||||||
|
* Licensed to Elasticsearch under one or more contributor
|
||||||
|
* license agreements. See the NOTICE file distributed with
|
||||||
|
* this work for additional information regarding copyright
|
||||||
|
* ownership. Elasticsearch licenses this file to you under
|
||||||
|
* the Apache License, Version 2.0 (the "License"); you may
|
||||||
|
* not use this file except in compliance with the License.
|
||||||
|
* You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing,
|
||||||
|
* software distributed under the License is distributed on an
|
||||||
|
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||||
|
* KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations
|
||||||
|
* under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.elasticsearch.action.bulk;
|
||||||
|
|
||||||
|
import org.elasticsearch.common.lease.Releasable;
|
||||||
|
|
||||||
|
import java.util.concurrent.atomic.AtomicLong;
|
||||||
|
|
||||||
|
public class WriteMemoryLimits {
|
||||||
|
|
||||||
|
private final AtomicLong coordinatingBytes = new AtomicLong(0);
|
||||||
|
private final AtomicLong primaryBytes = new AtomicLong(0);
|
||||||
|
private final AtomicLong replicaBytes = new AtomicLong(0);
|
||||||
|
|
||||||
|
public Releasable markCoordinatingOperationStarted(long bytes) {
|
||||||
|
coordinatingBytes.addAndGet(bytes);
|
||||||
|
return () -> coordinatingBytes.getAndAdd(-bytes);
|
||||||
|
}
|
||||||
|
|
||||||
|
public long getCoordinatingBytes() {
|
||||||
|
return coordinatingBytes.get();
|
||||||
|
}
|
||||||
|
|
||||||
|
public Releasable markPrimaryOperationStarted(long bytes) {
|
||||||
|
primaryBytes.addAndGet(bytes);
|
||||||
|
return () -> primaryBytes.getAndAdd(-bytes);
|
||||||
|
}
|
||||||
|
|
||||||
|
public long getPrimaryBytes() {
|
||||||
|
return primaryBytes.get();
|
||||||
|
}
|
||||||
|
|
||||||
|
public Releasable markReplicaOperationStarted(long bytes) {
|
||||||
|
replicaBytes.getAndAdd(bytes);
|
||||||
|
return () -> replicaBytes.getAndAdd(-bytes);
|
||||||
|
}
|
||||||
|
|
||||||
|
public long getReplicaBytes() {
|
||||||
|
return replicaBytes.get();
|
||||||
|
}
|
||||||
|
}
|
|
@ -19,6 +19,7 @@
|
||||||
|
|
||||||
package org.elasticsearch.action.delete;
|
package org.elasticsearch.action.delete;
|
||||||
|
|
||||||
|
import org.apache.lucene.util.RamUsageEstimator;
|
||||||
import org.elasticsearch.Version;
|
import org.elasticsearch.Version;
|
||||||
import org.elasticsearch.action.ActionRequestValidationException;
|
import org.elasticsearch.action.ActionRequestValidationException;
|
||||||
import org.elasticsearch.action.CompositeIndicesRequest;
|
import org.elasticsearch.action.CompositeIndicesRequest;
|
||||||
|
@ -53,6 +54,8 @@ import static org.elasticsearch.index.seqno.SequenceNumbers.UNASSIGNED_SEQ_NO;
|
||||||
public class DeleteRequest extends ReplicatedWriteRequest<DeleteRequest>
|
public class DeleteRequest extends ReplicatedWriteRequest<DeleteRequest>
|
||||||
implements DocWriteRequest<DeleteRequest>, CompositeIndicesRequest {
|
implements DocWriteRequest<DeleteRequest>, CompositeIndicesRequest {
|
||||||
|
|
||||||
|
private static final long SHALLOW_SIZE = RamUsageEstimator.shallowSizeOfInstance(DeleteRequest.class);
|
||||||
|
|
||||||
private static final ShardId NO_SHARD_ID = null;
|
private static final ShardId NO_SHARD_ID = null;
|
||||||
|
|
||||||
// Set to null initially so we can know to override in bulk requests that have a default type.
|
// Set to null initially so we can know to override in bulk requests that have a default type.
|
||||||
|
@ -340,4 +343,9 @@ public class DeleteRequest extends ReplicatedWriteRequest<DeleteRequest>
|
||||||
public String toString() {
|
public String toString() {
|
||||||
return "delete {[" + index + "][" + type() + "][" + id + "]}";
|
return "delete {[" + index + "][" + type() + "][" + id + "]}";
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long ramBytesUsed() {
|
||||||
|
return SHALLOW_SIZE + RamUsageEstimator.sizeOf(id);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -19,6 +19,7 @@
|
||||||
|
|
||||||
package org.elasticsearch.action.index;
|
package org.elasticsearch.action.index;
|
||||||
|
|
||||||
|
import org.apache.lucene.util.RamUsageEstimator;
|
||||||
import org.elasticsearch.ElasticsearchGenerationException;
|
import org.elasticsearch.ElasticsearchGenerationException;
|
||||||
import org.elasticsearch.Version;
|
import org.elasticsearch.Version;
|
||||||
import org.elasticsearch.action.ActionRequestValidationException;
|
import org.elasticsearch.action.ActionRequestValidationException;
|
||||||
|
@ -77,6 +78,8 @@ import static org.elasticsearch.index.seqno.SequenceNumbers.UNASSIGNED_SEQ_NO;
|
||||||
*/
|
*/
|
||||||
public class IndexRequest extends ReplicatedWriteRequest<IndexRequest> implements DocWriteRequest<IndexRequest>, CompositeIndicesRequest {
|
public class IndexRequest extends ReplicatedWriteRequest<IndexRequest> implements DocWriteRequest<IndexRequest>, CompositeIndicesRequest {
|
||||||
|
|
||||||
|
private static final long SHALLOW_SIZE = RamUsageEstimator.shallowSizeOfInstance(IndexRequest.class);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Max length of the source document to include into string()
|
* Max length of the source document to include into string()
|
||||||
*
|
*
|
||||||
|
@ -795,4 +798,9 @@ public class IndexRequest extends ReplicatedWriteRequest<IndexRequest> implement
|
||||||
public long getAutoGeneratedTimestamp() {
|
public long getAutoGeneratedTimestamp() {
|
||||||
return autoGeneratedTimestamp;
|
return autoGeneratedTimestamp;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long ramBytesUsed() {
|
||||||
|
return SHALLOW_SIZE + RamUsageEstimator.sizeOf(id) + (source == null ? 0 : source.ramBytesUsed());
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -20,6 +20,7 @@ package org.elasticsearch.action.resync;
|
||||||
|
|
||||||
import org.apache.logging.log4j.message.ParameterizedMessage;
|
import org.apache.logging.log4j.message.ParameterizedMessage;
|
||||||
import org.elasticsearch.action.ActionListener;
|
import org.elasticsearch.action.ActionListener;
|
||||||
|
import org.elasticsearch.action.bulk.WriteMemoryLimits;
|
||||||
import org.elasticsearch.action.support.ActionFilters;
|
import org.elasticsearch.action.support.ActionFilters;
|
||||||
import org.elasticsearch.action.support.replication.ReplicationOperation;
|
import org.elasticsearch.action.support.replication.ReplicationOperation;
|
||||||
import org.elasticsearch.action.support.replication.ReplicationResponse;
|
import org.elasticsearch.action.support.replication.ReplicationResponse;
|
||||||
|
@ -45,6 +46,7 @@ import org.elasticsearch.transport.TransportResponseHandler;
|
||||||
import org.elasticsearch.transport.TransportService;
|
import org.elasticsearch.transport.TransportService;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import java.util.stream.Stream;
|
||||||
|
|
||||||
public class TransportResyncReplicationAction extends TransportWriteAction<ResyncReplicationRequest,
|
public class TransportResyncReplicationAction extends TransportWriteAction<ResyncReplicationRequest,
|
||||||
ResyncReplicationRequest, ResyncReplicationResponse> implements PrimaryReplicaSyncer.SyncAction {
|
ResyncReplicationRequest, ResyncReplicationResponse> implements PrimaryReplicaSyncer.SyncAction {
|
||||||
|
@ -54,10 +56,12 @@ public class TransportResyncReplicationAction extends TransportWriteAction<Resyn
|
||||||
@Inject
|
@Inject
|
||||||
public TransportResyncReplicationAction(Settings settings, TransportService transportService,
|
public TransportResyncReplicationAction(Settings settings, TransportService transportService,
|
||||||
ClusterService clusterService, IndicesService indicesService, ThreadPool threadPool,
|
ClusterService clusterService, IndicesService indicesService, ThreadPool threadPool,
|
||||||
ShardStateAction shardStateAction, ActionFilters actionFilters) {
|
ShardStateAction shardStateAction, ActionFilters actionFilters,
|
||||||
|
WriteMemoryLimits writeMemoryLimits) {
|
||||||
super(settings, ACTION_NAME, transportService, clusterService, indicesService, threadPool, shardStateAction, actionFilters,
|
super(settings, ACTION_NAME, transportService, clusterService, indicesService, threadPool, shardStateAction, actionFilters,
|
||||||
ResyncReplicationRequest::new, ResyncReplicationRequest::new, ThreadPool.Names.WRITE,
|
ResyncReplicationRequest::new, ResyncReplicationRequest::new, ThreadPool.Names.WRITE,
|
||||||
true /* we should never reject resync because of thread pool capacity on primary */);
|
true, /* we should never reject resync because of thread pool capacity on primary */
|
||||||
|
writeMemoryLimits);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -83,21 +87,33 @@ public class TransportResyncReplicationAction extends TransportWriteAction<Resyn
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected void shardOperationOnPrimary(ResyncReplicationRequest request, IndexShard primary,
|
protected void dispatchedShardOperationOnPrimary(ResyncReplicationRequest request, IndexShard primary,
|
||||||
ActionListener<PrimaryResult<ResyncReplicationRequest, ResyncReplicationResponse>> listener) {
|
ActionListener<PrimaryResult<ResyncReplicationRequest, ResyncReplicationResponse>> listener) {
|
||||||
ActionListener.completeWith(listener,
|
ActionListener.completeWith(listener,
|
||||||
() -> new WritePrimaryResult<>(performOnPrimary(request), new ResyncReplicationResponse(), null, null, primary, logger));
|
() -> new WritePrimaryResult<>(performOnPrimary(request), new ResyncReplicationResponse(), null, null, primary, logger));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected long primaryOperationSize(ResyncReplicationRequest request) {
|
||||||
|
return Stream.of(request.getOperations()).mapToLong(Translog.Operation::estimateSize).sum();
|
||||||
|
}
|
||||||
|
|
||||||
public static ResyncReplicationRequest performOnPrimary(ResyncReplicationRequest request) {
|
public static ResyncReplicationRequest performOnPrimary(ResyncReplicationRequest request) {
|
||||||
return request;
|
return request;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected WriteReplicaResult<ResyncReplicationRequest> shardOperationOnReplica(ResyncReplicationRequest request,
|
protected void dispatchedShardOperationOnReplica(ResyncReplicationRequest request, IndexShard replica,
|
||||||
IndexShard replica) throws Exception {
|
ActionListener<ReplicaResult> listener) {
|
||||||
Translog.Location location = performOnReplica(request, replica);
|
ActionListener.completeWith(listener, () -> {
|
||||||
return new WriteReplicaResult<>(request, location, null, replica, logger);
|
Translog.Location location = performOnReplica(request, replica);
|
||||||
|
return new WriteReplicaResult<>(request, location, null, replica, logger);
|
||||||
|
});
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected long replicaOperationSize(ResyncReplicationRequest request) {
|
||||||
|
return Stream.of(request.getOperations()).mapToLong(Translog.Operation::estimateSize).sum();
|
||||||
}
|
}
|
||||||
|
|
||||||
public static Translog.Location performOnReplica(ResyncReplicationRequest request, IndexShard replica) throws Exception {
|
public static Translog.Location performOnReplica(ResyncReplicationRequest request, IndexShard replica) throws Exception {
|
||||||
|
|
|
@ -19,9 +19,6 @@
|
||||||
|
|
||||||
package org.elasticsearch.action.support;
|
package org.elasticsearch.action.support;
|
||||||
|
|
||||||
import org.apache.logging.log4j.LogManager;
|
|
||||||
import org.apache.logging.log4j.Logger;
|
|
||||||
import org.apache.logging.log4j.message.ParameterizedMessage;
|
|
||||||
import org.elasticsearch.action.ActionListener;
|
import org.elasticsearch.action.ActionListener;
|
||||||
import org.elasticsearch.transport.TransportChannel;
|
import org.elasticsearch.transport.TransportChannel;
|
||||||
import org.elasticsearch.transport.TransportRequest;
|
import org.elasticsearch.transport.TransportRequest;
|
||||||
|
@ -30,7 +27,6 @@ import org.elasticsearch.transport.TransportResponse;
|
||||||
public final class ChannelActionListener<
|
public final class ChannelActionListener<
|
||||||
Response extends TransportResponse, Request extends TransportRequest> implements ActionListener<Response> {
|
Response extends TransportResponse, Request extends TransportRequest> implements ActionListener<Response> {
|
||||||
|
|
||||||
private static final Logger logger = LogManager.getLogger(ChannelActionListener.class);
|
|
||||||
private final TransportChannel channel;
|
private final TransportChannel channel;
|
||||||
private final Request request;
|
private final Request request;
|
||||||
private final String actionName;
|
private final String actionName;
|
||||||
|
@ -52,12 +48,6 @@ public final class ChannelActionListener<
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void onFailure(Exception e) {
|
public void onFailure(Exception e) {
|
||||||
try {
|
TransportChannel.sendErrorResponse(channel, actionName, request, e);
|
||||||
channel.sendResponse(e);
|
|
||||||
} catch (Exception e1) {
|
|
||||||
e1.addSuppressed(e);
|
|
||||||
logger.warn(() -> new ParameterizedMessage(
|
|
||||||
"Failed to send error response for action [{}] and request [{}]", actionName, request), e1);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -214,13 +214,14 @@ public abstract class TransportReplicationAction<
|
||||||
ActionListener<PrimaryResult<ReplicaRequest, Response>> listener);
|
ActionListener<PrimaryResult<ReplicaRequest, Response>> listener);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Synchronously execute the specified replica operation. This is done under a permit from
|
* Execute the specified replica operation. This is done under a permit from
|
||||||
* {@link IndexShard#acquireReplicaOperationPermit(long, long, long, ActionListener, String, Object)}.
|
* {@link IndexShard#acquireReplicaOperationPermit(long, long, long, ActionListener, String, Object)}.
|
||||||
*
|
*
|
||||||
* @param shardRequest the request to the replica shard
|
* @param shardRequest the request to the replica shard
|
||||||
* @param replica the replica shard to perform the operation on
|
* @param replica the replica shard to perform the operation on
|
||||||
*/
|
*/
|
||||||
protected abstract ReplicaResult shardOperationOnReplica(ReplicaRequest shardRequest, IndexShard replica) throws Exception;
|
protected abstract void shardOperationOnReplica(ReplicaRequest shardRequest, IndexShard replica,
|
||||||
|
ActionListener<ReplicaResult> listener);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Cluster level block to check before request execution. Returning null means that no blocks need to be checked.
|
* Cluster level block to check before request execution. Returning null means that no blocks need to be checked.
|
||||||
|
@ -273,13 +274,31 @@ public abstract class TransportReplicationAction<
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
|
|
||||||
protected void handleOperationRequest(final Request request, final TransportChannel channel, Task task) {
|
private void handleOperationRequest(final Request request, final TransportChannel channel, Task task) {
|
||||||
execute(task, request, new ChannelActionListener<>(channel, actionName, request));
|
Releasable releasable = checkOperationLimits(request);
|
||||||
|
ActionListener<Response> listener =
|
||||||
|
ActionListener.runBefore(new ChannelActionListener<>(channel, actionName, request), releasable::close);
|
||||||
|
execute(task, request, listener);
|
||||||
|
}
|
||||||
|
|
||||||
|
protected Releasable checkOperationLimits(final Request request) {
|
||||||
|
return () -> {};
|
||||||
}
|
}
|
||||||
|
|
||||||
protected void handlePrimaryRequest(final ConcreteShardRequest<Request> request, final TransportChannel channel, final Task task) {
|
protected void handlePrimaryRequest(final ConcreteShardRequest<Request> request, final TransportChannel channel, final Task task) {
|
||||||
new AsyncPrimaryAction(
|
Releasable releasable = checkPrimaryLimits(request.getRequest());
|
||||||
request, new ChannelActionListener<>(channel, transportPrimaryAction, request), (ReplicationTask) task).run();
|
ActionListener<Response> listener =
|
||||||
|
ActionListener.runBefore(new ChannelActionListener<>(channel, transportPrimaryAction, request), releasable::close);
|
||||||
|
|
||||||
|
try {
|
||||||
|
new AsyncPrimaryAction(request, listener, (ReplicationTask) task).run();
|
||||||
|
} catch (RuntimeException e) {
|
||||||
|
listener.onFailure(e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
protected Releasable checkPrimaryLimits(final Request request) {
|
||||||
|
return () -> {};
|
||||||
}
|
}
|
||||||
|
|
||||||
class AsyncPrimaryAction extends AbstractRunnable {
|
class AsyncPrimaryAction extends AbstractRunnable {
|
||||||
|
@ -490,10 +509,21 @@ public abstract class TransportReplicationAction<
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
protected void handleReplicaRequest(final ConcreteReplicaRequest<ReplicaRequest> replicaRequest,
|
protected void handleReplicaRequest(final ConcreteReplicaRequest<ReplicaRequest> replicaRequest, final TransportChannel channel,
|
||||||
final TransportChannel channel, final Task task) {
|
final Task task) {
|
||||||
new AsyncReplicaAction(
|
Releasable releasable = checkReplicaLimits(replicaRequest.getRequest());
|
||||||
replicaRequest, new ChannelActionListener<>(channel, transportReplicaAction, replicaRequest), (ReplicationTask) task).run();
|
ActionListener<ReplicaResponse> listener =
|
||||||
|
ActionListener.runBefore(new ChannelActionListener<>(channel, transportReplicaAction, replicaRequest), releasable::close);
|
||||||
|
|
||||||
|
try {
|
||||||
|
new AsyncReplicaAction(replicaRequest, listener, (ReplicationTask) task).run();
|
||||||
|
} catch (RuntimeException e) {
|
||||||
|
listener.onFailure(e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
protected Releasable checkReplicaLimits(final ReplicaRequest request) {
|
||||||
|
return () -> {};
|
||||||
}
|
}
|
||||||
|
|
||||||
public static class RetryOnReplicaException extends ElasticsearchException {
|
public static class RetryOnReplicaException extends ElasticsearchException {
|
||||||
|
@ -532,27 +562,31 @@ public abstract class TransportReplicationAction<
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void onResponse(Releasable releasable) {
|
public void onResponse(Releasable releasable) {
|
||||||
|
assert replica.getActiveOperationsCount() != 0 : "must perform shard operation under a permit";
|
||||||
try {
|
try {
|
||||||
assert replica.getActiveOperationsCount() != 0 : "must perform shard operation under a permit";
|
shardOperationOnReplica(replicaRequest.getRequest(), replica, ActionListener.wrap((replicaResult) ->
|
||||||
final ReplicaResult replicaResult = shardOperationOnReplica(replicaRequest.getRequest(), replica);
|
replicaResult.runPostReplicaActions(
|
||||||
replicaResult.runPostReplicaActions(
|
ActionListener.wrap(r -> {
|
||||||
ActionListener.wrap(r -> {
|
final ReplicaResponse response =
|
||||||
final TransportReplicationAction.ReplicaResponse response =
|
new ReplicaResponse(replica.getLocalCheckpoint(), replica.getLastSyncedGlobalCheckpoint());
|
||||||
new ReplicaResponse(replica.getLocalCheckpoint(), replica.getLastSyncedGlobalCheckpoint());
|
releasable.close(); // release shard operation lock before responding to caller
|
||||||
releasable.close(); // release shard operation lock before responding to caller
|
if (logger.isTraceEnabled()) {
|
||||||
if (logger.isTraceEnabled()) {
|
logger.trace("action [{}] completed on shard [{}] for request [{}]", transportReplicaAction,
|
||||||
logger.trace("action [{}] completed on shard [{}] for request [{}]", transportReplicaAction,
|
replicaRequest.getRequest().shardId(),
|
||||||
replicaRequest.getRequest().shardId(),
|
replicaRequest.getRequest());
|
||||||
replicaRequest.getRequest());
|
}
|
||||||
}
|
setPhase(task, "finished");
|
||||||
setPhase(task, "finished");
|
onCompletionListener.onResponse(response);
|
||||||
onCompletionListener.onResponse(response);
|
}, e -> {
|
||||||
}, e -> {
|
Releasables.closeWhileHandlingException(releasable); // release shard operation lock before responding to caller
|
||||||
Releasables.closeWhileHandlingException(releasable); // release shard operation lock before responding to caller
|
responseWithFailure(e);
|
||||||
this.responseWithFailure(e);
|
})
|
||||||
})
|
), e -> {
|
||||||
);
|
Releasables.closeWhileHandlingException(releasable); // release shard operation lock before responding to caller
|
||||||
} catch (final Exception e) {
|
AsyncReplicaAction.this.onFailure(e);
|
||||||
|
}));
|
||||||
|
// TODO: Evaludate if we still need to catch this exception
|
||||||
|
} catch (Exception e) {
|
||||||
Releasables.closeWhileHandlingException(releasable); // release shard operation lock before responding to caller
|
Releasables.closeWhileHandlingException(releasable); // release shard operation lock before responding to caller
|
||||||
AsyncReplicaAction.this.onFailure(e);
|
AsyncReplicaAction.this.onFailure(e);
|
||||||
}
|
}
|
||||||
|
|
|
@ -22,6 +22,8 @@ package org.elasticsearch.action.support.replication;
|
||||||
import org.apache.logging.log4j.Logger;
|
import org.apache.logging.log4j.Logger;
|
||||||
import org.apache.logging.log4j.message.ParameterizedMessage;
|
import org.apache.logging.log4j.message.ParameterizedMessage;
|
||||||
import org.elasticsearch.action.ActionListener;
|
import org.elasticsearch.action.ActionListener;
|
||||||
|
import org.elasticsearch.action.ActionRunnable;
|
||||||
|
import org.elasticsearch.action.bulk.WriteMemoryLimits;
|
||||||
import org.elasticsearch.action.support.ActionFilters;
|
import org.elasticsearch.action.support.ActionFilters;
|
||||||
import org.elasticsearch.action.support.TransportActions;
|
import org.elasticsearch.action.support.TransportActions;
|
||||||
import org.elasticsearch.action.support.WriteRequest;
|
import org.elasticsearch.action.support.WriteRequest;
|
||||||
|
@ -32,6 +34,7 @@ import org.elasticsearch.cluster.routing.ShardRouting;
|
||||||
import org.elasticsearch.cluster.service.ClusterService;
|
import org.elasticsearch.cluster.service.ClusterService;
|
||||||
import org.elasticsearch.common.Nullable;
|
import org.elasticsearch.common.Nullable;
|
||||||
import org.elasticsearch.common.io.stream.Writeable;
|
import org.elasticsearch.common.io.stream.Writeable;
|
||||||
|
import org.elasticsearch.common.lease.Releasable;
|
||||||
import org.elasticsearch.common.settings.Settings;
|
import org.elasticsearch.common.settings.Settings;
|
||||||
import org.elasticsearch.index.engine.Engine;
|
import org.elasticsearch.index.engine.Engine;
|
||||||
import org.elasticsearch.index.mapper.MapperParsingException;
|
import org.elasticsearch.index.mapper.MapperParsingException;
|
||||||
|
@ -57,12 +60,45 @@ public abstract class TransportWriteAction<
|
||||||
Response extends ReplicationResponse & WriteResponse
|
Response extends ReplicationResponse & WriteResponse
|
||||||
> extends TransportReplicationAction<Request, ReplicaRequest, Response> {
|
> extends TransportReplicationAction<Request, ReplicaRequest, Response> {
|
||||||
|
|
||||||
|
private final boolean forceExecutionOnPrimary;
|
||||||
|
private final WriteMemoryLimits writeMemoryLimits;
|
||||||
|
private final String executor;
|
||||||
|
|
||||||
protected TransportWriteAction(Settings settings, String actionName, TransportService transportService,
|
protected TransportWriteAction(Settings settings, String actionName, TransportService transportService,
|
||||||
ClusterService clusterService, IndicesService indicesService, ThreadPool threadPool,
|
ClusterService clusterService, IndicesService indicesService, ThreadPool threadPool,
|
||||||
ShardStateAction shardStateAction, ActionFilters actionFilters, Writeable.Reader<Request> request,
|
ShardStateAction shardStateAction, ActionFilters actionFilters, Writeable.Reader<Request> request,
|
||||||
Writeable.Reader<ReplicaRequest> replicaRequest, String executor, boolean forceExecutionOnPrimary) {
|
Writeable.Reader<ReplicaRequest> replicaRequest, String executor, boolean forceExecutionOnPrimary,
|
||||||
|
WriteMemoryLimits writeMemoryLimits) {
|
||||||
|
// We pass ThreadPool.Names.SAME to the super class as we control the dispatching to the
|
||||||
|
// ThreadPool.Names.WRITE thread pool in this class.
|
||||||
super(settings, actionName, transportService, clusterService, indicesService, threadPool, shardStateAction, actionFilters,
|
super(settings, actionName, transportService, clusterService, indicesService, threadPool, shardStateAction, actionFilters,
|
||||||
request, replicaRequest, executor, true, forceExecutionOnPrimary);
|
request, replicaRequest, ThreadPool.Names.SAME, true, forceExecutionOnPrimary);
|
||||||
|
this.executor = executor;
|
||||||
|
this.forceExecutionOnPrimary = forceExecutionOnPrimary;
|
||||||
|
this.writeMemoryLimits = writeMemoryLimits;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Releasable checkOperationLimits(Request request) {
|
||||||
|
return writeMemoryLimits.markCoordinatingOperationStarted(primaryOperationSize(request));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Releasable checkPrimaryLimits(Request request) {
|
||||||
|
return writeMemoryLimits.markPrimaryOperationStarted(primaryOperationSize(request));
|
||||||
|
}
|
||||||
|
|
||||||
|
protected long primaryOperationSize(Request request) {
|
||||||
|
return 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Releasable checkReplicaLimits(ReplicaRequest request) {
|
||||||
|
return writeMemoryLimits.markReplicaOperationStarted(replicaOperationSize(request));
|
||||||
|
}
|
||||||
|
|
||||||
|
protected long replicaOperationSize(ReplicaRequest request) {
|
||||||
|
return 0;
|
||||||
}
|
}
|
||||||
|
|
||||||
/** Syncs operation result to the translog or throws a shard not available failure */
|
/** Syncs operation result to the translog or throws a shard not available failure */
|
||||||
|
@ -104,18 +140,48 @@ public abstract class TransportWriteAction<
|
||||||
* and failure async refresh is performed on the <code>primary</code> shard according to the <code>Request</code> refresh policy
|
* and failure async refresh is performed on the <code>primary</code> shard according to the <code>Request</code> refresh policy
|
||||||
*/
|
*/
|
||||||
@Override
|
@Override
|
||||||
protected abstract void shardOperationOnPrimary(
|
protected void shardOperationOnPrimary(
|
||||||
Request request, IndexShard primary, ActionListener<PrimaryResult<ReplicaRequest, Response>> listener);
|
Request request, IndexShard primary, ActionListener<PrimaryResult<ReplicaRequest, Response>> listener) {
|
||||||
|
threadPool.executor(executor).execute(new ActionRunnable<PrimaryResult<ReplicaRequest, Response>>(listener) {
|
||||||
|
@Override
|
||||||
|
protected void doRun() {
|
||||||
|
dispatchedShardOperationOnPrimary(request, primary, listener);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean isForceExecution() {
|
||||||
|
return forceExecutionOnPrimary;
|
||||||
|
}
|
||||||
|
});
|
||||||
|
}
|
||||||
|
|
||||||
|
protected abstract void dispatchedShardOperationOnPrimary(
|
||||||
|
Request request, IndexShard primary, ActionListener<PrimaryResult<ReplicaRequest, Response>> listener);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Called once per replica with a reference to the replica {@linkplain IndexShard} to modify.
|
* Called once per replica with a reference to the replica {@linkplain IndexShard} to modify.
|
||||||
*
|
*
|
||||||
* @return the result of the operation on replica, including current translog location and operation response and failure
|
* @param listener listener for the result of the operation on replica, including current translog location and operation
|
||||||
* async refresh is performed on the <code>replica</code> shard according to the <code>ReplicaRequest</code> refresh policy
|
* response and failure async refresh is performed on the <code>replica</code> shard according to the <code>ReplicaRequest</code>
|
||||||
|
* refresh policy
|
||||||
*/
|
*/
|
||||||
@Override
|
@Override
|
||||||
protected abstract WriteReplicaResult<ReplicaRequest> shardOperationOnReplica(
|
protected void shardOperationOnReplica(ReplicaRequest request, IndexShard replica, ActionListener<ReplicaResult> listener) {
|
||||||
ReplicaRequest request, IndexShard replica) throws Exception;
|
threadPool.executor(executor).execute(new ActionRunnable<ReplicaResult>(listener) {
|
||||||
|
@Override
|
||||||
|
protected void doRun() {
|
||||||
|
dispatchedShardOperationOnReplica(request, replica, listener);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean isForceExecution() {
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
});
|
||||||
|
}
|
||||||
|
|
||||||
|
protected abstract void dispatchedShardOperationOnReplica(
|
||||||
|
ReplicaRequest request, IndexShard replica, ActionListener<ReplicaResult> listener);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Result of taking the action on the primary.
|
* Result of taking the action on the primary.
|
||||||
|
|
|
@ -19,6 +19,7 @@
|
||||||
|
|
||||||
package org.elasticsearch.action.update;
|
package org.elasticsearch.action.update;
|
||||||
|
|
||||||
|
import org.apache.lucene.util.RamUsageEstimator;
|
||||||
import org.elasticsearch.Version;
|
import org.elasticsearch.Version;
|
||||||
import org.elasticsearch.action.ActionRequestValidationException;
|
import org.elasticsearch.action.ActionRequestValidationException;
|
||||||
import org.elasticsearch.action.DocWriteRequest;
|
import org.elasticsearch.action.DocWriteRequest;
|
||||||
|
@ -59,6 +60,9 @@ import static org.elasticsearch.index.seqno.SequenceNumbers.UNASSIGNED_SEQ_NO;
|
||||||
|
|
||||||
public class UpdateRequest extends InstanceShardOperationRequest<UpdateRequest>
|
public class UpdateRequest extends InstanceShardOperationRequest<UpdateRequest>
|
||||||
implements DocWriteRequest<UpdateRequest>, WriteRequest<UpdateRequest>, ToXContentObject {
|
implements DocWriteRequest<UpdateRequest>, WriteRequest<UpdateRequest>, ToXContentObject {
|
||||||
|
|
||||||
|
private static final long SHALLOW_SIZE = RamUsageEstimator.shallowSizeOfInstance(UpdateRequest.class);
|
||||||
|
|
||||||
private static ObjectParser<UpdateRequest, Void> PARSER;
|
private static ObjectParser<UpdateRequest, Void> PARSER;
|
||||||
|
|
||||||
private static final ParseField SCRIPT_FIELD = new ParseField("script");
|
private static final ParseField SCRIPT_FIELD = new ParseField("script");
|
||||||
|
@ -1010,4 +1014,16 @@ public class UpdateRequest extends InstanceShardOperationRequest<UpdateRequest>
|
||||||
res.append(", detect_noop[").append(detectNoop).append("]");
|
res.append(", detect_noop[").append(detectNoop).append("]");
|
||||||
return res.append("}").toString();
|
return res.append("}").toString();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long ramBytesUsed() {
|
||||||
|
long childRequestBytes = 0;
|
||||||
|
if (doc != null) {
|
||||||
|
childRequestBytes += doc.ramBytesUsed();
|
||||||
|
}
|
||||||
|
if (upsertRequest != null) {
|
||||||
|
childRequestBytes += upsertRequest.ramBytesUsed();
|
||||||
|
}
|
||||||
|
return SHALLOW_SIZE + RamUsageEstimator.sizeOf(id) + childRequestBytes;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -108,9 +108,11 @@ public class GlobalCheckpointSyncAction extends TransportReplicationAction<
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected ReplicaResult shardOperationOnReplica(final Request request, final IndexShard indexShard) throws Exception {
|
protected void shardOperationOnReplica(Request shardRequest, IndexShard replica, ActionListener<ReplicaResult> listener) {
|
||||||
maybeSyncTranslog(indexShard);
|
ActionListener.completeWith(listener, () -> {
|
||||||
return new ReplicaResult();
|
maybeSyncTranslog(replica);
|
||||||
|
return new ReplicaResult();
|
||||||
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
private void maybeSyncTranslog(final IndexShard indexShard) throws IOException {
|
private void maybeSyncTranslog(final IndexShard indexShard) throws IOException {
|
||||||
|
|
|
@ -38,7 +38,6 @@ import org.elasticsearch.common.io.stream.StreamInput;
|
||||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||||
import org.elasticsearch.common.settings.Settings;
|
import org.elasticsearch.common.settings.Settings;
|
||||||
import org.elasticsearch.common.util.concurrent.ThreadContext;
|
import org.elasticsearch.common.util.concurrent.ThreadContext;
|
||||||
import org.elasticsearch.gateway.WriteStateException;
|
|
||||||
import org.elasticsearch.index.IndexNotFoundException;
|
import org.elasticsearch.index.IndexNotFoundException;
|
||||||
import org.elasticsearch.index.shard.IndexShard;
|
import org.elasticsearch.index.shard.IndexShard;
|
||||||
import org.elasticsearch.index.shard.IndexShardClosedException;
|
import org.elasticsearch.index.shard.IndexShardClosedException;
|
||||||
|
@ -165,12 +164,14 @@ public class RetentionLeaseBackgroundSyncAction extends TransportReplicationActi
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected ReplicaResult shardOperationOnReplica(final Request request, final IndexShard replica) throws WriteStateException {
|
protected void shardOperationOnReplica(Request request, IndexShard replica, ActionListener<ReplicaResult> listener) {
|
||||||
Objects.requireNonNull(request);
|
ActionListener.completeWith(listener, () -> {
|
||||||
Objects.requireNonNull(replica);
|
Objects.requireNonNull(request);
|
||||||
replica.updateRetentionLeasesOnReplica(request.getRetentionLeases());
|
Objects.requireNonNull(replica);
|
||||||
replica.persistRetentionLeases();
|
replica.updateRetentionLeasesOnReplica(request.getRetentionLeases());
|
||||||
return new ReplicaResult();
|
replica.persistRetentionLeases();
|
||||||
|
return new ReplicaResult();
|
||||||
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
public static final class Request extends ReplicationRequest<Request> {
|
public static final class Request extends ReplicationRequest<Request> {
|
||||||
|
|
|
@ -25,6 +25,7 @@ import org.apache.logging.log4j.message.ParameterizedMessage;
|
||||||
import org.apache.lucene.store.AlreadyClosedException;
|
import org.apache.lucene.store.AlreadyClosedException;
|
||||||
import org.elasticsearch.ExceptionsHelper;
|
import org.elasticsearch.ExceptionsHelper;
|
||||||
import org.elasticsearch.action.ActionListener;
|
import org.elasticsearch.action.ActionListener;
|
||||||
|
import org.elasticsearch.action.bulk.WriteMemoryLimits;
|
||||||
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.WriteResponse;
|
import org.elasticsearch.action.support.WriteResponse;
|
||||||
|
@ -40,7 +41,6 @@ import org.elasticsearch.common.io.stream.StreamInput;
|
||||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||||
import org.elasticsearch.common.settings.Settings;
|
import org.elasticsearch.common.settings.Settings;
|
||||||
import org.elasticsearch.common.util.concurrent.ThreadContext;
|
import org.elasticsearch.common.util.concurrent.ThreadContext;
|
||||||
import org.elasticsearch.gateway.WriteStateException;
|
|
||||||
import org.elasticsearch.index.IndexNotFoundException;
|
import org.elasticsearch.index.IndexNotFoundException;
|
||||||
import org.elasticsearch.index.shard.IndexShard;
|
import org.elasticsearch.index.shard.IndexShard;
|
||||||
import org.elasticsearch.index.shard.IndexShardClosedException;
|
import org.elasticsearch.index.shard.IndexShardClosedException;
|
||||||
|
@ -79,7 +79,8 @@ public class RetentionLeaseSyncAction extends
|
||||||
final IndicesService indicesService,
|
final IndicesService indicesService,
|
||||||
final ThreadPool threadPool,
|
final ThreadPool threadPool,
|
||||||
final ShardStateAction shardStateAction,
|
final ShardStateAction shardStateAction,
|
||||||
final ActionFilters actionFilters) {
|
final ActionFilters actionFilters,
|
||||||
|
final WriteMemoryLimits writeMemoryLimits) {
|
||||||
super(
|
super(
|
||||||
settings,
|
settings,
|
||||||
ACTION_NAME,
|
ACTION_NAME,
|
||||||
|
@ -91,7 +92,7 @@ public class RetentionLeaseSyncAction extends
|
||||||
actionFilters,
|
actionFilters,
|
||||||
RetentionLeaseSyncAction.Request::new,
|
RetentionLeaseSyncAction.Request::new,
|
||||||
RetentionLeaseSyncAction.Request::new,
|
RetentionLeaseSyncAction.Request::new,
|
||||||
ThreadPool.Names.MANAGEMENT, false);
|
ThreadPool.Names.MANAGEMENT, false, writeMemoryLimits);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -146,7 +147,7 @@ public class RetentionLeaseSyncAction extends
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected void shardOperationOnPrimary(Request request, IndexShard primary,
|
protected void dispatchedShardOperationOnPrimary(Request request, IndexShard primary,
|
||||||
ActionListener<PrimaryResult<Request, Response>> listener) {
|
ActionListener<PrimaryResult<Request, Response>> listener) {
|
||||||
ActionListener.completeWith(listener, () -> {
|
ActionListener.completeWith(listener, () -> {
|
||||||
assert request.waitForActiveShards().equals(ActiveShardCount.NONE) : request.waitForActiveShards();
|
assert request.waitForActiveShards().equals(ActiveShardCount.NONE) : request.waitForActiveShards();
|
||||||
|
@ -158,14 +159,15 @@ public class RetentionLeaseSyncAction extends
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected WriteReplicaResult<Request> shardOperationOnReplica(
|
protected void dispatchedShardOperationOnReplica(Request request, IndexShard replica,
|
||||||
final Request request,
|
ActionListener<ReplicaResult> listener) {
|
||||||
final IndexShard replica) throws WriteStateException {
|
ActionListener.completeWith(listener, () -> {
|
||||||
Objects.requireNonNull(request);
|
Objects.requireNonNull(request);
|
||||||
Objects.requireNonNull(replica);
|
Objects.requireNonNull(replica);
|
||||||
replica.updateRetentionLeasesOnReplica(request.getRetentionLeases());
|
replica.updateRetentionLeasesOnReplica(request.getRetentionLeases());
|
||||||
replica.persistRetentionLeases();
|
replica.persistRetentionLeases();
|
||||||
return new WriteReplicaResult<>(request, null, null, replica, getLogger());
|
return new WriteReplicaResult<>(request, null, null, replica, getLogger());
|
||||||
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -31,6 +31,7 @@ import org.elasticsearch.Version;
|
||||||
import org.elasticsearch.action.ActionModule;
|
import org.elasticsearch.action.ActionModule;
|
||||||
import org.elasticsearch.action.ActionType;
|
import org.elasticsearch.action.ActionType;
|
||||||
import org.elasticsearch.action.admin.cluster.snapshots.status.TransportNodesSnapshotsStatus;
|
import org.elasticsearch.action.admin.cluster.snapshots.status.TransportNodesSnapshotsStatus;
|
||||||
|
import org.elasticsearch.action.bulk.WriteMemoryLimits;
|
||||||
import org.elasticsearch.action.search.SearchExecutionStatsCollector;
|
import org.elasticsearch.action.search.SearchExecutionStatsCollector;
|
||||||
import org.elasticsearch.action.search.SearchPhaseController;
|
import org.elasticsearch.action.search.SearchPhaseController;
|
||||||
import org.elasticsearch.action.search.SearchTransportService;
|
import org.elasticsearch.action.search.SearchTransportService;
|
||||||
|
@ -593,6 +594,7 @@ public class Node implements Closeable {
|
||||||
new PersistentTasksClusterService(settings, registry, clusterService, threadPool);
|
new PersistentTasksClusterService(settings, registry, clusterService, threadPool);
|
||||||
resourcesToClose.add(persistentTasksClusterService);
|
resourcesToClose.add(persistentTasksClusterService);
|
||||||
final PersistentTasksService persistentTasksService = new PersistentTasksService(clusterService, threadPool, client);
|
final PersistentTasksService persistentTasksService = new PersistentTasksService(clusterService, threadPool, client);
|
||||||
|
final WriteMemoryLimits bulkIndexingLimits = new WriteMemoryLimits();
|
||||||
|
|
||||||
modules.add(b -> {
|
modules.add(b -> {
|
||||||
b.bind(Node.class).toInstance(this);
|
b.bind(Node.class).toInstance(this);
|
||||||
|
@ -611,6 +613,7 @@ public class Node implements Closeable {
|
||||||
b.bind(ScriptService.class).toInstance(scriptService);
|
b.bind(ScriptService.class).toInstance(scriptService);
|
||||||
b.bind(AnalysisRegistry.class).toInstance(analysisModule.getAnalysisRegistry());
|
b.bind(AnalysisRegistry.class).toInstance(analysisModule.getAnalysisRegistry());
|
||||||
b.bind(IngestService.class).toInstance(ingestService);
|
b.bind(IngestService.class).toInstance(ingestService);
|
||||||
|
b.bind(WriteMemoryLimits.class).toInstance(bulkIndexingLimits);
|
||||||
b.bind(UsageService.class).toInstance(usageService);
|
b.bind(UsageService.class).toInstance(usageService);
|
||||||
b.bind(AggregationUsageService.class).toInstance(searchModule.getValuesSourceRegistry().getUsageService());
|
b.bind(AggregationUsageService.class).toInstance(searchModule.getValuesSourceRegistry().getUsageService());
|
||||||
b.bind(NamedWriteableRegistry.class).toInstance(namedWriteableRegistry);
|
b.bind(NamedWriteableRegistry.class).toInstance(namedWriteableRegistry);
|
||||||
|
|
|
@ -19,6 +19,9 @@
|
||||||
|
|
||||||
package org.elasticsearch.transport;
|
package org.elasticsearch.transport;
|
||||||
|
|
||||||
|
import org.apache.logging.log4j.LogManager;
|
||||||
|
import org.apache.logging.log4j.Logger;
|
||||||
|
import org.apache.logging.log4j.message.ParameterizedMessage;
|
||||||
import org.elasticsearch.Version;
|
import org.elasticsearch.Version;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
@ -28,6 +31,8 @@ import java.io.IOException;
|
||||||
*/
|
*/
|
||||||
public interface TransportChannel {
|
public interface TransportChannel {
|
||||||
|
|
||||||
|
Logger logger = LogManager.getLogger(TransportChannel.class);
|
||||||
|
|
||||||
String getProfileName();
|
String getProfileName();
|
||||||
|
|
||||||
String getChannelType();
|
String getChannelType();
|
||||||
|
@ -42,4 +47,17 @@ public interface TransportChannel {
|
||||||
default Version getVersion() {
|
default Version getVersion() {
|
||||||
return Version.CURRENT;
|
return Version.CURRENT;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* A helper method to send an exception and handle and log a subsequent exception
|
||||||
|
*/
|
||||||
|
static void sendErrorResponse(TransportChannel channel, String actionName, TransportRequest request, Exception e) {
|
||||||
|
try {
|
||||||
|
channel.sendResponse(e);
|
||||||
|
} catch (Exception sendException) {
|
||||||
|
sendException.addSuppressed(e);
|
||||||
|
logger.warn(() -> new ParameterizedMessage(
|
||||||
|
"Failed to send error response for action [{}] and request [{}]", actionName, request), sendException);
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -120,7 +120,7 @@ public class TransportBulkActionIndicesThatCannotBeCreatedTests extends ESTestCa
|
||||||
final ExecutorService direct = EsExecutors.newDirectExecutorService();
|
final ExecutorService direct = EsExecutors.newDirectExecutorService();
|
||||||
when(threadPool.executor(anyString())).thenReturn(direct);
|
when(threadPool.executor(anyString())).thenReturn(direct);
|
||||||
TransportBulkAction action = new TransportBulkAction(threadPool, mock(TransportService.class), clusterService,
|
TransportBulkAction action = new TransportBulkAction(threadPool, mock(TransportService.class), clusterService,
|
||||||
null, null, null, mock(ActionFilters.class), null, null) {
|
null, null, null, mock(ActionFilters.class), null, null, new WriteMemoryLimits()) {
|
||||||
@Override
|
@Override
|
||||||
void executeBulk(Task task, BulkRequest bulkRequest, long startTimeNanos, ActionListener<BulkResponse> listener,
|
void executeBulk(Task task, BulkRequest bulkRequest, long startTimeNanos, ActionListener<BulkResponse> listener,
|
||||||
AtomicArray<BulkItemResponse> responses, Map<String, IndexNotFoundException> indicesThatCannotBeCreated) {
|
AtomicArray<BulkItemResponse> responses, Map<String, IndexNotFoundException> indicesThatCannotBeCreated) {
|
||||||
|
|
|
@ -142,7 +142,7 @@ public class TransportBulkActionIngestTests extends ESTestCase {
|
||||||
new AutoCreateIndex(
|
new AutoCreateIndex(
|
||||||
SETTINGS, new ClusterSettings(SETTINGS, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS),
|
SETTINGS, new ClusterSettings(SETTINGS, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS),
|
||||||
new IndexNameExpressionResolver()
|
new IndexNameExpressionResolver()
|
||||||
)
|
), new WriteMemoryLimits()
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -27,7 +27,9 @@ import org.elasticsearch.action.bulk.TransportBulkActionTookTests.Resolver;
|
||||||
import org.elasticsearch.action.delete.DeleteRequest;
|
import org.elasticsearch.action.delete.DeleteRequest;
|
||||||
import org.elasticsearch.action.index.IndexRequest;
|
import org.elasticsearch.action.index.IndexRequest;
|
||||||
import org.elasticsearch.action.support.ActionFilters;
|
import org.elasticsearch.action.support.ActionFilters;
|
||||||
|
import org.elasticsearch.action.support.ActionTestUtils;
|
||||||
import org.elasticsearch.action.support.AutoCreateIndex;
|
import org.elasticsearch.action.support.AutoCreateIndex;
|
||||||
|
import org.elasticsearch.action.support.PlainActionFuture;
|
||||||
import org.elasticsearch.action.update.UpdateRequest;
|
import org.elasticsearch.action.update.UpdateRequest;
|
||||||
import org.elasticsearch.cluster.ClusterState;
|
import org.elasticsearch.cluster.ClusterState;
|
||||||
import org.elasticsearch.cluster.metadata.AliasMetadata;
|
import org.elasticsearch.cluster.metadata.AliasMetadata;
|
||||||
|
@ -78,7 +80,8 @@ public class TransportBulkActionTests extends ESTestCase {
|
||||||
TestTransportBulkAction() {
|
TestTransportBulkAction() {
|
||||||
super(TransportBulkActionTests.this.threadPool, transportService, clusterService, null, null,
|
super(TransportBulkActionTests.this.threadPool, transportService, clusterService, null, null,
|
||||||
null, new ActionFilters(Collections.emptySet()), new Resolver(),
|
null, new ActionFilters(Collections.emptySet()), new Resolver(),
|
||||||
new AutoCreateIndex(Settings.EMPTY, clusterService.getClusterSettings(), new Resolver()));
|
new AutoCreateIndex(Settings.EMPTY, clusterService.getClusterSettings(), new Resolver()),
|
||||||
|
new WriteMemoryLimits());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -120,38 +123,36 @@ public class TransportBulkActionTests extends ESTestCase {
|
||||||
public void testDeleteNonExistingDocDoesNotCreateIndex() throws Exception {
|
public void testDeleteNonExistingDocDoesNotCreateIndex() throws Exception {
|
||||||
BulkRequest bulkRequest = new BulkRequest().add(new DeleteRequest("index", "type", "id"));
|
BulkRequest bulkRequest = new BulkRequest().add(new DeleteRequest("index", "type", "id"));
|
||||||
|
|
||||||
bulkAction.execute(null, bulkRequest, ActionListener.wrap(response -> {
|
PlainActionFuture<BulkResponse> future = PlainActionFuture.newFuture();
|
||||||
assertFalse(bulkAction.indexCreated);
|
ActionTestUtils.execute(bulkAction, null, bulkRequest, future);
|
||||||
BulkItemResponse[] bulkResponses = ((BulkResponse) response).getItems();
|
|
||||||
assertEquals(bulkResponses.length, 1);
|
BulkResponse response = future.actionGet();
|
||||||
assertTrue(bulkResponses[0].isFailed());
|
assertFalse(bulkAction.indexCreated);
|
||||||
assertTrue(bulkResponses[0].getFailure().getCause() instanceof IndexNotFoundException);
|
BulkItemResponse[] bulkResponses = ((BulkResponse) response).getItems();
|
||||||
assertEquals("index", bulkResponses[0].getFailure().getIndex());
|
assertEquals(bulkResponses.length, 1);
|
||||||
}, exception -> {
|
assertTrue(bulkResponses[0].isFailed());
|
||||||
throw new AssertionError(exception);
|
assertTrue(bulkResponses[0].getFailure().getCause() instanceof IndexNotFoundException);
|
||||||
}));
|
assertEquals("index", bulkResponses[0].getFailure().getIndex());
|
||||||
}
|
}
|
||||||
|
|
||||||
public void testDeleteNonExistingDocExternalVersionCreatesIndex() throws Exception {
|
public void testDeleteNonExistingDocExternalVersionCreatesIndex() throws Exception {
|
||||||
BulkRequest bulkRequest = new BulkRequest()
|
BulkRequest bulkRequest = new BulkRequest()
|
||||||
.add(new DeleteRequest("index", "type", "id").versionType(VersionType.EXTERNAL).version(0));
|
.add(new DeleteRequest("index", "type", "id").versionType(VersionType.EXTERNAL).version(0));
|
||||||
|
|
||||||
bulkAction.execute(null, bulkRequest, ActionListener.wrap(response -> {
|
PlainActionFuture<BulkResponse> future = PlainActionFuture.newFuture();
|
||||||
assertTrue(bulkAction.indexCreated);
|
ActionTestUtils.execute(bulkAction, null, bulkRequest, future);
|
||||||
}, exception -> {
|
future.actionGet();
|
||||||
throw new AssertionError(exception);
|
assertTrue(bulkAction.indexCreated);
|
||||||
}));
|
|
||||||
}
|
}
|
||||||
|
|
||||||
public void testDeleteNonExistingDocExternalGteVersionCreatesIndex() throws Exception {
|
public void testDeleteNonExistingDocExternalGteVersionCreatesIndex() throws Exception {
|
||||||
BulkRequest bulkRequest = new BulkRequest()
|
BulkRequest bulkRequest = new BulkRequest()
|
||||||
.add(new DeleteRequest("index2", "type", "id").versionType(VersionType.EXTERNAL_GTE).version(0));
|
.add(new DeleteRequest("index2", "type", "id").versionType(VersionType.EXTERNAL_GTE).version(0));
|
||||||
|
|
||||||
bulkAction.execute(null, bulkRequest, ActionListener.wrap(response -> {
|
PlainActionFuture<BulkResponse> future = PlainActionFuture.newFuture();
|
||||||
assertTrue(bulkAction.indexCreated);
|
ActionTestUtils.execute(bulkAction, null, bulkRequest, future);
|
||||||
}, exception -> {
|
future.actionGet();
|
||||||
throw new AssertionError(exception);
|
assertTrue(bulkAction.indexCreated);
|
||||||
}));
|
|
||||||
}
|
}
|
||||||
|
|
||||||
public void testGetIndexWriteRequest() throws Exception {
|
public void testGetIndexWriteRequest() throws Exception {
|
||||||
|
|
|
@ -240,6 +240,7 @@ public class TransportBulkActionTookTests extends ESTestCase {
|
||||||
actionFilters,
|
actionFilters,
|
||||||
indexNameExpressionResolver,
|
indexNameExpressionResolver,
|
||||||
autoCreateIndex,
|
autoCreateIndex,
|
||||||
|
new WriteMemoryLimits(),
|
||||||
relativeTimeProvider);
|
relativeTimeProvider);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -20,6 +20,7 @@ package org.elasticsearch.action.resync;
|
||||||
|
|
||||||
import org.elasticsearch.Version;
|
import org.elasticsearch.Version;
|
||||||
import org.elasticsearch.action.ActionListener;
|
import org.elasticsearch.action.ActionListener;
|
||||||
|
import org.elasticsearch.action.bulk.WriteMemoryLimits;
|
||||||
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.cluster.ClusterState;
|
import org.elasticsearch.cluster.ClusterState;
|
||||||
|
@ -143,7 +144,8 @@ public class TransportResyncReplicationActionTests extends ESTestCase {
|
||||||
when(indexServices.indexServiceSafe(eq(index))).thenReturn(indexService);
|
when(indexServices.indexServiceSafe(eq(index))).thenReturn(indexService);
|
||||||
|
|
||||||
final TransportResyncReplicationAction action = new TransportResyncReplicationAction(Settings.EMPTY, transportService,
|
final TransportResyncReplicationAction action = new TransportResyncReplicationAction(Settings.EMPTY, transportService,
|
||||||
clusterService, indexServices, threadPool, shardStateAction, new ActionFilters(new HashSet<>()));
|
clusterService, indexServices, threadPool, shardStateAction, new ActionFilters(new HashSet<>()),
|
||||||
|
new WriteMemoryLimits());
|
||||||
|
|
||||||
assertThat(action.globalBlockLevel(), nullValue());
|
assertThat(action.globalBlockLevel(), nullValue());
|
||||||
assertThat(action.indexBlockLevel(), nullValue());
|
assertThat(action.indexBlockLevel(), nullValue());
|
||||||
|
|
|
@ -921,14 +921,17 @@ public class TransportReplicationActionTests extends ESTestCase {
|
||||||
final ReplicationTask task = maybeTask();
|
final ReplicationTask task = maybeTask();
|
||||||
TestAction action = new TestAction(Settings.EMPTY, "internal:testActionWithExceptions", transportService, clusterService,
|
TestAction action = new TestAction(Settings.EMPTY, "internal:testActionWithExceptions", transportService, clusterService,
|
||||||
shardStateAction, threadPool) {
|
shardStateAction, threadPool) {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected ReplicaResult shardOperationOnReplica(Request request, IndexShard replica) {
|
protected void shardOperationOnReplica(Request shardRequest, IndexShard replica, ActionListener<ReplicaResult> listener) {
|
||||||
assertIndexShardCounter(1);
|
ActionListener.completeWith(listener, () -> {
|
||||||
assertPhase(task, "replica");
|
assertIndexShardCounter(1);
|
||||||
if (throwException) {
|
assertPhase(task, "replica");
|
||||||
throw new ElasticsearchException("simulated");
|
if (throwException) {
|
||||||
}
|
throw new ElasticsearchException("simulated");
|
||||||
return new ReplicaResult();
|
}
|
||||||
|
return new ReplicaResult();
|
||||||
|
});
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
try {
|
try {
|
||||||
|
@ -1057,12 +1060,14 @@ public class TransportReplicationActionTests extends ESTestCase {
|
||||||
TestAction action = new TestAction(Settings.EMPTY, "internal:testActionWithExceptions", transportService, clusterService,
|
TestAction action = new TestAction(Settings.EMPTY, "internal:testActionWithExceptions", transportService, clusterService,
|
||||||
shardStateAction, threadPool) {
|
shardStateAction, threadPool) {
|
||||||
@Override
|
@Override
|
||||||
protected ReplicaResult shardOperationOnReplica(Request request, IndexShard replica) {
|
protected void shardOperationOnReplica(Request shardRequest, IndexShard replica, ActionListener<ReplicaResult> listener) {
|
||||||
assertPhase(task, "replica");
|
ActionListener.completeWith(listener, () -> {
|
||||||
if (throwException.get()) {
|
assertPhase(task, "replica");
|
||||||
throw new RetryOnReplicaException(shardId, "simulation");
|
if (throwException.get()) {
|
||||||
}
|
throw new RetryOnReplicaException(shardId, "simulation");
|
||||||
return new ReplicaResult();
|
}
|
||||||
|
return new ReplicaResult();
|
||||||
|
});
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
final PlainActionFuture<TransportResponse> listener = new PlainActionFuture<>();
|
final PlainActionFuture<TransportResponse> listener = new PlainActionFuture<>();
|
||||||
|
@ -1124,13 +1129,15 @@ public class TransportReplicationActionTests extends ESTestCase {
|
||||||
TestAction action = new TestAction(Settings.EMPTY, "internal:testActionWithExceptions", transportService, clusterService,
|
TestAction action = new TestAction(Settings.EMPTY, "internal:testActionWithExceptions", transportService, clusterService,
|
||||||
shardStateAction, threadPool) {
|
shardStateAction, threadPool) {
|
||||||
@Override
|
@Override
|
||||||
protected ReplicaResult shardOperationOnReplica(Request request, IndexShard replica) {
|
protected void shardOperationOnReplica(Request shardRequest, IndexShard replica, ActionListener<ReplicaResult> listener) {
|
||||||
assertPhase(task, "replica");
|
ActionListener.completeWith(listener, () -> {
|
||||||
if (throwException.get()) {
|
assertPhase(task, "replica");
|
||||||
throw new RetryOnReplicaException(shardId, "simulation");
|
if (throwException.get()) {
|
||||||
}
|
throw new RetryOnReplicaException(shardId, "simulation");
|
||||||
calledSuccessfully.set(true);
|
}
|
||||||
return new ReplicaResult();
|
calledSuccessfully.set(true);
|
||||||
|
return new ReplicaResult();
|
||||||
|
});
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
final PlainActionFuture<TransportResponse> listener = new PlainActionFuture<>();
|
final PlainActionFuture<TransportResponse> listener = new PlainActionFuture<>();
|
||||||
|
@ -1282,9 +1289,9 @@ public class TransportReplicationActionTests extends ESTestCase {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected ReplicaResult shardOperationOnReplica(Request request, IndexShard replica) {
|
protected void shardOperationOnReplica(Request request, IndexShard replica, ActionListener<ReplicaResult> listener) {
|
||||||
request.processedOnReplicas.incrementAndGet();
|
request.processedOnReplicas.incrementAndGet();
|
||||||
return new ReplicaResult();
|
listener.onResponse(new ReplicaResult());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -446,14 +446,14 @@ public class TransportReplicationAllPermitsAcquisitionTests extends IndexShardTe
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected ReplicaResult shardOperationOnReplica(Request shardRequest, IndexShard shard) throws Exception {
|
protected void shardOperationOnReplica(Request shardRequest, IndexShard replica, ActionListener<ReplicaResult> listener) {
|
||||||
assertEquals("Replica is always assigned to node 2 in this test", clusterService.state().nodes().get("_node2").getId(),
|
assertEquals("Replica is always assigned to node 2 in this test", clusterService.state().nodes().get("_node2").getId(),
|
||||||
shard.routingEntry().currentNodeId());
|
replica.routingEntry().currentNodeId());
|
||||||
executedOnReplica.set(true);
|
executedOnReplica.set(true);
|
||||||
// The TransportReplicationAction.getIndexShard() method is overridden for testing purpose but we double check here
|
// The TransportReplicationAction.getIndexShard() method is overridden for testing purpose but we double check here
|
||||||
// that the permit has been acquired on the replica shard
|
// that the permit has been acquired on the replica shard
|
||||||
assertSame(replica, shard);
|
assertSame(replica, replica);
|
||||||
return new ReplicaResult();
|
listener.onResponse(new ReplicaResult());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -505,10 +505,10 @@ public class TransportReplicationAllPermitsAcquisitionTests extends IndexShardTe
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected ReplicaResult shardOperationOnReplica(Request shardRequest, IndexShard shard) throws Exception {
|
protected void shardOperationOnReplica(Request shardRequest, IndexShard replica, ActionListener<ReplicaResult> listener) {
|
||||||
assertNoBlocks("block must not exist when executing the operation on replica shard: it should have been blocked before");
|
assertNoBlocks("block must not exist when executing the operation on replica shard: it should have been blocked before");
|
||||||
assertThat(shard.getActiveOperationsCount(), greaterThan(0));
|
assertThat(replica.getActiveOperationsCount(), greaterThan(0));
|
||||||
return super.shardOperationOnReplica(shardRequest, shard);
|
super.shardOperationOnReplica(shardRequest, replica, listener);
|
||||||
}
|
}
|
||||||
|
|
||||||
private void assertNoBlocks(final String error) {
|
private void assertNoBlocks(final String error) {
|
||||||
|
@ -551,9 +551,9 @@ public class TransportReplicationAllPermitsAcquisitionTests extends IndexShardTe
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected ReplicaResult shardOperationOnReplica(Request shardRequest, IndexShard shard) throws Exception {
|
protected void shardOperationOnReplica(Request shardRequest, IndexShard replica, ActionListener<ReplicaResult> listener) {
|
||||||
assertEquals("All permits must be acquired", IndexShard.OPERATIONS_BLOCKED, shard.getActiveOperationsCount());
|
assertEquals("All permits must be acquired", IndexShard.OPERATIONS_BLOCKED, replica.getActiveOperationsCount());
|
||||||
return super.shardOperationOnReplica(shardRequest, shard);
|
super.shardOperationOnReplica(shardRequest, replica, listener);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -21,6 +21,7 @@ package org.elasticsearch.action.support.replication;
|
||||||
|
|
||||||
import org.elasticsearch.ElasticsearchException;
|
import org.elasticsearch.ElasticsearchException;
|
||||||
import org.elasticsearch.action.ActionListener;
|
import org.elasticsearch.action.ActionListener;
|
||||||
|
import org.elasticsearch.action.bulk.WriteMemoryLimits;
|
||||||
import org.elasticsearch.action.support.ActionFilters;
|
import org.elasticsearch.action.support.ActionFilters;
|
||||||
import org.elasticsearch.action.support.ActionTestUtils;
|
import org.elasticsearch.action.support.ActionTestUtils;
|
||||||
import org.elasticsearch.action.support.PlainActionFuture;
|
import org.elasticsearch.action.support.PlainActionFuture;
|
||||||
|
@ -137,7 +138,7 @@ public class TransportWriteActionTests extends ESTestCase {
|
||||||
TestRequest request = new TestRequest();
|
TestRequest request = new TestRequest();
|
||||||
request.setRefreshPolicy(RefreshPolicy.NONE); // The default, but we'll set it anyway just to be explicit
|
request.setRefreshPolicy(RefreshPolicy.NONE); // The default, but we'll set it anyway just to be explicit
|
||||||
TestAction testAction = new TestAction();
|
TestAction testAction = new TestAction();
|
||||||
testAction.shardOperationOnPrimary(request, indexShard,
|
testAction.dispatchedShardOperationOnPrimary(request, indexShard,
|
||||||
ActionTestUtils.assertNoFailureListener(result -> {
|
ActionTestUtils.assertNoFailureListener(result -> {
|
||||||
CapturingActionListener<TestResponse> listener = new CapturingActionListener<>();
|
CapturingActionListener<TestResponse> listener = new CapturingActionListener<>();
|
||||||
result.runPostReplicationActions(ActionListener.map(listener, ignore -> result.finalResponseIfSuccessful));
|
result.runPostReplicationActions(ActionListener.map(listener, ignore -> result.finalResponseIfSuccessful));
|
||||||
|
@ -152,8 +153,9 @@ public class TransportWriteActionTests extends ESTestCase {
|
||||||
TestRequest request = new TestRequest();
|
TestRequest request = new TestRequest();
|
||||||
request.setRefreshPolicy(RefreshPolicy.NONE); // The default, but we'll set it anyway just to be explicit
|
request.setRefreshPolicy(RefreshPolicy.NONE); // The default, but we'll set it anyway just to be explicit
|
||||||
TestAction testAction = new TestAction();
|
TestAction testAction = new TestAction();
|
||||||
TransportWriteAction.WriteReplicaResult<TestRequest> result =
|
final PlainActionFuture<TransportReplicationAction.ReplicaResult> future = PlainActionFuture.newFuture();
|
||||||
testAction.shardOperationOnReplica(request, indexShard);
|
testAction.dispatchedShardOperationOnReplica(request, indexShard, future);
|
||||||
|
final TransportReplicationAction.ReplicaResult result = future.actionGet();
|
||||||
CapturingActionListener<TransportResponse.Empty> listener = new CapturingActionListener<>();
|
CapturingActionListener<TransportResponse.Empty> listener = new CapturingActionListener<>();
|
||||||
result.runPostReplicaActions(ActionListener.map(listener, ignore -> TransportResponse.Empty.INSTANCE));
|
result.runPostReplicaActions(ActionListener.map(listener, ignore -> TransportResponse.Empty.INSTANCE));
|
||||||
assertNotNull(listener.response);
|
assertNotNull(listener.response);
|
||||||
|
@ -166,7 +168,7 @@ public class TransportWriteActionTests extends ESTestCase {
|
||||||
TestRequest request = new TestRequest();
|
TestRequest request = new TestRequest();
|
||||||
request.setRefreshPolicy(RefreshPolicy.IMMEDIATE);
|
request.setRefreshPolicy(RefreshPolicy.IMMEDIATE);
|
||||||
TestAction testAction = new TestAction();
|
TestAction testAction = new TestAction();
|
||||||
testAction.shardOperationOnPrimary(request, indexShard,
|
testAction.dispatchedShardOperationOnPrimary(request, indexShard,
|
||||||
ActionTestUtils.assertNoFailureListener(result -> {
|
ActionTestUtils.assertNoFailureListener(result -> {
|
||||||
CapturingActionListener<TestResponse> listener = new CapturingActionListener<>();
|
CapturingActionListener<TestResponse> listener = new CapturingActionListener<>();
|
||||||
result.runPostReplicationActions(ActionListener.map(listener, ignore -> result.finalResponseIfSuccessful));
|
result.runPostReplicationActions(ActionListener.map(listener, ignore -> result.finalResponseIfSuccessful));
|
||||||
|
@ -182,8 +184,9 @@ public class TransportWriteActionTests extends ESTestCase {
|
||||||
TestRequest request = new TestRequest();
|
TestRequest request = new TestRequest();
|
||||||
request.setRefreshPolicy(RefreshPolicy.IMMEDIATE);
|
request.setRefreshPolicy(RefreshPolicy.IMMEDIATE);
|
||||||
TestAction testAction = new TestAction();
|
TestAction testAction = new TestAction();
|
||||||
TransportWriteAction.WriteReplicaResult<TestRequest> result =
|
final PlainActionFuture<TransportReplicationAction.ReplicaResult> future = PlainActionFuture.newFuture();
|
||||||
testAction.shardOperationOnReplica(request, indexShard);
|
testAction.dispatchedShardOperationOnReplica(request, indexShard, future);
|
||||||
|
final TransportReplicationAction.ReplicaResult result = future.actionGet();
|
||||||
CapturingActionListener<TransportResponse.Empty> listener = new CapturingActionListener<>();
|
CapturingActionListener<TransportResponse.Empty> listener = new CapturingActionListener<>();
|
||||||
result.runPostReplicaActions(ActionListener.map(listener, ignore -> TransportResponse.Empty.INSTANCE));
|
result.runPostReplicaActions(ActionListener.map(listener, ignore -> TransportResponse.Empty.INSTANCE));
|
||||||
assertNotNull(listener.response);
|
assertNotNull(listener.response);
|
||||||
|
@ -197,7 +200,7 @@ public class TransportWriteActionTests extends ESTestCase {
|
||||||
request.setRefreshPolicy(RefreshPolicy.WAIT_UNTIL);
|
request.setRefreshPolicy(RefreshPolicy.WAIT_UNTIL);
|
||||||
|
|
||||||
TestAction testAction = new TestAction();
|
TestAction testAction = new TestAction();
|
||||||
testAction.shardOperationOnPrimary(request, indexShard,
|
testAction.dispatchedShardOperationOnPrimary(request, indexShard,
|
||||||
ActionTestUtils.assertNoFailureListener(result -> {
|
ActionTestUtils.assertNoFailureListener(result -> {
|
||||||
CapturingActionListener<TestResponse> listener = new CapturingActionListener<>();
|
CapturingActionListener<TestResponse> listener = new CapturingActionListener<>();
|
||||||
result.runPostReplicationActions(ActionListener.map(listener, ignore -> result.finalResponseIfSuccessful));
|
result.runPostReplicationActions(ActionListener.map(listener, ignore -> result.finalResponseIfSuccessful));
|
||||||
|
@ -221,7 +224,9 @@ public class TransportWriteActionTests extends ESTestCase {
|
||||||
TestRequest request = new TestRequest();
|
TestRequest request = new TestRequest();
|
||||||
request.setRefreshPolicy(RefreshPolicy.WAIT_UNTIL);
|
request.setRefreshPolicy(RefreshPolicy.WAIT_UNTIL);
|
||||||
TestAction testAction = new TestAction();
|
TestAction testAction = new TestAction();
|
||||||
TransportWriteAction.WriteReplicaResult<TestRequest> result = testAction.shardOperationOnReplica(request, indexShard);
|
final PlainActionFuture<TransportReplicationAction.ReplicaResult> future = PlainActionFuture.newFuture();
|
||||||
|
testAction.dispatchedShardOperationOnReplica(request, indexShard, future);
|
||||||
|
final TransportReplicationAction.ReplicaResult result = future.actionGet();
|
||||||
CapturingActionListener<TransportResponse.Empty> listener = new CapturingActionListener<>();
|
CapturingActionListener<TransportResponse.Empty> listener = new CapturingActionListener<>();
|
||||||
result.runPostReplicaActions(ActionListener.map(listener, ignore -> TransportResponse.Empty.INSTANCE));
|
result.runPostReplicaActions(ActionListener.map(listener, ignore -> TransportResponse.Empty.INSTANCE));
|
||||||
assertNull(listener.response); // Haven't responded yet
|
assertNull(listener.response); // Haven't responded yet
|
||||||
|
@ -240,7 +245,7 @@ public class TransportWriteActionTests extends ESTestCase {
|
||||||
public void testDocumentFailureInShardOperationOnPrimary() throws Exception {
|
public void testDocumentFailureInShardOperationOnPrimary() throws Exception {
|
||||||
TestRequest request = new TestRequest();
|
TestRequest request = new TestRequest();
|
||||||
TestAction testAction = new TestAction(true, true);
|
TestAction testAction = new TestAction(true, true);
|
||||||
testAction.shardOperationOnPrimary(request, indexShard,
|
testAction.dispatchedShardOperationOnPrimary(request, indexShard,
|
||||||
ActionTestUtils.assertNoFailureListener(result -> {
|
ActionTestUtils.assertNoFailureListener(result -> {
|
||||||
CapturingActionListener<TestResponse> listener = new CapturingActionListener<>();
|
CapturingActionListener<TestResponse> listener = new CapturingActionListener<>();
|
||||||
result.runPostReplicationActions(ActionListener.map(listener, ignore -> result.finalResponseIfSuccessful));
|
result.runPostReplicationActions(ActionListener.map(listener, ignore -> result.finalResponseIfSuccessful));
|
||||||
|
@ -252,8 +257,9 @@ public class TransportWriteActionTests extends ESTestCase {
|
||||||
public void testDocumentFailureInShardOperationOnReplica() throws Exception {
|
public void testDocumentFailureInShardOperationOnReplica() throws Exception {
|
||||||
TestRequest request = new TestRequest();
|
TestRequest request = new TestRequest();
|
||||||
TestAction testAction = new TestAction(randomBoolean(), true);
|
TestAction testAction = new TestAction(randomBoolean(), true);
|
||||||
TransportWriteAction.WriteReplicaResult<TestRequest> result =
|
final PlainActionFuture<TransportReplicationAction.ReplicaResult> future = PlainActionFuture.newFuture();
|
||||||
testAction.shardOperationOnReplica(request, indexShard);
|
testAction.dispatchedShardOperationOnReplica(request, indexShard, future);
|
||||||
|
final TransportReplicationAction.ReplicaResult result = future.actionGet();
|
||||||
CapturingActionListener<TransportResponse.Empty> listener = new CapturingActionListener<>();
|
CapturingActionListener<TransportResponse.Empty> listener = new CapturingActionListener<>();
|
||||||
result.runPostReplicaActions(ActionListener.map(listener, ignore -> TransportResponse.Empty.INSTANCE));
|
result.runPostReplicaActions(ActionListener.map(listener, ignore -> TransportResponse.Empty.INSTANCE));
|
||||||
assertNull(listener.response);
|
assertNull(listener.response);
|
||||||
|
@ -360,7 +366,8 @@ public class TransportWriteActionTests extends ESTestCase {
|
||||||
super(Settings.EMPTY, "internal:test",
|
super(Settings.EMPTY, "internal:test",
|
||||||
new TransportService(Settings.EMPTY, mock(Transport.class), null, TransportService.NOOP_TRANSPORT_INTERCEPTOR,
|
new TransportService(Settings.EMPTY, mock(Transport.class), null, TransportService.NOOP_TRANSPORT_INTERCEPTOR,
|
||||||
x -> null, null, Collections.emptySet()), TransportWriteActionTests.this.clusterService, null, null, null,
|
x -> null, null, Collections.emptySet()), TransportWriteActionTests.this.clusterService, null, null, null,
|
||||||
new ActionFilters(new HashSet<>()), TestRequest::new, TestRequest::new, ThreadPool.Names.SAME, false);
|
new ActionFilters(new HashSet<>()), TestRequest::new, TestRequest::new, ThreadPool.Names.SAME, false,
|
||||||
|
new WriteMemoryLimits());
|
||||||
this.withDocumentFailureOnPrimary = withDocumentFailureOnPrimary;
|
this.withDocumentFailureOnPrimary = withDocumentFailureOnPrimary;
|
||||||
this.withDocumentFailureOnReplica = withDocumentFailureOnReplica;
|
this.withDocumentFailureOnReplica = withDocumentFailureOnReplica;
|
||||||
}
|
}
|
||||||
|
@ -369,7 +376,8 @@ public class TransportWriteActionTests extends ESTestCase {
|
||||||
ClusterService clusterService, ShardStateAction shardStateAction, ThreadPool threadPool) {
|
ClusterService clusterService, ShardStateAction shardStateAction, ThreadPool threadPool) {
|
||||||
super(settings, actionName, transportService, clusterService,
|
super(settings, actionName, transportService, clusterService,
|
||||||
mockIndicesService(clusterService), threadPool, shardStateAction,
|
mockIndicesService(clusterService), threadPool, shardStateAction,
|
||||||
new ActionFilters(new HashSet<>()), TestRequest::new, TestRequest::new, ThreadPool.Names.SAME, false);
|
new ActionFilters(new HashSet<>()), TestRequest::new, TestRequest::new, ThreadPool.Names.SAME, false,
|
||||||
|
new WriteMemoryLimits());
|
||||||
this.withDocumentFailureOnPrimary = false;
|
this.withDocumentFailureOnPrimary = false;
|
||||||
this.withDocumentFailureOnReplica = false;
|
this.withDocumentFailureOnReplica = false;
|
||||||
}
|
}
|
||||||
|
@ -381,7 +389,7 @@ public class TransportWriteActionTests extends ESTestCase {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected void shardOperationOnPrimary(
|
protected void dispatchedShardOperationOnPrimary(
|
||||||
TestRequest request, IndexShard primary, ActionListener<PrimaryResult<TestRequest, TestResponse>> listener) {
|
TestRequest request, IndexShard primary, ActionListener<PrimaryResult<TestRequest, TestResponse>> listener) {
|
||||||
ActionListener.completeWith(listener, () -> {
|
ActionListener.completeWith(listener, () -> {
|
||||||
if (withDocumentFailureOnPrimary) {
|
if (withDocumentFailureOnPrimary) {
|
||||||
|
@ -393,14 +401,16 @@ public class TransportWriteActionTests extends ESTestCase {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected WriteReplicaResult<TestRequest> shardOperationOnReplica(TestRequest request, IndexShard replica) throws Exception {
|
protected void dispatchedShardOperationOnReplica(TestRequest request, IndexShard replica, ActionListener<ReplicaResult> listener) {
|
||||||
final WriteReplicaResult<TestRequest> replicaResult;
|
ActionListener.completeWith(listener, () -> {
|
||||||
if (withDocumentFailureOnReplica) {
|
final WriteReplicaResult<TestRequest> replicaResult;
|
||||||
replicaResult = new WriteReplicaResult<>(request, null, new RuntimeException("simulated"), replica, logger);
|
if (withDocumentFailureOnReplica) {
|
||||||
} else {
|
replicaResult = new WriteReplicaResult<>(request, null, new RuntimeException("simulated"), replica, logger);
|
||||||
replicaResult = new WriteReplicaResult<>(request, location, null, replica, logger);
|
} else {
|
||||||
}
|
replicaResult = new WriteReplicaResult<>(request, location, null, replica, logger);
|
||||||
return replicaResult;
|
}
|
||||||
|
return replicaResult;
|
||||||
|
});
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -116,7 +116,8 @@ public class GlobalCheckpointSyncActionTests extends ESTestCase {
|
||||||
if (randomBoolean()) {
|
if (randomBoolean()) {
|
||||||
action.shardOperationOnPrimary(primaryRequest, indexShard, ActionTestUtils.assertNoFailureListener(r -> {}));
|
action.shardOperationOnPrimary(primaryRequest, indexShard, ActionTestUtils.assertNoFailureListener(r -> {}));
|
||||||
} else {
|
} else {
|
||||||
action.shardOperationOnReplica(new GlobalCheckpointSyncAction.Request(indexShard.shardId()), indexShard);
|
action.shardOperationOnReplica(new GlobalCheckpointSyncAction.Request(indexShard.shardId()), indexShard,
|
||||||
|
ActionTestUtils.assertNoFailureListener(r -> {}));
|
||||||
}
|
}
|
||||||
|
|
||||||
if (durability == Translog.Durability.ASYNC || lastSyncedGlobalCheckpoint == globalCheckpoint) {
|
if (durability == Translog.Durability.ASYNC || lastSyncedGlobalCheckpoint == globalCheckpoint) {
|
||||||
|
|
|
@ -23,6 +23,7 @@ import org.elasticsearch.action.ActionListener;
|
||||||
import org.elasticsearch.action.LatchedActionListener;
|
import org.elasticsearch.action.LatchedActionListener;
|
||||||
import org.elasticsearch.action.support.ActionFilters;
|
import org.elasticsearch.action.support.ActionFilters;
|
||||||
import org.elasticsearch.action.support.ActionTestUtils;
|
import org.elasticsearch.action.support.ActionTestUtils;
|
||||||
|
import org.elasticsearch.action.support.PlainActionFuture;
|
||||||
import org.elasticsearch.action.support.replication.TransportReplicationAction;
|
import org.elasticsearch.action.support.replication.TransportReplicationAction;
|
||||||
import org.elasticsearch.cluster.action.shard.ShardStateAction;
|
import org.elasticsearch.cluster.action.shard.ShardStateAction;
|
||||||
import org.elasticsearch.cluster.service.ClusterService;
|
import org.elasticsearch.cluster.service.ClusterService;
|
||||||
|
@ -149,7 +150,9 @@ public class RetentionLeaseBackgroundSyncActionTests extends ESTestCase {
|
||||||
final RetentionLeaseBackgroundSyncAction.Request request =
|
final RetentionLeaseBackgroundSyncAction.Request request =
|
||||||
new RetentionLeaseBackgroundSyncAction.Request(indexShard.shardId(), retentionLeases);
|
new RetentionLeaseBackgroundSyncAction.Request(indexShard.shardId(), retentionLeases);
|
||||||
|
|
||||||
final TransportReplicationAction.ReplicaResult result = action.shardOperationOnReplica(request, indexShard);
|
final PlainActionFuture<TransportReplicationAction.ReplicaResult> listener = PlainActionFuture.newFuture();
|
||||||
|
action.shardOperationOnReplica(request, indexShard, listener);
|
||||||
|
final TransportReplicationAction.ReplicaResult result = listener.actionGet();
|
||||||
// the retention leases on the shard should be updated
|
// the retention leases on the shard should be updated
|
||||||
verify(indexShard).updateRetentionLeasesOnReplica(retentionLeases);
|
verify(indexShard).updateRetentionLeasesOnReplica(retentionLeases);
|
||||||
// the retention leases on the shard should be persisted
|
// the retention leases on the shard should be persisted
|
||||||
|
|
|
@ -20,9 +20,11 @@
|
||||||
package org.elasticsearch.index.seqno;
|
package org.elasticsearch.index.seqno;
|
||||||
|
|
||||||
import org.elasticsearch.action.ActionListener;
|
import org.elasticsearch.action.ActionListener;
|
||||||
|
import org.elasticsearch.action.bulk.WriteMemoryLimits;
|
||||||
import org.elasticsearch.action.support.ActionFilters;
|
import org.elasticsearch.action.support.ActionFilters;
|
||||||
import org.elasticsearch.action.support.ActionTestUtils;
|
import org.elasticsearch.action.support.ActionTestUtils;
|
||||||
import org.elasticsearch.action.support.replication.TransportWriteAction;
|
import org.elasticsearch.action.support.PlainActionFuture;
|
||||||
|
import org.elasticsearch.action.support.replication.TransportReplicationAction;
|
||||||
import org.elasticsearch.cluster.action.shard.ShardStateAction;
|
import org.elasticsearch.cluster.action.shard.ShardStateAction;
|
||||||
import org.elasticsearch.cluster.service.ClusterService;
|
import org.elasticsearch.cluster.service.ClusterService;
|
||||||
import org.elasticsearch.common.settings.Settings;
|
import org.elasticsearch.common.settings.Settings;
|
||||||
|
@ -102,10 +104,11 @@ public class RetentionLeaseSyncActionTests extends ESTestCase {
|
||||||
indicesService,
|
indicesService,
|
||||||
threadPool,
|
threadPool,
|
||||||
shardStateAction,
|
shardStateAction,
|
||||||
new ActionFilters(Collections.emptySet()));
|
new ActionFilters(Collections.emptySet()),
|
||||||
|
new WriteMemoryLimits());
|
||||||
final RetentionLeases retentionLeases = mock(RetentionLeases.class);
|
final RetentionLeases retentionLeases = mock(RetentionLeases.class);
|
||||||
final RetentionLeaseSyncAction.Request request = new RetentionLeaseSyncAction.Request(indexShard.shardId(), retentionLeases);
|
final RetentionLeaseSyncAction.Request request = new RetentionLeaseSyncAction.Request(indexShard.shardId(), retentionLeases);
|
||||||
action.shardOperationOnPrimary(request, indexShard,
|
action.dispatchedShardOperationOnPrimary(request, indexShard,
|
||||||
ActionTestUtils.assertNoFailureListener(result -> {
|
ActionTestUtils.assertNoFailureListener(result -> {
|
||||||
// the retention leases on the shard should be persisted
|
// the retention leases on the shard should be persisted
|
||||||
verify(indexShard).persistRetentionLeases();
|
verify(indexShard).persistRetentionLeases();
|
||||||
|
@ -138,12 +141,14 @@ public class RetentionLeaseSyncActionTests extends ESTestCase {
|
||||||
indicesService,
|
indicesService,
|
||||||
threadPool,
|
threadPool,
|
||||||
shardStateAction,
|
shardStateAction,
|
||||||
new ActionFilters(Collections.emptySet()));
|
new ActionFilters(Collections.emptySet()),
|
||||||
|
new WriteMemoryLimits());
|
||||||
final RetentionLeases retentionLeases = mock(RetentionLeases.class);
|
final RetentionLeases retentionLeases = mock(RetentionLeases.class);
|
||||||
final RetentionLeaseSyncAction.Request request = new RetentionLeaseSyncAction.Request(indexShard.shardId(), retentionLeases);
|
final RetentionLeaseSyncAction.Request request = new RetentionLeaseSyncAction.Request(indexShard.shardId(), retentionLeases);
|
||||||
|
|
||||||
final TransportWriteAction.WriteReplicaResult<RetentionLeaseSyncAction.Request> result =
|
PlainActionFuture<TransportReplicationAction.ReplicaResult> listener = PlainActionFuture.newFuture();
|
||||||
action.shardOperationOnReplica(request, indexShard);
|
action.dispatchedShardOperationOnReplica(request, indexShard, listener);
|
||||||
|
final TransportReplicationAction.ReplicaResult result = listener.actionGet();
|
||||||
// the retention leases on the shard should be updated
|
// the retention leases on the shard should be updated
|
||||||
verify(indexShard).updateRetentionLeasesOnReplica(retentionLeases);
|
verify(indexShard).updateRetentionLeasesOnReplica(retentionLeases);
|
||||||
// the retention leases on the shard should be persisted
|
// the retention leases on the shard should be persisted
|
||||||
|
@ -176,7 +181,8 @@ public class RetentionLeaseSyncActionTests extends ESTestCase {
|
||||||
indicesService,
|
indicesService,
|
||||||
threadPool,
|
threadPool,
|
||||||
shardStateAction,
|
shardStateAction,
|
||||||
new ActionFilters(Collections.emptySet()));
|
new ActionFilters(Collections.emptySet()),
|
||||||
|
new WriteMemoryLimits());
|
||||||
|
|
||||||
assertNull(action.indexBlockLevel());
|
assertNull(action.indexBlockLevel());
|
||||||
}
|
}
|
||||||
|
|
|
@ -65,6 +65,7 @@ import org.elasticsearch.action.admin.indices.mapping.put.TransportPutMappingAct
|
||||||
import org.elasticsearch.action.admin.indices.shards.IndicesShardStoresAction;
|
import org.elasticsearch.action.admin.indices.shards.IndicesShardStoresAction;
|
||||||
import org.elasticsearch.action.admin.indices.shards.TransportIndicesShardStoresAction;
|
import org.elasticsearch.action.admin.indices.shards.TransportIndicesShardStoresAction;
|
||||||
import org.elasticsearch.action.bulk.BulkAction;
|
import org.elasticsearch.action.bulk.BulkAction;
|
||||||
|
import org.elasticsearch.action.bulk.WriteMemoryLimits;
|
||||||
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.bulk.TransportBulkAction;
|
import org.elasticsearch.action.bulk.TransportBulkAction;
|
||||||
|
@ -1484,7 +1485,8 @@ public class SnapshotResiliencyTests extends ESTestCase {
|
||||||
indicesService,
|
indicesService,
|
||||||
threadPool,
|
threadPool,
|
||||||
shardStateAction,
|
shardStateAction,
|
||||||
actionFilters)),
|
actionFilters,
|
||||||
|
new WriteMemoryLimits())),
|
||||||
new GlobalCheckpointSyncAction(
|
new GlobalCheckpointSyncAction(
|
||||||
settings,
|
settings,
|
||||||
transportService,
|
transportService,
|
||||||
|
@ -1510,7 +1512,7 @@ public class SnapshotResiliencyTests extends ESTestCase {
|
||||||
mappingUpdatedAction.setClient(client);
|
mappingUpdatedAction.setClient(client);
|
||||||
final TransportShardBulkAction transportShardBulkAction = new TransportShardBulkAction(settings, transportService,
|
final TransportShardBulkAction transportShardBulkAction = new TransportShardBulkAction(settings, transportService,
|
||||||
clusterService, indicesService, threadPool, shardStateAction, mappingUpdatedAction, new UpdateHelper(scriptService),
|
clusterService, indicesService, threadPool, shardStateAction, mappingUpdatedAction, new UpdateHelper(scriptService),
|
||||||
actionFilters);
|
actionFilters, new WriteMemoryLimits());
|
||||||
actions.put(BulkAction.INSTANCE,
|
actions.put(BulkAction.INSTANCE,
|
||||||
new TransportBulkAction(threadPool, transportService, clusterService,
|
new TransportBulkAction(threadPool, transportService, clusterService,
|
||||||
new IngestService(
|
new IngestService(
|
||||||
|
@ -1518,7 +1520,7 @@ public class SnapshotResiliencyTests extends ESTestCase {
|
||||||
new AnalysisModule(environment, Collections.emptyList()).getAnalysisRegistry(),
|
new AnalysisModule(environment, Collections.emptyList()).getAnalysisRegistry(),
|
||||||
Collections.emptyList(), client),
|
Collections.emptyList(), client),
|
||||||
transportShardBulkAction, client, actionFilters, indexNameExpressionResolver,
|
transportShardBulkAction, client, actionFilters, indexNameExpressionResolver,
|
||||||
new AutoCreateIndex(settings, clusterSettings, indexNameExpressionResolver)
|
new AutoCreateIndex(settings, clusterSettings, indexNameExpressionResolver), new WriteMemoryLimits()
|
||||||
));
|
));
|
||||||
final RestoreService restoreService = new RestoreService(
|
final RestoreService restoreService = new RestoreService(
|
||||||
clusterService, repositoriesService, allocationService,
|
clusterService, repositoriesService, allocationService,
|
||||||
|
|
|
@ -23,6 +23,7 @@ import org.elasticsearch.action.ActionListener;
|
||||||
import org.elasticsearch.action.ActionRequest;
|
import org.elasticsearch.action.ActionRequest;
|
||||||
import org.elasticsearch.action.ActionResponse;
|
import org.elasticsearch.action.ActionResponse;
|
||||||
import org.elasticsearch.common.CheckedConsumer;
|
import org.elasticsearch.common.CheckedConsumer;
|
||||||
|
import org.elasticsearch.tasks.Task;
|
||||||
|
|
||||||
import static org.elasticsearch.action.support.PlainActionFuture.newFuture;
|
import static org.elasticsearch.action.support.PlainActionFuture.newFuture;
|
||||||
|
|
||||||
|
@ -37,6 +38,16 @@ public class ActionTestUtils {
|
||||||
return future.actionGet();
|
return future.actionGet();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Executes the given action.
|
||||||
|
*
|
||||||
|
* This is a shim method to make execution publicly available in tests.
|
||||||
|
*/
|
||||||
|
public static <Request extends ActionRequest, Response extends ActionResponse>
|
||||||
|
void execute(TransportAction<Request, Response> action, Task task, Request request, ActionListener<Response> listener) {
|
||||||
|
action.execute(task, request, listener);
|
||||||
|
}
|
||||||
|
|
||||||
public static <T> ActionListener<T> assertNoFailureListener(CheckedConsumer<T, Exception> consumer) {
|
public static <T> ActionListener<T> assertNoFailureListener(CheckedConsumer<T, Exception> consumer) {
|
||||||
return ActionListener.wrap(consumer, e -> {
|
return ActionListener.wrap(consumer, e -> {
|
||||||
throw new AssertionError(e);
|
throw new AssertionError(e);
|
||||||
|
|
|
@ -9,6 +9,7 @@ package org.elasticsearch.xpack.ccr.action.bulk;
|
||||||
import org.apache.logging.log4j.Logger;
|
import org.apache.logging.log4j.Logger;
|
||||||
import org.elasticsearch.ExceptionsHelper;
|
import org.elasticsearch.ExceptionsHelper;
|
||||||
import org.elasticsearch.action.ActionListener;
|
import org.elasticsearch.action.ActionListener;
|
||||||
|
import org.elasticsearch.action.bulk.WriteMemoryLimits;
|
||||||
import org.elasticsearch.action.support.ActionFilters;
|
import org.elasticsearch.action.support.ActionFilters;
|
||||||
import org.elasticsearch.action.support.replication.TransportWriteAction;
|
import org.elasticsearch.action.support.replication.TransportWriteAction;
|
||||||
import org.elasticsearch.cluster.action.shard.ShardStateAction;
|
import org.elasticsearch.cluster.action.shard.ShardStateAction;
|
||||||
|
@ -43,7 +44,8 @@ public class TransportBulkShardOperationsAction
|
||||||
final IndicesService indicesService,
|
final IndicesService indicesService,
|
||||||
final ThreadPool threadPool,
|
final ThreadPool threadPool,
|
||||||
final ShardStateAction shardStateAction,
|
final ShardStateAction shardStateAction,
|
||||||
final ActionFilters actionFilters) {
|
final ActionFilters actionFilters,
|
||||||
|
final WriteMemoryLimits writeMemoryLimits) {
|
||||||
super(
|
super(
|
||||||
settings,
|
settings,
|
||||||
BulkShardOperationsAction.NAME,
|
BulkShardOperationsAction.NAME,
|
||||||
|
@ -55,11 +57,11 @@ public class TransportBulkShardOperationsAction
|
||||||
actionFilters,
|
actionFilters,
|
||||||
BulkShardOperationsRequest::new,
|
BulkShardOperationsRequest::new,
|
||||||
BulkShardOperationsRequest::new,
|
BulkShardOperationsRequest::new,
|
||||||
ThreadPool.Names.WRITE, false);
|
ThreadPool.Names.WRITE, false, writeMemoryLimits);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected void shardOperationOnPrimary(BulkShardOperationsRequest request, IndexShard primary,
|
protected void dispatchedShardOperationOnPrimary(BulkShardOperationsRequest request, IndexShard primary,
|
||||||
ActionListener<PrimaryResult<BulkShardOperationsRequest, BulkShardOperationsResponse>> listener) {
|
ActionListener<PrimaryResult<BulkShardOperationsRequest, BulkShardOperationsResponse>> listener) {
|
||||||
if (logger.isTraceEnabled()) {
|
if (logger.isTraceEnabled()) {
|
||||||
logger.trace("index [{}] on the following primary shard {}", request.getOperations(), primary.routingEntry());
|
logger.trace("index [{}] on the following primary shard {}", request.getOperations(), primary.routingEntry());
|
||||||
|
@ -68,6 +70,11 @@ public class TransportBulkShardOperationsAction
|
||||||
request.getOperations(), request.getMaxSeqNoOfUpdatesOrDeletes(), primary, logger));
|
request.getOperations(), request.getMaxSeqNoOfUpdatesOrDeletes(), primary, logger));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected long primaryOperationSize(BulkShardOperationsRequest request) {
|
||||||
|
return request.getOperations().stream().mapToLong(Translog.Operation::estimateSize).sum();
|
||||||
|
}
|
||||||
|
|
||||||
public static Translog.Operation rewriteOperationWithPrimaryTerm(Translog.Operation operation, long primaryTerm) {
|
public static Translog.Operation rewriteOperationWithPrimaryTerm(Translog.Operation operation, long primaryTerm) {
|
||||||
final Translog.Operation operationWithPrimaryTerm;
|
final Translog.Operation operationWithPrimaryTerm;
|
||||||
switch (operation.opType()) {
|
switch (operation.opType()) {
|
||||||
|
@ -160,12 +167,19 @@ public class TransportBulkShardOperationsAction
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected WriteReplicaResult<BulkShardOperationsRequest> shardOperationOnReplica(
|
protected void dispatchedShardOperationOnReplica(BulkShardOperationsRequest request, IndexShard replica,
|
||||||
final BulkShardOperationsRequest request, final IndexShard replica) throws Exception {
|
ActionListener<ReplicaResult> listener) {
|
||||||
if (logger.isTraceEnabled()) {
|
ActionListener.completeWith(listener, () -> {
|
||||||
logger.trace("index [{}] on the following replica shard {}", request.getOperations(), replica.routingEntry());
|
if (logger.isTraceEnabled()) {
|
||||||
}
|
logger.trace("index [{}] on the following replica shard {}", request.getOperations(), replica.routingEntry());
|
||||||
return shardOperationOnReplica(request, replica, logger);
|
}
|
||||||
|
return shardOperationOnReplica(request, replica, logger);
|
||||||
|
});
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected long replicaOperationSize(BulkShardOperationsRequest request) {
|
||||||
|
return request.getOperations().stream().mapToLong(Translog.Operation::estimateSize).sum();
|
||||||
}
|
}
|
||||||
|
|
||||||
// public for testing purposes only
|
// public for testing purposes only
|
||||||
|
|
Loading…
Reference in New Issue