From d7265e6b396779670383976b98f25a2987dcc5cd Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Mon, 13 Apr 2020 15:53:10 -0600 Subject: [PATCH 01/41] Add post indexing hooks --- .../action/support/replication/ReplicationOperation.java | 6 ++++++ .../support/replication/TransportReplicationAction.java | 7 +++---- .../close/TransportVerifyShardBeforeCloseActionTests.java | 3 +++ .../support/replication/ReplicationOperationTests.java | 5 +++++ .../index/replication/ESIndexLevelReplicationTestCase.java | 3 +++ 5 files changed, 20 insertions(+), 4 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/support/replication/ReplicationOperation.java b/server/src/main/java/org/elasticsearch/action/support/replication/ReplicationOperation.java index e24a3dbe381c3..472469a8908e0 100644 --- a/server/src/main/java/org/elasticsearch/action/support/replication/ReplicationOperation.java +++ b/server/src/main/java/org/elasticsearch/action/support/replication/ReplicationOperation.java @@ -303,6 +303,7 @@ private void finish() { failuresArray ) ); + primaryResult.primaryCoordinationComplete(); resultListener.onResponse(primaryResult); } } @@ -494,5 +495,10 @@ public interface PrimaryResult> { * @param listener calllback that is invoked after post replication actions have completed * */ void runPostReplicationActions(ActionListener listener); + + /** + * Called when all primary coordination and replication is complete + * */ + void primaryCoordinationComplete(); } } diff --git a/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java b/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java index ad0faff1926d0..5854e924ec46e 100644 --- a/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java +++ b/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java @@ -395,10 +395,6 @@ protected void adaptResponse(Response response, IndexShard indexShard) { } - protected ActionListener wrapResponseActionListener(ActionListener listener, IndexShard shard) { - return listener; - } - public static class PrimaryResult, Response extends ReplicationResponse> implements ReplicationOperation.PrimaryResult { @@ -443,6 +439,9 @@ public void runPostReplicationActions(ActionListener listener) { listener.onResponse(null); } } + + @Override + public void primaryCoordinationComplete() {} } public static class ReplicaResult { diff --git a/server/src/test/java/org/elasticsearch/action/admin/indices/close/TransportVerifyShardBeforeCloseActionTests.java b/server/src/test/java/org/elasticsearch/action/admin/indices/close/TransportVerifyShardBeforeCloseActionTests.java index 076e538300039..73f6d83ab1631 100644 --- a/server/src/test/java/org/elasticsearch/action/admin/indices/close/TransportVerifyShardBeforeCloseActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/admin/indices/close/TransportVerifyShardBeforeCloseActionTests.java @@ -352,6 +352,9 @@ public void runPostReplicationActions(ActionListener listener) { listener.onResponse(null); } + @Override + public void primaryCoordinationComplete() {} + public ReplicationResponse.ShardInfo getShardInfo() { return shardInfo.get(); } diff --git a/server/src/test/java/org/elasticsearch/action/support/replication/ReplicationOperationTests.java b/server/src/test/java/org/elasticsearch/action/support/replication/ReplicationOperationTests.java index 6830a8613f953..39dd82c69d196 100644 --- a/server/src/test/java/org/elasticsearch/action/support/replication/ReplicationOperationTests.java +++ b/server/src/test/java/org/elasticsearch/action/support/replication/ReplicationOperationTests.java @@ -515,6 +515,11 @@ public void runPostReplicationActions(ActionListener listener) { listener.onResponse(null); } + @Override + public void primaryCoordinationComplete() { + + } + public ShardInfo getShardInfo() { return shardInfo; } diff --git a/test/framework/src/main/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java b/test/framework/src/main/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java index 3ff568aa9bc56..75f0dace393d1 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java @@ -742,6 +742,9 @@ public void setShardInfo(ReplicationResponse.ShardInfo shardInfo) { public void runPostReplicationActions(ActionListener listener) { listener.onResponse(null); } + + @Override + public void primaryCoordinationComplete() {} } } From 0a782cd30cc2e32e54d8452323133672fa091d74 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Mon, 13 Apr 2020 16:42:00 -0600 Subject: [PATCH 02/41] Add post indexing hooks --- .../org/elasticsearch/action/bulk/BulkIndexingBreaker.java | 4 ++++ 1 file changed, 4 insertions(+) create mode 100644 server/src/main/java/org/elasticsearch/action/bulk/BulkIndexingBreaker.java diff --git a/server/src/main/java/org/elasticsearch/action/bulk/BulkIndexingBreaker.java b/server/src/main/java/org/elasticsearch/action/bulk/BulkIndexingBreaker.java new file mode 100644 index 0000000000000..1f1ca018e8840 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/action/bulk/BulkIndexingBreaker.java @@ -0,0 +1,4 @@ +package org.elasticsearch.action.bulk; + +public class BulkIndexingBreaker { +} From 84279ee0629822a5fa9cdf8b61040ce8ee2196a8 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Mon, 13 Apr 2020 16:58:20 -0600 Subject: [PATCH 03/41] Changes --- .../elasticsearch/action/DocWriteRequest.java | 18 ++++ .../action/bulk/BulkIndexingBreaker.java | 85 +++++++++++++++++++ .../action/bulk/TransportShardBulkAction.java | 25 ++++++ .../replication/TransportWriteAction.java | 13 +++ 4 files changed, 141 insertions(+) diff --git a/server/src/main/java/org/elasticsearch/action/DocWriteRequest.java b/server/src/main/java/org/elasticsearch/action/DocWriteRequest.java index da7eb9868e331..39905e5907708 100644 --- a/server/src/main/java/org/elasticsearch/action/DocWriteRequest.java +++ b/server/src/main/java/org/elasticsearch/action/DocWriteRequest.java @@ -29,6 +29,7 @@ import java.io.IOException; import java.util.Locale; +import java.util.stream.Stream; import static org.elasticsearch.action.ValidateActions.addValidationError; import static org.elasticsearch.index.seqno.SequenceNumbers.UNASSIGNED_PRIMARY_TERM; @@ -256,4 +257,21 @@ static ActionRequestValidationException validateSeqNoBasedCASParams( return validationException; } + + static long writeSizeInBytes(Stream> requestStream) { + return requestStream.mapToLong(request -> { + // TODO: Add constant sizes for DELETE, etc + if (request instanceof IndexRequest) { + if (((IndexRequest) request).source() != null) { + return ((IndexRequest) request).source().length(); + } + } else if (request instanceof UpdateRequest) { + IndexRequest doc = ((UpdateRequest) request).doc(); + if (doc != null && doc.source() != null) { + return ((UpdateRequest) request).doc().source().length(); + } + } + return 0; + }).sum(); + } } diff --git a/server/src/main/java/org/elasticsearch/action/bulk/BulkIndexingBreaker.java b/server/src/main/java/org/elasticsearch/action/bulk/BulkIndexingBreaker.java index 1f1ca018e8840..d9b2d412ca3f9 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/BulkIndexingBreaker.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/BulkIndexingBreaker.java @@ -1,4 +1,89 @@ +/* + * 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.util.concurrent.EsRejectedExecutionException; +import org.elasticsearch.monitor.jvm.JvmInfo; + +import java.util.concurrent.atomic.AtomicLong; + public class BulkIndexingBreaker { + + // TODO: Configurable? Allow dedicating coordinating nodes to have a higher value? + private static final long FORTY_PER_HEAP_SIZE = (long) (JvmInfo.jvmInfo().getMem().getHeapMax().getBytes() * 0.6); + private static final long THIRTY_PER_HEAP_SIZE = (long) (JvmInfo.jvmInfo().getMem().getHeapMax().getBytes() * 0.3); + + private final AtomicLong pendingBytes = new AtomicLong(0); + + public void markCoordinatingOperationStarted(long bytes) { + long pendingWithOperation = pendingBytes.addAndGet(bytes); + + if (pendingWithOperation > FORTY_PER_HEAP_SIZE) { + decrementPendingBytes(bytes); + long pendingPreOperation = pendingWithOperation - bytes; + throw new EsRejectedExecutionException("rejected execution of coordinating indexing operation [" + + "pending_bytes=" + pendingPreOperation + ", " + + "operation_bytes=" + bytes + "," + + "max_pending_bytes=" + FORTY_PER_HEAP_SIZE + "]", false); + } + } + + public void markCoordinatingOperationFinished(long bytes) { + decrementPendingBytes(bytes); + } + + public void markPrimaryOperationStarted(long bytes) { + long pendingWithOperation = pendingBytes.addAndGet(bytes); + + if (pendingWithOperation > THIRTY_PER_HEAP_SIZE) { + decrementPendingBytes(bytes); + long pendingPreOperation = pendingWithOperation - bytes; + throw new EsRejectedExecutionException("rejected execution of primary shard operation [" + + "pending_bytes=" + pendingPreOperation + ", " + + "operation_bytes=" + bytes + "," + + "max_pending_bytes=" + THIRTY_PER_HEAP_SIZE + "]", false); + } + } + + public void markPrimaryOperationFinished(long bytes) { + decrementPendingBytes(bytes); + } + + public void markReplicaOperationStarted(long bytes) { + long pendingWithOperation = pendingBytes.addAndGet(bytes); + + if (pendingWithOperation > FORTY_PER_HEAP_SIZE) { + decrementPendingBytes(bytes); + long pendingPreOperation = pendingWithOperation - bytes; + throw new EsRejectedExecutionException("rejected execution of replica shard operation [" + + "pending_bytes=" + pendingPreOperation + ", " + + "operation_bytes=" + bytes + "," + + "max_pending_bytes=" + FORTY_PER_HEAP_SIZE + "]", false); + } + } + + public void markReplicaOperationFinished(long bytes) { + decrementPendingBytes(bytes); + } + + private void decrementPendingBytes(long operationSizeInBytes) { + pendingBytes.getAndAdd(-operationSizeInBytes); + } } diff --git a/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java b/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java index 3eded2888e564..cb0ad3959d948 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java @@ -77,6 +77,7 @@ import java.util.concurrent.Executor; import java.util.function.Consumer; import java.util.function.LongSupplier; +import java.util.stream.Stream; /** Performs shard-level bulk (index, delete or update) operations */ public class TransportShardBulkAction extends TransportWriteAction { @@ -112,6 +113,15 @@ protected BulkShardResponse newResponseInstance(StreamInput in) throws IOExcepti @Override protected void shardOperationOnPrimary(BulkShardRequest request, IndexShard primary, ActionListener> listener) { + long operationSizeInBytes; + final String localNodeId = clusterService.localNode().getId(); + if (localNodeId.equals(request.getParentTask().getNodeId())) { + // If we are still on the coordinating node, we have already accounted for the bytes + operationSizeInBytes = 0; + } else { + operationSizeInBytes = operationSizeInBytes(request.items()); + } + ClusterStateObserver observer = new ClusterStateObserver(clusterService, request.timeout(), logger, threadPool.getThreadContext()); performOnPrimary(request, primary, updateHelper, threadPool::absoluteTimeInMillis, (update, shardId, mappingListener) -> { @@ -405,6 +415,15 @@ private static BulkItemResponse processUpdateResponse(final UpdateRequest update @Override public WriteReplicaResult shardOperationOnReplica(BulkShardRequest request, IndexShard replica) throws Exception { + long operationSizeInBytes; + final String localNodeId = clusterService.localNode().getId(); + if (localNodeId.equals(request.getParentTask().getNodeId())) { + // If we are still on the coordinating node, we have already accounted for the bytes + operationSizeInBytes = 0; + } else { + operationSizeInBytes = operationSizeInBytes(request.items()); + } + final Translog.Location location = performOnReplica(request, replica); return new WriteReplicaResult<>(request, location, null, replica, logger); } @@ -478,4 +497,10 @@ private static Engine.Result performOpOnReplica(DocWriteResponse primaryResponse } return result; } + + private static long operationSizeInBytes(BulkItemRequest[] items) { + return DocWriteRequest.writeSizeInBytes(Stream.of(items).map(BulkItemRequest::request)); + } + + } diff --git a/server/src/main/java/org/elasticsearch/action/support/replication/TransportWriteAction.java b/server/src/main/java/org/elasticsearch/action/support/replication/TransportWriteAction.java index 07f8e96b4e796..4fceb39462a10 100644 --- a/server/src/main/java/org/elasticsearch/action/support/replication/TransportWriteAction.java +++ b/server/src/main/java/org/elasticsearch/action/support/replication/TransportWriteAction.java @@ -127,14 +127,22 @@ public static class WritePrimaryResult {}); + } + + public WritePrimaryResult(ReplicaRequest request, @Nullable Response finalResponse, + @Nullable Location location, @Nullable Exception operationFailure, + IndexShard primary, Logger logger, Runnable postPrimaryCoordination) { super(request, finalResponse, operationFailure); this.location = location; this.primary = primary; this.logger = logger; + this.postPrimaryCoordination = postPrimaryCoordination; assert location == null || operationFailure == null : "expected either failure to be null or translog location to be null, " + "but found: [" + location + "] translog location and [" + operationFailure + "] failure"; @@ -163,6 +171,11 @@ public void onFailure(Exception ex) { }, logger).run(); } } + + @Override + public void primaryCoordinationComplete() { + postPrimaryCoordination.run(); + } } /** From c9ad59c3a518bce7e3f80d69c285fe2cb31cd213 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Thu, 16 Apr 2020 14:30:46 -0600 Subject: [PATCH 04/41] Changes --- .../elasticsearch/action/DocWriteRequest.java | 10 +++--- ...ker.java => BulkIndexingMemoryLimits.java} | 31 ++++++++++------- .../action/bulk/TransportBulkAction.java | 18 +++++++--- .../action/bulk/TransportShardBulkAction.java | 33 ++++++++++++++----- .../TransportReplicationAction.java | 20 ++++++++--- .../java/org/elasticsearch/node/Node.java | 3 ++ ...ActionIndicesThatCannotBeCreatedTests.java | 3 +- .../bulk/TransportBulkActionIngestTests.java | 2 +- .../action/bulk/TransportBulkActionTests.java | 3 +- .../bulk/TransportBulkActionTookTests.java | 12 ++++--- .../snapshots/SnapshotResiliencyTests.java | 6 ++-- 11 files changed, 99 insertions(+), 42 deletions(-) rename server/src/main/java/org/elasticsearch/action/bulk/{BulkIndexingBreaker.java => BulkIndexingMemoryLimits.java} (72%) diff --git a/server/src/main/java/org/elasticsearch/action/DocWriteRequest.java b/server/src/main/java/org/elasticsearch/action/DocWriteRequest.java index 39905e5907708..1a643908ec98c 100644 --- a/server/src/main/java/org/elasticsearch/action/DocWriteRequest.java +++ b/server/src/main/java/org/elasticsearch/action/DocWriteRequest.java @@ -258,20 +258,22 @@ static ActionRequestValidationException validateSeqNoBasedCASParams( return validationException; } + // A heuristic for the bytes overhead of a single indexing operation + int WRITE_REQUEST_BYTES_OVERHEAD = 256; + static long writeSizeInBytes(Stream> requestStream) { return requestStream.mapToLong(request -> { - // TODO: Add constant sizes for DELETE, etc if (request instanceof IndexRequest) { if (((IndexRequest) request).source() != null) { - return ((IndexRequest) request).source().length(); + return ((IndexRequest) request).source().length() + WRITE_REQUEST_BYTES_OVERHEAD; } } else if (request instanceof UpdateRequest) { IndexRequest doc = ((UpdateRequest) request).doc(); if (doc != null && doc.source() != null) { - return ((UpdateRequest) request).doc().source().length(); + return ((UpdateRequest) request).doc().source().length() + WRITE_REQUEST_BYTES_OVERHEAD; } } - return 0; + return WRITE_REQUEST_BYTES_OVERHEAD; }).sum(); } } diff --git a/server/src/main/java/org/elasticsearch/action/bulk/BulkIndexingBreaker.java b/server/src/main/java/org/elasticsearch/action/bulk/BulkIndexingMemoryLimits.java similarity index 72% rename from server/src/main/java/org/elasticsearch/action/bulk/BulkIndexingBreaker.java rename to server/src/main/java/org/elasticsearch/action/bulk/BulkIndexingMemoryLimits.java index d9b2d412ca3f9..cffe6c774bda6 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/BulkIndexingBreaker.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/BulkIndexingMemoryLimits.java @@ -19,29 +19,38 @@ package org.elasticsearch.action.bulk; +import org.elasticsearch.common.settings.Setting; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.util.concurrent.EsRejectedExecutionException; -import org.elasticsearch.monitor.jvm.JvmInfo; import java.util.concurrent.atomic.AtomicLong; -public class BulkIndexingBreaker { +public class BulkIndexingMemoryLimits { - // TODO: Configurable? Allow dedicating coordinating nodes to have a higher value? - private static final long FORTY_PER_HEAP_SIZE = (long) (JvmInfo.jvmInfo().getMem().getHeapMax().getBytes() * 0.6); - private static final long THIRTY_PER_HEAP_SIZE = (long) (JvmInfo.jvmInfo().getMem().getHeapMax().getBytes() * 0.3); + public static final Setting MAX_INDEXING_BYTES = + Setting.memorySizeSetting("indices.memory.queued_indexing_bytes.limit", "25%", Setting.Property.NodeScope); + + private final long primaryIndexingLimits; + private final long replicaIndexingLimits; private final AtomicLong pendingBytes = new AtomicLong(0); + public BulkIndexingMemoryLimits(Settings settings) { + this.primaryIndexingLimits = MAX_INDEXING_BYTES.get(settings).getBytes(); + this.replicaIndexingLimits = Math.round(primaryIndexingLimits * 2); + } + public void markCoordinatingOperationStarted(long bytes) { long pendingWithOperation = pendingBytes.addAndGet(bytes); - if (pendingWithOperation > FORTY_PER_HEAP_SIZE) { + if (pendingWithOperation > replicaIndexingLimits) { decrementPendingBytes(bytes); long pendingPreOperation = pendingWithOperation - bytes; throw new EsRejectedExecutionException("rejected execution of coordinating indexing operation [" + "pending_bytes=" + pendingPreOperation + ", " + "operation_bytes=" + bytes + "," + - "max_pending_bytes=" + FORTY_PER_HEAP_SIZE + "]", false); + "max_pending_bytes=" + primaryIndexingLimits + "]", false); } } @@ -52,13 +61,13 @@ public void markCoordinatingOperationFinished(long bytes) { public void markPrimaryOperationStarted(long bytes) { long pendingWithOperation = pendingBytes.addAndGet(bytes); - if (pendingWithOperation > THIRTY_PER_HEAP_SIZE) { + if (pendingWithOperation > replicaIndexingLimits) { decrementPendingBytes(bytes); long pendingPreOperation = pendingWithOperation - bytes; throw new EsRejectedExecutionException("rejected execution of primary shard operation [" + "pending_bytes=" + pendingPreOperation + ", " + "operation_bytes=" + bytes + "," + - "max_pending_bytes=" + THIRTY_PER_HEAP_SIZE + "]", false); + "max_pending_bytes=" + replicaIndexingLimits + "]", false); } } @@ -69,13 +78,13 @@ public void markPrimaryOperationFinished(long bytes) { public void markReplicaOperationStarted(long bytes) { long pendingWithOperation = pendingBytes.addAndGet(bytes); - if (pendingWithOperation > FORTY_PER_HEAP_SIZE) { + if (pendingWithOperation > primaryIndexingLimits) { decrementPendingBytes(bytes); long pendingPreOperation = pendingWithOperation - bytes; throw new EsRejectedExecutionException("rejected execution of replica shard operation [" + "pending_bytes=" + pendingPreOperation + ", " + "operation_bytes=" + bytes + "," + - "max_pending_bytes=" + FORTY_PER_HEAP_SIZE + "]", false); + "max_pending_bytes=" + primaryIndexingLimits + "]", false); } } diff --git a/server/src/main/java/org/elasticsearch/action/bulk/TransportBulkAction.java b/server/src/main/java/org/elasticsearch/action/bulk/TransportBulkAction.java index b6aefef24b60e..58a74aeb2043d 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/TransportBulkAction.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/TransportBulkAction.java @@ -106,21 +106,23 @@ public class TransportBulkAction extends HandledTransportAction docWriteRequest) { IndexRequest indexRequest = null; if (docWriteRequest instanceof IndexRequest) { indexRequest = (IndexRequest) docWriteRequest; @@ -153,7 +156,12 @@ public static IndexRequest getIndexWriteRequest(DocWriteRequest docWriteRequest) } @Override - protected void doExecute(Task task, BulkRequest bulkRequest, ActionListener listener) { + protected void doExecute(Task task, BulkRequest bulkRequest, ActionListener initialListener) { + final long indexingBytes = DocWriteRequest.writeSizeInBytes(bulkRequest.requests.stream()); + indexingMemoryLimits.markCoordinatingOperationStarted(indexingBytes); + final Runnable release = () -> indexingMemoryLimits.markCoordinatingOperationFinished(indexingBytes); + final ActionListener listener = ActionListener.runAfter(initialListener, release); + final long startTime = relativeTime(); final AtomicArray responses = new AtomicArray<>(bulkRequest.requests.size()); diff --git a/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java b/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java index cb0ad3959d948..fa19f0791ee37 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java @@ -51,8 +51,10 @@ import org.elasticsearch.common.compress.CompressedXContent; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.lease.Releasable; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.common.util.concurrent.EsRejectedExecutionException; import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.common.xcontent.XContentType; @@ -89,15 +91,18 @@ public class TransportShardBulkAction extends TransportWriteAction> listener) { - long operationSizeInBytes; + protected Releasable checkPrimaryLimits(BulkShardRequest request) throws EsRejectedExecutionException { + super.checkPrimaryLimits(request); final String localNodeId = clusterService.localNode().getId(); + long operationSizeInBytes; if (localNodeId.equals(request.getParentTask().getNodeId())) { - // If we are still on the coordinating node, we have already accounted for the bytes + // If we are on the TransportBulkAction coordinating node, we have already accounted for the bytes operationSizeInBytes = 0; } else { operationSizeInBytes = operationSizeInBytes(request.items()); } + indexingMemoryLimits.markPrimaryOperationStarted(operationSizeInBytes); + return () -> indexingMemoryLimits.markPrimaryOperationFinished(operationSizeInBytes); + } + @Override + protected void shardOperationOnPrimary(BulkShardRequest request, IndexShard primary, + ActionListener> listener) { ClusterStateObserver observer = new ClusterStateObserver(clusterService, request.timeout(), logger, threadPool.getThreadContext()); performOnPrimary(request, primary, updateHelper, threadPool::absoluteTimeInMillis, (update, shardId, mappingListener) -> { @@ -414,16 +425,22 @@ private static BulkItemResponse processUpdateResponse(final UpdateRequest update } @Override - public WriteReplicaResult shardOperationOnReplica(BulkShardRequest request, IndexShard replica) throws Exception { - long operationSizeInBytes; + protected Releasable checkReplicaLimits(BulkShardRequest request) throws EsRejectedExecutionException { + super.checkReplicaLimits(request); final String localNodeId = clusterService.localNode().getId(); + long operationSizeInBytes; if (localNodeId.equals(request.getParentTask().getNodeId())) { - // If we are still on the coordinating node, we have already accounted for the bytes + // If we are on the TransportBulkAction coordinating node, we have already accounted for the bytes operationSizeInBytes = 0; } else { operationSizeInBytes = operationSizeInBytes(request.items()); } + indexingMemoryLimits.markReplicaOperationStarted(operationSizeInBytes); + return () -> indexingMemoryLimits.markReplicaOperationFinished(operationSizeInBytes); + } + @Override + public WriteReplicaResult shardOperationOnReplica(BulkShardRequest request, IndexShard replica) throws Exception { final Translog.Location location = performOnReplica(request, replica); return new WriteReplicaResult<>(request, location, null, replica, logger); } diff --git a/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java b/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java index 5854e924ec46e..f9e1b5fb2d62f 100644 --- a/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java +++ b/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java @@ -251,8 +251,13 @@ protected void handleOperationRequest(final Request request, final TransportChan } protected void handlePrimaryRequest(final ConcreteShardRequest request, final TransportChannel channel, final Task task) { - new AsyncPrimaryAction( - request, new ChannelActionListener<>(channel, transportPrimaryAction, request), (ReplicationTask) task).run(); + final Releasable releasable = checkPrimaryLimits(request.request); + final ActionListener channelListener = new ChannelActionListener<>(channel, transportPrimaryAction, request); + new AsyncPrimaryAction(request, ActionListener.runAfter(channelListener, releasable::close), (ReplicationTask) task).run(); + } + + protected Releasable checkPrimaryLimits(final Request request) { + return () -> {}; } class AsyncPrimaryAction extends AbstractRunnable { @@ -464,10 +469,15 @@ public void runPostReplicaActions(ActionListener listener) { } } - protected void handleReplicaRequest(final ConcreteReplicaRequest replicaRequest, + protected void handleReplicaRequest(final ConcreteReplicaRequest request, final TransportChannel channel, final Task task) { - new AsyncReplicaAction( - replicaRequest, new ChannelActionListener<>(channel, transportReplicaAction, replicaRequest), (ReplicationTask) task).run(); + final Releasable releasable = checkReplicaLimits(request.getRequest()); + final ActionListener channelListener = new ChannelActionListener<>(channel, transportReplicaAction, request); + new AsyncReplicaAction(request, ActionListener.runAfter(channelListener, releasable::close), (ReplicationTask) task).run(); + } + + protected Releasable checkReplicaLimits(final ReplicaRequest request) { + return () -> {}; } public static class RetryOnReplicaException extends ElasticsearchException { diff --git a/server/src/main/java/org/elasticsearch/node/Node.java b/server/src/main/java/org/elasticsearch/node/Node.java index 09eac49137296..c91c407c932cd 100644 --- a/server/src/main/java/org/elasticsearch/node/Node.java +++ b/server/src/main/java/org/elasticsearch/node/Node.java @@ -30,6 +30,7 @@ import org.elasticsearch.Version; import org.elasticsearch.action.ActionModule; import org.elasticsearch.action.ActionType; +import org.elasticsearch.action.bulk.BulkIndexingMemoryLimits; import org.elasticsearch.action.search.SearchExecutionStatsCollector; import org.elasticsearch.action.search.SearchPhaseController; import org.elasticsearch.action.search.SearchTransportService; @@ -547,6 +548,7 @@ protected Node(final Environment initialEnvironment, new PersistentTasksClusterService(settings, registry, clusterService, threadPool); resourcesToClose.add(persistentTasksClusterService); final PersistentTasksService persistentTasksService = new PersistentTasksService(clusterService, threadPool, client); + final BulkIndexingMemoryLimits bulkIndexingLimits = new BulkIndexingMemoryLimits(settings); modules.add(b -> { b.bind(Node.class).toInstance(this); @@ -565,6 +567,7 @@ protected Node(final Environment initialEnvironment, b.bind(ScriptService.class).toInstance(scriptService); b.bind(AnalysisRegistry.class).toInstance(analysisModule.getAnalysisRegistry()); b.bind(IngestService.class).toInstance(ingestService); + b.bind(BulkIndexingMemoryLimits.class).toInstance(bulkIndexingLimits); b.bind(UsageService.class).toInstance(usageService); b.bind(NamedWriteableRegistry.class).toInstance(namedWriteableRegistry); b.bind(MetadataUpgrader.class).toInstance(metadataUpgrader); diff --git a/server/src/test/java/org/elasticsearch/action/bulk/TransportBulkActionIndicesThatCannotBeCreatedTests.java b/server/src/test/java/org/elasticsearch/action/bulk/TransportBulkActionIndicesThatCannotBeCreatedTests.java index e287704698532..85807d8be9769 100644 --- a/server/src/test/java/org/elasticsearch/action/bulk/TransportBulkActionIndicesThatCannotBeCreatedTests.java +++ b/server/src/test/java/org/elasticsearch/action/bulk/TransportBulkActionIndicesThatCannotBeCreatedTests.java @@ -31,6 +31,7 @@ import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.concurrent.AtomicArray; import org.elasticsearch.common.util.concurrent.EsExecutors; @@ -119,7 +120,7 @@ private void indicesThatCannotBeCreatedTestCase(Set expected, final ExecutorService direct = EsExecutors.newDirectExecutorService(); when(threadPool.executor(anyString())).thenReturn(direct); TransportBulkAction action = new TransportBulkAction(threadPool, mock(TransportService.class), clusterService, - null, null, mock(ActionFilters.class), null, null) { + null, null, mock(ActionFilters.class), null, null, new BulkIndexingMemoryLimits(Settings.EMPTY)) { @Override void executeBulk(Task task, BulkRequest bulkRequest, long startTimeNanos, ActionListener listener, AtomicArray responses, Map indicesThatCannotBeCreated) { diff --git a/server/src/test/java/org/elasticsearch/action/bulk/TransportBulkActionIngestTests.java b/server/src/test/java/org/elasticsearch/action/bulk/TransportBulkActionIngestTests.java index 154e0c1af26bc..ffb659830c013 100644 --- a/server/src/test/java/org/elasticsearch/action/bulk/TransportBulkActionIngestTests.java +++ b/server/src/test/java/org/elasticsearch/action/bulk/TransportBulkActionIngestTests.java @@ -142,7 +142,7 @@ null, new ActionFilters(Collections.emptySet()), null, new AutoCreateIndex( SETTINGS, new ClusterSettings(SETTINGS, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS), new IndexNameExpressionResolver() - ) + ), new BulkIndexingMemoryLimits(Settings.EMPTY) ); } @Override diff --git a/server/src/test/java/org/elasticsearch/action/bulk/TransportBulkActionTests.java b/server/src/test/java/org/elasticsearch/action/bulk/TransportBulkActionTests.java index 3f3404a9270ae..8c0d0c1ea7464 100644 --- a/server/src/test/java/org/elasticsearch/action/bulk/TransportBulkActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/bulk/TransportBulkActionTests.java @@ -72,7 +72,8 @@ class TestTransportBulkAction extends TransportBulkAction { TestTransportBulkAction() { super(TransportBulkActionTests.this.threadPool, transportService, clusterService, null, null, new ActionFilters(Collections.emptySet()), new Resolver(), - new AutoCreateIndex(Settings.EMPTY, clusterService.getClusterSettings(), new Resolver())); + new AutoCreateIndex(Settings.EMPTY, clusterService.getClusterSettings(), new Resolver()), + new BulkIndexingMemoryLimits(Settings.EMPTY)); } @Override diff --git a/server/src/test/java/org/elasticsearch/action/bulk/TransportBulkActionTookTests.java b/server/src/test/java/org/elasticsearch/action/bulk/TransportBulkActionTookTests.java index 2b2c22cd9be5c..7ac5952e0e719 100644 --- a/server/src/test/java/org/elasticsearch/action/bulk/TransportBulkActionTookTests.java +++ b/server/src/test/java/org/elasticsearch/action/bulk/TransportBulkActionTookTests.java @@ -118,7 +118,8 @@ void doExecute(ActionType action, Request request, ActionListener Date: Thu, 16 Apr 2020 18:30:48 -0600 Subject: [PATCH 05/41] Add tests --- .../action/bulk/BulkIndexingMemoryLimits.java | 34 +--- .../action/bulk/TransportBulkAction.java | 16 +- .../action/bulk/TransportShardBulkAction.java | 18 +-- .../common/settings/ClusterSettings.java | 2 + .../bulk/BulkIndexingMemoryLimitsIT.java | 151 ++++++++++++++++++ ...ActionIndicesThatCannotBeCreatedTests.java | 3 +- .../bulk/TransportBulkActionIngestTests.java | 2 +- .../action/bulk/TransportBulkActionTests.java | 3 +- .../bulk/TransportBulkActionTookTests.java | 12 +- .../snapshots/SnapshotResiliencyTests.java | 2 +- 10 files changed, 170 insertions(+), 73 deletions(-) create mode 100644 server/src/test/java/org/elasticsearch/action/bulk/BulkIndexingMemoryLimitsIT.java diff --git a/server/src/main/java/org/elasticsearch/action/bulk/BulkIndexingMemoryLimits.java b/server/src/main/java/org/elasticsearch/action/bulk/BulkIndexingMemoryLimits.java index cffe6c774bda6..428009a203cb5 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/BulkIndexingMemoryLimits.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/BulkIndexingMemoryLimits.java @@ -32,42 +32,23 @@ public class BulkIndexingMemoryLimits { Setting.memorySizeSetting("indices.memory.queued_indexing_bytes.limit", "25%", Setting.Property.NodeScope); private final long primaryIndexingLimits; - private final long replicaIndexingLimits; private final AtomicLong pendingBytes = new AtomicLong(0); public BulkIndexingMemoryLimits(Settings settings) { this.primaryIndexingLimits = MAX_INDEXING_BYTES.get(settings).getBytes(); - this.replicaIndexingLimits = Math.round(primaryIndexingLimits * 2); - } - - public void markCoordinatingOperationStarted(long bytes) { - long pendingWithOperation = pendingBytes.addAndGet(bytes); - - if (pendingWithOperation > replicaIndexingLimits) { - decrementPendingBytes(bytes); - long pendingPreOperation = pendingWithOperation - bytes; - throw new EsRejectedExecutionException("rejected execution of coordinating indexing operation [" + - "pending_bytes=" + pendingPreOperation + ", " + - "operation_bytes=" + bytes + "," + - "max_pending_bytes=" + primaryIndexingLimits + "]", false); - } - } - - public void markCoordinatingOperationFinished(long bytes) { - decrementPendingBytes(bytes); } public void markPrimaryOperationStarted(long bytes) { long pendingWithOperation = pendingBytes.addAndGet(bytes); - if (pendingWithOperation > replicaIndexingLimits) { + if (pendingWithOperation > primaryIndexingLimits) { decrementPendingBytes(bytes); long pendingPreOperation = pendingWithOperation - bytes; throw new EsRejectedExecutionException("rejected execution of primary shard operation [" + "pending_bytes=" + pendingPreOperation + ", " + "operation_bytes=" + bytes + "," + - "max_pending_bytes=" + replicaIndexingLimits + "]", false); + "max_pending_bytes=" + primaryIndexingLimits + "]", false); } } @@ -76,16 +57,7 @@ public void markPrimaryOperationFinished(long bytes) { } public void markReplicaOperationStarted(long bytes) { - long pendingWithOperation = pendingBytes.addAndGet(bytes); - - if (pendingWithOperation > primaryIndexingLimits) { - decrementPendingBytes(bytes); - long pendingPreOperation = pendingWithOperation - bytes; - throw new EsRejectedExecutionException("rejected execution of replica shard operation [" + - "pending_bytes=" + pendingPreOperation + ", " + - "operation_bytes=" + bytes + "," + - "max_pending_bytes=" + primaryIndexingLimits + "]", false); - } + pendingBytes.getAndAdd(bytes); } public void markReplicaOperationFinished(long bytes) { diff --git a/server/src/main/java/org/elasticsearch/action/bulk/TransportBulkAction.java b/server/src/main/java/org/elasticsearch/action/bulk/TransportBulkAction.java index 58a74aeb2043d..b671be92c774d 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/TransportBulkAction.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/TransportBulkAction.java @@ -106,23 +106,21 @@ public class TransportBulkAction extends HandledTransportAction docWriteReque } @Override - protected void doExecute(Task task, BulkRequest bulkRequest, ActionListener initialListener) { - final long indexingBytes = DocWriteRequest.writeSizeInBytes(bulkRequest.requests.stream()); - indexingMemoryLimits.markCoordinatingOperationStarted(indexingBytes); - final Runnable release = () -> indexingMemoryLimits.markCoordinatingOperationFinished(indexingBytes); - final ActionListener listener = ActionListener.runAfter(initialListener, release); - + protected void doExecute(Task task, BulkRequest bulkRequest, ActionListener listener) { final long startTime = relativeTime(); final AtomicArray responses = new AtomicArray<>(bulkRequest.requests.size()); diff --git a/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java b/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java index fa19f0791ee37..002443e315cf7 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java @@ -118,14 +118,7 @@ protected BulkShardResponse newResponseInstance(StreamInput in) throws IOExcepti @Override protected Releasable checkPrimaryLimits(BulkShardRequest request) throws EsRejectedExecutionException { super.checkPrimaryLimits(request); - final String localNodeId = clusterService.localNode().getId(); - long operationSizeInBytes; - if (localNodeId.equals(request.getParentTask().getNodeId())) { - // If we are on the TransportBulkAction coordinating node, we have already accounted for the bytes - operationSizeInBytes = 0; - } else { - operationSizeInBytes = operationSizeInBytes(request.items()); - } + long operationSizeInBytes = operationSizeInBytes(request.items()); indexingMemoryLimits.markPrimaryOperationStarted(operationSizeInBytes); return () -> indexingMemoryLimits.markPrimaryOperationFinished(operationSizeInBytes); } @@ -427,14 +420,7 @@ private static BulkItemResponse processUpdateResponse(final UpdateRequest update @Override protected Releasable checkReplicaLimits(BulkShardRequest request) throws EsRejectedExecutionException { super.checkReplicaLimits(request); - final String localNodeId = clusterService.localNode().getId(); - long operationSizeInBytes; - if (localNodeId.equals(request.getParentTask().getNodeId())) { - // If we are on the TransportBulkAction coordinating node, we have already accounted for the bytes - operationSizeInBytes = 0; - } else { - operationSizeInBytes = operationSizeInBytes(request.items()); - } + long operationSizeInBytes = operationSizeInBytes(request.items()); indexingMemoryLimits.markReplicaOperationStarted(operationSizeInBytes); return () -> indexingMemoryLimits.markReplicaOperationFinished(operationSizeInBytes); } diff --git a/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java b/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java index f31b311048e61..a7ec9d1c3fc37 100644 --- a/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java +++ b/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java @@ -21,6 +21,7 @@ import org.apache.logging.log4j.LogManager; import org.elasticsearch.action.admin.cluster.configuration.TransportAddVotingConfigExclusionsAction; import org.elasticsearch.action.admin.indices.close.TransportCloseIndexAction; +import org.elasticsearch.action.bulk.BulkIndexingMemoryLimits; import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.action.support.AutoCreateIndex; import org.elasticsearch.action.support.DestructiveOperations; @@ -186,6 +187,7 @@ public void apply(Settings value, Settings current, Settings previous) { BalancedShardsAllocator.INDEX_BALANCE_FACTOR_SETTING, BalancedShardsAllocator.SHARD_BALANCE_FACTOR_SETTING, BalancedShardsAllocator.THRESHOLD_SETTING, + BulkIndexingMemoryLimits.MAX_INDEXING_BYTES, ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING, ConcurrentRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_CLUSTER_CONCURRENT_REBALANCE_SETTING, DanglingIndicesState.AUTO_IMPORT_DANGLING_INDICES_SETTING, diff --git a/server/src/test/java/org/elasticsearch/action/bulk/BulkIndexingMemoryLimitsIT.java b/server/src/test/java/org/elasticsearch/action/bulk/BulkIndexingMemoryLimitsIT.java new file mode 100644 index 0000000000000..c990da232c578 --- /dev/null +++ b/server/src/test/java/org/elasticsearch/action/bulk/BulkIndexingMemoryLimitsIT.java @@ -0,0 +1,151 @@ +/* + * 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.DocWriteRequest; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.plugins.Plugin; +import org.elasticsearch.rest.RestStatus; +import org.elasticsearch.test.ESIntegTestCase; +import org.elasticsearch.test.InternalSettingsPlugin; +import org.elasticsearch.test.transport.MockTransportService; +import org.elasticsearch.transport.TransportService; + +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.concurrent.CountDownLatch; + +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; +import static org.hamcrest.Matchers.greaterThan; +import static org.hamcrest.Matchers.lessThan; + +@ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.SUITE, numDataNodes = 2) +public class BulkIndexingMemoryLimitsIT 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) + .put("indices.memory.queued_indexing_bytes.limit", "1KB") + .build(); + } + + @Override + protected Collection> nodePlugins() { + return Arrays.asList(MockTransportService.TestPlugin.class, InternalSettingsPlugin.class); + } + + @Override + protected int numberOfReplicas() { + return 1; + } + + @Override + protected int numberOfShards() { + return 1; + } + + public void testRejectionDueToRequestOverMemoryLimit() { + final String index = "test"; + assertAcked(prepareCreate(index)); + ensureGreen(index); + final BulkRequest lessThan1KB = new BulkRequest(); + for (int i = 0; i < 3; ++i) { + lessThan1KB.add(new IndexRequest(index).source(Collections.singletonMap("key", "value" + i))); + } + + final BulkRequest moreThan1KB = new BulkRequest(); + for (int i = 0; i < 4; ++i) { + moreThan1KB.add(new IndexRequest(index).source(Collections.singletonMap("key", "value" + i))); + } + assertThat(DocWriteRequest.writeSizeInBytes(lessThan1KB.requests.stream()), lessThan(1024L)); + assertFalse(client().bulk(lessThan1KB).actionGet().hasFailures()); + + assertThat(DocWriteRequest.writeSizeInBytes(moreThan1KB.requests.stream()), greaterThan(1024L)); + final ActionFuture bulkFuture2 = client().bulk(moreThan1KB); + final BulkResponse failedResponses = bulkFuture2.actionGet(); + for (BulkItemResponse response : failedResponses) { + assertEquals(RestStatus.TOO_MANY_REQUESTS, response.getFailure().getStatus()); + } + } + + public void testRejectionDueToConcurrentRequestsOverMemoryLimit() throws InterruptedException { + final String index = "test"; + assertAcked(prepareCreate(index)); + ensureGreen(); + + final CountDownLatch replicationStarted = new CountDownLatch(1); + final CountDownLatch replicationBlock = new CountDownLatch(1); + + for (TransportService transportService : internalCluster().getInstances(TransportService.class)) { + final MockTransportService mockTransportService = (MockTransportService) transportService; + mockTransportService.addSendBehavior((connection, requestId, action, request, options) -> { + if (action.equals(TransportShardBulkAction.ACTION_NAME + "[r]")) { + try { + replicationStarted.countDown(); + replicationBlock.await(); + } catch (InterruptedException e) { + throw new IllegalStateException(e); + } + } + connection.sendRequest(requestId, action, request, options); + }); + } + + final BulkRequest lessThan1KB = new BulkRequest(); + for (int i = 0; i < 3; ++i) { + lessThan1KB.add(new IndexRequest(index).source(Collections.singletonMap("key", "value" + i))); + } + assertThat(DocWriteRequest.writeSizeInBytes(lessThan1KB.requests.stream()), lessThan(1024L)); + + final BulkRequest rejectedRequest = new BulkRequest(); + for (int i = 0; i < 2; ++i) { + rejectedRequest.add(new IndexRequest(index).source(Collections.singletonMap("key", "value" + i))); + } + assertThat(DocWriteRequest.writeSizeInBytes(rejectedRequest.requests.stream()), lessThan(1024L)); + + try { + final ActionFuture successFuture = client().bulk(lessThan1KB); + replicationStarted.await(); + + final ActionFuture rejectedFuture = client().bulk(rejectedRequest); + final BulkResponse failedResponses = rejectedFuture.actionGet(); + assertTrue(failedResponses.hasFailures()); + for (BulkItemResponse response : failedResponses) { + assertEquals(RestStatus.TOO_MANY_REQUESTS, response.getFailure().getStatus()); + } + + replicationBlock.countDown(); + + final BulkResponse successResponses = successFuture.actionGet(); + assertFalse(successResponses.hasFailures()); + } finally { + for (TransportService transportService : internalCluster().getInstances(TransportService.class)) { + final MockTransportService mockTransportService = (MockTransportService) transportService; + mockTransportService.clearAllRules(); + } + } + + } +} diff --git a/server/src/test/java/org/elasticsearch/action/bulk/TransportBulkActionIndicesThatCannotBeCreatedTests.java b/server/src/test/java/org/elasticsearch/action/bulk/TransportBulkActionIndicesThatCannotBeCreatedTests.java index 85807d8be9769..e287704698532 100644 --- a/server/src/test/java/org/elasticsearch/action/bulk/TransportBulkActionIndicesThatCannotBeCreatedTests.java +++ b/server/src/test/java/org/elasticsearch/action/bulk/TransportBulkActionIndicesThatCannotBeCreatedTests.java @@ -31,7 +31,6 @@ import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.cluster.service.ClusterService; -import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.concurrent.AtomicArray; import org.elasticsearch.common.util.concurrent.EsExecutors; @@ -120,7 +119,7 @@ private void indicesThatCannotBeCreatedTestCase(Set expected, final ExecutorService direct = EsExecutors.newDirectExecutorService(); when(threadPool.executor(anyString())).thenReturn(direct); TransportBulkAction action = new TransportBulkAction(threadPool, mock(TransportService.class), clusterService, - null, null, mock(ActionFilters.class), null, null, new BulkIndexingMemoryLimits(Settings.EMPTY)) { + null, null, mock(ActionFilters.class), null, null) { @Override void executeBulk(Task task, BulkRequest bulkRequest, long startTimeNanos, ActionListener listener, AtomicArray responses, Map indicesThatCannotBeCreated) { diff --git a/server/src/test/java/org/elasticsearch/action/bulk/TransportBulkActionIngestTests.java b/server/src/test/java/org/elasticsearch/action/bulk/TransportBulkActionIngestTests.java index ffb659830c013..154e0c1af26bc 100644 --- a/server/src/test/java/org/elasticsearch/action/bulk/TransportBulkActionIngestTests.java +++ b/server/src/test/java/org/elasticsearch/action/bulk/TransportBulkActionIngestTests.java @@ -142,7 +142,7 @@ null, new ActionFilters(Collections.emptySet()), null, new AutoCreateIndex( SETTINGS, new ClusterSettings(SETTINGS, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS), new IndexNameExpressionResolver() - ), new BulkIndexingMemoryLimits(Settings.EMPTY) + ) ); } @Override diff --git a/server/src/test/java/org/elasticsearch/action/bulk/TransportBulkActionTests.java b/server/src/test/java/org/elasticsearch/action/bulk/TransportBulkActionTests.java index 8c0d0c1ea7464..3f3404a9270ae 100644 --- a/server/src/test/java/org/elasticsearch/action/bulk/TransportBulkActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/bulk/TransportBulkActionTests.java @@ -72,8 +72,7 @@ class TestTransportBulkAction extends TransportBulkAction { TestTransportBulkAction() { super(TransportBulkActionTests.this.threadPool, transportService, clusterService, null, null, new ActionFilters(Collections.emptySet()), new Resolver(), - new AutoCreateIndex(Settings.EMPTY, clusterService.getClusterSettings(), new Resolver()), - new BulkIndexingMemoryLimits(Settings.EMPTY)); + new AutoCreateIndex(Settings.EMPTY, clusterService.getClusterSettings(), new Resolver())); } @Override diff --git a/server/src/test/java/org/elasticsearch/action/bulk/TransportBulkActionTookTests.java b/server/src/test/java/org/elasticsearch/action/bulk/TransportBulkActionTookTests.java index 7ac5952e0e719..2b2c22cd9be5c 100644 --- a/server/src/test/java/org/elasticsearch/action/bulk/TransportBulkActionTookTests.java +++ b/server/src/test/java/org/elasticsearch/action/bulk/TransportBulkActionTookTests.java @@ -118,8 +118,7 @@ void doExecute(ActionType action, Request request, ActionListener Date: Thu, 16 Apr 2020 18:38:24 -0600 Subject: [PATCH 06/41] Changes --- .../support/replication/ReplicationOperation.java | 6 ------ .../replication/TransportReplicationAction.java | 3 --- .../support/replication/TransportWriteAction.java | 13 ------------- .../TransportVerifyShardBeforeCloseActionTests.java | 3 --- .../replication/ReplicationOperationTests.java | 5 ----- .../ESIndexLevelReplicationTestCase.java | 3 --- 6 files changed, 33 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/support/replication/ReplicationOperation.java b/server/src/main/java/org/elasticsearch/action/support/replication/ReplicationOperation.java index 472469a8908e0..e24a3dbe381c3 100644 --- a/server/src/main/java/org/elasticsearch/action/support/replication/ReplicationOperation.java +++ b/server/src/main/java/org/elasticsearch/action/support/replication/ReplicationOperation.java @@ -303,7 +303,6 @@ private void finish() { failuresArray ) ); - primaryResult.primaryCoordinationComplete(); resultListener.onResponse(primaryResult); } } @@ -495,10 +494,5 @@ public interface PrimaryResult> { * @param listener calllback that is invoked after post replication actions have completed * */ void runPostReplicationActions(ActionListener listener); - - /** - * Called when all primary coordination and replication is complete - * */ - void primaryCoordinationComplete(); } } diff --git a/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java b/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java index f9e1b5fb2d62f..a6995b4f78a55 100644 --- a/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java +++ b/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java @@ -444,9 +444,6 @@ public void runPostReplicationActions(ActionListener listener) { listener.onResponse(null); } } - - @Override - public void primaryCoordinationComplete() {} } public static class ReplicaResult { diff --git a/server/src/main/java/org/elasticsearch/action/support/replication/TransportWriteAction.java b/server/src/main/java/org/elasticsearch/action/support/replication/TransportWriteAction.java index 4fceb39462a10..07f8e96b4e796 100644 --- a/server/src/main/java/org/elasticsearch/action/support/replication/TransportWriteAction.java +++ b/server/src/main/java/org/elasticsearch/action/support/replication/TransportWriteAction.java @@ -127,22 +127,14 @@ public static class WritePrimaryResult {}); - } - - public WritePrimaryResult(ReplicaRequest request, @Nullable Response finalResponse, - @Nullable Location location, @Nullable Exception operationFailure, - IndexShard primary, Logger logger, Runnable postPrimaryCoordination) { super(request, finalResponse, operationFailure); this.location = location; this.primary = primary; this.logger = logger; - this.postPrimaryCoordination = postPrimaryCoordination; assert location == null || operationFailure == null : "expected either failure to be null or translog location to be null, " + "but found: [" + location + "] translog location and [" + operationFailure + "] failure"; @@ -171,11 +163,6 @@ public void onFailure(Exception ex) { }, logger).run(); } } - - @Override - public void primaryCoordinationComplete() { - postPrimaryCoordination.run(); - } } /** diff --git a/server/src/test/java/org/elasticsearch/action/admin/indices/close/TransportVerifyShardBeforeCloseActionTests.java b/server/src/test/java/org/elasticsearch/action/admin/indices/close/TransportVerifyShardBeforeCloseActionTests.java index 73f6d83ab1631..076e538300039 100644 --- a/server/src/test/java/org/elasticsearch/action/admin/indices/close/TransportVerifyShardBeforeCloseActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/admin/indices/close/TransportVerifyShardBeforeCloseActionTests.java @@ -352,9 +352,6 @@ public void runPostReplicationActions(ActionListener listener) { listener.onResponse(null); } - @Override - public void primaryCoordinationComplete() {} - public ReplicationResponse.ShardInfo getShardInfo() { return shardInfo.get(); } diff --git a/server/src/test/java/org/elasticsearch/action/support/replication/ReplicationOperationTests.java b/server/src/test/java/org/elasticsearch/action/support/replication/ReplicationOperationTests.java index 39dd82c69d196..6830a8613f953 100644 --- a/server/src/test/java/org/elasticsearch/action/support/replication/ReplicationOperationTests.java +++ b/server/src/test/java/org/elasticsearch/action/support/replication/ReplicationOperationTests.java @@ -515,11 +515,6 @@ public void runPostReplicationActions(ActionListener listener) { listener.onResponse(null); } - @Override - public void primaryCoordinationComplete() { - - } - public ShardInfo getShardInfo() { return shardInfo; } diff --git a/test/framework/src/main/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java b/test/framework/src/main/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java index 75f0dace393d1..3ff568aa9bc56 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java @@ -742,9 +742,6 @@ public void setShardInfo(ReplicationResponse.ShardInfo shardInfo) { public void runPostReplicationActions(ActionListener listener) { listener.onResponse(null); } - - @Override - public void primaryCoordinationComplete() {} } } From b0a6568e55a553dde1e3b70085814f86fc2f762f Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Wed, 29 Apr 2020 17:47:04 -0600 Subject: [PATCH 07/41] Changes --- .../action/bulk/BulkIndexingMemoryLimits.java | 125 +++++++++++++++++- .../action/bulk/TransportShardBulkAction.java | 2 + .../action/support/ChannelActionListener.java | 12 +- .../TransportReplicationAction.java | 37 ++++-- .../common/breaker/CircuitBreaker.java | 6 + .../transport/InboundHandler.java | 29 +--- .../transport/RequestHandlerRegistry.java | 55 +++++++- .../transport/TaskTransportChannel.java | 21 ++- .../transport/TransportChannel.java | 19 +++ .../transport/TransportExecutor.java | 25 ++++ .../transport/TransportRequestHandler.java | 5 + .../transport/TransportService.java | 50 +------ .../transport/InboundHandlerTests.java | 6 +- 13 files changed, 277 insertions(+), 115 deletions(-) create mode 100644 server/src/main/java/org/elasticsearch/transport/TransportExecutor.java diff --git a/server/src/main/java/org/elasticsearch/action/bulk/BulkIndexingMemoryLimits.java b/server/src/main/java/org/elasticsearch/action/bulk/BulkIndexingMemoryLimits.java index 428009a203cb5..31810e90ccd37 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/BulkIndexingMemoryLimits.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/BulkIndexingMemoryLimits.java @@ -19,36 +19,50 @@ package org.elasticsearch.action.bulk; +import org.apache.logging.log4j.Logger; +import org.elasticsearch.action.DocWriteRequest; +import org.elasticsearch.common.breaker.CircuitBreaker; +import org.elasticsearch.common.breaker.CircuitBreakingException; import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.util.concurrent.EsRejectedExecutionException; import java.util.concurrent.atomic.AtomicLong; +import java.util.stream.Stream; -public class BulkIndexingMemoryLimits { +public class BulkIndexingMemoryLimits implements CircuitBreaker { public static final Setting MAX_INDEXING_BYTES = Setting.memorySizeSetting("indices.memory.queued_indexing_bytes.limit", "25%", Setting.Property.NodeScope); - private final long primaryIndexingLimits; + private final Logger logger; + private final AtomicLong trippedCount = new AtomicLong(0); + private final long indexingLimits; + private final long replicaLimits; private final AtomicLong pendingBytes = new AtomicLong(0); public BulkIndexingMemoryLimits(Settings settings) { - this.primaryIndexingLimits = MAX_INDEXING_BYTES.get(settings).getBytes(); + this(settings, null); + } + + public BulkIndexingMemoryLimits(Settings settings, Logger logger) { + this.indexingLimits = MAX_INDEXING_BYTES.get(settings).getBytes(); + this.logger = logger; + this.replicaLimits = (long) (indexingLimits * 1.5); } public void markPrimaryOperationStarted(long bytes) { long pendingWithOperation = pendingBytes.addAndGet(bytes); - if (pendingWithOperation > primaryIndexingLimits) { + if (pendingWithOperation > indexingLimits) { decrementPendingBytes(bytes); long pendingPreOperation = pendingWithOperation - bytes; throw new EsRejectedExecutionException("rejected execution of primary shard operation [" + "pending_bytes=" + pendingPreOperation + ", " + "operation_bytes=" + bytes + "," + - "max_pending_bytes=" + primaryIndexingLimits + "]", false); + "max_pending_bytes=" + indexingLimits + "]", false); } } @@ -67,4 +81,105 @@ public void markReplicaOperationFinished(long bytes) { private void decrementPendingBytes(long operationSizeInBytes) { pendingBytes.getAndAdd(-operationSizeInBytes); } + + public void addOperationAndMaybeBreak(long operationBytes) { + addEstimateBytesAndMaybeBreak(operationBytes, "indexing", true); + } + + public void addReplicationOperationAndMaybeBreak(long operationBytes) { + addEstimateBytesAndMaybeBreak(operationBytes, "replication", true); + } + + public void markOperationFinished(long bytes) { + addWithoutBreaking(-bytes); + } + + @Override + public double addEstimateBytesAndMaybeBreak(long bytes, String label) throws CircuitBreakingException { + return addEstimateBytesAndMaybeBreak(bytes, label, false); + } + + private double addEstimateBytesAndMaybeBreak(long bytes, String label, boolean isReplication) throws CircuitBreakingException { + long byteLimit = isReplication ? replicaLimits : indexingLimits; + long newUsed; + long currentUsed; + do { + currentUsed = pendingBytes.get(); + newUsed = currentUsed + bytes; + long newUsedWithOverhead = newUsed; + if (logger.isTraceEnabled()) { + logger.trace("[{}] Adding [{}][{}] to used bytes [new used: [{}], limit: {} [{}], estimate: {} [{}]]", + getName(), + new ByteSizeValue(bytes), label, new ByteSizeValue(newUsed), + byteLimit, new ByteSizeValue(byteLimit), + newUsedWithOverhead, new ByteSizeValue(newUsedWithOverhead)); + } + if (byteLimit > 0 && newUsedWithOverhead > byteLimit) { + logger.warn("[{}] New used memory {} [{}] for data of [{}] would be larger than configured breaker: {} [{}], breaking", + getName(), + newUsedWithOverhead, new ByteSizeValue(newUsedWithOverhead), label, + byteLimit, new ByteSizeValue(byteLimit)); + circuitBreak(label, newUsedWithOverhead, isReplication); + } + // Attempt to set the new used value, but make sure it hasn't changed + // underneath us, if it has, keep trying until we are able to set it + } while (!this.pendingBytes.compareAndSet(currentUsed, newUsed)); + return newUsed; + } + + @Override + public void circuitBreak(String fieldName, long bytesNeeded) { + circuitBreak(fieldName, bytesNeeded, false); + } + + private void circuitBreak(String fieldName, long bytesNeeded, boolean isReplication) { + this.trippedCount.incrementAndGet(); + final long byteLimit; + if (isReplication) { + byteLimit = replicaLimits; + } else { + byteLimit = indexingLimits; + } + final String message = "[" + getName() + "] Data too large, data for [" + fieldName + "]" + + " would be [" + bytesNeeded + "/" + new ByteSizeValue(bytesNeeded) + "]" + + ", which is larger than the limit of [" + + byteLimit + "/" + new ByteSizeValue(byteLimit) + "]"; + logger.debug("{}", message); + throw new CircuitBreakingException(message, bytesNeeded, indexingLimits, getDurability()); + } + + @Override + public long addWithoutBreaking(long bytes) { + return pendingBytes.getAndAdd(bytes); + } + + @Override + public long getUsed() { + return pendingBytes.get(); + } + + @Override + public long getLimit() { + return indexingLimits; + } + + @Override + public double getOverhead() { + return 1.0; + } + + @Override + public long getTrippedCount() { + return trippedCount.get(); + } + + @Override + public String getName() { + return CircuitBreaker.INDEXING; + } + + @Override + public Durability getDurability() { + return CircuitBreaker.Durability.TRANSIENT; + } } diff --git a/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java b/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java index 4d772b9a9430a..5fded2ea7de0e 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java @@ -46,6 +46,7 @@ import org.elasticsearch.cluster.metadata.IndexMetadata; import org.elasticsearch.cluster.metadata.MappingMetadata; import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.common.breaker.CircuitBreaker; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.compress.CompressedXContent; @@ -69,6 +70,7 @@ import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.translog.Translog; import org.elasticsearch.indices.IndicesService; +import org.elasticsearch.indices.breaker.CircuitBreakerService; import org.elasticsearch.node.NodeClosedException; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportRequestOptions; diff --git a/server/src/main/java/org/elasticsearch/action/support/ChannelActionListener.java b/server/src/main/java/org/elasticsearch/action/support/ChannelActionListener.java index b93298eefe871..835546282d6cd 100644 --- a/server/src/main/java/org/elasticsearch/action/support/ChannelActionListener.java +++ b/server/src/main/java/org/elasticsearch/action/support/ChannelActionListener.java @@ -19,9 +19,6 @@ 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.transport.TransportChannel; import org.elasticsearch.transport.TransportRequest; @@ -30,7 +27,6 @@ public final class ChannelActionListener< Response extends TransportResponse, Request extends TransportRequest> implements ActionListener { - private static final Logger logger = LogManager.getLogger(ChannelActionListener.class); private final TransportChannel channel; private final Request request; private final String actionName; @@ -52,12 +48,6 @@ public void onResponse(Response response) { @Override public void onFailure(Exception e) { - try { - 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); - } + TransportChannel.sendErrorResponse(channel, actionName, request, e); } } diff --git a/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java b/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java index a6995b4f78a55..d0316967072b4 100644 --- a/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java +++ b/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java @@ -72,6 +72,7 @@ import org.elasticsearch.transport.TransportChannel; import org.elasticsearch.transport.TransportException; import org.elasticsearch.transport.TransportRequest; +import org.elasticsearch.transport.TransportRequestHandler; import org.elasticsearch.transport.TransportRequestOptions; import org.elasticsearch.transport.TransportResponseHandler; import org.elasticsearch.transport.TransportService; @@ -139,11 +140,31 @@ protected TransportReplicationAction(Settings settings, String actionName, Trans transportService.registerRequestHandler(actionName, ThreadPool.Names.SAME, requestReader, this::handleOperationRequest); transportService.registerRequestHandler(transportPrimaryAction, executor, forceExecutionOnPrimary, true, - in -> new ConcreteShardRequest<>(requestReader, in), this::handlePrimaryRequest); + in -> new ConcreteShardRequest<>(requestReader, in), new TransportRequestHandler<>() { + @Override + public void messageReceived(ConcreteShardRequest request, TransportChannel channel, Task task) { + handlePrimaryRequest(request, channel, task); + } + + @Override + public Releasable preDispatchValidation(ConcreteShardRequest request) { + return checkPrimaryLimits(request.request); + } + }); // we must never reject on because of thread pool capacity on replicas transportService.registerRequestHandler(transportReplicaAction, executor, true, true, - in -> new ConcreteReplicaRequest<>(replicaRequestReader, in), this::handleReplicaRequest); + in -> new ConcreteReplicaRequest<>(replicaRequestReader, in), new TransportRequestHandler<>() { + @Override + public void messageReceived(ConcreteReplicaRequest request, TransportChannel channel, Task task) { + handleReplicaRequest(request, channel, task); + } + + @Override + public Releasable preDispatchValidation(ConcreteReplicaRequest request) { + return checkReplicaLimits(request.getRequest()); + } + }); this.transportOptions = transportOptions(settings); @@ -251,9 +272,8 @@ protected void handleOperationRequest(final Request request, final TransportChan } protected void handlePrimaryRequest(final ConcreteShardRequest request, final TransportChannel channel, final Task task) { - final Releasable releasable = checkPrimaryLimits(request.request); - final ActionListener channelListener = new ChannelActionListener<>(channel, transportPrimaryAction, request); - new AsyncPrimaryAction(request, ActionListener.runAfter(channelListener, releasable::close), (ReplicationTask) task).run(); + new AsyncPrimaryAction( + request, new ChannelActionListener<>(channel, transportPrimaryAction, request), (ReplicationTask) task).run(); } protected Releasable checkPrimaryLimits(final Request request) { @@ -466,11 +486,10 @@ public void runPostReplicaActions(ActionListener listener) { } } - protected void handleReplicaRequest(final ConcreteReplicaRequest request, + protected void handleReplicaRequest(final ConcreteReplicaRequest replicaRequest, final TransportChannel channel, final Task task) { - final Releasable releasable = checkReplicaLimits(request.getRequest()); - final ActionListener channelListener = new ChannelActionListener<>(channel, transportReplicaAction, request); - new AsyncReplicaAction(request, ActionListener.runAfter(channelListener, releasable::close), (ReplicationTask) task).run(); + new AsyncReplicaAction( + replicaRequest, new ChannelActionListener<>(channel, transportReplicaAction, replicaRequest), (ReplicationTask) task).run(); } protected Releasable checkReplicaLimits(final ReplicaRequest request) { diff --git a/server/src/main/java/org/elasticsearch/common/breaker/CircuitBreaker.java b/server/src/main/java/org/elasticsearch/common/breaker/CircuitBreaker.java index 2908856b84f1b..222c071a8c5c4 100644 --- a/server/src/main/java/org/elasticsearch/common/breaker/CircuitBreaker.java +++ b/server/src/main/java/org/elasticsearch/common/breaker/CircuitBreaker.java @@ -60,6 +60,12 @@ public interface CircuitBreaker { * segments. */ String ACCOUNTING = "accounting"; + /** + * The accounting breaker tracks things held in memory that is independent + * of the request lifecycle. This includes memory used by Lucene for + * segments. + */ + String INDEXING = "indexing"; enum Type { // A regular or ChildMemoryCircuitBreaker diff --git a/server/src/main/java/org/elasticsearch/transport/InboundHandler.java b/server/src/main/java/org/elasticsearch/transport/InboundHandler.java index d9b5f405db3d9..820c0f6b527ab 100644 --- a/server/src/main/java/org/elasticsearch/transport/InboundHandler.java +++ b/server/src/main/java/org/elasticsearch/transport/InboundHandler.java @@ -166,7 +166,7 @@ private void handleRequest(TcpChannel channel, Head throw new IllegalStateException("Message not fully read (request) for requestId [" + requestId + "], action [" + action + "], available [" + stream.available() + "]; resetting"); } - threadPool.executor(reg.getExecutor()).execute(new RequestHandler<>(reg, request, transportChannel)); + reg.dispatchMessage(request, transportChannel); } } catch (Exception e) { sendErrorResponse(action, transportChannel, e); @@ -239,31 +239,4 @@ private StreamInput namedWriteableStream(StreamInput delegate) { static void assertRemoteVersion(StreamInput in, Version version) { assert version.equals(in.getVersion()) : "Stream version [" + in.getVersion() + "] does not match version [" + version + "]"; } - - private static class RequestHandler extends AbstractRunnable { - private final RequestHandlerRegistry reg; - private final T request; - private final TransportChannel transportChannel; - - RequestHandler(RequestHandlerRegistry reg, T request, TransportChannel transportChannel) { - this.reg = reg; - this.request = request; - this.transportChannel = transportChannel; - } - - @Override - protected void doRun() throws Exception { - reg.processMessageReceived(request, transportChannel); - } - - @Override - public boolean isForceExecution() { - return reg.isForceExecution(); - } - - @Override - public void onFailure(Exception e) { - sendErrorResponse(reg.getAction(), transportChannel, e); - } - } } diff --git a/server/src/main/java/org/elasticsearch/transport/RequestHandlerRegistry.java b/server/src/main/java/org/elasticsearch/transport/RequestHandlerRegistry.java index 2730a52437bbe..add1e7c696d7e 100644 --- a/server/src/main/java/org/elasticsearch/transport/RequestHandlerRegistry.java +++ b/server/src/main/java/org/elasticsearch/transport/RequestHandlerRegistry.java @@ -21,14 +21,20 @@ import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.common.lease.Releasable; +import org.elasticsearch.common.lease.Releasables; +import org.elasticsearch.common.util.concurrent.AbstractRunnable; import org.elasticsearch.tasks.Task; import org.elasticsearch.tasks.TaskManager; +import org.elasticsearch.threadpool.ThreadPool; import java.io.IOException; +import java.util.concurrent.atomic.AtomicReference; public class RequestHandlerRegistry { private final String action; + private final ThreadPool threadPool; private final TransportRequestHandler handler; private final boolean forceExecution; private final boolean canTripCircuitBreaker; @@ -36,11 +42,12 @@ public class RequestHandlerRegistry { private final TaskManager taskManager; private final Writeable.Reader requestReader; - public RequestHandlerRegistry(String action, Writeable.Reader requestReader, TaskManager taskManager, + public RequestHandlerRegistry(String action, Writeable.Reader requestReader, ThreadPool threadPool, TaskManager taskManager, TransportRequestHandler handler, String executor, boolean forceExecution, boolean canTripCircuitBreaker) { this.action = action; this.requestReader = requestReader; + this.threadPool = threadPool; this.handler = handler; this.forceExecution = forceExecution; this.canTripCircuitBreaker = canTripCircuitBreaker; @@ -57,14 +64,22 @@ public Request newRequest(StreamInput in) throws IOException { } public void processMessageReceived(Request request, TransportChannel channel) throws Exception { + processMessageReceived(request, channel, new AtomicReference<>(() -> {})); + } + + private void processMessageReceived(Request request, TransportChannel channel, AtomicReference releasable) + throws Exception { final Task task = taskManager.register(channel.getChannelType(), action, request); + Releasable[] releasables = new Releasable[2]; boolean success = false; try { - handler.messageReceived(request, new TaskTransportChannel(taskManager, task, channel), task); + releasables[0] = () -> taskManager.unregister(task); + releasables[1] = releasable.get(); + handler.messageReceived(request, new TaskTransportChannel(channel, releasables), task); success = true; } finally { if (success == false) { - taskManager.unregister(task); + Releasables.close(releasables); } } } @@ -85,6 +100,38 @@ public TransportRequestHandler getHandler() { return handler; } + public void dispatchMessage(Request request, TransportChannel channel) { + final AtomicReference releasable = new AtomicReference<>(); + try { + releasable.set(handler.preDispatchValidation(request)); + threadPool.executor(executor).execute(new AbstractRunnable() { + @Override + public void onFailure(Exception e) { + Releasables.close(releasable.get()); + TransportChannel.sendErrorResponse(channel, action, request, e); + } + + @Override + protected void doRun() throws Exception { + processMessageReceived(request, channel, releasable); + } + + @Override + public boolean isForceExecution() { + return forceExecution; + } + + @Override + public String toString() { + return "processing of [" + action + "]: " + request; + } + }); + } catch (Exception e) { + Releasables.close(releasable.get()); + TransportChannel.sendErrorResponse(channel, action, request, e); + } + } + @Override public String toString() { return handler.toString(); @@ -92,7 +139,7 @@ public String toString() { public static RequestHandlerRegistry replaceHandler(RequestHandlerRegistry registry, TransportRequestHandler handler) { - return new RequestHandlerRegistry<>(registry.action, registry.requestReader, registry.taskManager, handler, + return new RequestHandlerRegistry<>(registry.action, registry.requestReader, registry.threadPool, registry.taskManager, handler, registry.executor, registry.forceExecution, registry.canTripCircuitBreaker); } } diff --git a/server/src/main/java/org/elasticsearch/transport/TaskTransportChannel.java b/server/src/main/java/org/elasticsearch/transport/TaskTransportChannel.java index aa659906019ce..b766787db6112 100644 --- a/server/src/main/java/org/elasticsearch/transport/TaskTransportChannel.java +++ b/server/src/main/java/org/elasticsearch/transport/TaskTransportChannel.java @@ -20,22 +20,19 @@ package org.elasticsearch.transport; import org.elasticsearch.Version; -import org.elasticsearch.tasks.Task; -import org.elasticsearch.tasks.TaskManager; +import org.elasticsearch.common.lease.Releasable; +import org.elasticsearch.common.lease.Releasables; import java.io.IOException; public class TaskTransportChannel implements TransportChannel { - private final Task task; - - private final TaskManager taskManager; private final TransportChannel channel; + private final Releasable[] releasables; - TaskTransportChannel(TaskManager taskManager, Task task, TransportChannel channel) { + TaskTransportChannel(TransportChannel channel, Releasable[] releasables) { this.channel = channel; - this.task = task; - this.taskManager = taskManager; + this.releasables = releasables; } @Override @@ -50,13 +47,13 @@ public String getChannelType() { @Override public void sendResponse(TransportResponse response) throws IOException { - endTask(); + release(); channel.sendResponse(response); } @Override public void sendResponse(Exception exception) throws IOException { - endTask(); + release(); channel.sendResponse(exception); } @@ -69,7 +66,7 @@ public TransportChannel getChannel() { return channel; } - private void endTask() { - taskManager.unregister(task); + private void release() { + Releasables.close(releasables); } } diff --git a/server/src/main/java/org/elasticsearch/transport/TransportChannel.java b/server/src/main/java/org/elasticsearch/transport/TransportChannel.java index 17e538f04c43b..c2cce0ea40dde 100644 --- a/server/src/main/java/org/elasticsearch/transport/TransportChannel.java +++ b/server/src/main/java/org/elasticsearch/transport/TransportChannel.java @@ -19,7 +19,11 @@ 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.action.support.ChannelActionListener; import java.io.IOException; @@ -28,6 +32,8 @@ */ public interface TransportChannel { + Logger logger = LogManager.getLogger(ChannelActionListener.class); + String getProfileName(); String getChannelType(); @@ -42,4 +48,17 @@ public interface TransportChannel { default Version getVersion() { 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); + } + } } diff --git a/server/src/main/java/org/elasticsearch/transport/TransportExecutor.java b/server/src/main/java/org/elasticsearch/transport/TransportExecutor.java new file mode 100644 index 0000000000000..ea2ab4ba90915 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/transport/TransportExecutor.java @@ -0,0 +1,25 @@ +/* + * 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.transport; + +public class TransportExecutor { + + +} diff --git a/server/src/main/java/org/elasticsearch/transport/TransportRequestHandler.java b/server/src/main/java/org/elasticsearch/transport/TransportRequestHandler.java index be95798806847..495ce13be6e38 100644 --- a/server/src/main/java/org/elasticsearch/transport/TransportRequestHandler.java +++ b/server/src/main/java/org/elasticsearch/transport/TransportRequestHandler.java @@ -19,9 +19,14 @@ package org.elasticsearch.transport; +import org.elasticsearch.common.lease.Releasable; import org.elasticsearch.tasks.Task; public interface TransportRequestHandler { void messageReceived(T request, TransportChannel channel, Task task) throws Exception; + + default Releasable preDispatchValidation(T request) throws Exception { + return () -> {}; + }; } diff --git a/server/src/main/java/org/elasticsearch/transport/TransportService.java b/server/src/main/java/org/elasticsearch/transport/TransportService.java index 7f42e175db774..261561884c7da 100644 --- a/server/src/main/java/org/elasticsearch/transport/TransportService.java +++ b/server/src/main/java/org/elasticsearch/transport/TransportService.java @@ -721,50 +721,18 @@ protected void doRun() throws Exception { } } - private void sendLocalRequest(long requestId, final String action, final TransportRequest request, TransportRequestOptions options) { + @SuppressWarnings("unchecked") + private void sendLocalRequest(long requestId, final String action, final TransportRequest request, + TransportRequestOptions options) { final DirectResponseChannel channel = new DirectResponseChannel(localNode, action, requestId, this, threadPool); try { onRequestSent(localNode, requestId, action, request, options); onRequestReceived(requestId, action); - final RequestHandlerRegistry reg = getRequestHandler(action); + final RequestHandlerRegistry reg = (RequestHandlerRegistry) getRequestHandler(action); if (reg == null) { throw new ActionNotFoundTransportException("Action [" + action + "] not found"); } - final String executor = reg.getExecutor(); - if (ThreadPool.Names.SAME.equals(executor)) { - //noinspection unchecked - reg.processMessageReceived(request, channel); - } else { - threadPool.executor(executor).execute(new AbstractRunnable() { - @Override - protected void doRun() throws Exception { - //noinspection unchecked - reg.processMessageReceived(request, channel); - } - - @Override - public boolean isForceExecution() { - return reg.isForceExecution(); - } - - @Override - public void onFailure(Exception e) { - try { - channel.sendResponse(e); - } catch (Exception inner) { - inner.addSuppressed(e); - logger.warn(() -> new ParameterizedMessage( - "failed to notify channel of error message for action [{}]", action), inner); - } - } - - @Override - public String toString() { - return "processing of [" + requestId + "][" + action + "]: " + request; - } - }); - } - + reg.dispatchMessage((T) request, channel); } catch (Exception e) { try { channel.sendResponse(e); @@ -845,11 +813,7 @@ public static boolean isValidActionName(String actionName) { public void registerRequestHandler(String action, String executor, Writeable.Reader requestReader, TransportRequestHandler handler) { - validateActionName(action); - handler = interceptor.interceptHandler(action, executor, false, handler); - RequestHandlerRegistry reg = new RequestHandlerRegistry<>( - action, requestReader, taskManager, handler, executor, false, true); - transport.registerRequestHandler(reg); + registerRequestHandler(action, executor, false, true, requestReader, handler); } /** @@ -870,7 +834,7 @@ public void registerRequestHandler(String act validateActionName(action); handler = interceptor.interceptHandler(action, executor, forceExecution, handler); RequestHandlerRegistry reg = new RequestHandlerRegistry<>( - action, requestReader, taskManager, handler, executor, forceExecution, canTripCircuitBreaker); + action, requestReader, threadPool, taskManager, handler, executor, forceExecution, canTripCircuitBreaker); transport.registerRequestHandler(reg); } diff --git a/server/src/test/java/org/elasticsearch/transport/InboundHandlerTests.java b/server/src/test/java/org/elasticsearch/transport/InboundHandlerTests.java index 2d98a1d50a101..d320f1996c42c 100644 --- a/server/src/test/java/org/elasticsearch/transport/InboundHandlerTests.java +++ b/server/src/test/java/org/elasticsearch/transport/InboundHandlerTests.java @@ -77,8 +77,8 @@ public void tearDown() throws Exception { public void testPing() throws Exception { AtomicReference channelCaptor = new AtomicReference<>(); - RequestHandlerRegistry registry = new RequestHandlerRegistry<>("test-request", TestRequest::new, taskManager, - (request, channel, task) -> channelCaptor.set(channel), ThreadPool.Names.SAME, false, true); + RequestHandlerRegistry registry = new RequestHandlerRegistry<>("test-request", TestRequest::new, threadPool, + taskManager, (request, channel, task) -> channelCaptor.set(channel), ThreadPool.Names.SAME, false, true); requestHandlers.registerHandler(registry); handler.inboundMessage(channel, new InboundMessage(null, true)); @@ -119,7 +119,7 @@ public TestResponse read(StreamInput in) throws IOException { return new TestResponse(in); } }, null, action)); - RequestHandlerRegistry registry = new RequestHandlerRegistry<>(action, TestRequest::new, taskManager, + RequestHandlerRegistry registry = new RequestHandlerRegistry<>(action, TestRequest::new, threadPool, taskManager, (request, channel, task) -> { channelCaptor.set(channel); requestCaptor.set(request); From dcfb545eda063dc9306e9c47376c20760855c828 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Wed, 29 Apr 2020 18:01:37 -0600 Subject: [PATCH 08/41] Changes --- .../action/bulk/BulkIndexingMemoryLimits.java | 2 -- .../action/bulk/TransportShardBulkAction.java | 2 -- .../common/breaker/CircuitBreaker.java | 6 +---- .../transport/TransportExecutor.java | 25 ------------------- 4 files changed, 1 insertion(+), 34 deletions(-) delete mode 100644 server/src/main/java/org/elasticsearch/transport/TransportExecutor.java diff --git a/server/src/main/java/org/elasticsearch/action/bulk/BulkIndexingMemoryLimits.java b/server/src/main/java/org/elasticsearch/action/bulk/BulkIndexingMemoryLimits.java index 31810e90ccd37..5af08d19c8223 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/BulkIndexingMemoryLimits.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/BulkIndexingMemoryLimits.java @@ -20,7 +20,6 @@ package org.elasticsearch.action.bulk; import org.apache.logging.log4j.Logger; -import org.elasticsearch.action.DocWriteRequest; import org.elasticsearch.common.breaker.CircuitBreaker; import org.elasticsearch.common.breaker.CircuitBreakingException; import org.elasticsearch.common.settings.Setting; @@ -29,7 +28,6 @@ import org.elasticsearch.common.util.concurrent.EsRejectedExecutionException; import java.util.concurrent.atomic.AtomicLong; -import java.util.stream.Stream; public class BulkIndexingMemoryLimits implements CircuitBreaker { diff --git a/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java b/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java index 5fded2ea7de0e..4d772b9a9430a 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java @@ -46,7 +46,6 @@ import org.elasticsearch.cluster.metadata.IndexMetadata; import org.elasticsearch.cluster.metadata.MappingMetadata; import org.elasticsearch.cluster.service.ClusterService; -import org.elasticsearch.common.breaker.CircuitBreaker; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.compress.CompressedXContent; @@ -70,7 +69,6 @@ import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.translog.Translog; import org.elasticsearch.indices.IndicesService; -import org.elasticsearch.indices.breaker.CircuitBreakerService; import org.elasticsearch.node.NodeClosedException; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportRequestOptions; diff --git a/server/src/main/java/org/elasticsearch/common/breaker/CircuitBreaker.java b/server/src/main/java/org/elasticsearch/common/breaker/CircuitBreaker.java index 222c071a8c5c4..163b2deae289c 100644 --- a/server/src/main/java/org/elasticsearch/common/breaker/CircuitBreaker.java +++ b/server/src/main/java/org/elasticsearch/common/breaker/CircuitBreaker.java @@ -60,11 +60,7 @@ public interface CircuitBreaker { * segments. */ String ACCOUNTING = "accounting"; - /** - * The accounting breaker tracks things held in memory that is independent - * of the request lifecycle. This includes memory used by Lucene for - * segments. - */ + // TODO: Description String INDEXING = "indexing"; enum Type { diff --git a/server/src/main/java/org/elasticsearch/transport/TransportExecutor.java b/server/src/main/java/org/elasticsearch/transport/TransportExecutor.java deleted file mode 100644 index ea2ab4ba90915..0000000000000 --- a/server/src/main/java/org/elasticsearch/transport/TransportExecutor.java +++ /dev/null @@ -1,25 +0,0 @@ -/* - * 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.transport; - -public class TransportExecutor { - - -} From 755a6cbd6f0be54917e66cc7fca76217870597dd Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Tue, 12 May 2020 12:47:09 -0600 Subject: [PATCH 09/41] Move --- .../org/elasticsearch/action/bulk/BulkIndexingMemoryLimitsIT.java | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename server/src/{test => internalClusterTest}/java/org/elasticsearch/action/bulk/BulkIndexingMemoryLimitsIT.java (100%) diff --git a/server/src/test/java/org/elasticsearch/action/bulk/BulkIndexingMemoryLimitsIT.java b/server/src/internalClusterTest/java/org/elasticsearch/action/bulk/BulkIndexingMemoryLimitsIT.java similarity index 100% rename from server/src/test/java/org/elasticsearch/action/bulk/BulkIndexingMemoryLimitsIT.java rename to server/src/internalClusterTest/java/org/elasticsearch/action/bulk/BulkIndexingMemoryLimitsIT.java From 2ccb5b6070bb1c11bbf5b68ddda0405529a43efd Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Wed, 13 May 2020 15:48:44 -0600 Subject: [PATCH 10/41] Changes --- .../bulk/BulkIndexingMemoryLimitsIT.java | 4 +- .../action/bulk/BulkIndexingMemoryLimits.java | 126 ++---------------- .../java/org/elasticsearch/node/Node.java | 2 +- .../snapshots/SnapshotResiliencyTests.java | 2 +- 4 files changed, 13 insertions(+), 121 deletions(-) diff --git a/server/src/internalClusterTest/java/org/elasticsearch/action/bulk/BulkIndexingMemoryLimitsIT.java b/server/src/internalClusterTest/java/org/elasticsearch/action/bulk/BulkIndexingMemoryLimitsIT.java index c990da232c578..4c5b998bddc0b 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/action/bulk/BulkIndexingMemoryLimitsIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/action/bulk/BulkIndexingMemoryLimitsIT.java @@ -7,7 +7,7 @@ * 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 + * 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 @@ -47,7 +47,7 @@ protected Settings nodeSettings(int nodeOrdinal) { .put(super.nodeSettings(nodeOrdinal)) // Need at least two threads because we are going to block one .put("thread_pool.write.size", 2) - .put("indices.memory.queued_indexing_bytes.limit", "1KB") + .put("indices.indexing_bytes.limit", "1KB") .build(); } diff --git a/server/src/main/java/org/elasticsearch/action/bulk/BulkIndexingMemoryLimits.java b/server/src/main/java/org/elasticsearch/action/bulk/BulkIndexingMemoryLimits.java index 5af08d19c8223..71f8c6489e642 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/BulkIndexingMemoryLimits.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/BulkIndexingMemoryLimits.java @@ -19,9 +19,7 @@ package org.elasticsearch.action.bulk; -import org.apache.logging.log4j.Logger; -import org.elasticsearch.common.breaker.CircuitBreaker; -import org.elasticsearch.common.breaker.CircuitBreakingException; +import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.ByteSizeValue; @@ -29,32 +27,26 @@ import java.util.concurrent.atomic.AtomicLong; -public class BulkIndexingMemoryLimits implements CircuitBreaker { +public class BulkIndexingMemoryLimits { public static final Setting MAX_INDEXING_BYTES = - Setting.memorySizeSetting("indices.memory.queued_indexing_bytes.limit", "25%", Setting.Property.NodeScope); + Setting.memorySizeSetting("indices.indexing_bytes.limit", "20%", Setting.Property.NodeScope, Setting.Property.Dynamic); - private final Logger logger; - private final AtomicLong trippedCount = new AtomicLong(0); - private final long indexingLimits; - private final long replicaLimits; + private volatile long indexingLimits; private final AtomicLong pendingBytes = new AtomicLong(0); - public BulkIndexingMemoryLimits(Settings settings) { - this(settings, null); - } - - public BulkIndexingMemoryLimits(Settings settings, Logger logger) { + public BulkIndexingMemoryLimits(Settings settings, ClusterSettings clusterSettings) { this.indexingLimits = MAX_INDEXING_BYTES.get(settings).getBytes(); - this.logger = logger; - this.replicaLimits = (long) (indexingLimits * 1.5); + clusterSettings.addSettingsUpdateConsumer(MAX_INDEXING_BYTES, value -> indexingLimits = value.getBytes()); } public void markPrimaryOperationStarted(long bytes) { long pendingWithOperation = pendingBytes.addAndGet(bytes); + System.err.println("ACQUIRE " + pendingWithOperation); if (pendingWithOperation > indexingLimits) { + System.err.println("FAILED " + bytes); decrementPendingBytes(bytes); long pendingPreOperation = pendingWithOperation - bytes; throw new EsRejectedExecutionException("rejected execution of primary shard operation [" + @@ -65,6 +57,7 @@ public void markPrimaryOperationStarted(long bytes) { } public void markPrimaryOperationFinished(long bytes) { + System.err.println("RELEASE " + bytes); decrementPendingBytes(bytes); } @@ -79,105 +72,4 @@ public void markReplicaOperationFinished(long bytes) { private void decrementPendingBytes(long operationSizeInBytes) { pendingBytes.getAndAdd(-operationSizeInBytes); } - - public void addOperationAndMaybeBreak(long operationBytes) { - addEstimateBytesAndMaybeBreak(operationBytes, "indexing", true); - } - - public void addReplicationOperationAndMaybeBreak(long operationBytes) { - addEstimateBytesAndMaybeBreak(operationBytes, "replication", true); - } - - public void markOperationFinished(long bytes) { - addWithoutBreaking(-bytes); - } - - @Override - public double addEstimateBytesAndMaybeBreak(long bytes, String label) throws CircuitBreakingException { - return addEstimateBytesAndMaybeBreak(bytes, label, false); - } - - private double addEstimateBytesAndMaybeBreak(long bytes, String label, boolean isReplication) throws CircuitBreakingException { - long byteLimit = isReplication ? replicaLimits : indexingLimits; - long newUsed; - long currentUsed; - do { - currentUsed = pendingBytes.get(); - newUsed = currentUsed + bytes; - long newUsedWithOverhead = newUsed; - if (logger.isTraceEnabled()) { - logger.trace("[{}] Adding [{}][{}] to used bytes [new used: [{}], limit: {} [{}], estimate: {} [{}]]", - getName(), - new ByteSizeValue(bytes), label, new ByteSizeValue(newUsed), - byteLimit, new ByteSizeValue(byteLimit), - newUsedWithOverhead, new ByteSizeValue(newUsedWithOverhead)); - } - if (byteLimit > 0 && newUsedWithOverhead > byteLimit) { - logger.warn("[{}] New used memory {} [{}] for data of [{}] would be larger than configured breaker: {} [{}], breaking", - getName(), - newUsedWithOverhead, new ByteSizeValue(newUsedWithOverhead), label, - byteLimit, new ByteSizeValue(byteLimit)); - circuitBreak(label, newUsedWithOverhead, isReplication); - } - // Attempt to set the new used value, but make sure it hasn't changed - // underneath us, if it has, keep trying until we are able to set it - } while (!this.pendingBytes.compareAndSet(currentUsed, newUsed)); - return newUsed; - } - - @Override - public void circuitBreak(String fieldName, long bytesNeeded) { - circuitBreak(fieldName, bytesNeeded, false); - } - - private void circuitBreak(String fieldName, long bytesNeeded, boolean isReplication) { - this.trippedCount.incrementAndGet(); - final long byteLimit; - if (isReplication) { - byteLimit = replicaLimits; - } else { - byteLimit = indexingLimits; - } - final String message = "[" + getName() + "] Data too large, data for [" + fieldName + "]" + - " would be [" + bytesNeeded + "/" + new ByteSizeValue(bytesNeeded) + "]" + - ", which is larger than the limit of [" + - byteLimit + "/" + new ByteSizeValue(byteLimit) + "]"; - logger.debug("{}", message); - throw new CircuitBreakingException(message, bytesNeeded, indexingLimits, getDurability()); - } - - @Override - public long addWithoutBreaking(long bytes) { - return pendingBytes.getAndAdd(bytes); - } - - @Override - public long getUsed() { - return pendingBytes.get(); - } - - @Override - public long getLimit() { - return indexingLimits; - } - - @Override - public double getOverhead() { - return 1.0; - } - - @Override - public long getTrippedCount() { - return trippedCount.get(); - } - - @Override - public String getName() { - return CircuitBreaker.INDEXING; - } - - @Override - public Durability getDurability() { - return CircuitBreaker.Durability.TRANSIENT; - } } diff --git a/server/src/main/java/org/elasticsearch/node/Node.java b/server/src/main/java/org/elasticsearch/node/Node.java index 95e950eb0f438..59d77178dd1c2 100644 --- a/server/src/main/java/org/elasticsearch/node/Node.java +++ b/server/src/main/java/org/elasticsearch/node/Node.java @@ -552,7 +552,7 @@ protected Node(final Environment initialEnvironment, new PersistentTasksClusterService(settings, registry, clusterService, threadPool); resourcesToClose.add(persistentTasksClusterService); final PersistentTasksService persistentTasksService = new PersistentTasksService(clusterService, threadPool, client); - final BulkIndexingMemoryLimits bulkIndexingLimits = new BulkIndexingMemoryLimits(settings); + final BulkIndexingMemoryLimits bulkIndexingLimits = new BulkIndexingMemoryLimits(settings, clusterService.getClusterSettings()); modules.add(b -> { b.bind(Node.class).toInstance(this); diff --git a/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java b/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java index ec13b103016a7..5cd6879a1b138 100644 --- a/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java +++ b/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java @@ -1497,7 +1497,7 @@ allocationService, new AliasValidator(), environment, indexScopedSettings, actionFilters, indexNameExpressionResolver )); final MappingUpdatedAction mappingUpdatedAction = new MappingUpdatedAction(settings, clusterSettings); - final BulkIndexingMemoryLimits indexingMemoryLimits = new BulkIndexingMemoryLimits(settings); + final BulkIndexingMemoryLimits indexingMemoryLimits = new BulkIndexingMemoryLimits(settings, clusterSettings); mappingUpdatedAction.setClient(client); actions.put(BulkAction.INSTANCE, new TransportBulkAction(threadPool, transportService, clusterService, From 524b965161b7d76ac698c10d0d5d9cad3b256662 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Wed, 13 May 2020 15:50:45 -0600 Subject: [PATCH 11/41] Remove --- .../elasticsearch/action/bulk/BulkIndexingMemoryLimits.java | 3 --- 1 file changed, 3 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/bulk/BulkIndexingMemoryLimits.java b/server/src/main/java/org/elasticsearch/action/bulk/BulkIndexingMemoryLimits.java index 71f8c6489e642..6a1e388d4612c 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/BulkIndexingMemoryLimits.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/BulkIndexingMemoryLimits.java @@ -43,10 +43,8 @@ public BulkIndexingMemoryLimits(Settings settings, ClusterSettings clusterSettin public void markPrimaryOperationStarted(long bytes) { long pendingWithOperation = pendingBytes.addAndGet(bytes); - System.err.println("ACQUIRE " + pendingWithOperation); if (pendingWithOperation > indexingLimits) { - System.err.println("FAILED " + bytes); decrementPendingBytes(bytes); long pendingPreOperation = pendingWithOperation - bytes; throw new EsRejectedExecutionException("rejected execution of primary shard operation [" + @@ -57,7 +55,6 @@ public void markPrimaryOperationStarted(long bytes) { } public void markPrimaryOperationFinished(long bytes) { - System.err.println("RELEASE " + bytes); decrementPendingBytes(bytes); } From a9afc77d8f0f3c512ba6dd35ef3f88fdb6b6ccdc Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Tue, 19 May 2020 13:19:57 -0600 Subject: [PATCH 12/41] WIP --- ...TransportVerifyShardBeforeCloseAction.java | 8 ++-- .../action/bulk/TransportShardBulkAction.java | 5 ++- .../TransportResyncReplicationAction.java | 6 +-- .../TransportReplicationAction.java | 37 +++++++------------ .../replication/TransportWriteAction.java | 26 ++++++++++--- .../seqno/GlobalCheckpointSyncAction.java | 8 ++-- .../index/seqno/RetentionLeaseSyncAction.java | 4 +- ...ReplicationAllPermitsAcquisitionTests.java | 20 +++++----- .../TransportWriteActionTests.java | 12 +++--- .../GlobalCheckpointSyncActionTests.java | 3 +- .../seqno/RetentionLeaseSyncActionTests.java | 4 +- .../TransportBulkShardOperationsAction.java | 4 +- 12 files changed, 73 insertions(+), 64 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/close/TransportVerifyShardBeforeCloseAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/close/TransportVerifyShardBeforeCloseAction.java index 348855b7252a5..bcf6ccea3129b 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/close/TransportVerifyShardBeforeCloseAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/close/TransportVerifyShardBeforeCloseAction.java @@ -96,9 +96,11 @@ protected void shardOperationOnPrimary(final ShardRequest shardRequest, final In } @Override - protected ReplicaResult shardOperationOnReplica(final ShardRequest shardRequest, final IndexShard replica) throws IOException { - executeShardOperation(shardRequest, replica); - return new ReplicaResult(); + protected void shardOperationOnReplica(ShardRequest shardRequest, IndexShard replica, ActionListener listener) { + ActionListener.completeWith(listener, () -> { + executeShardOperation(shardRequest, replica); + return new ReplicaResult(); + }); } private void executeShardOperation(final ShardRequest request, final IndexShard indexShard) throws IOException { diff --git a/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java b/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java index 4d772b9a9430a..8bb7a76a894d4 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java @@ -124,7 +124,7 @@ protected Releasable checkPrimaryLimits(BulkShardRequest request) throws EsRejec } @Override - protected void shardOperationOnPrimary(BulkShardRequest request, IndexShard primary, + protected void dispatchedShardOperationOnPrimary(BulkShardRequest request, IndexShard primary, ActionListener> listener) { ClusterStateObserver observer = new ClusterStateObserver(clusterService, request.timeout(), logger, threadPool.getThreadContext()); performOnPrimary(request, primary, updateHelper, threadPool::absoluteTimeInMillis, @@ -429,7 +429,8 @@ protected Releasable checkReplicaLimits(BulkShardRequest request) throws EsRejec } @Override - public WriteReplicaResult shardOperationOnReplica(BulkShardRequest request, IndexShard replica) throws Exception { + public WriteReplicaResult dispatchedShardOperationOnReplica(BulkShardRequest request, IndexShard replica) + throws Exception { final long startBulkTime = System.nanoTime(); final Translog.Location location = performOnReplica(request, replica); replica.getBulkOperationListener().afterBulk(request.totalSizeInBytes(), System.nanoTime() - startBulkTime); diff --git a/server/src/main/java/org/elasticsearch/action/resync/TransportResyncReplicationAction.java b/server/src/main/java/org/elasticsearch/action/resync/TransportResyncReplicationAction.java index 94dde512aa105..64abdf4301a82 100644 --- a/server/src/main/java/org/elasticsearch/action/resync/TransportResyncReplicationAction.java +++ b/server/src/main/java/org/elasticsearch/action/resync/TransportResyncReplicationAction.java @@ -83,7 +83,7 @@ public ClusterBlockLevel indexBlockLevel() { } @Override - protected void shardOperationOnPrimary(ResyncReplicationRequest request, IndexShard primary, + protected void dispatchedShardOperationOnPrimary(ResyncReplicationRequest request, IndexShard primary, ActionListener> listener) { ActionListener.completeWith(listener, () -> new WritePrimaryResult<>(performOnPrimary(request), new ResyncReplicationResponse(), null, null, primary, logger)); @@ -94,8 +94,8 @@ public static ResyncReplicationRequest performOnPrimary(ResyncReplicationRequest } @Override - protected WriteReplicaResult shardOperationOnReplica(ResyncReplicationRequest request, - IndexShard replica) throws Exception { + protected WriteReplicaResult dispatchedShardOperationOnReplica(ResyncReplicationRequest request, + IndexShard replica) throws Exception { Translog.Location location = performOnReplica(request, replica); return new WriteReplicaResult<>(request, location, null, replica, logger); } diff --git a/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java b/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java index b66a254568d05..6754f8d971602 100644 --- a/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java +++ b/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java @@ -31,6 +31,7 @@ import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.ActiveShardCount; import org.elasticsearch.action.support.ChannelActionListener; +import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.action.support.TransportAction; import org.elasticsearch.action.support.TransportActions; import org.elasticsearch.client.transport.NoNodeAvailableException; @@ -74,7 +75,6 @@ import org.elasticsearch.transport.TransportChannel; import org.elasticsearch.transport.TransportException; import org.elasticsearch.transport.TransportRequest; -import org.elasticsearch.transport.TransportRequestHandler; import org.elasticsearch.transport.TransportRequestOptions; import org.elasticsearch.transport.TransportResponseHandler; import org.elasticsearch.transport.TransportService; @@ -162,31 +162,11 @@ protected TransportReplicationAction(Settings settings, String actionName, Trans transportService.registerRequestHandler(actionName, ThreadPool.Names.SAME, requestReader, this::handleOperationRequest); transportService.registerRequestHandler(transportPrimaryAction, executor, forceExecutionOnPrimary, true, - in -> new ConcreteShardRequest<>(requestReader, in), new TransportRequestHandler<>() { - @Override - public void messageReceived(ConcreteShardRequest request, TransportChannel channel, Task task) { - handlePrimaryRequest(request, channel, task); - } - - @Override - public Releasable preDispatchValidation(ConcreteShardRequest request) { - return checkPrimaryLimits(request.request); - } - }); + in -> new ConcreteShardRequest<>(requestReader, in), this::handlePrimaryRequest); // we must never reject on because of thread pool capacity on replicas transportService.registerRequestHandler(transportReplicaAction, executor, true, true, - in -> new ConcreteReplicaRequest<>(replicaRequestReader, in), new TransportRequestHandler<>() { - @Override - public void messageReceived(ConcreteReplicaRequest request, TransportChannel channel, Task task) { - handleReplicaRequest(request, channel, task); - } - - @Override - public Releasable preDispatchValidation(ConcreteReplicaRequest request) { - return checkReplicaLimits(request.getRequest()); - } - }); + in -> new ConcreteReplicaRequest<>(replicaRequestReader, in), this::handleReplicaRequest); this.transportOptions = transportOptions(settings); @@ -240,7 +220,16 @@ protected abstract void shardOperationOnPrimary(Request shardRequest, IndexShard * @param shardRequest the request to the replica shard * @param replica the replica shard to perform the operation on */ - protected abstract ReplicaResult shardOperationOnReplica(ReplicaRequest shardRequest, IndexShard replica) throws Exception; + protected ReplicaResult shardOperationOnReplica(ReplicaRequest shardRequest, IndexShard replica) throws Exception { + PlainActionFuture listener = PlainActionFuture.newFuture(); + shardOperationOnReplica(shardRequest, replica, listener); + return listener.actionGet(); + } + + protected void shardOperationOnReplica(ReplicaRequest shardRequest, IndexShard replica, + ActionListener listener) { + + }; /** * Cluster level block to check before request execution. Returning null means that no blocks need to be checked. diff --git a/server/src/main/java/org/elasticsearch/action/support/replication/TransportWriteAction.java b/server/src/main/java/org/elasticsearch/action/support/replication/TransportWriteAction.java index 07f8e96b4e796..afbaf7de2db07 100644 --- a/server/src/main/java/org/elasticsearch/action/support/replication/TransportWriteAction.java +++ b/server/src/main/java/org/elasticsearch/action/support/replication/TransportWriteAction.java @@ -104,18 +104,32 @@ protected ReplicationOperation.Replicas newReplicasProxy() { * and failure async refresh is performed on the primary shard according to the Request refresh policy */ @Override - protected abstract void shardOperationOnPrimary( - Request request, IndexShard primary, ActionListener> listener); + protected void shardOperationOnPrimary( + Request request, IndexShard primary, ActionListener> listener) { + dispatchedShardOperationOnPrimary(request, primary, listener); + } + + protected abstract void dispatchedShardOperationOnPrimary( + Request request, IndexShard primary, ActionListener> listener); /** * 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 - * async refresh is performed on the replica shard according to the ReplicaRequest refresh policy + * @param listener listener for the result of the operation on replica, including current translog location and operation + * response and failure async refresh is performed on the replica shard according to the ReplicaRequest + * refresh policy */ @Override - protected abstract WriteReplicaResult shardOperationOnReplica( - ReplicaRequest request, IndexShard replica) throws Exception; + protected void shardOperationOnReplica(ReplicaRequest shardRequest, IndexShard replica, ActionListener listener) { + try { + listener.onResponse(dispatchedShardOperationOnReplica(shardRequest, replica)); + } catch (Exception e) { + listener.onFailure(e); + } + } + + protected abstract WriteReplicaResult dispatchedShardOperationOnReplica( + ReplicaRequest request, IndexShard replica) throws Exception; /** * Result of taking the action on the primary. diff --git a/server/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointSyncAction.java b/server/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointSyncAction.java index 0cdecbf4306dc..17d29575f23c7 100644 --- a/server/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointSyncAction.java +++ b/server/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointSyncAction.java @@ -90,9 +90,11 @@ protected void shardOperationOnPrimary(Request request, IndexShard indexShard, } @Override - protected ReplicaResult shardOperationOnReplica(final Request request, final IndexShard indexShard) throws Exception { - maybeSyncTranslog(indexShard); - return new ReplicaResult(); + protected void shardOperationOnReplica(Request shardRequest, IndexShard replica, ActionListener listener) { + ActionListener.completeWith(listener, () -> { + maybeSyncTranslog(replica); + return new ReplicaResult(); + }); } private void maybeSyncTranslog(final IndexShard indexShard) throws IOException { diff --git a/server/src/main/java/org/elasticsearch/index/seqno/RetentionLeaseSyncAction.java b/server/src/main/java/org/elasticsearch/index/seqno/RetentionLeaseSyncAction.java index 6b906792afc05..153947267c451 100644 --- a/server/src/main/java/org/elasticsearch/index/seqno/RetentionLeaseSyncAction.java +++ b/server/src/main/java/org/elasticsearch/index/seqno/RetentionLeaseSyncAction.java @@ -142,7 +142,7 @@ public void handleException(TransportException e) { } @Override - protected void shardOperationOnPrimary(Request request, IndexShard primary, + protected void dispatchedShardOperationOnPrimary(Request request, IndexShard primary, ActionListener> listener) { ActionListener.completeWith(listener, () -> { assert request.waitForActiveShards().equals(ActiveShardCount.NONE) : request.waitForActiveShards(); @@ -154,7 +154,7 @@ protected void shardOperationOnPrimary(Request request, IndexShard primary, } @Override - protected WriteReplicaResult shardOperationOnReplica( + protected WriteReplicaResult dispatchedShardOperationOnReplica( final Request request, final IndexShard replica) throws WriteStateException { Objects.requireNonNull(request); diff --git a/server/src/test/java/org/elasticsearch/action/support/replication/TransportReplicationAllPermitsAcquisitionTests.java b/server/src/test/java/org/elasticsearch/action/support/replication/TransportReplicationAllPermitsAcquisitionTests.java index e2ce61047c4d0..901bd28cb7591 100644 --- a/server/src/test/java/org/elasticsearch/action/support/replication/TransportReplicationAllPermitsAcquisitionTests.java +++ b/server/src/test/java/org/elasticsearch/action/support/replication/TransportReplicationAllPermitsAcquisitionTests.java @@ -446,14 +446,14 @@ protected void shardOperationOnPrimary(Request shardRequest, IndexShard shard, } @Override - protected ReplicaResult shardOperationOnReplica(Request shardRequest, IndexShard shard) throws Exception { + protected void shardOperationOnReplica(Request shardRequest, IndexShard replica, ActionListener listener) { 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); // The TransportReplicationAction.getIndexShard() method is overridden for testing purpose but we double check here // that the permit has been acquired on the replica shard - assertSame(replica, shard); - return new ReplicaResult(); + assertSame(replica, replica); + listener.onResponse(new ReplicaResult()); } } @@ -505,10 +505,10 @@ protected void shardOperationOnPrimary(Request shardRequest, IndexShard shard, } @Override - protected ReplicaResult shardOperationOnReplica(Request shardRequest, IndexShard shard) throws Exception { + protected void shardOperationOnReplica(Request shardRequest, IndexShard replica, ActionListener listener) { assertNoBlocks("block must not exist when executing the operation on replica shard: it should have been blocked before"); - assertThat(shard.getActiveOperationsCount(), greaterThan(0)); - return super.shardOperationOnReplica(shardRequest, shard); + assertThat(replica.getActiveOperationsCount(), greaterThan(0)); + super.shardOperationOnReplica(shardRequest, replica, listener); } private void assertNoBlocks(final String error) { @@ -551,9 +551,9 @@ protected void shardOperationOnPrimary(Request shardRequest, IndexShard shard, } @Override - protected ReplicaResult shardOperationOnReplica(Request shardRequest, IndexShard shard) throws Exception { - assertEquals("All permits must be acquired", IndexShard.OPERATIONS_BLOCKED, shard.getActiveOperationsCount()); - return super.shardOperationOnReplica(shardRequest, shard); + protected void shardOperationOnReplica(Request shardRequest, IndexShard replica, ActionListener listener) { + assertEquals("All permits must be acquired", IndexShard.OPERATIONS_BLOCKED, replica.getActiveOperationsCount()); + super.shardOperationOnReplica(shardRequest, replica, listener); } } diff --git a/server/src/test/java/org/elasticsearch/action/support/replication/TransportWriteActionTests.java b/server/src/test/java/org/elasticsearch/action/support/replication/TransportWriteActionTests.java index 42a696c3b2583..25b4066555efb 100644 --- a/server/src/test/java/org/elasticsearch/action/support/replication/TransportWriteActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/support/replication/TransportWriteActionTests.java @@ -153,7 +153,7 @@ public void testReplicaNoRefreshCall() throws Exception { request.setRefreshPolicy(RefreshPolicy.NONE); // The default, but we'll set it anyway just to be explicit TestAction testAction = new TestAction(); TransportWriteAction.WriteReplicaResult result = - testAction.shardOperationOnReplica(request, indexShard); + testAction.dispatchedShardOperationOnReplica(request, indexShard); CapturingActionListener listener = new CapturingActionListener<>(); result.runPostReplicaActions(ActionListener.map(listener, ignore -> TransportResponse.Empty.INSTANCE)); assertNotNull(listener.response); @@ -183,7 +183,7 @@ public void testReplicaImmediateRefresh() throws Exception { request.setRefreshPolicy(RefreshPolicy.IMMEDIATE); TestAction testAction = new TestAction(); TransportWriteAction.WriteReplicaResult result = - testAction.shardOperationOnReplica(request, indexShard); + testAction.dispatchedShardOperationOnReplica(request, indexShard); CapturingActionListener listener = new CapturingActionListener<>(); result.runPostReplicaActions(ActionListener.map(listener, ignore -> TransportResponse.Empty.INSTANCE)); assertNotNull(listener.response); @@ -221,7 +221,7 @@ public void testReplicaWaitForRefresh() throws Exception { TestRequest request = new TestRequest(); request.setRefreshPolicy(RefreshPolicy.WAIT_UNTIL); TestAction testAction = new TestAction(); - TransportWriteAction.WriteReplicaResult result = testAction.shardOperationOnReplica(request, indexShard); + TransportWriteAction.WriteReplicaResult result = testAction.dispatchedShardOperationOnReplica(request, indexShard); CapturingActionListener listener = new CapturingActionListener<>(); result.runPostReplicaActions(ActionListener.map(listener, ignore -> TransportResponse.Empty.INSTANCE)); assertNull(listener.response); // Haven't responded yet @@ -253,7 +253,7 @@ public void testDocumentFailureInShardOperationOnReplica() throws Exception { TestRequest request = new TestRequest(); TestAction testAction = new TestAction(randomBoolean(), true); TransportWriteAction.WriteReplicaResult result = - testAction.shardOperationOnReplica(request, indexShard); + testAction.dispatchedShardOperationOnReplica(request, indexShard); CapturingActionListener listener = new CapturingActionListener<>(); result.runPostReplicaActions(ActionListener.map(listener, ignore -> TransportResponse.Empty.INSTANCE)); assertNull(listener.response); @@ -381,7 +381,7 @@ protected TestResponse newResponseInstance(StreamInput in) throws IOException { } @Override - protected void shardOperationOnPrimary( + protected void dispatchedShardOperationOnPrimary( TestRequest request, IndexShard primary, ActionListener> listener) { ActionListener.completeWith(listener, () -> { if (withDocumentFailureOnPrimary) { @@ -393,7 +393,7 @@ protected void shardOperationOnPrimary( } @Override - protected WriteReplicaResult shardOperationOnReplica(TestRequest request, IndexShard replica) throws Exception { + protected WriteReplicaResult dispatchedShardOperationOnReplica(TestRequest request, IndexShard replica) { final WriteReplicaResult replicaResult; if (withDocumentFailureOnReplica) { replicaResult = new WriteReplicaResult<>(request, null, new RuntimeException("simulated"), replica, logger); diff --git a/server/src/test/java/org/elasticsearch/index/seqno/GlobalCheckpointSyncActionTests.java b/server/src/test/java/org/elasticsearch/index/seqno/GlobalCheckpointSyncActionTests.java index 8a1dc44a3e3c7..af2855a489a84 100644 --- a/server/src/test/java/org/elasticsearch/index/seqno/GlobalCheckpointSyncActionTests.java +++ b/server/src/test/java/org/elasticsearch/index/seqno/GlobalCheckpointSyncActionTests.java @@ -114,7 +114,8 @@ public void testTranslogSyncAfterGlobalCheckpointSync() throws Exception { if (randomBoolean()) { action.shardOperationOnPrimary(primaryRequest, indexShard, ActionTestUtils.assertNoFailureListener(r -> {})); } 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) { diff --git a/server/src/test/java/org/elasticsearch/index/seqno/RetentionLeaseSyncActionTests.java b/server/src/test/java/org/elasticsearch/index/seqno/RetentionLeaseSyncActionTests.java index 1999873464ddd..e2935fdf0cf00 100644 --- a/server/src/test/java/org/elasticsearch/index/seqno/RetentionLeaseSyncActionTests.java +++ b/server/src/test/java/org/elasticsearch/index/seqno/RetentionLeaseSyncActionTests.java @@ -104,7 +104,7 @@ public void testRetentionLeaseSyncActionOnPrimary() { new ActionFilters(Collections.emptySet())); final RetentionLeases retentionLeases = mock(RetentionLeases.class); final RetentionLeaseSyncAction.Request request = new RetentionLeaseSyncAction.Request(indexShard.shardId(), retentionLeases); - action.shardOperationOnPrimary(request, indexShard, + action.dispatchedShardOperationOnPrimary(request, indexShard, ActionTestUtils.assertNoFailureListener(result -> { // the retention leases on the shard should be persisted verify(indexShard).persistRetentionLeases(); @@ -142,7 +142,7 @@ public void testRetentionLeaseSyncActionOnReplica() throws WriteStateException { final RetentionLeaseSyncAction.Request request = new RetentionLeaseSyncAction.Request(indexShard.shardId(), retentionLeases); final TransportWriteAction.WriteReplicaResult result = - action.shardOperationOnReplica(request, indexShard); + action.dispatchedShardOperationOnReplica(request, indexShard); // the retention leases on the shard should be updated verify(indexShard).updateRetentionLeasesOnReplica(retentionLeases); // the retention leases on the shard should be persisted diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/bulk/TransportBulkShardOperationsAction.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/bulk/TransportBulkShardOperationsAction.java index 48f41ef527de1..323dfb85cada3 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/bulk/TransportBulkShardOperationsAction.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/bulk/TransportBulkShardOperationsAction.java @@ -59,7 +59,7 @@ public TransportBulkShardOperationsAction( } @Override - protected void shardOperationOnPrimary(BulkShardOperationsRequest request, IndexShard primary, + protected void dispatchedShardOperationOnPrimary(BulkShardOperationsRequest request, IndexShard primary, ActionListener> listener) { if (logger.isTraceEnabled()) { logger.trace("index [{}] on the following primary shard {}", request.getOperations(), primary.routingEntry()); @@ -158,7 +158,7 @@ public static WritePrimaryResult shardOperationOnReplica( + protected WriteReplicaResult dispatchedShardOperationOnReplica( final BulkShardOperationsRequest request, final IndexShard replica) throws Exception { if (logger.isTraceEnabled()) { logger.trace("index [{}] on the following replica shard {}", request.getOperations(), replica.routingEntry()); From 832dec023f53482336ab9454c29d7635b8daf9d8 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Tue, 19 May 2020 13:39:03 -0600 Subject: [PATCH 13/41] Changes --- ...tReplicationActionRetryOnClosedNodeIT.java | 4 +- .../flush/TransportShardFlushAction.java | 10 ++-- .../refresh/TransportShardRefreshAction.java | 11 ++-- .../TransportReplicationAction.java | 20 ++++---- .../RetentionLeaseBackgroundSyncAction.java | 15 +++--- .../TransportReplicationActionTests.java | 51 +++++++++++-------- ...tentionLeaseBackgroundSyncActionTests.java | 5 +- 7 files changed, 65 insertions(+), 51 deletions(-) diff --git a/server/src/internalClusterTest/java/org/elasticsearch/action/support/replication/TransportReplicationActionRetryOnClosedNodeIT.java b/server/src/internalClusterTest/java/org/elasticsearch/action/support/replication/TransportReplicationActionRetryOnClosedNodeIT.java index 227fc1581a730..75cf424df4628 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/action/support/replication/TransportReplicationActionRetryOnClosedNodeIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/action/support/replication/TransportReplicationActionRetryOnClosedNodeIT.java @@ -118,8 +118,8 @@ protected void shardOperationOnPrimary(Request shardRequest, IndexShard primary, } @Override - protected ReplicaResult shardOperationOnReplica(Request shardRequest, IndexShard replica) { - return new ReplicaResult(); + protected void shardOperationOnReplica(Request shardRequest, IndexShard replica, ActionListener listener) { + listener.onResponse(new ReplicaResult()); } } diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/flush/TransportShardFlushAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/flush/TransportShardFlushAction.java index 397ce43747d53..2c5fd39557a0f 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/flush/TransportShardFlushAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/flush/TransportShardFlushAction.java @@ -75,10 +75,12 @@ protected void shardOperationOnPrimary(ShardFlushRequest shardRequest, IndexShar } @Override - protected ReplicaResult shardOperationOnReplica(ShardFlushRequest request, IndexShard replica) { - replica.flush(request.getRequest()); - logger.trace("{} flush request executed on replica", replica.shardId()); - return new ReplicaResult(); + protected void shardOperationOnReplica(ShardFlushRequest request, IndexShard replica, ActionListener listener) { + ActionListener.completeWith(listener, () -> { + replica.flush(request.getRequest()); + logger.trace("{} flush request executed on replica", replica.shardId()); + return new ReplicaResult(); + }); } // TODO: Remove this transition in 9.0 diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/refresh/TransportShardRefreshAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/refresh/TransportShardRefreshAction.java index a0ce70503958c..3485028708d75 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/refresh/TransportShardRefreshAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/refresh/TransportShardRefreshAction.java @@ -68,9 +68,12 @@ protected void shardOperationOnPrimary(BasicReplicationRequest shardRequest, Ind } @Override - protected ReplicaResult shardOperationOnReplica(BasicReplicationRequest request, IndexShard replica) { - replica.refresh("api"); - logger.trace("{} refresh request executed on replica", replica.shardId()); - return new ReplicaResult(); + protected void shardOperationOnReplica(BasicReplicationRequest request, IndexShard replica, + ActionListener listener) { + ActionListener.completeWith(listener, () -> { + replica.refresh("api"); + logger.trace("{} refresh request executed on replica", replica.shardId()); + return new ReplicaResult(); + }); } } diff --git a/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java b/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java index 6754f8d971602..55e6a2997f468 100644 --- a/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java +++ b/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java @@ -213,23 +213,21 @@ protected void resolveRequest(final IndexMetadata indexMetadata, final Request r protected abstract void shardOperationOnPrimary(Request shardRequest, IndexShard primary, ActionListener> listener); + private ReplicaResult shardOperationOnReplica(ReplicaRequest shardRequest, IndexShard replica) throws Exception { + PlainActionFuture listener = PlainActionFuture.newFuture(); + shardOperationOnReplica(shardRequest, replica, listener); + return listener.actionGet(); + } + /** - * 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)}. * * @param shardRequest the request to the replica shard * @param replica the replica shard to perform the operation on */ - protected ReplicaResult shardOperationOnReplica(ReplicaRequest shardRequest, IndexShard replica) throws Exception { - PlainActionFuture listener = PlainActionFuture.newFuture(); - shardOperationOnReplica(shardRequest, replica, listener); - return listener.actionGet(); - } - - protected void shardOperationOnReplica(ReplicaRequest shardRequest, IndexShard replica, - ActionListener listener) { - - }; + protected abstract void shardOperationOnReplica(ReplicaRequest shardRequest, IndexShard replica, + ActionListener listener); /** * Cluster level block to check before request execution. Returning null means that no blocks need to be checked. diff --git a/server/src/main/java/org/elasticsearch/index/seqno/RetentionLeaseBackgroundSyncAction.java b/server/src/main/java/org/elasticsearch/index/seqno/RetentionLeaseBackgroundSyncAction.java index e4b82b3417551..75558dc423f57 100644 --- a/server/src/main/java/org/elasticsearch/index/seqno/RetentionLeaseBackgroundSyncAction.java +++ b/server/src/main/java/org/elasticsearch/index/seqno/RetentionLeaseBackgroundSyncAction.java @@ -38,7 +38,6 @@ import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.concurrent.ThreadContext; -import org.elasticsearch.gateway.WriteStateException; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.IndexShardClosedException; import org.elasticsearch.index.shard.ShardId; @@ -161,12 +160,14 @@ protected void shardOperationOnPrimary( } @Override - protected ReplicaResult shardOperationOnReplica(final Request request, final IndexShard replica) throws WriteStateException { - Objects.requireNonNull(request); - Objects.requireNonNull(replica); - replica.updateRetentionLeasesOnReplica(request.getRetentionLeases()); - replica.persistRetentionLeases(); - return new ReplicaResult(); + protected void shardOperationOnReplica(Request request, IndexShard replica, ActionListener listener) { + ActionListener.completeWith(listener, () -> { + Objects.requireNonNull(request); + Objects.requireNonNull(replica); + replica.updateRetentionLeasesOnReplica(request.getRetentionLeases()); + replica.persistRetentionLeases(); + return new ReplicaResult(); + }); } public static final class Request extends ReplicationRequest { diff --git a/server/src/test/java/org/elasticsearch/action/support/replication/TransportReplicationActionTests.java b/server/src/test/java/org/elasticsearch/action/support/replication/TransportReplicationActionTests.java index fd6cacf773932..25f27d2abfb3b 100644 --- a/server/src/test/java/org/elasticsearch/action/support/replication/TransportReplicationActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/support/replication/TransportReplicationActionTests.java @@ -921,14 +921,17 @@ public void testReplicasCounter() { final ReplicationTask task = maybeTask(); TestAction action = new TestAction(Settings.EMPTY, "internal:testActionWithExceptions", transportService, clusterService, shardStateAction, threadPool) { + @Override - protected ReplicaResult shardOperationOnReplica(Request request, IndexShard replica) { - assertIndexShardCounter(1); - assertPhase(task, "replica"); - if (throwException) { - throw new ElasticsearchException("simulated"); - } - return new ReplicaResult(); + protected void shardOperationOnReplica(Request shardRequest, IndexShard replica, ActionListener listener) { + ActionListener.completeWith(listener, () -> { + assertIndexShardCounter(1); + assertPhase(task, "replica"); + if (throwException) { + throw new ElasticsearchException("simulated"); + } + return new ReplicaResult(); + }); } }; try { @@ -1057,12 +1060,14 @@ public void testRetryOnReplica() throws Exception { TestAction action = new TestAction(Settings.EMPTY, "internal:testActionWithExceptions", transportService, clusterService, shardStateAction, threadPool) { @Override - protected ReplicaResult shardOperationOnReplica(Request request, IndexShard replica) { - assertPhase(task, "replica"); - if (throwException.get()) { - throw new RetryOnReplicaException(shardId, "simulation"); - } - return new ReplicaResult(); + protected void shardOperationOnReplica(Request shardRequest, IndexShard replica, ActionListener listener) { + ActionListener.completeWith(listener, () -> { + assertPhase(task, "replica"); + if (throwException.get()) { + throw new RetryOnReplicaException(shardId, "simulation"); + } + return new ReplicaResult(); + }); } }; final PlainActionFuture listener = new PlainActionFuture<>(); @@ -1124,13 +1129,15 @@ public void testRetryOnReplicaWithRealTransport() throws Exception { TestAction action = new TestAction(Settings.EMPTY, "internal:testActionWithExceptions", transportService, clusterService, shardStateAction, threadPool) { @Override - protected ReplicaResult shardOperationOnReplica(Request request, IndexShard replica) { - assertPhase(task, "replica"); - if (throwException.get()) { - throw new RetryOnReplicaException(shardId, "simulation"); - } - calledSuccessfully.set(true); - return new ReplicaResult(); + protected void shardOperationOnReplica(Request shardRequest, IndexShard replica, ActionListener listener) { + ActionListener.completeWith(listener, () -> { + assertPhase(task, "replica"); + if (throwException.get()) { + throw new RetryOnReplicaException(shardId, "simulation"); + } + calledSuccessfully.set(true); + return new ReplicaResult(); + }); } }; final PlainActionFuture listener = new PlainActionFuture<>(); @@ -1282,9 +1289,9 @@ protected void shardOperationOnPrimary(Request shardRequest, IndexShard primary, } @Override - protected ReplicaResult shardOperationOnReplica(Request request, IndexShard replica) { + protected void shardOperationOnReplica(Request request, IndexShard replica, ActionListener listener) { request.processedOnReplicas.incrementAndGet(); - return new ReplicaResult(); + listener.onResponse(new ReplicaResult()); } } diff --git a/server/src/test/java/org/elasticsearch/index/seqno/RetentionLeaseBackgroundSyncActionTests.java b/server/src/test/java/org/elasticsearch/index/seqno/RetentionLeaseBackgroundSyncActionTests.java index faccbc7ff482f..66cf37599c6c8 100644 --- a/server/src/test/java/org/elasticsearch/index/seqno/RetentionLeaseBackgroundSyncActionTests.java +++ b/server/src/test/java/org/elasticsearch/index/seqno/RetentionLeaseBackgroundSyncActionTests.java @@ -21,6 +21,7 @@ import org.elasticsearch.action.LatchedActionListener; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.ActionTestUtils; +import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.action.support.replication.TransportReplicationAction; import org.elasticsearch.cluster.action.shard.ShardStateAction; import org.elasticsearch.cluster.service.ClusterService; @@ -147,7 +148,9 @@ public void testRetentionLeaseBackgroundSyncActionOnReplica() throws WriteStateE final RetentionLeaseBackgroundSyncAction.Request request = new RetentionLeaseBackgroundSyncAction.Request(indexShard.shardId(), retentionLeases); - final TransportReplicationAction.ReplicaResult result = action.shardOperationOnReplica(request, indexShard); + final PlainActionFuture listener = PlainActionFuture.newFuture(); + action.shardOperationOnReplica(request, indexShard, listener); + final TransportReplicationAction.ReplicaResult result = listener.actionGet(); // the retention leases on the shard should be updated verify(indexShard).updateRetentionLeasesOnReplica(retentionLeases); // the retention leases on the shard should be persisted From 275df87635615769bffd679825a77ffc8e6d24f3 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Tue, 19 May 2020 13:50:34 -0600 Subject: [PATCH 14/41] WIP --- .../TransportReplicationAction.java | 27 +++++++------------ 1 file changed, 9 insertions(+), 18 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java b/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java index 55e6a2997f468..481b3686c490a 100644 --- a/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java +++ b/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java @@ -31,7 +31,6 @@ import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.ActiveShardCount; import org.elasticsearch.action.support.ChannelActionListener; -import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.action.support.TransportAction; import org.elasticsearch.action.support.TransportActions; import org.elasticsearch.client.transport.NoNodeAvailableException; @@ -213,12 +212,6 @@ protected void resolveRequest(final IndexMetadata indexMetadata, final Request r protected abstract void shardOperationOnPrimary(Request shardRequest, IndexShard primary, ActionListener> listener); - private ReplicaResult shardOperationOnReplica(ReplicaRequest shardRequest, IndexShard replica) throws Exception { - PlainActionFuture listener = PlainActionFuture.newFuture(); - shardOperationOnReplica(shardRequest, replica, listener); - return listener.actionGet(); - } - /** * Execute the specified replica operation. This is done under a permit from * {@link IndexShard#acquireReplicaOperationPermit(long, long, long, ActionListener, String, Object)}. @@ -547,30 +540,28 @@ private final class AsyncReplicaAction extends AbstractRunnable implements Actio @Override public void onResponse(Releasable releasable) { - try { - assert replica.getActiveOperationsCount() != 0 : "must perform shard operation under a permit"; - final ReplicaResult replicaResult = shardOperationOnReplica(replicaRequest.getRequest(), replica); + assert replica.getActiveOperationsCount() != 0 : "must perform shard operation under a permit"; + shardOperationOnReplica(replicaRequest.getRequest(), replica, ActionListener.wrap((replicaResult) -> replicaResult.runPostReplicaActions( ActionListener.wrap(r -> { - final TransportReplicationAction.ReplicaResponse response = + final ReplicaResponse response = new ReplicaResponse(replica.getLocalCheckpoint(), replica.getLastSyncedGlobalCheckpoint()); releasable.close(); // release shard operation lock before responding to caller if (logger.isTraceEnabled()) { logger.trace("action [{}] completed on shard [{}] for request [{}]", transportReplicaAction, - replicaRequest.getRequest().shardId(), - replicaRequest.getRequest()); + replicaRequest.getRequest().shardId(), + replicaRequest.getRequest()); } setPhase(task, "finished"); onCompletionListener.onResponse(response); - }, e -> { + }, e -> { Releasables.closeWhileHandlingException(releasable); // release shard operation lock before responding to caller - this.responseWithFailure(e); + responseWithFailure(e); }) - ); - } catch (final Exception e) { + ), e -> { Releasables.closeWhileHandlingException(releasable); // release shard operation lock before responding to caller AsyncReplicaAction.this.onFailure(e); - } + })); } @Override From 87af5561c04854509d74660d79037615d1870dc4 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Tue, 19 May 2020 14:18:43 -0600 Subject: [PATCH 15/41] Changes --- .../action/bulk/TransportShardBulkAction.java | 13 ++++--- .../TransportResyncReplicationAction.java | 10 +++-- .../replication/TransportWriteAction.java | 30 ++++++++++----- .../index/seqno/RetentionLeaseSyncAction.java | 18 ++++----- .../TransportWriteActionTests.java | 37 +++++++++++-------- .../seqno/RetentionLeaseSyncActionTests.java | 8 ++-- .../TransportBulkShardOperationsAction.java | 14 ++++--- 7 files changed, 77 insertions(+), 53 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java b/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java index 8bb7a76a894d4..a7c0ccbca8ebb 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java @@ -429,12 +429,13 @@ protected Releasable checkReplicaLimits(BulkShardRequest request) throws EsRejec } @Override - public WriteReplicaResult dispatchedShardOperationOnReplica(BulkShardRequest request, IndexShard replica) - throws Exception { - final long startBulkTime = System.nanoTime(); - final Translog.Location location = performOnReplica(request, replica); - replica.getBulkOperationListener().afterBulk(request.totalSizeInBytes(), System.nanoTime() - startBulkTime); - return new WriteReplicaResult<>(request, location, null, replica, logger); + protected void dispatchedShardOperationOnReplica(BulkShardRequest request, IndexShard replica, ActionListener listener) { + ActionListener.completeWith(listener, () -> { + final long startBulkTime = System.nanoTime(); + final Translog.Location location = performOnReplica(request, replica); + replica.getBulkOperationListener().afterBulk(request.totalSizeInBytes(), System.nanoTime() - startBulkTime); + return new WriteReplicaResult<>(request, location, null, replica, logger); + }); } public static Translog.Location performOnReplica(BulkShardRequest request, IndexShard replica) throws Exception { diff --git a/server/src/main/java/org/elasticsearch/action/resync/TransportResyncReplicationAction.java b/server/src/main/java/org/elasticsearch/action/resync/TransportResyncReplicationAction.java index 64abdf4301a82..7fb352c955da6 100644 --- a/server/src/main/java/org/elasticsearch/action/resync/TransportResyncReplicationAction.java +++ b/server/src/main/java/org/elasticsearch/action/resync/TransportResyncReplicationAction.java @@ -94,10 +94,12 @@ public static ResyncReplicationRequest performOnPrimary(ResyncReplicationRequest } @Override - protected WriteReplicaResult dispatchedShardOperationOnReplica(ResyncReplicationRequest request, - IndexShard replica) throws Exception { - Translog.Location location = performOnReplica(request, replica); - return new WriteReplicaResult<>(request, location, null, replica, logger); + protected void dispatchedShardOperationOnReplica(ResyncReplicationRequest request, IndexShard replica, + ActionListener listener) { + ActionListener.completeWith(listener, () -> { + Translog.Location location = performOnReplica(request, replica); + return new WriteReplicaResult<>(request, location, null, replica, logger); + }); } public static Translog.Location performOnReplica(ResyncReplicationRequest request, IndexShard replica) throws Exception { diff --git a/server/src/main/java/org/elasticsearch/action/support/replication/TransportWriteAction.java b/server/src/main/java/org/elasticsearch/action/support/replication/TransportWriteAction.java index afbaf7de2db07..a099d8591c989 100644 --- a/server/src/main/java/org/elasticsearch/action/support/replication/TransportWriteAction.java +++ b/server/src/main/java/org/elasticsearch/action/support/replication/TransportWriteAction.java @@ -22,6 +22,7 @@ import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.ActionRunnable; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.TransportActions; import org.elasticsearch.action.support.WriteRequest; @@ -57,12 +58,15 @@ public abstract class TransportWriteAction< Response extends ReplicationResponse & WriteResponse > extends TransportReplicationAction { + private final String executor; + protected TransportWriteAction(Settings settings, String actionName, TransportService transportService, ClusterService clusterService, IndicesService indicesService, ThreadPool threadPool, ShardStateAction shardStateAction, ActionFilters actionFilters, Writeable.Reader request, Writeable.Reader replicaRequest, String executor, boolean forceExecutionOnPrimary) { super(settings, actionName, transportService, clusterService, indicesService, threadPool, shardStateAction, actionFilters, - request, replicaRequest, executor, true, forceExecutionOnPrimary); + request, replicaRequest, ThreadPool.Names.SAME, true, forceExecutionOnPrimary); + this.executor = executor; } /** Syncs operation result to the translog or throws a shard not available failure */ @@ -106,7 +110,12 @@ protected ReplicationOperation.Replicas newReplicasProxy() { @Override protected void shardOperationOnPrimary( Request request, IndexShard primary, ActionListener> listener) { - dispatchedShardOperationOnPrimary(request, primary, listener); + threadPool.executor(executor).execute(new ActionRunnable<>(listener) { + @Override + protected void doRun() { + dispatchedShardOperationOnPrimary(request, primary, listener); + } + }); } protected abstract void dispatchedShardOperationOnPrimary( @@ -120,16 +129,17 @@ protected abstract void dispatchedShardOperationOnPrimary( * refresh policy */ @Override - protected void shardOperationOnReplica(ReplicaRequest shardRequest, IndexShard replica, ActionListener listener) { - try { - listener.onResponse(dispatchedShardOperationOnReplica(shardRequest, replica)); - } catch (Exception e) { - listener.onFailure(e); - } + protected void shardOperationOnReplica(ReplicaRequest request, IndexShard replica, ActionListener listener) { + threadPool.executor(executor).execute(new ActionRunnable<>(listener) { + @Override + protected void doRun() { + dispatchedShardOperationOnReplica(request, replica, listener); + } + }); } - protected abstract WriteReplicaResult dispatchedShardOperationOnReplica( - ReplicaRequest request, IndexShard replica) throws Exception; + protected abstract void dispatchedShardOperationOnReplica( + ReplicaRequest request, IndexShard replica, ActionListener listener); /** * Result of taking the action on the primary. diff --git a/server/src/main/java/org/elasticsearch/index/seqno/RetentionLeaseSyncAction.java b/server/src/main/java/org/elasticsearch/index/seqno/RetentionLeaseSyncAction.java index 153947267c451..44d6eca94092c 100644 --- a/server/src/main/java/org/elasticsearch/index/seqno/RetentionLeaseSyncAction.java +++ b/server/src/main/java/org/elasticsearch/index/seqno/RetentionLeaseSyncAction.java @@ -40,7 +40,6 @@ import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.concurrent.ThreadContext; -import org.elasticsearch.gateway.WriteStateException; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.IndexShardClosedException; import org.elasticsearch.index.shard.ShardId; @@ -154,14 +153,15 @@ protected void dispatchedShardOperationOnPrimary(Request request, IndexShard pri } @Override - protected WriteReplicaResult dispatchedShardOperationOnReplica( - final Request request, - final IndexShard replica) throws WriteStateException { - Objects.requireNonNull(request); - Objects.requireNonNull(replica); - replica.updateRetentionLeasesOnReplica(request.getRetentionLeases()); - replica.persistRetentionLeases(); - return new WriteReplicaResult<>(request, null, null, replica, getLogger()); + protected void dispatchedShardOperationOnReplica(Request request, IndexShard replica, + ActionListener listener) { + ActionListener.completeWith(listener, () -> { + Objects.requireNonNull(request); + Objects.requireNonNull(replica); + replica.updateRetentionLeasesOnReplica(request.getRetentionLeases()); + replica.persistRetentionLeases(); + return new WriteReplicaResult<>(request, null, null, replica, getLogger()); + }); } @Override diff --git a/server/src/test/java/org/elasticsearch/action/support/replication/TransportWriteActionTests.java b/server/src/test/java/org/elasticsearch/action/support/replication/TransportWriteActionTests.java index 25b4066555efb..bcddef3e240ed 100644 --- a/server/src/test/java/org/elasticsearch/action/support/replication/TransportWriteActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/support/replication/TransportWriteActionTests.java @@ -152,8 +152,9 @@ public void testReplicaNoRefreshCall() throws Exception { TestRequest request = new TestRequest(); request.setRefreshPolicy(RefreshPolicy.NONE); // The default, but we'll set it anyway just to be explicit TestAction testAction = new TestAction(); - TransportWriteAction.WriteReplicaResult result = - testAction.dispatchedShardOperationOnReplica(request, indexShard); + final PlainActionFuture future = PlainActionFuture.newFuture(); + testAction.dispatchedShardOperationOnReplica(request, indexShard, future); + final TransportReplicationAction.ReplicaResult result = future.actionGet(); CapturingActionListener listener = new CapturingActionListener<>(); result.runPostReplicaActions(ActionListener.map(listener, ignore -> TransportResponse.Empty.INSTANCE)); assertNotNull(listener.response); @@ -182,8 +183,9 @@ public void testReplicaImmediateRefresh() throws Exception { TestRequest request = new TestRequest(); request.setRefreshPolicy(RefreshPolicy.IMMEDIATE); TestAction testAction = new TestAction(); - TransportWriteAction.WriteReplicaResult result = - testAction.dispatchedShardOperationOnReplica(request, indexShard); + final PlainActionFuture future = PlainActionFuture.newFuture(); + testAction.dispatchedShardOperationOnReplica(request, indexShard, future); + final TransportReplicationAction.ReplicaResult result = future.actionGet(); CapturingActionListener listener = new CapturingActionListener<>(); result.runPostReplicaActions(ActionListener.map(listener, ignore -> TransportResponse.Empty.INSTANCE)); assertNotNull(listener.response); @@ -221,7 +223,9 @@ public void testReplicaWaitForRefresh() throws Exception { TestRequest request = new TestRequest(); request.setRefreshPolicy(RefreshPolicy.WAIT_UNTIL); TestAction testAction = new TestAction(); - TransportWriteAction.WriteReplicaResult result = testAction.dispatchedShardOperationOnReplica(request, indexShard); + final PlainActionFuture future = PlainActionFuture.newFuture(); + testAction.dispatchedShardOperationOnReplica(request, indexShard, future); + final TransportReplicationAction.ReplicaResult result = future.actionGet(); CapturingActionListener listener = new CapturingActionListener<>(); result.runPostReplicaActions(ActionListener.map(listener, ignore -> TransportResponse.Empty.INSTANCE)); assertNull(listener.response); // Haven't responded yet @@ -252,8 +256,9 @@ public void testDocumentFailureInShardOperationOnPrimary() throws Exception { public void testDocumentFailureInShardOperationOnReplica() throws Exception { TestRequest request = new TestRequest(); TestAction testAction = new TestAction(randomBoolean(), true); - TransportWriteAction.WriteReplicaResult result = - testAction.dispatchedShardOperationOnReplica(request, indexShard); + final PlainActionFuture future = PlainActionFuture.newFuture(); + testAction.dispatchedShardOperationOnReplica(request, indexShard, future); + final TransportReplicationAction.ReplicaResult result = future.actionGet(); CapturingActionListener listener = new CapturingActionListener<>(); result.runPostReplicaActions(ActionListener.map(listener, ignore -> TransportResponse.Empty.INSTANCE)); assertNull(listener.response); @@ -393,14 +398,16 @@ protected void dispatchedShardOperationOnPrimary( } @Override - protected WriteReplicaResult dispatchedShardOperationOnReplica(TestRequest request, IndexShard replica) { - final WriteReplicaResult replicaResult; - if (withDocumentFailureOnReplica) { - replicaResult = new WriteReplicaResult<>(request, null, new RuntimeException("simulated"), replica, logger); - } else { - replicaResult = new WriteReplicaResult<>(request, location, null, replica, logger); - } - return replicaResult; + protected void dispatchedShardOperationOnReplica(TestRequest request, IndexShard replica, ActionListener listener) { + ActionListener.completeWith(listener, () -> { + final WriteReplicaResult replicaResult; + if (withDocumentFailureOnReplica) { + replicaResult = new WriteReplicaResult<>(request, null, new RuntimeException("simulated"), replica, logger); + } else { + replicaResult = new WriteReplicaResult<>(request, location, null, replica, logger); + } + return replicaResult; + }); } } diff --git a/server/src/test/java/org/elasticsearch/index/seqno/RetentionLeaseSyncActionTests.java b/server/src/test/java/org/elasticsearch/index/seqno/RetentionLeaseSyncActionTests.java index e2935fdf0cf00..797925ab389b4 100644 --- a/server/src/test/java/org/elasticsearch/index/seqno/RetentionLeaseSyncActionTests.java +++ b/server/src/test/java/org/elasticsearch/index/seqno/RetentionLeaseSyncActionTests.java @@ -20,7 +20,8 @@ import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.support.ActionFilters; 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.service.ClusterService; import org.elasticsearch.common.settings.Settings; @@ -141,8 +142,9 @@ public void testRetentionLeaseSyncActionOnReplica() throws WriteStateException { final RetentionLeases retentionLeases = mock(RetentionLeases.class); final RetentionLeaseSyncAction.Request request = new RetentionLeaseSyncAction.Request(indexShard.shardId(), retentionLeases); - final TransportWriteAction.WriteReplicaResult result = - action.dispatchedShardOperationOnReplica(request, indexShard); + PlainActionFuture listener = PlainActionFuture.newFuture(); + action.dispatchedShardOperationOnReplica(request, indexShard, listener); + final TransportReplicationAction.ReplicaResult result = listener.actionGet(); // the retention leases on the shard should be updated verify(indexShard).updateRetentionLeasesOnReplica(retentionLeases); // the retention leases on the shard should be persisted diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/bulk/TransportBulkShardOperationsAction.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/bulk/TransportBulkShardOperationsAction.java index 323dfb85cada3..e00e0cc3a5a9e 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/bulk/TransportBulkShardOperationsAction.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/bulk/TransportBulkShardOperationsAction.java @@ -158,12 +158,14 @@ public static WritePrimaryResult dispatchedShardOperationOnReplica( - final BulkShardOperationsRequest request, final IndexShard replica) throws Exception { - if (logger.isTraceEnabled()) { - logger.trace("index [{}] on the following replica shard {}", request.getOperations(), replica.routingEntry()); - } - return shardOperationOnReplica(request, replica, logger); + protected void dispatchedShardOperationOnReplica(BulkShardOperationsRequest request, IndexShard replica, + ActionListener listener) { + ActionListener.completeWith(listener, () -> { + if (logger.isTraceEnabled()) { + logger.trace("index [{}] on the following replica shard {}", request.getOperations(), replica.routingEntry()); + } + return shardOperationOnReplica(request, replica, logger); + }); } // public for testing purposes only From 185c24cf85126e3f714c179788f6d281a19571d9 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Tue, 26 May 2020 16:18:22 -0600 Subject: [PATCH 16/41] Changes --- .../TransportReplicationAction.java | 39 ++++++++++++++----- 1 file changed, 30 insertions(+), 9 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java b/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java index 481b3686c490a..b571eb8c4ae3c 100644 --- a/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java +++ b/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java @@ -27,6 +27,7 @@ import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionListenerResponseHandler; import org.elasticsearch.action.ActionResponse; +import org.elasticsearch.action.ActionRunnable; import org.elasticsearch.action.UnavailableShardsException; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.ActiveShardCount; @@ -160,11 +161,11 @@ protected TransportReplicationAction(Settings settings, String actionName, Trans transportService.registerRequestHandler(actionName, ThreadPool.Names.SAME, requestReader, this::handleOperationRequest); - transportService.registerRequestHandler(transportPrimaryAction, executor, forceExecutionOnPrimary, true, + transportService.registerRequestHandler(transportPrimaryAction, ThreadPool.Names.SAME, forceExecutionOnPrimary, true, in -> new ConcreteShardRequest<>(requestReader, in), this::handlePrimaryRequest); // we must never reject on because of thread pool capacity on replicas - transportService.registerRequestHandler(transportReplicaAction, executor, true, true, + transportService.registerRequestHandler(transportReplicaAction, ThreadPool.Names.SAME, true, true, in -> new ConcreteReplicaRequest<>(replicaRequestReader, in), this::handleReplicaRequest); this.transportOptions = transportOptions(settings); @@ -274,12 +275,23 @@ boolean isRetryableClusterBlockException(final Throwable e) { } protected void handleOperationRequest(final Request request, final TransportChannel channel, Task task) { - execute(task, request, new ChannelActionListener<>(channel, actionName, request)); + Releasable releasable = checkPrimaryLimits(request); + ActionListener listener = + ActionListener.runAfter(new ChannelActionListener<>(channel, actionName, request), releasable::close); + execute(task, request, listener); } protected void handlePrimaryRequest(final ConcreteShardRequest request, final TransportChannel channel, final Task task) { - new AsyncPrimaryAction( - request, new ChannelActionListener<>(channel, transportPrimaryAction, request), (ReplicationTask) task).run(); + Releasable releasable = checkPrimaryLimits(request.getRequest()); + ActionListener listener = + ActionListener.runAfter(new ChannelActionListener<>(channel, transportPrimaryAction, request), releasable::close); + + threadPool.executor(executor).execute(new ActionRunnable<>(listener) { + @Override + protected void doRun() { + new AsyncPrimaryAction(request, listener, (ReplicationTask) task).run(); + } + }); } protected Releasable checkPrimaryLimits(final Request request) { @@ -494,10 +506,19 @@ public void runPostReplicaActions(ActionListener listener) { } } - protected void handleReplicaRequest(final ConcreteReplicaRequest replicaRequest, - final TransportChannel channel, final Task task) { - new AsyncReplicaAction( - replicaRequest, new ChannelActionListener<>(channel, transportReplicaAction, replicaRequest), (ReplicationTask) task).run(); + protected void handleReplicaRequest(final ConcreteReplicaRequest replicaRequest, final TransportChannel channel, + final Task task) { + Releasable releasable = checkReplicaLimits(replicaRequest.getRequest()); + ActionListener listener = + ActionListener.runAfter(new ChannelActionListener<>(channel, transportReplicaAction, replicaRequest), releasable::close); + + threadPool.executor(executor).execute(new ActionRunnable<>(listener) { + @Override + protected void doRun() { + new AsyncReplicaAction( + replicaRequest, listener, (ReplicationTask) task).run(); + } + }); } protected Releasable checkReplicaLimits(final ReplicaRequest request) { From eaa6eaadef67ed1fdfdf2030c233d7a032f36495 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Fri, 29 May 2020 15:40:30 -0600 Subject: [PATCH 17/41] Change --- ...LimitsIT.java => WriteMemoryLimitsIT.java} | 2 +- .../action/bulk/TransportShardBulkAction.java | 24 ++----------- ...moryLimits.java => WriteMemoryLimits.java} | 8 ++--- .../TransportResyncReplicationAction.java | 7 ++-- .../replication/TransportWriteAction.java | 35 +++++++++++++++++-- .../common/settings/ClusterSettings.java | 4 +-- .../index/seqno/RetentionLeaseSyncAction.java | 6 ++-- .../java/org/elasticsearch/node/Node.java | 6 ++-- ...TransportResyncReplicationActionTests.java | 5 ++- .../TransportWriteActionTests.java | 8 +++-- .../seqno/RetentionLeaseSyncActionTests.java | 11 ++++-- .../snapshots/SnapshotResiliencyTests.java | 7 ++-- .../TransportBulkShardOperationsAction.java | 6 ++-- 13 files changed, 80 insertions(+), 49 deletions(-) rename server/src/internalClusterTest/java/org/elasticsearch/action/bulk/{BulkIndexingMemoryLimitsIT.java => WriteMemoryLimitsIT.java} (98%) rename server/src/main/java/org/elasticsearch/action/bulk/{BulkIndexingMemoryLimits.java => WriteMemoryLimits.java} (90%) diff --git a/server/src/internalClusterTest/java/org/elasticsearch/action/bulk/BulkIndexingMemoryLimitsIT.java b/server/src/internalClusterTest/java/org/elasticsearch/action/bulk/WriteMemoryLimitsIT.java similarity index 98% rename from server/src/internalClusterTest/java/org/elasticsearch/action/bulk/BulkIndexingMemoryLimitsIT.java rename to server/src/internalClusterTest/java/org/elasticsearch/action/bulk/WriteMemoryLimitsIT.java index 4c5b998bddc0b..4c62ede35a165 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/action/bulk/BulkIndexingMemoryLimitsIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/action/bulk/WriteMemoryLimitsIT.java @@ -39,7 +39,7 @@ import static org.hamcrest.Matchers.lessThan; @ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.SUITE, numDataNodes = 2) -public class BulkIndexingMemoryLimitsIT extends ESIntegTestCase { +public class WriteMemoryLimitsIT extends ESIntegTestCase { @Override protected Settings nodeSettings(int nodeOrdinal) { diff --git a/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java b/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java index a7c0ccbca8ebb..c099e2027d097 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java @@ -51,10 +51,8 @@ import org.elasticsearch.common.compress.CompressedXContent; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.io.stream.StreamInput; -import org.elasticsearch.common.lease.Releasable; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; -import org.elasticsearch.common.util.concurrent.EsRejectedExecutionException; import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.common.xcontent.XContentType; @@ -91,18 +89,16 @@ public class TransportShardBulkAction extends TransportWriteAction indexingMemoryLimits.markPrimaryOperationFinished(operationSizeInBytes); - } - @Override protected void dispatchedShardOperationOnPrimary(BulkShardRequest request, IndexShard primary, ActionListener> listener) { @@ -420,14 +408,6 @@ private static BulkItemResponse processUpdateResponse(final UpdateRequest update return response; } - @Override - protected Releasable checkReplicaLimits(BulkShardRequest request) throws EsRejectedExecutionException { - super.checkReplicaLimits(request); - long operationSizeInBytes = operationSizeInBytes(request.items()); - indexingMemoryLimits.markReplicaOperationStarted(operationSizeInBytes); - return () -> indexingMemoryLimits.markReplicaOperationFinished(operationSizeInBytes); - } - @Override protected void dispatchedShardOperationOnReplica(BulkShardRequest request, IndexShard replica, ActionListener listener) { ActionListener.completeWith(listener, () -> { diff --git a/server/src/main/java/org/elasticsearch/action/bulk/BulkIndexingMemoryLimits.java b/server/src/main/java/org/elasticsearch/action/bulk/WriteMemoryLimits.java similarity index 90% rename from server/src/main/java/org/elasticsearch/action/bulk/BulkIndexingMemoryLimits.java rename to server/src/main/java/org/elasticsearch/action/bulk/WriteMemoryLimits.java index 6a1e388d4612c..dff39f0b9ae89 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/BulkIndexingMemoryLimits.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/WriteMemoryLimits.java @@ -27,7 +27,7 @@ import java.util.concurrent.atomic.AtomicLong; -public class BulkIndexingMemoryLimits { +public class WriteMemoryLimits { public static final Setting MAX_INDEXING_BYTES = Setting.memorySizeSetting("indices.indexing_bytes.limit", "20%", Setting.Property.NodeScope, Setting.Property.Dynamic); @@ -36,15 +36,15 @@ public class BulkIndexingMemoryLimits { private final AtomicLong pendingBytes = new AtomicLong(0); - public BulkIndexingMemoryLimits(Settings settings, ClusterSettings clusterSettings) { + public WriteMemoryLimits(Settings settings, ClusterSettings clusterSettings) { this.indexingLimits = MAX_INDEXING_BYTES.get(settings).getBytes(); clusterSettings.addSettingsUpdateConsumer(MAX_INDEXING_BYTES, value -> indexingLimits = value.getBytes()); } - public void markPrimaryOperationStarted(long bytes) { + public void markPrimaryOperationStarted(long bytes, boolean forceExecution) { long pendingWithOperation = pendingBytes.addAndGet(bytes); - if (pendingWithOperation > indexingLimits) { + if (forceExecution== false && pendingWithOperation > indexingLimits) { decrementPendingBytes(bytes); long pendingPreOperation = pendingWithOperation - bytes; throw new EsRejectedExecutionException("rejected execution of primary shard operation [" + diff --git a/server/src/main/java/org/elasticsearch/action/resync/TransportResyncReplicationAction.java b/server/src/main/java/org/elasticsearch/action/resync/TransportResyncReplicationAction.java index 7fb352c955da6..044f12c315b12 100644 --- a/server/src/main/java/org/elasticsearch/action/resync/TransportResyncReplicationAction.java +++ b/server/src/main/java/org/elasticsearch/action/resync/TransportResyncReplicationAction.java @@ -20,6 +20,7 @@ import org.apache.logging.log4j.message.ParameterizedMessage; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.bulk.WriteMemoryLimits; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.replication.ReplicationOperation; import org.elasticsearch.action.support.replication.ReplicationResponse; @@ -54,10 +55,12 @@ public class TransportResyncReplicationAction extends TransportWriteAction extends TransportReplicationAction { + private final WriteMemoryLimits writeMemoryLimits; private final String executor; + private final boolean forceExecutionOnPrimary; protected TransportWriteAction(Settings settings, String actionName, TransportService transportService, ClusterService clusterService, IndicesService indicesService, ThreadPool threadPool, ShardStateAction shardStateAction, ActionFilters actionFilters, Writeable.Reader request, - Writeable.Reader replicaRequest, String executor, boolean forceExecutionOnPrimary) { + Writeable.Reader replicaRequest, String executor, boolean forceExecutionOnPrimary, + WriteMemoryLimits writeMemoryLimits) { super(settings, actionName, transportService, clusterService, indicesService, threadPool, shardStateAction, actionFilters, - request, replicaRequest, ThreadPool.Names.SAME, true, forceExecutionOnPrimary); + request, replicaRequest, ThreadPool.Names.SAME, true, forceExecutionOnPrimary); this.executor = executor; + this.writeMemoryLimits = writeMemoryLimits; + this.forceExecutionOnPrimary = forceExecutionOnPrimary; + } + + @Override + protected Releasable checkPrimaryLimits(Request request) { + super.checkPrimaryLimits(request); + long operationSizeInBytes = primaryOperationSize(request); + writeMemoryLimits.markPrimaryOperationStarted(operationSizeInBytes, forceExecutionOnPrimary); + return () -> writeMemoryLimits.markPrimaryOperationFinished(operationSizeInBytes); + } + + protected long primaryOperationSize(Request request) { + return 4096; + } + + @Override + protected Releasable checkReplicaLimits(ReplicaRequest request) { + super.checkReplicaLimits(request); + long operationSizeInBytes = replicaOperationSize(request); + writeMemoryLimits.markReplicaOperationStarted(operationSizeInBytes); + return () -> writeMemoryLimits.markReplicaOperationFinished(operationSizeInBytes); + } + + protected long replicaOperationSize(ReplicaRequest request) { + return 4096; } /** Syncs operation result to the translog or throws a shard not available failure */ diff --git a/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java b/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java index e81b853ae3aaf..c8c12f79931b2 100644 --- a/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java +++ b/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java @@ -21,7 +21,7 @@ import org.apache.logging.log4j.LogManager; import org.elasticsearch.action.admin.cluster.configuration.TransportAddVotingConfigExclusionsAction; import org.elasticsearch.action.admin.indices.close.TransportCloseIndexAction; -import org.elasticsearch.action.bulk.BulkIndexingMemoryLimits; +import org.elasticsearch.action.bulk.WriteMemoryLimits; import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.action.support.AutoCreateIndex; import org.elasticsearch.action.support.DestructiveOperations; @@ -190,7 +190,7 @@ public void apply(Settings value, Settings current, Settings previous) { BalancedShardsAllocator.INDEX_BALANCE_FACTOR_SETTING, BalancedShardsAllocator.SHARD_BALANCE_FACTOR_SETTING, BalancedShardsAllocator.THRESHOLD_SETTING, - BulkIndexingMemoryLimits.MAX_INDEXING_BYTES, + WriteMemoryLimits.MAX_INDEXING_BYTES, BreakerSettings.CIRCUIT_BREAKER_LIMIT_SETTING, BreakerSettings.CIRCUIT_BREAKER_OVERHEAD_SETTING, BreakerSettings.CIRCUIT_BREAKER_TYPE, diff --git a/server/src/main/java/org/elasticsearch/index/seqno/RetentionLeaseSyncAction.java b/server/src/main/java/org/elasticsearch/index/seqno/RetentionLeaseSyncAction.java index 44d6eca94092c..4833a1a07631a 100644 --- a/server/src/main/java/org/elasticsearch/index/seqno/RetentionLeaseSyncAction.java +++ b/server/src/main/java/org/elasticsearch/index/seqno/RetentionLeaseSyncAction.java @@ -25,6 +25,7 @@ import org.apache.lucene.store.AlreadyClosedException; import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.bulk.WriteMemoryLimits; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.ActiveShardCount; import org.elasticsearch.action.support.WriteResponse; @@ -77,7 +78,8 @@ public RetentionLeaseSyncAction( final IndicesService indicesService, final ThreadPool threadPool, final ShardStateAction shardStateAction, - final ActionFilters actionFilters) { + final ActionFilters actionFilters, + final WriteMemoryLimits writeMemoryLimits) { super( settings, ACTION_NAME, @@ -89,7 +91,7 @@ public RetentionLeaseSyncAction( actionFilters, RetentionLeaseSyncAction.Request::new, RetentionLeaseSyncAction.Request::new, - ThreadPool.Names.MANAGEMENT, false); + ThreadPool.Names.MANAGEMENT, false, writeMemoryLimits); } @Override diff --git a/server/src/main/java/org/elasticsearch/node/Node.java b/server/src/main/java/org/elasticsearch/node/Node.java index 9ba0094799df4..6720590654fda 100644 --- a/server/src/main/java/org/elasticsearch/node/Node.java +++ b/server/src/main/java/org/elasticsearch/node/Node.java @@ -30,7 +30,7 @@ import org.elasticsearch.Version; import org.elasticsearch.action.ActionModule; import org.elasticsearch.action.ActionType; -import org.elasticsearch.action.bulk.BulkIndexingMemoryLimits; +import org.elasticsearch.action.bulk.WriteMemoryLimits; import org.elasticsearch.action.search.SearchExecutionStatsCollector; import org.elasticsearch.action.search.SearchPhaseController; import org.elasticsearch.action.search.SearchTransportService; @@ -565,7 +565,7 @@ protected Node(final Environment initialEnvironment, new PersistentTasksClusterService(settings, registry, clusterService, threadPool); resourcesToClose.add(persistentTasksClusterService); final PersistentTasksService persistentTasksService = new PersistentTasksService(clusterService, threadPool, client); - final BulkIndexingMemoryLimits bulkIndexingLimits = new BulkIndexingMemoryLimits(settings, clusterService.getClusterSettings()); + final WriteMemoryLimits bulkIndexingLimits = new WriteMemoryLimits(settings, clusterService.getClusterSettings()); modules.add(b -> { b.bind(Node.class).toInstance(this); @@ -584,7 +584,7 @@ protected Node(final Environment initialEnvironment, b.bind(ScriptService.class).toInstance(scriptService); b.bind(AnalysisRegistry.class).toInstance(analysisModule.getAnalysisRegistry()); b.bind(IngestService.class).toInstance(ingestService); - b.bind(BulkIndexingMemoryLimits.class).toInstance(bulkIndexingLimits); + b.bind(WriteMemoryLimits.class).toInstance(bulkIndexingLimits); b.bind(UsageService.class).toInstance(usageService); b.bind(AggregationUsageService.class).toInstance(searchModule.getValuesSourceRegistry().getUsageService()); b.bind(NamedWriteableRegistry.class).toInstance(namedWriteableRegistry); diff --git a/server/src/test/java/org/elasticsearch/action/resync/TransportResyncReplicationActionTests.java b/server/src/test/java/org/elasticsearch/action/resync/TransportResyncReplicationActionTests.java index cfb94c3465dc5..5fc382737566c 100644 --- a/server/src/test/java/org/elasticsearch/action/resync/TransportResyncReplicationActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/resync/TransportResyncReplicationActionTests.java @@ -20,6 +20,7 @@ import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.bulk.WriteMemoryLimits; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.cluster.ClusterState; @@ -34,6 +35,7 @@ import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.lease.Releasable; import org.elasticsearch.common.network.NetworkService; +import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.PageCacheRecycler; import org.elasticsearch.index.Index; @@ -143,7 +145,8 @@ public void testResyncDoesNotBlockOnPrimaryAction() throws Exception { when(indexServices.indexServiceSafe(eq(index))).thenReturn(indexService); 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(Settings.EMPTY, new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS))); assertThat(action.globalBlockLevel(), nullValue()); assertThat(action.indexBlockLevel(), nullValue()); diff --git a/server/src/test/java/org/elasticsearch/action/support/replication/TransportWriteActionTests.java b/server/src/test/java/org/elasticsearch/action/support/replication/TransportWriteActionTests.java index bcddef3e240ed..38f7e11c72db8 100644 --- a/server/src/test/java/org/elasticsearch/action/support/replication/TransportWriteActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/support/replication/TransportWriteActionTests.java @@ -21,6 +21,7 @@ import org.elasticsearch.ElasticsearchException; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.bulk.WriteMemoryLimits; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.ActionTestUtils; import org.elasticsearch.action.support.PlainActionFuture; @@ -39,6 +40,7 @@ import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.lease.Releasable; +import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.index.Index; import org.elasticsearch.index.IndexService; @@ -365,7 +367,8 @@ protected TestAction(boolean withDocumentFailureOnPrimary, boolean withDocumentF super(Settings.EMPTY, "internal:test", new TransportService(Settings.EMPTY, mock(Transport.class), null, TransportService.NOOP_TRANSPORT_INTERCEPTOR, 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(Settings.EMPTY, new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS))); this.withDocumentFailureOnPrimary = withDocumentFailureOnPrimary; this.withDocumentFailureOnReplica = withDocumentFailureOnReplica; } @@ -374,7 +377,8 @@ protected TestAction(Settings settings, String actionName, TransportService tran ClusterService clusterService, ShardStateAction shardStateAction, ThreadPool threadPool) { super(settings, actionName, transportService, clusterService, 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(Settings.EMPTY, new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS))); this.withDocumentFailureOnPrimary = false; this.withDocumentFailureOnReplica = false; } diff --git a/server/src/test/java/org/elasticsearch/index/seqno/RetentionLeaseSyncActionTests.java b/server/src/test/java/org/elasticsearch/index/seqno/RetentionLeaseSyncActionTests.java index 74b168e60575e..4c5539e75e300 100644 --- a/server/src/test/java/org/elasticsearch/index/seqno/RetentionLeaseSyncActionTests.java +++ b/server/src/test/java/org/elasticsearch/index/seqno/RetentionLeaseSyncActionTests.java @@ -20,12 +20,14 @@ package org.elasticsearch.index.seqno; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.bulk.WriteMemoryLimits; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.ActionTestUtils; import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.action.support.replication.TransportReplicationAction; import org.elasticsearch.cluster.action.shard.ShardStateAction; import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.gateway.WriteStateException; @@ -104,7 +106,8 @@ public void testRetentionLeaseSyncActionOnPrimary() { indicesService, threadPool, shardStateAction, - new ActionFilters(Collections.emptySet())); + new ActionFilters(Collections.emptySet()), + new WriteMemoryLimits(Settings.EMPTY, new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS))); final RetentionLeases retentionLeases = mock(RetentionLeases.class); final RetentionLeaseSyncAction.Request request = new RetentionLeaseSyncAction.Request(indexShard.shardId(), retentionLeases); action.dispatchedShardOperationOnPrimary(request, indexShard, @@ -140,7 +143,8 @@ public void testRetentionLeaseSyncActionOnReplica() throws WriteStateException { indicesService, threadPool, shardStateAction, - new ActionFilters(Collections.emptySet())); + new ActionFilters(Collections.emptySet()), + new WriteMemoryLimits(Settings.EMPTY, new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS))); final RetentionLeases retentionLeases = mock(RetentionLeases.class); final RetentionLeaseSyncAction.Request request = new RetentionLeaseSyncAction.Request(indexShard.shardId(), retentionLeases); @@ -178,7 +182,8 @@ public void testBlocks() { indicesService, threadPool, shardStateAction, - new ActionFilters(Collections.emptySet())); + new ActionFilters(Collections.emptySet()), + new WriteMemoryLimits(Settings.EMPTY, new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS))); assertNull(action.indexBlockLevel()); } diff --git a/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java b/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java index 5cd6879a1b138..63230b1a1b9dd 100644 --- a/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java +++ b/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java @@ -63,7 +63,7 @@ import org.elasticsearch.action.admin.indices.shards.IndicesShardStoresAction; import org.elasticsearch.action.admin.indices.shards.TransportIndicesShardStoresAction; import org.elasticsearch.action.bulk.BulkAction; -import org.elasticsearch.action.bulk.BulkIndexingMemoryLimits; +import org.elasticsearch.action.bulk.WriteMemoryLimits; import org.elasticsearch.action.bulk.BulkRequest; import org.elasticsearch.action.bulk.BulkResponse; import org.elasticsearch.action.bulk.TransportBulkAction; @@ -1483,7 +1483,8 @@ public void onFailure(final Exception e) { indicesService, threadPool, shardStateAction, - actionFilters)), + actionFilters, + new WriteMemoryLimits(settings, clusterSettings))), RetentionLeaseSyncer.EMPTY, client); final MetadataCreateIndexService metadataCreateIndexService = new MetadataCreateIndexService(settings, clusterService, @@ -1497,7 +1498,7 @@ allocationService, new AliasValidator(), environment, indexScopedSettings, actionFilters, indexNameExpressionResolver )); final MappingUpdatedAction mappingUpdatedAction = new MappingUpdatedAction(settings, clusterSettings); - final BulkIndexingMemoryLimits indexingMemoryLimits = new BulkIndexingMemoryLimits(settings, clusterSettings); + final WriteMemoryLimits indexingMemoryLimits = new WriteMemoryLimits(settings, clusterSettings); mappingUpdatedAction.setClient(client); actions.put(BulkAction.INSTANCE, new TransportBulkAction(threadPool, transportService, clusterService, diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/bulk/TransportBulkShardOperationsAction.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/bulk/TransportBulkShardOperationsAction.java index e00e0cc3a5a9e..b654d82fc339a 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/bulk/TransportBulkShardOperationsAction.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/bulk/TransportBulkShardOperationsAction.java @@ -9,6 +9,7 @@ import org.apache.logging.log4j.Logger; import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.bulk.WriteMemoryLimits; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.replication.TransportWriteAction; import org.elasticsearch.cluster.action.shard.ShardStateAction; @@ -43,7 +44,8 @@ public TransportBulkShardOperationsAction( final IndicesService indicesService, final ThreadPool threadPool, final ShardStateAction shardStateAction, - final ActionFilters actionFilters) { + final ActionFilters actionFilters, + final WriteMemoryLimits writeMemoryLimits) { super( settings, BulkShardOperationsAction.NAME, @@ -55,7 +57,7 @@ public TransportBulkShardOperationsAction( actionFilters, BulkShardOperationsRequest::new, BulkShardOperationsRequest::new, - ThreadPool.Names.WRITE, false); + ThreadPool.Names.WRITE, false, writeMemoryLimits); } @Override From 718e343e4a50aa3aae8eaae526043fcee455e153 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Fri, 29 May 2020 16:28:47 -0600 Subject: [PATCH 18/41] Changes --- .../action/bulk/WriteMemoryLimitsIT.java | 1 - .../action/bulk/WriteMemoryLimits.java | 34 +++---------------- .../common/settings/ClusterSettings.java | 2 -- .../java/org/elasticsearch/node/Node.java | 2 +- ...TransportResyncReplicationActionTests.java | 3 +- .../TransportWriteActionTests.java | 5 ++- .../seqno/RetentionLeaseSyncActionTests.java | 7 ++-- .../snapshots/SnapshotResiliencyTests.java | 4 +-- 8 files changed, 14 insertions(+), 44 deletions(-) diff --git a/server/src/internalClusterTest/java/org/elasticsearch/action/bulk/WriteMemoryLimitsIT.java b/server/src/internalClusterTest/java/org/elasticsearch/action/bulk/WriteMemoryLimitsIT.java index 4c62ede35a165..5da8ecb08cc6a 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/action/bulk/WriteMemoryLimitsIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/action/bulk/WriteMemoryLimitsIT.java @@ -47,7 +47,6 @@ protected Settings nodeSettings(int nodeOrdinal) { .put(super.nodeSettings(nodeOrdinal)) // Need at least two threads because we are going to block one .put("thread_pool.write.size", 2) - .put("indices.indexing_bytes.limit", "1KB") .build(); } diff --git a/server/src/main/java/org/elasticsearch/action/bulk/WriteMemoryLimits.java b/server/src/main/java/org/elasticsearch/action/bulk/WriteMemoryLimits.java index dff39f0b9ae89..c71b1604e6cbd 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/WriteMemoryLimits.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/WriteMemoryLimits.java @@ -19,39 +19,15 @@ package org.elasticsearch.action.bulk; -import org.elasticsearch.common.settings.ClusterSettings; -import org.elasticsearch.common.settings.Setting; -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.unit.ByteSizeValue; -import org.elasticsearch.common.util.concurrent.EsRejectedExecutionException; - import java.util.concurrent.atomic.AtomicLong; public class WriteMemoryLimits { - public static final Setting MAX_INDEXING_BYTES = - Setting.memorySizeSetting("indices.indexing_bytes.limit", "20%", Setting.Property.NodeScope, Setting.Property.Dynamic); - - private volatile long indexingLimits; - - private final AtomicLong pendingBytes = new AtomicLong(0); - - public WriteMemoryLimits(Settings settings, ClusterSettings clusterSettings) { - this.indexingLimits = MAX_INDEXING_BYTES.get(settings).getBytes(); - clusterSettings.addSettingsUpdateConsumer(MAX_INDEXING_BYTES, value -> indexingLimits = value.getBytes()); - } + private final AtomicLong primaryBytes = new AtomicLong(0); + private final AtomicLong replicaBytes = new AtomicLong(0); public void markPrimaryOperationStarted(long bytes, boolean forceExecution) { - long pendingWithOperation = pendingBytes.addAndGet(bytes); - - if (forceExecution== false && pendingWithOperation > indexingLimits) { - decrementPendingBytes(bytes); - long pendingPreOperation = pendingWithOperation - bytes; - throw new EsRejectedExecutionException("rejected execution of primary shard operation [" + - "pending_bytes=" + pendingPreOperation + ", " + - "operation_bytes=" + bytes + "," + - "max_pending_bytes=" + indexingLimits + "]", false); - } + primaryBytes.addAndGet(bytes); } public void markPrimaryOperationFinished(long bytes) { @@ -59,7 +35,7 @@ public void markPrimaryOperationFinished(long bytes) { } public void markReplicaOperationStarted(long bytes) { - pendingBytes.getAndAdd(bytes); + replicaBytes.getAndAdd(bytes); } public void markReplicaOperationFinished(long bytes) { @@ -67,6 +43,6 @@ public void markReplicaOperationFinished(long bytes) { } private void decrementPendingBytes(long operationSizeInBytes) { - pendingBytes.getAndAdd(-operationSizeInBytes); + replicaBytes.getAndAdd(-operationSizeInBytes); } } diff --git a/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java b/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java index c8c12f79931b2..f7ff73e90faea 100644 --- a/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java +++ b/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java @@ -21,7 +21,6 @@ import org.apache.logging.log4j.LogManager; import org.elasticsearch.action.admin.cluster.configuration.TransportAddVotingConfigExclusionsAction; import org.elasticsearch.action.admin.indices.close.TransportCloseIndexAction; -import org.elasticsearch.action.bulk.WriteMemoryLimits; import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.action.support.AutoCreateIndex; import org.elasticsearch.action.support.DestructiveOperations; @@ -190,7 +189,6 @@ public void apply(Settings value, Settings current, Settings previous) { BalancedShardsAllocator.INDEX_BALANCE_FACTOR_SETTING, BalancedShardsAllocator.SHARD_BALANCE_FACTOR_SETTING, BalancedShardsAllocator.THRESHOLD_SETTING, - WriteMemoryLimits.MAX_INDEXING_BYTES, BreakerSettings.CIRCUIT_BREAKER_LIMIT_SETTING, BreakerSettings.CIRCUIT_BREAKER_OVERHEAD_SETTING, BreakerSettings.CIRCUIT_BREAKER_TYPE, diff --git a/server/src/main/java/org/elasticsearch/node/Node.java b/server/src/main/java/org/elasticsearch/node/Node.java index 6720590654fda..3e41c0b20192e 100644 --- a/server/src/main/java/org/elasticsearch/node/Node.java +++ b/server/src/main/java/org/elasticsearch/node/Node.java @@ -565,7 +565,7 @@ protected Node(final Environment initialEnvironment, new PersistentTasksClusterService(settings, registry, clusterService, threadPool); resourcesToClose.add(persistentTasksClusterService); final PersistentTasksService persistentTasksService = new PersistentTasksService(clusterService, threadPool, client); - final WriteMemoryLimits bulkIndexingLimits = new WriteMemoryLimits(settings, clusterService.getClusterSettings()); + final WriteMemoryLimits bulkIndexingLimits = new WriteMemoryLimits(); modules.add(b -> { b.bind(Node.class).toInstance(this); diff --git a/server/src/test/java/org/elasticsearch/action/resync/TransportResyncReplicationActionTests.java b/server/src/test/java/org/elasticsearch/action/resync/TransportResyncReplicationActionTests.java index 5fc382737566c..6861fb4aa89a2 100644 --- a/server/src/test/java/org/elasticsearch/action/resync/TransportResyncReplicationActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/resync/TransportResyncReplicationActionTests.java @@ -35,7 +35,6 @@ import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.lease.Releasable; import org.elasticsearch.common.network.NetworkService; -import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.PageCacheRecycler; import org.elasticsearch.index.Index; @@ -146,7 +145,7 @@ public void testResyncDoesNotBlockOnPrimaryAction() throws Exception { final TransportResyncReplicationAction action = new TransportResyncReplicationAction(Settings.EMPTY, transportService, clusterService, indexServices, threadPool, shardStateAction, new ActionFilters(new HashSet<>()), - new WriteMemoryLimits(Settings.EMPTY, new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS))); + new WriteMemoryLimits()); assertThat(action.globalBlockLevel(), nullValue()); assertThat(action.indexBlockLevel(), nullValue()); diff --git a/server/src/test/java/org/elasticsearch/action/support/replication/TransportWriteActionTests.java b/server/src/test/java/org/elasticsearch/action/support/replication/TransportWriteActionTests.java index 38f7e11c72db8..f0e51da276bfb 100644 --- a/server/src/test/java/org/elasticsearch/action/support/replication/TransportWriteActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/support/replication/TransportWriteActionTests.java @@ -40,7 +40,6 @@ import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.lease.Releasable; -import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.index.Index; import org.elasticsearch.index.IndexService; @@ -368,7 +367,7 @@ protected TestAction(boolean withDocumentFailureOnPrimary, boolean withDocumentF new TransportService(Settings.EMPTY, mock(Transport.class), null, TransportService.NOOP_TRANSPORT_INTERCEPTOR, x -> null, null, Collections.emptySet()), TransportWriteActionTests.this.clusterService, null, null, null, new ActionFilters(new HashSet<>()), TestRequest::new, TestRequest::new, ThreadPool.Names.SAME, false, - new WriteMemoryLimits(Settings.EMPTY, new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS))); + new WriteMemoryLimits()); this.withDocumentFailureOnPrimary = withDocumentFailureOnPrimary; this.withDocumentFailureOnReplica = withDocumentFailureOnReplica; } @@ -378,7 +377,7 @@ protected TestAction(Settings settings, String actionName, TransportService tran super(settings, actionName, transportService, clusterService, mockIndicesService(clusterService), threadPool, shardStateAction, new ActionFilters(new HashSet<>()), TestRequest::new, TestRequest::new, ThreadPool.Names.SAME, false, - new WriteMemoryLimits(Settings.EMPTY, new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS))); + new WriteMemoryLimits()); this.withDocumentFailureOnPrimary = false; this.withDocumentFailureOnReplica = false; } diff --git a/server/src/test/java/org/elasticsearch/index/seqno/RetentionLeaseSyncActionTests.java b/server/src/test/java/org/elasticsearch/index/seqno/RetentionLeaseSyncActionTests.java index 4c5539e75e300..7cfaf06f3e581 100644 --- a/server/src/test/java/org/elasticsearch/index/seqno/RetentionLeaseSyncActionTests.java +++ b/server/src/test/java/org/elasticsearch/index/seqno/RetentionLeaseSyncActionTests.java @@ -27,7 +27,6 @@ import org.elasticsearch.action.support.replication.TransportReplicationAction; import org.elasticsearch.cluster.action.shard.ShardStateAction; import org.elasticsearch.cluster.service.ClusterService; -import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.gateway.WriteStateException; @@ -107,7 +106,7 @@ public void testRetentionLeaseSyncActionOnPrimary() { threadPool, shardStateAction, new ActionFilters(Collections.emptySet()), - new WriteMemoryLimits(Settings.EMPTY, new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS))); + new WriteMemoryLimits()); final RetentionLeases retentionLeases = mock(RetentionLeases.class); final RetentionLeaseSyncAction.Request request = new RetentionLeaseSyncAction.Request(indexShard.shardId(), retentionLeases); action.dispatchedShardOperationOnPrimary(request, indexShard, @@ -144,7 +143,7 @@ public void testRetentionLeaseSyncActionOnReplica() throws WriteStateException { threadPool, shardStateAction, new ActionFilters(Collections.emptySet()), - new WriteMemoryLimits(Settings.EMPTY, new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS))); + new WriteMemoryLimits()); final RetentionLeases retentionLeases = mock(RetentionLeases.class); final RetentionLeaseSyncAction.Request request = new RetentionLeaseSyncAction.Request(indexShard.shardId(), retentionLeases); @@ -183,7 +182,7 @@ public void testBlocks() { threadPool, shardStateAction, new ActionFilters(Collections.emptySet()), - new WriteMemoryLimits(Settings.EMPTY, new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS))); + new WriteMemoryLimits()); assertNull(action.indexBlockLevel()); } diff --git a/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java b/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java index 63230b1a1b9dd..b71738ea6dd37 100644 --- a/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java +++ b/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java @@ -1484,7 +1484,7 @@ public void onFailure(final Exception e) { threadPool, shardStateAction, actionFilters, - new WriteMemoryLimits(settings, clusterSettings))), + new WriteMemoryLimits())), RetentionLeaseSyncer.EMPTY, client); final MetadataCreateIndexService metadataCreateIndexService = new MetadataCreateIndexService(settings, clusterService, @@ -1498,7 +1498,7 @@ allocationService, new AliasValidator(), environment, indexScopedSettings, actionFilters, indexNameExpressionResolver )); final MappingUpdatedAction mappingUpdatedAction = new MappingUpdatedAction(settings, clusterSettings); - final WriteMemoryLimits indexingMemoryLimits = new WriteMemoryLimits(settings, clusterSettings); + final WriteMemoryLimits indexingMemoryLimits = new WriteMemoryLimits(); mappingUpdatedAction.setClient(client); actions.put(BulkAction.INSTANCE, new TransportBulkAction(threadPool, transportService, clusterService, From 668715ea626a8aec3583b5460995f5128a155ba4 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Fri, 29 May 2020 17:04:37 -0600 Subject: [PATCH 19/41] WIP --- .../action/bulk/WriteMemoryLimitsIT.java | 8 +++--- .../elasticsearch/action/DocWriteRequest.java | 2 +- .../action/bulk/TransportBulkAction.java | 25 ++++++++++++++++--- .../action/bulk/WriteMemoryLimits.java | 10 +++++--- .../replication/TransportWriteAction.java | 12 ++++++--- ...ActionIndicesThatCannotBeCreatedTests.java | 2 +- .../bulk/TransportBulkActionIngestTests.java | 2 +- .../action/bulk/TransportBulkActionTests.java | 3 ++- .../bulk/TransportBulkActionTookTests.java | 1 + .../snapshots/SnapshotResiliencyTests.java | 3 ++- 10 files changed, 47 insertions(+), 21 deletions(-) diff --git a/server/src/internalClusterTest/java/org/elasticsearch/action/bulk/WriteMemoryLimitsIT.java b/server/src/internalClusterTest/java/org/elasticsearch/action/bulk/WriteMemoryLimitsIT.java index 5da8ecb08cc6a..599561d889c63 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/action/bulk/WriteMemoryLimitsIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/action/bulk/WriteMemoryLimitsIT.java @@ -78,10 +78,10 @@ public void testRejectionDueToRequestOverMemoryLimit() { for (int i = 0; i < 4; ++i) { moreThan1KB.add(new IndexRequest(index).source(Collections.singletonMap("key", "value" + i))); } - assertThat(DocWriteRequest.writeSizeInBytes(lessThan1KB.requests.stream()), lessThan(1024L)); +// assertThat(DocWriteRequest.writeSizeInBytes(lessThan1KB.requests.stream()), lessThan(1024L)); assertFalse(client().bulk(lessThan1KB).actionGet().hasFailures()); - assertThat(DocWriteRequest.writeSizeInBytes(moreThan1KB.requests.stream()), greaterThan(1024L)); +// assertThat(DocWriteRequest.writeSizeInBytes(moreThan1KB.requests.stream()), greaterThan(1024L)); final ActionFuture bulkFuture2 = client().bulk(moreThan1KB); final BulkResponse failedResponses = bulkFuture2.actionGet(); for (BulkItemResponse response : failedResponses) { @@ -116,13 +116,13 @@ public void testRejectionDueToConcurrentRequestsOverMemoryLimit() throws Interru for (int i = 0; i < 3; ++i) { lessThan1KB.add(new IndexRequest(index).source(Collections.singletonMap("key", "value" + i))); } - assertThat(DocWriteRequest.writeSizeInBytes(lessThan1KB.requests.stream()), lessThan(1024L)); +// assertThat(DocWriteRequest.writeSizeInBytes(lessThan1KB.requests.stream()), lessThan(1024L)); final BulkRequest rejectedRequest = new BulkRequest(); for (int i = 0; i < 2; ++i) { rejectedRequest.add(new IndexRequest(index).source(Collections.singletonMap("key", "value" + i))); } - assertThat(DocWriteRequest.writeSizeInBytes(rejectedRequest.requests.stream()), lessThan(1024L)); +// assertThat(DocWriteRequest.writeSizeInBytes(rejectedRequest.requests.stream()), lessThan(1024L)); try { final ActionFuture successFuture = client().bulk(lessThan1KB); diff --git a/server/src/main/java/org/elasticsearch/action/DocWriteRequest.java b/server/src/main/java/org/elasticsearch/action/DocWriteRequest.java index 1a643908ec98c..594aef9885bfd 100644 --- a/server/src/main/java/org/elasticsearch/action/DocWriteRequest.java +++ b/server/src/main/java/org/elasticsearch/action/DocWriteRequest.java @@ -259,7 +259,7 @@ static ActionRequestValidationException validateSeqNoBasedCASParams( } // A heuristic for the bytes overhead of a single indexing operation - int WRITE_REQUEST_BYTES_OVERHEAD = 256; + int WRITE_REQUEST_BYTES_OVERHEAD = 4096; static long writeSizeInBytes(Stream> requestStream) { return requestStream.mapToLong(request -> { diff --git a/server/src/main/java/org/elasticsearch/action/bulk/TransportBulkAction.java b/server/src/main/java/org/elasticsearch/action/bulk/TransportBulkAction.java index 94fb04279de6b..0f441998c5683 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/TransportBulkAction.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/TransportBulkAction.java @@ -58,6 +58,7 @@ import org.elasticsearch.cluster.metadata.MetadataIndexTemplateService; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.lease.Releasable; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.concurrent.AbstractRunnable; @@ -108,21 +109,22 @@ public class TransportBulkAction extends HandledTransportAction docWriteReque @Override protected void doExecute(Task task, BulkRequest bulkRequest, ActionListener listener) { + long indexingBytes = DocWriteRequest.writeSizeInBytes(bulkRequest.requests.stream()); + writeMemoryLimits.markOperationStarted(indexingBytes); + final Releasable releasable = () -> writeMemoryLimits.markOperationFinished(indexingBytes); + final ActionListener releasingListener = ActionListener.runAfter(listener, releasable::close); + threadPool.executor(ThreadPool.Names.WRITE).execute(new ActionRunnable<>(releasingListener) { + @Override + protected void doRun() { + doDispatchedExecute(task, bulkRequest, releasingListener); + } + }); + } + + protected void doDispatchedExecute(Task task, BulkRequest bulkRequest, ActionListener listener) { final long startTime = relativeTime(); final AtomicArray responses = new AtomicArray<>(bulkRequest.requests.size()); @@ -517,6 +533,7 @@ protected void doRun() { if (task != null) { bulkShardRequest.setParentTask(nodeId, task.getId()); } + threadPool.getThreadContext().putTransient(WriteMemoryLimits.WRITE_BYTES_MARKED, true); client.executeLocally(TransportShardBulkAction.TYPE, bulkShardRequest, new ActionListener<>() { @Override public void onResponse(BulkShardResponse bulkShardResponse) { diff --git a/server/src/main/java/org/elasticsearch/action/bulk/WriteMemoryLimits.java b/server/src/main/java/org/elasticsearch/action/bulk/WriteMemoryLimits.java index c71b1604e6cbd..bf461d3aac255 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/WriteMemoryLimits.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/WriteMemoryLimits.java @@ -23,14 +23,16 @@ public class WriteMemoryLimits { - private final AtomicLong primaryBytes = new AtomicLong(0); + public static final String WRITE_BYTES_MARKED = "write_bytes_marked"; + + private final AtomicLong primaryCoordinatingBytes = new AtomicLong(0); private final AtomicLong replicaBytes = new AtomicLong(0); - public void markPrimaryOperationStarted(long bytes, boolean forceExecution) { - primaryBytes.addAndGet(bytes); + public void markOperationStarted(long bytes) { + primaryCoordinatingBytes.addAndGet(bytes); } - public void markPrimaryOperationFinished(long bytes) { + public void markOperationFinished(long bytes) { decrementPendingBytes(bytes); } diff --git a/server/src/main/java/org/elasticsearch/action/support/replication/TransportWriteAction.java b/server/src/main/java/org/elasticsearch/action/support/replication/TransportWriteAction.java index 4cb3d4900d05f..f0df55eea7850 100644 --- a/server/src/main/java/org/elasticsearch/action/support/replication/TransportWriteAction.java +++ b/server/src/main/java/org/elasticsearch/action/support/replication/TransportWriteAction.java @@ -62,7 +62,6 @@ public abstract class TransportWriteAction< private final WriteMemoryLimits writeMemoryLimits; private final String executor; - private final boolean forceExecutionOnPrimary; protected TransportWriteAction(Settings settings, String actionName, TransportService transportService, ClusterService clusterService, IndicesService indicesService, ThreadPool threadPool, @@ -73,15 +72,20 @@ protected TransportWriteAction(Settings settings, String actionName, TransportSe request, replicaRequest, ThreadPool.Names.SAME, true, forceExecutionOnPrimary); this.executor = executor; this.writeMemoryLimits = writeMemoryLimits; - this.forceExecutionOnPrimary = forceExecutionOnPrimary; } @Override protected Releasable checkPrimaryLimits(Request request) { super.checkPrimaryLimits(request); long operationSizeInBytes = primaryOperationSize(request); - writeMemoryLimits.markPrimaryOperationStarted(operationSizeInBytes, forceExecutionOnPrimary); - return () -> writeMemoryLimits.markPrimaryOperationFinished(operationSizeInBytes); + Object writeBytesMarked = threadPool.getThreadContext().getTransient(WriteMemoryLimits.WRITE_BYTES_MARKED); + if (Boolean.TRUE.equals(writeBytesMarked)) { + return () -> {}; + } else { + threadPool.getThreadContext().putTransient(WriteMemoryLimits.WRITE_BYTES_MARKED, true); + writeMemoryLimits.markOperationStarted(operationSizeInBytes); + return () -> writeMemoryLimits.markOperationFinished(operationSizeInBytes); + } } protected long primaryOperationSize(Request request) { diff --git a/server/src/test/java/org/elasticsearch/action/bulk/TransportBulkActionIndicesThatCannotBeCreatedTests.java b/server/src/test/java/org/elasticsearch/action/bulk/TransportBulkActionIndicesThatCannotBeCreatedTests.java index e99d483787db3..3939cd98f5f88 100644 --- a/server/src/test/java/org/elasticsearch/action/bulk/TransportBulkActionIndicesThatCannotBeCreatedTests.java +++ b/server/src/test/java/org/elasticsearch/action/bulk/TransportBulkActionIndicesThatCannotBeCreatedTests.java @@ -120,7 +120,7 @@ private void indicesThatCannotBeCreatedTestCase(Set expected, final ExecutorService direct = EsExecutors.newDirectExecutorService(); when(threadPool.executor(anyString())).thenReturn(direct); TransportBulkAction action = new TransportBulkAction(threadPool, mock(TransportService.class), clusterService, - null, null, mock(ActionFilters.class), null, null) { + null, null, mock(ActionFilters.class), null, null, new WriteMemoryLimits()) { @Override void executeBulk(Task task, BulkRequest bulkRequest, long startTimeNanos, ActionListener listener, AtomicArray responses, Map indicesThatCannotBeCreated) { diff --git a/server/src/test/java/org/elasticsearch/action/bulk/TransportBulkActionIngestTests.java b/server/src/test/java/org/elasticsearch/action/bulk/TransportBulkActionIngestTests.java index f2d617230c5c8..129a98459fdec 100644 --- a/server/src/test/java/org/elasticsearch/action/bulk/TransportBulkActionIngestTests.java +++ b/server/src/test/java/org/elasticsearch/action/bulk/TransportBulkActionIngestTests.java @@ -143,7 +143,7 @@ null, new ActionFilters(Collections.emptySet()), null, new AutoCreateIndex( SETTINGS, new ClusterSettings(SETTINGS, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS), new IndexNameExpressionResolver() - ) + ), new WriteMemoryLimits() ); } @Override diff --git a/server/src/test/java/org/elasticsearch/action/bulk/TransportBulkActionTests.java b/server/src/test/java/org/elasticsearch/action/bulk/TransportBulkActionTests.java index c16b2f132ba8e..7e2dc739f5def 100644 --- a/server/src/test/java/org/elasticsearch/action/bulk/TransportBulkActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/bulk/TransportBulkActionTests.java @@ -75,7 +75,8 @@ class TestTransportBulkAction extends TransportBulkAction { TestTransportBulkAction() { super(TransportBulkActionTests.this.threadPool, transportService, clusterService, null, 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 diff --git a/server/src/test/java/org/elasticsearch/action/bulk/TransportBulkActionTookTests.java b/server/src/test/java/org/elasticsearch/action/bulk/TransportBulkActionTookTests.java index e084a7a7bdcbc..89408f3d17f91 100644 --- a/server/src/test/java/org/elasticsearch/action/bulk/TransportBulkActionTookTests.java +++ b/server/src/test/java/org/elasticsearch/action/bulk/TransportBulkActionTookTests.java @@ -233,6 +233,7 @@ static class TestTransportBulkAction extends TransportBulkAction { actionFilters, indexNameExpressionResolver, autoCreateIndex, + new WriteMemoryLimits(), relativeTimeProvider); } diff --git a/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java b/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java index b71738ea6dd37..d9d5bbd9bccb0 100644 --- a/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java +++ b/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java @@ -1507,7 +1507,8 @@ allocationService, new AliasValidator(), environment, indexScopedSettings, new AnalysisModule(environment, Collections.emptyList()).getAnalysisRegistry(), Collections.emptyList(), client), client, actionFilters, indexNameExpressionResolver, - new AutoCreateIndex(settings, clusterSettings, indexNameExpressionResolver) + new AutoCreateIndex(settings, clusterSettings, indexNameExpressionResolver), + new WriteMemoryLimits() )); final TransportShardBulkAction transportShardBulkAction = new TransportShardBulkAction(settings, transportService, clusterService, indicesService, threadPool, shardStateAction, mappingUpdatedAction, new UpdateHelper(scriptService), From bc76b9a1c9cc890ca9441e73ef5ae6e73e66242c Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Tue, 2 Jun 2020 18:26:04 -0600 Subject: [PATCH 20/41] Changes --- .../elasticsearch/action/DocWriteRequest.java | 9 +++------ .../action/bulk/BulkShardRequest.java | 19 ++++++++++++++++++- .../action/bulk/TransportBulkAction.java | 2 +- .../action/bulk/TransportShardBulkAction.java | 11 +++++++++++ .../action/bulk/WriteMemoryLimits.java | 13 ++++++------- .../replication/TransportWriteAction.java | 4 ++-- 6 files changed, 41 insertions(+), 17 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/DocWriteRequest.java b/server/src/main/java/org/elasticsearch/action/DocWriteRequest.java index 594aef9885bfd..b59845a940a3b 100644 --- a/server/src/main/java/org/elasticsearch/action/DocWriteRequest.java +++ b/server/src/main/java/org/elasticsearch/action/DocWriteRequest.java @@ -258,22 +258,19 @@ static ActionRequestValidationException validateSeqNoBasedCASParams( return validationException; } - // A heuristic for the bytes overhead of a single indexing operation - int WRITE_REQUEST_BYTES_OVERHEAD = 4096; - static long writeSizeInBytes(Stream> requestStream) { return requestStream.mapToLong(request -> { if (request instanceof IndexRequest) { if (((IndexRequest) request).source() != null) { - return ((IndexRequest) request).source().length() + WRITE_REQUEST_BYTES_OVERHEAD; + return ((IndexRequest) request).source().length(); } } else if (request instanceof UpdateRequest) { IndexRequest doc = ((UpdateRequest) request).doc(); if (doc != null && doc.source() != null) { - return ((UpdateRequest) request).doc().source().length() + WRITE_REQUEST_BYTES_OVERHEAD; + return ((UpdateRequest) request).doc().source().length(); } } - return WRITE_REQUEST_BYTES_OVERHEAD; + return 0; }).sum(); } } diff --git a/server/src/main/java/org/elasticsearch/action/bulk/BulkShardRequest.java b/server/src/main/java/org/elasticsearch/action/bulk/BulkShardRequest.java index 540db58da864c..005608e7d9f6a 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/BulkShardRequest.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/BulkShardRequest.java @@ -31,10 +31,12 @@ import java.io.IOException; import java.util.HashSet; import java.util.Set; +import java.util.concurrent.atomic.AtomicBoolean; public class BulkShardRequest extends ReplicatedWriteRequest { - private BulkItemRequest[] items; + private final AtomicBoolean bytesAccounted; + private final BulkItemRequest[] items; public BulkShardRequest(StreamInput in) throws IOException { super(in); @@ -44,11 +46,17 @@ public BulkShardRequest(StreamInput in) throws IOException { items[i] = new BulkItemRequest(in); } } + this.bytesAccounted = new AtomicBoolean(false); } public BulkShardRequest(ShardId shardId, RefreshPolicy refreshPolicy, BulkItemRequest[] items) { + this(shardId, refreshPolicy, items, false); + } + + public BulkShardRequest(ShardId shardId, RefreshPolicy refreshPolicy, BulkItemRequest[] items, boolean bytesAccounted) { super(shardId); this.items = items; + this.bytesAccounted = new AtomicBoolean(bytesAccounted); setRefreshPolicy(refreshPolicy); } @@ -154,4 +162,13 @@ public void onRetry() { } } } + + public boolean isBytesAccounted() { + return bytesAccounted.get(); + } + + public void markBytesAccounted() { + assert bytesAccounted.get() == false; + bytesAccounted.set(true); + } } diff --git a/server/src/main/java/org/elasticsearch/action/bulk/TransportBulkAction.java b/server/src/main/java/org/elasticsearch/action/bulk/TransportBulkAction.java index 0f441998c5683..4e06476f214e1 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/TransportBulkAction.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/TransportBulkAction.java @@ -526,7 +526,7 @@ protected void doRun() { final ShardId shardId = entry.getKey(); final List requests = entry.getValue(); BulkShardRequest bulkShardRequest = new BulkShardRequest(shardId, bulkRequest.getRefreshPolicy(), - requests.toArray(new BulkItemRequest[requests.size()])); + requests.toArray(new BulkItemRequest[requests.size()]), true); bulkShardRequest.waitForActiveShards(bulkRequest.waitForActiveShards()); bulkShardRequest.timeout(bulkRequest.timeout()); bulkShardRequest.routedBasedOnClusterVersion(clusterState.version()); diff --git a/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java b/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java index c099e2027d097..a519eac5a17d8 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java @@ -51,6 +51,7 @@ import org.elasticsearch.common.compress.CompressedXContent; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.lease.Releasable; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.xcontent.ToXContent; @@ -140,6 +141,11 @@ public void onTimeout(TimeValue timeout) { ); } + @Override + protected long primaryOperationSize(BulkShardRequest request) { + return DocWriteRequest.writeSizeInBytes(Stream.of(request.items()).map(BulkItemRequest::request)); + } + public static void performOnPrimary( BulkShardRequest request, IndexShard primary, @@ -418,6 +424,11 @@ protected void dispatchedShardOperationOnReplica(BulkShardRequest request, Index }); } + @Override + protected long replicaOperationSize(BulkShardRequest request) { + return DocWriteRequest.writeSizeInBytes(Stream.of(request.items()).map(BulkItemRequest::request)); + } + public static Translog.Location performOnReplica(BulkShardRequest request, IndexShard replica) throws Exception { Translog.Location location = null; for (int i = 0; i < request.items().length; i++) { diff --git a/server/src/main/java/org/elasticsearch/action/bulk/WriteMemoryLimits.java b/server/src/main/java/org/elasticsearch/action/bulk/WriteMemoryLimits.java index bf461d3aac255..a0a360c884a7c 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/WriteMemoryLimits.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/WriteMemoryLimits.java @@ -23,28 +23,27 @@ public class WriteMemoryLimits { + // A heuristic for the bytes overhead of a single write operation + public static final int WRITE_REQUEST_BYTES_OVERHEAD = 4096; public static final String WRITE_BYTES_MARKED = "write_bytes_marked"; private final AtomicLong primaryCoordinatingBytes = new AtomicLong(0); private final AtomicLong replicaBytes = new AtomicLong(0); public void markOperationStarted(long bytes) { - primaryCoordinatingBytes.addAndGet(bytes); + primaryCoordinatingBytes.addAndGet(WRITE_REQUEST_BYTES_OVERHEAD + bytes); } public void markOperationFinished(long bytes) { - decrementPendingBytes(bytes); + primaryCoordinatingBytes.getAndAdd(-(WRITE_REQUEST_BYTES_OVERHEAD + bytes)); } public void markReplicaOperationStarted(long bytes) { - replicaBytes.getAndAdd(bytes); + replicaBytes.getAndAdd(WRITE_REQUEST_BYTES_OVERHEAD + bytes); } public void markReplicaOperationFinished(long bytes) { - decrementPendingBytes(bytes); + replicaBytes.getAndAdd(-(WRITE_REQUEST_BYTES_OVERHEAD + bytes)); } - private void decrementPendingBytes(long operationSizeInBytes) { - replicaBytes.getAndAdd(-operationSizeInBytes); - } } diff --git a/server/src/main/java/org/elasticsearch/action/support/replication/TransportWriteAction.java b/server/src/main/java/org/elasticsearch/action/support/replication/TransportWriteAction.java index f0df55eea7850..fd54847234456 100644 --- a/server/src/main/java/org/elasticsearch/action/support/replication/TransportWriteAction.java +++ b/server/src/main/java/org/elasticsearch/action/support/replication/TransportWriteAction.java @@ -89,7 +89,7 @@ protected Releasable checkPrimaryLimits(Request request) { } protected long primaryOperationSize(Request request) { - return 4096; + return 0; } @Override @@ -101,7 +101,7 @@ protected Releasable checkReplicaLimits(ReplicaRequest request) { } protected long replicaOperationSize(ReplicaRequest request) { - return 4096; + return 0; } /** Syncs operation result to the translog or throws a shard not available failure */ From 2b581b547921cc7bfc9b29ce8d6e9751a4d7d3ea Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Tue, 2 Jun 2020 18:49:50 -0600 Subject: [PATCH 21/41] Changes --- .../elasticsearch/action/bulk/WriteMemoryLimitsIT.java | 3 --- .../action/bulk/TransportShardBulkAction.java | 1 - .../bulk/TransportBulkShardOperationsAction.java | 10 ++++++++++ 3 files changed, 10 insertions(+), 4 deletions(-) diff --git a/server/src/internalClusterTest/java/org/elasticsearch/action/bulk/WriteMemoryLimitsIT.java b/server/src/internalClusterTest/java/org/elasticsearch/action/bulk/WriteMemoryLimitsIT.java index 599561d889c63..6f26292debdda 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/action/bulk/WriteMemoryLimitsIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/action/bulk/WriteMemoryLimitsIT.java @@ -19,7 +19,6 @@ package org.elasticsearch.action.bulk; import org.elasticsearch.action.ActionFuture; -import org.elasticsearch.action.DocWriteRequest; import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.plugins.Plugin; @@ -35,8 +34,6 @@ import java.util.concurrent.CountDownLatch; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; -import static org.hamcrest.Matchers.greaterThan; -import static org.hamcrest.Matchers.lessThan; @ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.SUITE, numDataNodes = 2) public class WriteMemoryLimitsIT extends ESIntegTestCase { diff --git a/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java b/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java index a519eac5a17d8..31758d7f86192 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java @@ -51,7 +51,6 @@ import org.elasticsearch.common.compress.CompressedXContent; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.io.stream.StreamInput; -import org.elasticsearch.common.lease.Releasable; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.xcontent.ToXContent; diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/bulk/TransportBulkShardOperationsAction.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/bulk/TransportBulkShardOperationsAction.java index b654d82fc339a..6918c4a96678a 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/bulk/TransportBulkShardOperationsAction.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/bulk/TransportBulkShardOperationsAction.java @@ -70,6 +70,11 @@ protected void dispatchedShardOperationOnPrimary(BulkShardOperationsRequest requ 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) { final Translog.Operation operationWithPrimaryTerm; switch (operation.opType()) { @@ -170,6 +175,11 @@ protected void dispatchedShardOperationOnReplica(BulkShardOperationsRequest requ }); } + @Override + protected long replicaOperationSize(BulkShardOperationsRequest request) { + return request.getOperations().stream().mapToLong(Translog.Operation::estimateSize).sum(); + } + // public for testing purposes only public static WriteReplicaResult shardOperationOnReplica( final BulkShardOperationsRequest request, final IndexShard replica, final Logger logger) throws IOException { From d5aaf0e220ac8b717f8cbbd8f9a5cd095ed38ab8 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Tue, 2 Jun 2020 19:04:05 -0600 Subject: [PATCH 22/41] Delete --- .../action/bulk/BulkShardRequest.java | 19 +----- .../action/bulk/TransportBulkAction.java | 2 +- .../transport/InboundHandler.java | 29 ++++++++- .../transport/RequestHandlerRegistry.java | 62 +++---------------- .../transport/TaskTransportChannel.java | 15 ++--- .../transport/TransportRequestHandler.java | 5 -- .../transport/TransportService.java | 50 ++++++++++++--- .../transport/InboundHandlerTests.java | 6 +- 8 files changed, 88 insertions(+), 100 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/bulk/BulkShardRequest.java b/server/src/main/java/org/elasticsearch/action/bulk/BulkShardRequest.java index 005608e7d9f6a..540db58da864c 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/BulkShardRequest.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/BulkShardRequest.java @@ -31,12 +31,10 @@ import java.io.IOException; import java.util.HashSet; import java.util.Set; -import java.util.concurrent.atomic.AtomicBoolean; public class BulkShardRequest extends ReplicatedWriteRequest { - private final AtomicBoolean bytesAccounted; - private final BulkItemRequest[] items; + private BulkItemRequest[] items; public BulkShardRequest(StreamInput in) throws IOException { super(in); @@ -46,17 +44,11 @@ public BulkShardRequest(StreamInput in) throws IOException { items[i] = new BulkItemRequest(in); } } - this.bytesAccounted = new AtomicBoolean(false); } public BulkShardRequest(ShardId shardId, RefreshPolicy refreshPolicy, BulkItemRequest[] items) { - this(shardId, refreshPolicy, items, false); - } - - public BulkShardRequest(ShardId shardId, RefreshPolicy refreshPolicy, BulkItemRequest[] items, boolean bytesAccounted) { super(shardId); this.items = items; - this.bytesAccounted = new AtomicBoolean(bytesAccounted); setRefreshPolicy(refreshPolicy); } @@ -162,13 +154,4 @@ public void onRetry() { } } } - - public boolean isBytesAccounted() { - return bytesAccounted.get(); - } - - public void markBytesAccounted() { - assert bytesAccounted.get() == false; - bytesAccounted.set(true); - } } diff --git a/server/src/main/java/org/elasticsearch/action/bulk/TransportBulkAction.java b/server/src/main/java/org/elasticsearch/action/bulk/TransportBulkAction.java index 4e06476f214e1..0f441998c5683 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/TransportBulkAction.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/TransportBulkAction.java @@ -526,7 +526,7 @@ protected void doRun() { final ShardId shardId = entry.getKey(); final List requests = entry.getValue(); BulkShardRequest bulkShardRequest = new BulkShardRequest(shardId, bulkRequest.getRefreshPolicy(), - requests.toArray(new BulkItemRequest[requests.size()]), true); + requests.toArray(new BulkItemRequest[requests.size()])); bulkShardRequest.waitForActiveShards(bulkRequest.waitForActiveShards()); bulkShardRequest.timeout(bulkRequest.timeout()); bulkShardRequest.routedBasedOnClusterVersion(clusterState.version()); diff --git a/server/src/main/java/org/elasticsearch/transport/InboundHandler.java b/server/src/main/java/org/elasticsearch/transport/InboundHandler.java index 3e129e548ddaf..fa7533299b687 100644 --- a/server/src/main/java/org/elasticsearch/transport/InboundHandler.java +++ b/server/src/main/java/org/elasticsearch/transport/InboundHandler.java @@ -173,7 +173,7 @@ private void handleRequest(TcpChannel channel, Head throw new IllegalStateException("Message not fully read (request) for requestId [" + requestId + "], action [" + action + "], available [" + stream.available() + "]; resetting"); } - reg.dispatchMessage(request, transportChannel); + threadPool.executor(reg.getExecutor()).execute(new RequestHandler<>(reg, request, transportChannel)); } } catch (Exception e) { sendErrorResponse(action, transportChannel, e); @@ -246,4 +246,31 @@ private StreamInput namedWriteableStream(StreamInput delegate) { static void assertRemoteVersion(StreamInput in, Version version) { assert version.equals(in.getVersion()) : "Stream version [" + in.getVersion() + "] does not match version [" + version + "]"; } + + private static class RequestHandler extends AbstractRunnable { + private final RequestHandlerRegistry reg; + private final T request; + private final TransportChannel transportChannel; + + RequestHandler(RequestHandlerRegistry reg, T request, TransportChannel transportChannel) { + this.reg = reg; + this.request = request; + this.transportChannel = transportChannel; + } + + @Override + protected void doRun() throws Exception { + reg.processMessageReceived(request, transportChannel); + } + + @Override + public boolean isForceExecution() { + return reg.isForceExecution(); + } + + @Override + public void onFailure(Exception e) { + sendErrorResponse(reg.getAction(), transportChannel, e); + } + } } diff --git a/server/src/main/java/org/elasticsearch/transport/RequestHandlerRegistry.java b/server/src/main/java/org/elasticsearch/transport/RequestHandlerRegistry.java index 36a38c709826f..0c1922d35430d 100644 --- a/server/src/main/java/org/elasticsearch/transport/RequestHandlerRegistry.java +++ b/server/src/main/java/org/elasticsearch/transport/RequestHandlerRegistry.java @@ -23,19 +23,15 @@ import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.common.lease.Releasable; import org.elasticsearch.common.lease.Releasables; -import org.elasticsearch.common.util.concurrent.AbstractRunnable; import org.elasticsearch.tasks.CancellableTask; import org.elasticsearch.tasks.Task; import org.elasticsearch.tasks.TaskManager; -import org.elasticsearch.threadpool.ThreadPool; import java.io.IOException; -import java.util.concurrent.atomic.AtomicReference; public class RequestHandlerRegistry { private final String action; - private final ThreadPool threadPool; private final TransportRequestHandler handler; private final boolean forceExecution; private final boolean canTripCircuitBreaker; @@ -43,12 +39,11 @@ public class RequestHandlerRegistry { private final TaskManager taskManager; private final Writeable.Reader requestReader; - public RequestHandlerRegistry(String action, Writeable.Reader requestReader, ThreadPool threadPool, TaskManager taskManager, + public RequestHandlerRegistry(String action, Writeable.Reader requestReader, TaskManager taskManager, TransportRequestHandler handler, String executor, boolean forceExecution, boolean canTripCircuitBreaker) { this.action = action; this.requestReader = requestReader; - this.threadPool = threadPool; this.handler = handler; this.forceExecution = forceExecution; this.canTripCircuitBreaker = canTripCircuitBreaker; @@ -65,30 +60,19 @@ public Request newRequest(StreamInput in) throws IOException { } public void processMessageReceived(Request request, TransportChannel channel) throws Exception { - processMessageReceived(request, channel, new AtomicReference<>(() -> {})); - } - - private void processMessageReceived(Request request, TransportChannel channel, AtomicReference releasable) - throws Exception { final Task task = taskManager.register(channel.getChannelType(), action, request); - boolean success = false; - // TODO: Review releasable logic - Releasable[] releasables = new Releasable[2]; - releasables[0] = () -> taskManager.unregister(task); + Releasable unregisterTask = () -> taskManager.unregister(task); try { if (channel instanceof TcpTransportChannel && task instanceof CancellableTask) { final TcpChannel tcpChannel = ((TcpTransportChannel) channel).getChannel(); final Releasable stopTracking = taskManager.startTrackingCancellableChannelTask(tcpChannel, (CancellableTask) task); - releasables[0] = Releasables.wrap(releasables[0], stopTracking); - releasables[1] = releasable.get(); + unregisterTask = Releasables.wrap(unregisterTask, stopTracking); } - final TaskTransportChannel taskTransportChannel = new TaskTransportChannel(channel, releasables); + final TaskTransportChannel taskTransportChannel = new TaskTransportChannel(channel, unregisterTask); handler.messageReceived(request, taskTransportChannel, task); - success = true; + unregisterTask = null; } finally { - if (success == false) { - Releasables.close(releasables); - } + Releasables.close(unregisterTask); } } @@ -108,38 +92,6 @@ public TransportRequestHandler getHandler() { return handler; } - public void dispatchMessage(Request request, TransportChannel channel) { - final AtomicReference releasable = new AtomicReference<>(); - try { - releasable.set(handler.preDispatchValidation(request)); - threadPool.executor(executor).execute(new AbstractRunnable() { - @Override - public void onFailure(Exception e) { - Releasables.close(releasable.get()); - TransportChannel.sendErrorResponse(channel, action, request, e); - } - - @Override - protected void doRun() throws Exception { - processMessageReceived(request, channel, releasable); - } - - @Override - public boolean isForceExecution() { - return forceExecution; - } - - @Override - public String toString() { - return "processing of [" + action + "]: " + request; - } - }); - } catch (Exception e) { - Releasables.close(releasable.get()); - TransportChannel.sendErrorResponse(channel, action, request, e); - } - } - @Override public String toString() { return handler.toString(); @@ -147,7 +99,7 @@ public String toString() { public static RequestHandlerRegistry replaceHandler(RequestHandlerRegistry registry, TransportRequestHandler handler) { - return new RequestHandlerRegistry<>(registry.action, registry.requestReader, registry.threadPool, registry.taskManager, handler, + return new RequestHandlerRegistry<>(registry.action, registry.requestReader, registry.taskManager, handler, registry.executor, registry.forceExecution, registry.canTripCircuitBreaker); } } diff --git a/server/src/main/java/org/elasticsearch/transport/TaskTransportChannel.java b/server/src/main/java/org/elasticsearch/transport/TaskTransportChannel.java index 2f6b411b117b4..142fae1a9968a 100644 --- a/server/src/main/java/org/elasticsearch/transport/TaskTransportChannel.java +++ b/server/src/main/java/org/elasticsearch/transport/TaskTransportChannel.java @@ -21,18 +21,17 @@ import org.elasticsearch.Version; import org.elasticsearch.common.lease.Releasable; -import org.elasticsearch.common.lease.Releasables; import java.io.IOException; public class TaskTransportChannel implements TransportChannel { private final TransportChannel channel; - private final Releasable[] releasables; + private final Releasable onTaskFinished; - TaskTransportChannel(TransportChannel channel, Releasable[] releasables) { + TaskTransportChannel(TransportChannel channel, Releasable onTaskFinished) { this.channel = channel; - this.releasables = releasables; + this.onTaskFinished = onTaskFinished; } @Override @@ -48,7 +47,7 @@ public String getChannelType() { @Override public void sendResponse(TransportResponse response) throws IOException { try { - release(); + onTaskFinished.close(); } finally { channel.sendResponse(response); } @@ -57,7 +56,7 @@ public void sendResponse(TransportResponse response) throws IOException { @Override public void sendResponse(Exception exception) throws IOException { try { - release(); + onTaskFinished.close(); } finally { channel.sendResponse(exception); } @@ -71,8 +70,4 @@ public Version getVersion() { public TransportChannel getChannel() { return channel; } - - private void release() { - Releasables.close(releasables); - } } diff --git a/server/src/main/java/org/elasticsearch/transport/TransportRequestHandler.java b/server/src/main/java/org/elasticsearch/transport/TransportRequestHandler.java index 495ce13be6e38..be95798806847 100644 --- a/server/src/main/java/org/elasticsearch/transport/TransportRequestHandler.java +++ b/server/src/main/java/org/elasticsearch/transport/TransportRequestHandler.java @@ -19,14 +19,9 @@ package org.elasticsearch.transport; -import org.elasticsearch.common.lease.Releasable; import org.elasticsearch.tasks.Task; public interface TransportRequestHandler { void messageReceived(T request, TransportChannel channel, Task task) throws Exception; - - default Releasable preDispatchValidation(T request) throws Exception { - return () -> {}; - }; } diff --git a/server/src/main/java/org/elasticsearch/transport/TransportService.java b/server/src/main/java/org/elasticsearch/transport/TransportService.java index 261561884c7da..7f42e175db774 100644 --- a/server/src/main/java/org/elasticsearch/transport/TransportService.java +++ b/server/src/main/java/org/elasticsearch/transport/TransportService.java @@ -721,18 +721,50 @@ protected void doRun() throws Exception { } } - @SuppressWarnings("unchecked") - private void sendLocalRequest(long requestId, final String action, final TransportRequest request, - TransportRequestOptions options) { + private void sendLocalRequest(long requestId, final String action, final TransportRequest request, TransportRequestOptions options) { final DirectResponseChannel channel = new DirectResponseChannel(localNode, action, requestId, this, threadPool); try { onRequestSent(localNode, requestId, action, request, options); onRequestReceived(requestId, action); - final RequestHandlerRegistry reg = (RequestHandlerRegistry) getRequestHandler(action); + final RequestHandlerRegistry reg = getRequestHandler(action); if (reg == null) { throw new ActionNotFoundTransportException("Action [" + action + "] not found"); } - reg.dispatchMessage((T) request, channel); + final String executor = reg.getExecutor(); + if (ThreadPool.Names.SAME.equals(executor)) { + //noinspection unchecked + reg.processMessageReceived(request, channel); + } else { + threadPool.executor(executor).execute(new AbstractRunnable() { + @Override + protected void doRun() throws Exception { + //noinspection unchecked + reg.processMessageReceived(request, channel); + } + + @Override + public boolean isForceExecution() { + return reg.isForceExecution(); + } + + @Override + public void onFailure(Exception e) { + try { + channel.sendResponse(e); + } catch (Exception inner) { + inner.addSuppressed(e); + logger.warn(() -> new ParameterizedMessage( + "failed to notify channel of error message for action [{}]", action), inner); + } + } + + @Override + public String toString() { + return "processing of [" + requestId + "][" + action + "]: " + request; + } + }); + } + } catch (Exception e) { try { channel.sendResponse(e); @@ -813,7 +845,11 @@ public static boolean isValidActionName(String actionName) { public void registerRequestHandler(String action, String executor, Writeable.Reader requestReader, TransportRequestHandler handler) { - registerRequestHandler(action, executor, false, true, requestReader, handler); + validateActionName(action); + handler = interceptor.interceptHandler(action, executor, false, handler); + RequestHandlerRegistry reg = new RequestHandlerRegistry<>( + action, requestReader, taskManager, handler, executor, false, true); + transport.registerRequestHandler(reg); } /** @@ -834,7 +870,7 @@ public void registerRequestHandler(String act validateActionName(action); handler = interceptor.interceptHandler(action, executor, forceExecution, handler); RequestHandlerRegistry reg = new RequestHandlerRegistry<>( - action, requestReader, threadPool, taskManager, handler, executor, forceExecution, canTripCircuitBreaker); + action, requestReader, taskManager, handler, executor, forceExecution, canTripCircuitBreaker); transport.registerRequestHandler(reg); } diff --git a/server/src/test/java/org/elasticsearch/transport/InboundHandlerTests.java b/server/src/test/java/org/elasticsearch/transport/InboundHandlerTests.java index 37e9990bcda68..71a464efd8d37 100644 --- a/server/src/test/java/org/elasticsearch/transport/InboundHandlerTests.java +++ b/server/src/test/java/org/elasticsearch/transport/InboundHandlerTests.java @@ -90,8 +90,8 @@ public void tearDown() throws Exception { public void testPing() throws Exception { AtomicReference channelCaptor = new AtomicReference<>(); - RequestHandlerRegistry registry = new RequestHandlerRegistry<>("test-request", TestRequest::new, threadPool, - taskManager, (request, channel, task) -> channelCaptor.set(channel), ThreadPool.Names.SAME, false, true); + RequestHandlerRegistry registry = new RequestHandlerRegistry<>("test-request", TestRequest::new, taskManager, + (request, channel, task) -> channelCaptor.set(channel), ThreadPool.Names.SAME, false, true); requestHandlers.registerHandler(registry); handler.inboundMessage(channel, new InboundMessage(null, true)); @@ -132,7 +132,7 @@ public TestResponse read(StreamInput in) throws IOException { return new TestResponse(in); } }, null, action)); - RequestHandlerRegistry registry = new RequestHandlerRegistry<>(action, TestRequest::new, threadPool, taskManager, + RequestHandlerRegistry registry = new RequestHandlerRegistry<>(action, TestRequest::new, taskManager, (request, channel, task) -> { channelCaptor.set(channel); requestCaptor.set(request); From 777569a12a1f7c5dc5cc69216767c9d9c53c34e5 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Tue, 2 Jun 2020 19:21:47 -0600 Subject: [PATCH 23/41] Move --- .../java/org/elasticsearch/action/bulk/TransportBulkAction.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/action/bulk/TransportBulkAction.java b/server/src/main/java/org/elasticsearch/action/bulk/TransportBulkAction.java index 0f441998c5683..8efcca6fedd43 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/TransportBulkAction.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/TransportBulkAction.java @@ -160,6 +160,7 @@ public static IndexRequest getIndexWriteRequest(DocWriteRequest docWriteReque protected void doExecute(Task task, BulkRequest bulkRequest, ActionListener listener) { long indexingBytes = DocWriteRequest.writeSizeInBytes(bulkRequest.requests.stream()); writeMemoryLimits.markOperationStarted(indexingBytes); + threadPool.getThreadContext().putTransient(WriteMemoryLimits.WRITE_BYTES_MARKED, true); final Releasable releasable = () -> writeMemoryLimits.markOperationFinished(indexingBytes); final ActionListener releasingListener = ActionListener.runAfter(listener, releasable::close); threadPool.executor(ThreadPool.Names.WRITE).execute(new ActionRunnable<>(releasingListener) { @@ -533,7 +534,6 @@ protected void doRun() { if (task != null) { bulkShardRequest.setParentTask(nodeId, task.getId()); } - threadPool.getThreadContext().putTransient(WriteMemoryLimits.WRITE_BYTES_MARKED, true); client.executeLocally(TransportShardBulkAction.TYPE, bulkShardRequest, new ActionListener<>() { @Override public void onResponse(BulkShardResponse bulkShardResponse) { From 31459dcb2cdf23dd2a03228601feb2ae56dfdb9b Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Tue, 2 Jun 2020 19:36:34 -0600 Subject: [PATCH 24/41] Change --- .../java/org/elasticsearch/action/bulk/TransportBulkAction.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/action/bulk/TransportBulkAction.java b/server/src/main/java/org/elasticsearch/action/bulk/TransportBulkAction.java index 8efcca6fedd43..6d5d81c0ff141 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/TransportBulkAction.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/TransportBulkAction.java @@ -160,12 +160,12 @@ public static IndexRequest getIndexWriteRequest(DocWriteRequest docWriteReque protected void doExecute(Task task, BulkRequest bulkRequest, ActionListener listener) { long indexingBytes = DocWriteRequest.writeSizeInBytes(bulkRequest.requests.stream()); writeMemoryLimits.markOperationStarted(indexingBytes); - threadPool.getThreadContext().putTransient(WriteMemoryLimits.WRITE_BYTES_MARKED, true); final Releasable releasable = () -> writeMemoryLimits.markOperationFinished(indexingBytes); final ActionListener releasingListener = ActionListener.runAfter(listener, releasable::close); threadPool.executor(ThreadPool.Names.WRITE).execute(new ActionRunnable<>(releasingListener) { @Override protected void doRun() { + threadPool.getThreadContext().putTransient(WriteMemoryLimits.WRITE_BYTES_MARKED, true); doDispatchedExecute(task, bulkRequest, releasingListener); } }); From 903ed490304aa6b853deb542ffec14bc3b3146f7 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Tue, 2 Jun 2020 20:06:02 -0600 Subject: [PATCH 25/41] Change --- .../org/elasticsearch/action/bulk/TransportBulkAction.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/bulk/TransportBulkAction.java b/server/src/main/java/org/elasticsearch/action/bulk/TransportBulkAction.java index 6d5d81c0ff141..770bb01533b22 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/TransportBulkAction.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/TransportBulkAction.java @@ -706,7 +706,7 @@ private void processBulkIndexIngestRequest(Task task, BulkRequest original, Acti // before we continue the bulk request we should fork back on a write thread: if (originalThread == Thread.currentThread()) { assert Thread.currentThread().getName().contains(ThreadPool.Names.WRITE); - doExecute(task, bulkRequest, actionListener); + doDispatchedExecute(task, bulkRequest, actionListener); } else { threadPool.executor(ThreadPool.Names.WRITE).execute(new AbstractRunnable() { @Override @@ -716,7 +716,7 @@ public void onFailure(Exception e) { @Override protected void doRun() throws Exception { - doExecute(task, bulkRequest, actionListener); + doDispatchedExecute(task, bulkRequest, actionListener); } @Override From dfe5e8e71f026f029cc9f047ff2d657f69eb7ca5 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Wed, 3 Jun 2020 11:05:51 -0600 Subject: [PATCH 26/41] Changes --- .../action/bulk/BulkShardRequest.java | 13 ++++++++- .../action/bulk/TransportBulkAction.java | 5 ++-- .../action/bulk/TransportShardBulkAction.java | 10 +++++++ .../action/bulk/WriteMemoryLimits.java | 18 ++++++++++-- .../TransportResyncReplicationAction.java | 11 ++++++++ .../TransportReplicationAction.java | 6 +++- .../replication/TransportWriteAction.java | 28 ++++++++++--------- 7 files changed, 71 insertions(+), 20 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/bulk/BulkShardRequest.java b/server/src/main/java/org/elasticsearch/action/bulk/BulkShardRequest.java index 540db58da864c..5fd4af01498f1 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/BulkShardRequest.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/BulkShardRequest.java @@ -31,10 +31,12 @@ import java.io.IOException; import java.util.HashSet; import java.util.Set; +import java.util.concurrent.atomic.AtomicBoolean; public class BulkShardRequest extends ReplicatedWriteRequest { - private BulkItemRequest[] items; + private final AtomicBoolean bytesAccounted = new AtomicBoolean(false); + private final BulkItemRequest[] items; public BulkShardRequest(StreamInput in) throws IOException { super(in); @@ -154,4 +156,13 @@ public void onRetry() { } } } + + public void markCoordinatingBytesAccounted() { + assert bytesAccounted.get() == false; + bytesAccounted.set(true); + } + + public boolean isCoordinatingBytesAccounted() { + return bytesAccounted.get(); + } } diff --git a/server/src/main/java/org/elasticsearch/action/bulk/TransportBulkAction.java b/server/src/main/java/org/elasticsearch/action/bulk/TransportBulkAction.java index 770bb01533b22..34d9436cdeb48 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/TransportBulkAction.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/TransportBulkAction.java @@ -159,13 +159,11 @@ public static IndexRequest getIndexWriteRequest(DocWriteRequest docWriteReque @Override protected void doExecute(Task task, BulkRequest bulkRequest, ActionListener listener) { long indexingBytes = DocWriteRequest.writeSizeInBytes(bulkRequest.requests.stream()); - writeMemoryLimits.markOperationStarted(indexingBytes); - final Releasable releasable = () -> writeMemoryLimits.markOperationFinished(indexingBytes); + final Releasable releasable = writeMemoryLimits.markCoordinatingOperationStarted(indexingBytes); final ActionListener releasingListener = ActionListener.runAfter(listener, releasable::close); threadPool.executor(ThreadPool.Names.WRITE).execute(new ActionRunnable<>(releasingListener) { @Override protected void doRun() { - threadPool.getThreadContext().putTransient(WriteMemoryLimits.WRITE_BYTES_MARKED, true); doDispatchedExecute(task, bulkRequest, releasingListener); } }); @@ -534,6 +532,7 @@ protected void doRun() { if (task != null) { bulkShardRequest.setParentTask(nodeId, task.getId()); } + bulkShardRequest.markCoordinatingBytesAccounted(); client.executeLocally(TransportShardBulkAction.TYPE, bulkShardRequest, new ActionListener<>() { @Override public void onResponse(BulkShardResponse bulkShardResponse) { diff --git a/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java b/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java index 31758d7f86192..2b51dfea95721 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java @@ -111,6 +111,16 @@ protected BulkShardResponse newResponseInstance(StreamInput in) throws IOExcepti return new BulkShardResponse(in); } + @Override + protected boolean coordinatingBytesNeedAccounted(BulkShardRequest request) { + if (request.isCoordinatingBytesAccounted()) { + return false; + } else { + request.markCoordinatingBytesAccounted(); + return true; + } + } + @Override protected void dispatchedShardOperationOnPrimary(BulkShardRequest request, IndexShard primary, ActionListener> listener) { diff --git a/server/src/main/java/org/elasticsearch/action/bulk/WriteMemoryLimits.java b/server/src/main/java/org/elasticsearch/action/bulk/WriteMemoryLimits.java index a0a360c884a7c..3227a47eccef6 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/WriteMemoryLimits.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/WriteMemoryLimits.java @@ -19,6 +19,8 @@ package org.elasticsearch.action.bulk; +import org.elasticsearch.common.lease.Releasable; + import java.util.concurrent.atomic.AtomicLong; public class WriteMemoryLimits { @@ -27,9 +29,21 @@ public class WriteMemoryLimits { public static final int WRITE_REQUEST_BYTES_OVERHEAD = 4096; public static final String WRITE_BYTES_MARKED = "write_bytes_marked"; + private final AtomicLong coordinatingBytes = new AtomicLong(0); private final AtomicLong primaryCoordinatingBytes = new AtomicLong(0); + private final AtomicLong primaryBytes = new AtomicLong(0); private final AtomicLong replicaBytes = new AtomicLong(0); + public Releasable markCoordinatingOperationStarted(long bytes) { + coordinatingBytes.addAndGet(WRITE_REQUEST_BYTES_OVERHEAD + bytes); + return () -> coordinatingBytes.getAndAdd(-(WRITE_REQUEST_BYTES_OVERHEAD + bytes)); + } + + public Releasable markPrimaryOperationStarted(long bytes) { + primaryBytes.addAndGet(WRITE_REQUEST_BYTES_OVERHEAD + bytes); + return () -> primaryBytes.getAndAdd(-(WRITE_REQUEST_BYTES_OVERHEAD + bytes)); + } + public void markOperationStarted(long bytes) { primaryCoordinatingBytes.addAndGet(WRITE_REQUEST_BYTES_OVERHEAD + bytes); } @@ -38,12 +52,12 @@ public void markOperationFinished(long bytes) { primaryCoordinatingBytes.getAndAdd(-(WRITE_REQUEST_BYTES_OVERHEAD + bytes)); } - public void markReplicaOperationStarted(long bytes) { + public Releasable markReplicaOperationStarted(long bytes) { replicaBytes.getAndAdd(WRITE_REQUEST_BYTES_OVERHEAD + bytes); + return () -> replicaBytes.getAndAdd(-(WRITE_REQUEST_BYTES_OVERHEAD + bytes)); } public void markReplicaOperationFinished(long bytes) { replicaBytes.getAndAdd(-(WRITE_REQUEST_BYTES_OVERHEAD + bytes)); } - } diff --git a/server/src/main/java/org/elasticsearch/action/resync/TransportResyncReplicationAction.java b/server/src/main/java/org/elasticsearch/action/resync/TransportResyncReplicationAction.java index 044f12c315b12..638371f414111 100644 --- a/server/src/main/java/org/elasticsearch/action/resync/TransportResyncReplicationAction.java +++ b/server/src/main/java/org/elasticsearch/action/resync/TransportResyncReplicationAction.java @@ -46,6 +46,7 @@ import org.elasticsearch.transport.TransportService; import java.io.IOException; +import java.util.stream.Stream; public class TransportResyncReplicationAction extends TransportWriteAction implements PrimaryReplicaSyncer.SyncAction { @@ -92,6 +93,11 @@ protected void dispatchedShardOperationOnPrimary(ResyncReplicationRequest reques () -> 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) { return request; } @@ -105,6 +111,11 @@ protected void dispatchedShardOperationOnReplica(ResyncReplicationRequest reques }); } + @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 { Translog.Location location = null; /* diff --git a/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java b/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java index b571eb8c4ae3c..96305420e138c 100644 --- a/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java +++ b/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java @@ -275,12 +275,16 @@ boolean isRetryableClusterBlockException(final Throwable e) { } protected void handleOperationRequest(final Request request, final TransportChannel channel, Task task) { - Releasable releasable = checkPrimaryLimits(request); + Releasable releasable = checkOperationLimits(request); ActionListener listener = ActionListener.runAfter(new ChannelActionListener<>(channel, actionName, request), releasable::close); execute(task, request, listener); } + protected Releasable checkOperationLimits(final Request request) { + return () -> {}; + } + protected void handlePrimaryRequest(final ConcreteShardRequest request, final TransportChannel channel, final Task task) { Releasable releasable = checkPrimaryLimits(request.getRequest()); ActionListener listener = diff --git a/server/src/main/java/org/elasticsearch/action/support/replication/TransportWriteAction.java b/server/src/main/java/org/elasticsearch/action/support/replication/TransportWriteAction.java index fd54847234456..c2647e938224c 100644 --- a/server/src/main/java/org/elasticsearch/action/support/replication/TransportWriteAction.java +++ b/server/src/main/java/org/elasticsearch/action/support/replication/TransportWriteAction.java @@ -75,29 +75,31 @@ protected TransportWriteAction(Settings settings, String actionName, TransportSe } @Override - protected Releasable checkPrimaryLimits(Request request) { - super.checkPrimaryLimits(request); - long operationSizeInBytes = primaryOperationSize(request); - Object writeBytesMarked = threadPool.getThreadContext().getTransient(WriteMemoryLimits.WRITE_BYTES_MARKED); - if (Boolean.TRUE.equals(writeBytesMarked)) { - return () -> {}; + protected Releasable checkOperationLimits(Request request) { + if (coordinatingBytesNeedAccounted(request)) { + long operationSizeInBytes = primaryOperationSize(request); + return writeMemoryLimits.markCoordinatingOperationStarted(operationSizeInBytes); } else { - threadPool.getThreadContext().putTransient(WriteMemoryLimits.WRITE_BYTES_MARKED, true); - writeMemoryLimits.markOperationStarted(operationSizeInBytes); - return () -> writeMemoryLimits.markOperationFinished(operationSizeInBytes); + return () -> {}; } } + protected boolean coordinatingBytesNeedAccounted(Request request) { + return false; + } + + @Override + protected Releasable checkPrimaryLimits(Request request) { + return writeMemoryLimits.markPrimaryOperationStarted(primaryOperationSize(request)); + } + protected long primaryOperationSize(Request request) { return 0; } @Override protected Releasable checkReplicaLimits(ReplicaRequest request) { - super.checkReplicaLimits(request); - long operationSizeInBytes = replicaOperationSize(request); - writeMemoryLimits.markReplicaOperationStarted(operationSizeInBytes); - return () -> writeMemoryLimits.markReplicaOperationFinished(operationSizeInBytes); + return writeMemoryLimits.markReplicaOperationStarted(replicaOperationSize(request)); } protected long replicaOperationSize(ReplicaRequest request) { From 7708977b9e9c8cec44ebc7cb541637b725536155 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Wed, 3 Jun 2020 15:00:40 -0600 Subject: [PATCH 27/41] Tests --- .../action/bulk/TransportBulkActionTests.java | 39 +++++++++---------- .../TransportWriteActionTests.java | 8 ++-- 2 files changed, 23 insertions(+), 24 deletions(-) diff --git a/server/src/test/java/org/elasticsearch/action/bulk/TransportBulkActionTests.java b/server/src/test/java/org/elasticsearch/action/bulk/TransportBulkActionTests.java index 7e2dc739f5def..04fb89bf6f75f 100644 --- a/server/src/test/java/org/elasticsearch/action/bulk/TransportBulkActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/bulk/TransportBulkActionTests.java @@ -28,6 +28,7 @@ import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.ActionTestUtils; import org.elasticsearch.action.support.AutoCreateIndex; +import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.action.update.UpdateRequest; import org.elasticsearch.cluster.metadata.AliasMetadata; import org.elasticsearch.cluster.metadata.IndexMetadata; @@ -118,38 +119,36 @@ public void tearDown() throws Exception { public void testDeleteNonExistingDocDoesNotCreateIndex() throws Exception { BulkRequest bulkRequest = new BulkRequest().add(new DeleteRequest("index").id("id")); - ActionTestUtils.execute(bulkAction, null, bulkRequest, ActionListener.wrap(response -> { - assertFalse(bulkAction.indexCreated); - BulkItemResponse[] bulkResponses = ((BulkResponse) response).getItems(); - assertEquals(bulkResponses.length, 1); - assertTrue(bulkResponses[0].isFailed()); - assertTrue(bulkResponses[0].getFailure().getCause() instanceof IndexNotFoundException); - assertEquals("index", bulkResponses[0].getFailure().getIndex()); - }, exception -> { - throw new AssertionError(exception); - })); + PlainActionFuture future = PlainActionFuture.newFuture(); + ActionTestUtils.execute(bulkAction, null, bulkRequest, future); + + BulkResponse response = future.actionGet(); + assertFalse(bulkAction.indexCreated); + BulkItemResponse[] bulkResponses = ((BulkResponse) response).getItems(); + assertEquals(bulkResponses.length, 1); + assertTrue(bulkResponses[0].isFailed()); + assertTrue(bulkResponses[0].getFailure().getCause() instanceof IndexNotFoundException); + assertEquals("index", bulkResponses[0].getFailure().getIndex()); } public void testDeleteNonExistingDocExternalVersionCreatesIndex() throws Exception { BulkRequest bulkRequest = new BulkRequest() .add(new DeleteRequest("index").id("id").versionType(VersionType.EXTERNAL).version(0)); - ActionTestUtils.execute(bulkAction, null, bulkRequest, ActionListener.wrap(response -> { - assertTrue(bulkAction.indexCreated); - }, exception -> { - throw new AssertionError(exception); - })); + PlainActionFuture future = PlainActionFuture.newFuture(); + ActionTestUtils.execute(bulkAction, null, bulkRequest, future); + future.actionGet(); + assertTrue(bulkAction.indexCreated); } public void testDeleteNonExistingDocExternalGteVersionCreatesIndex() throws Exception { BulkRequest bulkRequest = new BulkRequest() .add(new DeleteRequest("index2").id("id").versionType(VersionType.EXTERNAL_GTE).version(0)); - ActionTestUtils.execute(bulkAction, null, bulkRequest, ActionListener.wrap(response -> { - assertTrue(bulkAction.indexCreated); - }, exception -> { - throw new AssertionError(exception); - })); + PlainActionFuture future = PlainActionFuture.newFuture(); + ActionTestUtils.execute(bulkAction, null, bulkRequest, future); + future.actionGet(); + assertTrue(bulkAction.indexCreated); } public void testGetIndexWriteRequest() throws Exception { diff --git a/server/src/test/java/org/elasticsearch/action/support/replication/TransportWriteActionTests.java b/server/src/test/java/org/elasticsearch/action/support/replication/TransportWriteActionTests.java index f0e51da276bfb..c1ef04b10d43d 100644 --- a/server/src/test/java/org/elasticsearch/action/support/replication/TransportWriteActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/support/replication/TransportWriteActionTests.java @@ -138,7 +138,7 @@ public void testPrimaryNoRefreshCall() throws Exception { TestRequest request = new TestRequest(); request.setRefreshPolicy(RefreshPolicy.NONE); // The default, but we'll set it anyway just to be explicit TestAction testAction = new TestAction(); - testAction.shardOperationOnPrimary(request, indexShard, + testAction.dispatchedShardOperationOnPrimary(request, indexShard, ActionTestUtils.assertNoFailureListener(result -> { CapturingActionListener listener = new CapturingActionListener<>(); result.runPostReplicationActions(ActionListener.map(listener, ignore -> result.finalResponseIfSuccessful)); @@ -168,7 +168,7 @@ public void testPrimaryImmediateRefresh() throws Exception { TestRequest request = new TestRequest(); request.setRefreshPolicy(RefreshPolicy.IMMEDIATE); TestAction testAction = new TestAction(); - testAction.shardOperationOnPrimary(request, indexShard, + testAction.dispatchedShardOperationOnPrimary(request, indexShard, ActionTestUtils.assertNoFailureListener(result -> { CapturingActionListener listener = new CapturingActionListener<>(); result.runPostReplicationActions(ActionListener.map(listener, ignore -> result.finalResponseIfSuccessful)); @@ -200,7 +200,7 @@ public void testPrimaryWaitForRefresh() throws Exception { request.setRefreshPolicy(RefreshPolicy.WAIT_UNTIL); TestAction testAction = new TestAction(); - testAction.shardOperationOnPrimary(request, indexShard, + testAction.dispatchedShardOperationOnPrimary(request, indexShard, ActionTestUtils.assertNoFailureListener(result -> { CapturingActionListener listener = new CapturingActionListener<>(); result.runPostReplicationActions(ActionListener.map(listener, ignore -> result.finalResponseIfSuccessful)); @@ -245,7 +245,7 @@ public void testReplicaWaitForRefresh() throws Exception { public void testDocumentFailureInShardOperationOnPrimary() throws Exception { TestRequest request = new TestRequest(); TestAction testAction = new TestAction(true, true); - testAction.shardOperationOnPrimary(request, indexShard, + testAction.dispatchedShardOperationOnPrimary(request, indexShard, ActionTestUtils.assertNoFailureListener(result -> { CapturingActionListener listener = new CapturingActionListener<>(); result.runPostReplicationActions(ActionListener.map(listener, ignore -> result.finalResponseIfSuccessful)); From 5295df9ac235e37c3b36970edff4d5df2e675aae Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Wed, 3 Jun 2020 15:46:24 -0600 Subject: [PATCH 28/41] Changes --- .../java/org/elasticsearch/action/bulk/WriteMemoryLimitsIT.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/server/src/internalClusterTest/java/org/elasticsearch/action/bulk/WriteMemoryLimitsIT.java b/server/src/internalClusterTest/java/org/elasticsearch/action/bulk/WriteMemoryLimitsIT.java index 6f26292debdda..45ff6b07223f1 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/action/bulk/WriteMemoryLimitsIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/action/bulk/WriteMemoryLimitsIT.java @@ -18,6 +18,7 @@ */ package org.elasticsearch.action.bulk; +import org.apache.lucene.util.LuceneTestCase; import org.elasticsearch.action.ActionFuture; import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.common.settings.Settings; @@ -35,6 +36,7 @@ import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; +@LuceneTestCase.AwaitsFix(bugUrl = "") @ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.SUITE, numDataNodes = 2) public class WriteMemoryLimitsIT extends ESIntegTestCase { From 2227059ff6e0d8f3e3d8f98f86098635ba90fb9b Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Wed, 3 Jun 2020 15:59:30 -0600 Subject: [PATCH 29/41] Mute tests --- .../action/admin/cluster/node/tasks/TaskStorageRetryIT.java | 2 ++ .../org/elasticsearch/action/bulk/BulkProcessorRetryIT.java | 2 ++ .../org/elasticsearch/action/bulk/WriteMemoryLimitsIT.java | 5 +++++ 3 files changed, 9 insertions(+) diff --git a/server/src/internalClusterTest/java/org/elasticsearch/action/admin/cluster/node/tasks/TaskStorageRetryIT.java b/server/src/internalClusterTest/java/org/elasticsearch/action/admin/cluster/node/tasks/TaskStorageRetryIT.java index a2e645b457a8a..a62960311df35 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/action/admin/cluster/node/tasks/TaskStorageRetryIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/action/admin/cluster/node/tasks/TaskStorageRetryIT.java @@ -19,6 +19,7 @@ package org.elasticsearch.action.admin.cluster.node.tasks; +import org.apache.lucene.util.LuceneTestCase; import org.elasticsearch.action.admin.cluster.node.tasks.get.GetTaskResponse; import org.elasticsearch.action.support.PlainListenableActionFuture; import org.elasticsearch.client.node.NodeClient; @@ -41,6 +42,7 @@ * Makes sure that tasks that attempt to store themselves on completion retry if * they don't succeed at first. */ +@LuceneTestCase.AwaitsFix(bugUrl = "") public class TaskStorageRetryIT extends ESSingleNodeTestCase { @Override protected Collection> getPlugins() { diff --git a/server/src/internalClusterTest/java/org/elasticsearch/action/bulk/BulkProcessorRetryIT.java b/server/src/internalClusterTest/java/org/elasticsearch/action/bulk/BulkProcessorRetryIT.java index 0bb0b89c0042c..36d207211703b 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/action/bulk/BulkProcessorRetryIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/action/bulk/BulkProcessorRetryIT.java @@ -18,6 +18,7 @@ */ package org.elasticsearch.action.bulk; +import org.apache.lucene.util.LuceneTestCase; import org.elasticsearch.action.admin.indices.refresh.RefreshRequest; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.common.settings.Settings; @@ -40,6 +41,7 @@ import static org.hamcrest.Matchers.lessThan; import static org.hamcrest.Matchers.lessThanOrEqualTo; +@LuceneTestCase.AwaitsFix(bugUrl = "") @ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.SUITE, numDataNodes = 2) public class BulkProcessorRetryIT extends ESIntegTestCase { private static final String INDEX_NAME = "test"; diff --git a/server/src/internalClusterTest/java/org/elasticsearch/action/bulk/WriteMemoryLimitsIT.java b/server/src/internalClusterTest/java/org/elasticsearch/action/bulk/WriteMemoryLimitsIT.java index 45ff6b07223f1..f7330fecda0c4 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/action/bulk/WriteMemoryLimitsIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/action/bulk/WriteMemoryLimitsIT.java @@ -64,6 +64,11 @@ protected int numberOfShards() { return 1; } + @Override + public void setUp() throws Exception { + super.setUp(); + } + public void testRejectionDueToRequestOverMemoryLimit() { final String index = "test"; assertAcked(prepareCreate(index)); From 1e421f08dab1cea387cbfb91464ac9705973d918 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Wed, 3 Jun 2020 19:38:27 -0600 Subject: [PATCH 30/41] Changes --- .../action/bulk/WriteMemoryLimitsIT.java | 167 +++++++++++------- .../action/bulk/WriteMemoryLimits.java | 18 +- 2 files changed, 109 insertions(+), 76 deletions(-) diff --git a/server/src/internalClusterTest/java/org/elasticsearch/action/bulk/WriteMemoryLimitsIT.java b/server/src/internalClusterTest/java/org/elasticsearch/action/bulk/WriteMemoryLimitsIT.java index f7330fecda0c4..e4ab8e42b49c5 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/action/bulk/WriteMemoryLimitsIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/action/bulk/WriteMemoryLimitsIT.java @@ -18,12 +18,15 @@ */ package org.elasticsearch.action.bulk; -import org.apache.lucene.util.LuceneTestCase; 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.action.index.IndexResponse; +import org.elasticsearch.cluster.metadata.IndexMetadata; +import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.plugins.Plugin; -import org.elasticsearch.rest.RestStatus; import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.InternalSettingsPlugin; import org.elasticsearch.test.transport.MockTransportService; @@ -33,10 +36,10 @@ 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; -@LuceneTestCase.AwaitsFix(bugUrl = "") @ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.SUITE, numDataNodes = 2) public class WriteMemoryLimitsIT extends ESIntegTestCase { @@ -64,91 +67,123 @@ protected int numberOfShards() { return 1; } - @Override - public void setUp() throws Exception { - super.setUp(); - } - - public void testRejectionDueToRequestOverMemoryLimit() { + public void testWriteBytesAreIncremented() throws Exception { final String index = "test"; - assertAcked(prepareCreate(index)); + 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 lessThan1KB = new BulkRequest(); - for (int i = 0; i < 3; ++i) { - lessThan1KB.add(new IndexRequest(index).source(Collections.singletonMap("key", "value" + i))); + int totalSourceLength = 0; + for (int i = 0; i < 80; ++i) { + IndexRequest request = new IndexRequest(index).source(Collections.singletonMap("key", randomAlphaOfLength(50))); + totalSourceLength += request.source().length(); + lessThan1KB.add(request); } - final BulkRequest moreThan1KB = new BulkRequest(); - for (int i = 0; i < 4; ++i) { - moreThan1KB.add(new IndexRequest(index).source(Collections.singletonMap("key", "value" + i))); - } -// assertThat(DocWriteRequest.writeSizeInBytes(lessThan1KB.requests.stream()), lessThan(1024L)); - assertFalse(client().bulk(lessThan1KB).actionGet().hasFailures()); - -// assertThat(DocWriteRequest.writeSizeInBytes(moreThan1KB.requests.stream()), greaterThan(1024L)); - final ActionFuture bulkFuture2 = client().bulk(moreThan1KB); - final BulkResponse failedResponses = bulkFuture2.actionGet(); - for (BulkItemResponse response : failedResponses) { - assertEquals(RestStatus.TOO_MANY_REQUESTS, response.getFailure().getStatus()); - } - } - public void testRejectionDueToConcurrentRequestsOverMemoryLimit() throws InterruptedException { - final String index = "test"; - assertAcked(prepareCreate(index)); - ensureGreen(); - final CountDownLatch replicationStarted = new CountDownLatch(1); - final CountDownLatch replicationBlock = new CountDownLatch(1); + final int operationSize = totalSourceLength + WriteMemoryLimits.WRITE_REQUEST_BYTES_OVERHEAD; + + try { + final ActionFuture successFuture = client(replicaName).bulk(lessThan1KB); + replicationSendPointReached.await(); - for (TransportService transportService : internalCluster().getInstances(TransportService.class)) { - final MockTransportService mockTransportService = (MockTransportService) transportService; - mockTransportService.addSendBehavior((connection, requestId, action, request, options) -> { - if (action.equals(TransportShardBulkAction.ACTION_NAME + "[r]")) { + WriteMemoryLimits primaryWriteLimits = internalCluster().getInstance(WriteMemoryLimits.class, primaryName); + WriteMemoryLimits replicaWriteLimits = internalCluster().getInstance(WriteMemoryLimits.class, replicaName); + + assertEquals(operationSize, primaryWriteLimits.getCoordinatingBytes()); + assertEquals(operationSize, primaryWriteLimits.getPrimaryBytes()); + assertEquals(0, primaryWriteLimits.getReplicaBytes()); + assertEquals(operationSize, replicaWriteLimits.getCoordinatingBytes()); + assertEquals(0, replicaWriteLimits.getPrimaryBytes()); + assertEquals(0, replicaWriteLimits.getReplicaBytes()); + + replicaTransportService.addSendBehavior((connection, requestId, action, request, options) -> { + if (action.equals(TransportShardBulkAction.ACTION_NAME)) { try { - replicationStarted.countDown(); - replicationBlock.await(); + newActionsSendPointReached.countDown(); + latchBlockingReplication.await(); } catch (InterruptedException e) { throw new IllegalStateException(e); } } connection.sendRequest(requestId, action, request, options); }); - } - final BulkRequest lessThan1KB = new BulkRequest(); - for (int i = 0; i < 3; ++i) { - lessThan1KB.add(new IndexRequest(index).source(Collections.singletonMap("key", "value" + i))); - } -// assertThat(DocWriteRequest.writeSizeInBytes(lessThan1KB.requests.stream()), lessThan(1024L)); + IndexRequest request1 = new IndexRequest(index).source(Collections.singletonMap("key", randomAlphaOfLength(50))); + IndexRequest request2 = new IndexRequest(index).source(Collections.singletonMap("key", randomAlphaOfLength(50))); + ActionFuture future1 = client(replicaName).index(request1); + ActionFuture future2 = client(replicaName).index(request2); - final BulkRequest rejectedRequest = new BulkRequest(); - for (int i = 0; i < 2; ++i) { - rejectedRequest.add(new IndexRequest(index).source(Collections.singletonMap("key", "value" + i))); - } -// assertThat(DocWriteRequest.writeSizeInBytes(rejectedRequest.requests.stream()), lessThan(1024L)); + newActionsSendPointReached.await(); + latchBlockingReplicationSend.countDown(); - try { - final ActionFuture successFuture = client().bulk(lessThan1KB); - replicationStarted.await(); - - final ActionFuture rejectedFuture = client().bulk(rejectedRequest); - final BulkResponse failedResponses = rejectedFuture.actionGet(); - assertTrue(failedResponses.hasFailures()); - for (BulkItemResponse response : failedResponses) { - assertEquals(RestStatus.TOO_MANY_REQUESTS, response.getFailure().getStatus()); - } + int newOperationSizes = request1.source().length() + request2.source().length() + + (WriteMemoryLimits.WRITE_REQUEST_BYTES_OVERHEAD * 2); + + assertEquals(operationSize + newOperationSizes, replicaWriteLimits.getCoordinatingBytes()); + assertBusy(() -> assertEquals(operationSize, replicaWriteLimits.getReplicaBytes())); - replicationBlock.countDown(); + latchBlockingReplication.countDown(); - final BulkResponse successResponses = successFuture.actionGet(); - assertFalse(successResponses.hasFailures()); + successFuture.actionGet(); + future1.actionGet(); + future2.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 { - for (TransportService transportService : internalCluster().getInstances(TransportService.class)) { - final MockTransportService mockTransportService = (MockTransportService) transportService; - mockTransportService.clearAllRules(); + if (latchBlockingReplicationSend.getCount() > 0) { + latchBlockingReplicationSend.countDown(); + } + if (latchBlockingReplication.getCount() > 0) { + latchBlockingReplicationSend.countDown(); } + primaryTransportService.clearAllRules(); } - } } diff --git a/server/src/main/java/org/elasticsearch/action/bulk/WriteMemoryLimits.java b/server/src/main/java/org/elasticsearch/action/bulk/WriteMemoryLimits.java index 3227a47eccef6..d4dbb2d27e1dd 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/WriteMemoryLimits.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/WriteMemoryLimits.java @@ -27,10 +27,8 @@ public class WriteMemoryLimits { // A heuristic for the bytes overhead of a single write operation public static final int WRITE_REQUEST_BYTES_OVERHEAD = 4096; - public static final String WRITE_BYTES_MARKED = "write_bytes_marked"; private final AtomicLong coordinatingBytes = new AtomicLong(0); - private final AtomicLong primaryCoordinatingBytes = new AtomicLong(0); private final AtomicLong primaryBytes = new AtomicLong(0); private final AtomicLong replicaBytes = new AtomicLong(0); @@ -39,17 +37,17 @@ public Releasable markCoordinatingOperationStarted(long bytes) { return () -> coordinatingBytes.getAndAdd(-(WRITE_REQUEST_BYTES_OVERHEAD + bytes)); } + public long getCoordinatingBytes() { + return coordinatingBytes.get(); + } + public Releasable markPrimaryOperationStarted(long bytes) { primaryBytes.addAndGet(WRITE_REQUEST_BYTES_OVERHEAD + bytes); return () -> primaryBytes.getAndAdd(-(WRITE_REQUEST_BYTES_OVERHEAD + bytes)); } - public void markOperationStarted(long bytes) { - primaryCoordinatingBytes.addAndGet(WRITE_REQUEST_BYTES_OVERHEAD + bytes); - } - - public void markOperationFinished(long bytes) { - primaryCoordinatingBytes.getAndAdd(-(WRITE_REQUEST_BYTES_OVERHEAD + bytes)); + public long getPrimaryBytes() { + return primaryBytes.get(); } public Releasable markReplicaOperationStarted(long bytes) { @@ -57,7 +55,7 @@ public Releasable markReplicaOperationStarted(long bytes) { return () -> replicaBytes.getAndAdd(-(WRITE_REQUEST_BYTES_OVERHEAD + bytes)); } - public void markReplicaOperationFinished(long bytes) { - replicaBytes.getAndAdd(-(WRITE_REQUEST_BYTES_OVERHEAD + bytes)); + public long getReplicaBytes() { + return replicaBytes.get(); } } From d461c36e07ee613580ab6a430d96769eb00c7c6a Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Mon, 8 Jun 2020 18:12:14 -0600 Subject: [PATCH 31/41] WIP --- .../action/bulk/WriteMemoryLimitsIT.java | 34 +++++++++++-------- .../action/bulk/TransportBulkAction.java | 13 +++---- .../TransportReplicationAction.java | 21 +++--------- .../replication/TransportWriteAction.java | 12 +++++++ .../common/breaker/CircuitBreaker.java | 2 -- .../transport/TransportChannel.java | 3 +- 6 files changed, 41 insertions(+), 44 deletions(-) diff --git a/server/src/internalClusterTest/java/org/elasticsearch/action/bulk/WriteMemoryLimitsIT.java b/server/src/internalClusterTest/java/org/elasticsearch/action/bulk/WriteMemoryLimitsIT.java index e4ab8e42b49c5..f50064ba101a1 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/action/bulk/WriteMemoryLimitsIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/action/bulk/WriteMemoryLimitsIT.java @@ -30,6 +30,7 @@ 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; @@ -120,8 +121,6 @@ public void testWriteBytesAreIncremented() throws Exception { lessThan1KB.add(request); } - - final int operationSize = totalSourceLength + WriteMemoryLimits.WRITE_REQUEST_BYTES_OVERHEAD; try { @@ -138,26 +137,33 @@ public void testWriteBytesAreIncremented() throws Exception { assertEquals(0, replicaWriteLimits.getPrimaryBytes()); assertEquals(0, replicaWriteLimits.getReplicaBytes()); - replicaTransportService.addSendBehavior((connection, requestId, action, request, options) -> { - if (action.equals(TransportShardBulkAction.ACTION_NAME)) { - try { - newActionsSendPointReached.countDown(); - latchBlockingReplication.await(); - } catch (InterruptedException e) { - throw new IllegalStateException(e); - } + 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); } - connection.sendRequest(requestId, action, request, options); }); + newActionsSendPointReached.await(); + latchBlockingReplicationSend.countDown(); IndexRequest request1 = new IndexRequest(index).source(Collections.singletonMap("key", randomAlphaOfLength(50))); IndexRequest request2 = new IndexRequest(index).source(Collections.singletonMap("key", randomAlphaOfLength(50))); + ActionFuture future1 = client(replicaName).index(request1); ActionFuture future2 = client(replicaName).index(request2); - newActionsSendPointReached.await(); - latchBlockingReplicationSend.countDown(); - int newOperationSizes = request1.source().length() + request2.source().length() + (WriteMemoryLimits.WRITE_REQUEST_BYTES_OVERHEAD * 2); diff --git a/server/src/main/java/org/elasticsearch/action/bulk/TransportBulkAction.java b/server/src/main/java/org/elasticsearch/action/bulk/TransportBulkAction.java index 34d9436cdeb48..d6dc52239d74c 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/TransportBulkAction.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/TransportBulkAction.java @@ -161,15 +161,10 @@ protected void doExecute(Task task, BulkRequest bulkRequest, ActionListener releasingListener = ActionListener.runAfter(listener, releasable::close); - threadPool.executor(ThreadPool.Names.WRITE).execute(new ActionRunnable<>(releasingListener) { - @Override - protected void doRun() { - doDispatchedExecute(task, bulkRequest, releasingListener); - } - }); + doInternalExecute(task, bulkRequest, releasingListener); } - protected void doDispatchedExecute(Task task, BulkRequest bulkRequest, ActionListener listener) { + protected void doInternalExecute(Task task, BulkRequest bulkRequest, ActionListener listener) { final long startTime = relativeTime(); final AtomicArray responses = new AtomicArray<>(bulkRequest.requests.size()); @@ -705,7 +700,7 @@ private void processBulkIndexIngestRequest(Task task, BulkRequest original, Acti // before we continue the bulk request we should fork back on a write thread: if (originalThread == Thread.currentThread()) { assert Thread.currentThread().getName().contains(ThreadPool.Names.WRITE); - doDispatchedExecute(task, bulkRequest, actionListener); + doInternalExecute(task, bulkRequest, actionListener); } else { threadPool.executor(ThreadPool.Names.WRITE).execute(new AbstractRunnable() { @Override @@ -715,7 +710,7 @@ public void onFailure(Exception e) { @Override protected void doRun() throws Exception { - doDispatchedExecute(task, bulkRequest, actionListener); + doInternalExecute(task, bulkRequest, actionListener); } @Override diff --git a/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java b/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java index 96305420e138c..d34b96538f831 100644 --- a/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java +++ b/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java @@ -27,7 +27,6 @@ import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionListenerResponseHandler; import org.elasticsearch.action.ActionResponse; -import org.elasticsearch.action.ActionRunnable; import org.elasticsearch.action.UnavailableShardsException; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.ActiveShardCount; @@ -161,11 +160,11 @@ protected TransportReplicationAction(Settings settings, String actionName, Trans transportService.registerRequestHandler(actionName, ThreadPool.Names.SAME, requestReader, this::handleOperationRequest); - transportService.registerRequestHandler(transportPrimaryAction, ThreadPool.Names.SAME, forceExecutionOnPrimary, true, + transportService.registerRequestHandler(transportPrimaryAction, executor, forceExecutionOnPrimary, true, in -> new ConcreteShardRequest<>(requestReader, in), this::handlePrimaryRequest); // we must never reject on because of thread pool capacity on replicas - transportService.registerRequestHandler(transportReplicaAction, ThreadPool.Names.SAME, true, true, + transportService.registerRequestHandler(transportReplicaAction, executor, true, true, in -> new ConcreteReplicaRequest<>(replicaRequestReader, in), this::handleReplicaRequest); this.transportOptions = transportOptions(settings); @@ -290,12 +289,7 @@ protected void handlePrimaryRequest(final ConcreteShardRequest request, ActionListener listener = ActionListener.runAfter(new ChannelActionListener<>(channel, transportPrimaryAction, request), releasable::close); - threadPool.executor(executor).execute(new ActionRunnable<>(listener) { - @Override - protected void doRun() { - new AsyncPrimaryAction(request, listener, (ReplicationTask) task).run(); - } - }); + new AsyncPrimaryAction(request, listener, (ReplicationTask) task).run(); } protected Releasable checkPrimaryLimits(final Request request) { @@ -515,14 +509,7 @@ protected void handleReplicaRequest(final ConcreteReplicaRequest Releasable releasable = checkReplicaLimits(replicaRequest.getRequest()); ActionListener listener = ActionListener.runAfter(new ChannelActionListener<>(channel, transportReplicaAction, replicaRequest), releasable::close); - - threadPool.executor(executor).execute(new ActionRunnable<>(listener) { - @Override - protected void doRun() { - new AsyncReplicaAction( - replicaRequest, listener, (ReplicationTask) task).run(); - } - }); + new AsyncReplicaAction(replicaRequest, listener, (ReplicationTask) task).run(); } protected Releasable checkReplicaLimits(final ReplicaRequest request) { diff --git a/server/src/main/java/org/elasticsearch/action/support/replication/TransportWriteAction.java b/server/src/main/java/org/elasticsearch/action/support/replication/TransportWriteAction.java index c2647e938224c..fa57bdf620ece 100644 --- a/server/src/main/java/org/elasticsearch/action/support/replication/TransportWriteAction.java +++ b/server/src/main/java/org/elasticsearch/action/support/replication/TransportWriteAction.java @@ -60,6 +60,7 @@ public abstract class TransportWriteAction< Response extends ReplicationResponse & WriteResponse > extends TransportReplicationAction { + private final boolean forceExecutionOnPrimary; private final WriteMemoryLimits writeMemoryLimits; private final String executor; @@ -71,6 +72,7 @@ protected TransportWriteAction(Settings settings, String actionName, TransportSe super(settings, actionName, transportService, clusterService, indicesService, threadPool, shardStateAction, actionFilters, request, replicaRequest, ThreadPool.Names.SAME, true, forceExecutionOnPrimary); this.executor = executor; + this.forceExecutionOnPrimary = forceExecutionOnPrimary; this.writeMemoryLimits = writeMemoryLimits; } @@ -152,6 +154,11 @@ protected void shardOperationOnPrimary( protected void doRun() { dispatchedShardOperationOnPrimary(request, primary, listener); } + + @Override + public boolean isForceExecution() { + return forceExecutionOnPrimary; + } }); } @@ -172,6 +179,11 @@ protected void shardOperationOnReplica(ReplicaRequest request, IndexShard replic protected void doRun() { dispatchedShardOperationOnReplica(request, replica, listener); } + + @Override + public boolean isForceExecution() { + return true; + } }); } diff --git a/server/src/main/java/org/elasticsearch/common/breaker/CircuitBreaker.java b/server/src/main/java/org/elasticsearch/common/breaker/CircuitBreaker.java index 9b2a5898c5821..7e5b6b2f42228 100644 --- a/server/src/main/java/org/elasticsearch/common/breaker/CircuitBreaker.java +++ b/server/src/main/java/org/elasticsearch/common/breaker/CircuitBreaker.java @@ -60,8 +60,6 @@ public interface CircuitBreaker { * segments. */ String ACCOUNTING = "accounting"; - // TODO: Description - String INDEXING = "indexing"; enum Type { // A regular or ChildMemoryCircuitBreaker diff --git a/server/src/main/java/org/elasticsearch/transport/TransportChannel.java b/server/src/main/java/org/elasticsearch/transport/TransportChannel.java index c2cce0ea40dde..bdec69131156a 100644 --- a/server/src/main/java/org/elasticsearch/transport/TransportChannel.java +++ b/server/src/main/java/org/elasticsearch/transport/TransportChannel.java @@ -23,7 +23,6 @@ import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; import org.elasticsearch.Version; -import org.elasticsearch.action.support.ChannelActionListener; import java.io.IOException; @@ -32,7 +31,7 @@ */ public interface TransportChannel { - Logger logger = LogManager.getLogger(ChannelActionListener.class); + Logger logger = LogManager.getLogger(TransportChannel.class); String getProfileName(); From f06985626cd0f60f9c155e26a75d7969127ff059 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Tue, 9 Jun 2020 18:32:07 -0600 Subject: [PATCH 32/41] Changes --- .../node/tasks/TaskStorageRetryIT.java | 2 - .../action/bulk/BulkProcessorRetryIT.java | 2 - .../action/bulk/TransportShardBulkAction.java | 2 +- .../TransportReplicationAction.java | 56 ++++++++++++------- .../replication/TransportWriteAction.java | 8 ++- 5 files changed, 41 insertions(+), 29 deletions(-) diff --git a/server/src/internalClusterTest/java/org/elasticsearch/action/admin/cluster/node/tasks/TaskStorageRetryIT.java b/server/src/internalClusterTest/java/org/elasticsearch/action/admin/cluster/node/tasks/TaskStorageRetryIT.java index a62960311df35..a2e645b457a8a 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/action/admin/cluster/node/tasks/TaskStorageRetryIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/action/admin/cluster/node/tasks/TaskStorageRetryIT.java @@ -19,7 +19,6 @@ package org.elasticsearch.action.admin.cluster.node.tasks; -import org.apache.lucene.util.LuceneTestCase; import org.elasticsearch.action.admin.cluster.node.tasks.get.GetTaskResponse; import org.elasticsearch.action.support.PlainListenableActionFuture; import org.elasticsearch.client.node.NodeClient; @@ -42,7 +41,6 @@ * Makes sure that tasks that attempt to store themselves on completion retry if * they don't succeed at first. */ -@LuceneTestCase.AwaitsFix(bugUrl = "") public class TaskStorageRetryIT extends ESSingleNodeTestCase { @Override protected Collection> getPlugins() { diff --git a/server/src/internalClusterTest/java/org/elasticsearch/action/bulk/BulkProcessorRetryIT.java b/server/src/internalClusterTest/java/org/elasticsearch/action/bulk/BulkProcessorRetryIT.java index 36d207211703b..0bb0b89c0042c 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/action/bulk/BulkProcessorRetryIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/action/bulk/BulkProcessorRetryIT.java @@ -18,7 +18,6 @@ */ package org.elasticsearch.action.bulk; -import org.apache.lucene.util.LuceneTestCase; import org.elasticsearch.action.admin.indices.refresh.RefreshRequest; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.common.settings.Settings; @@ -41,7 +40,6 @@ import static org.hamcrest.Matchers.lessThan; import static org.hamcrest.Matchers.lessThanOrEqualTo; -@LuceneTestCase.AwaitsFix(bugUrl = "") @ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.SUITE, numDataNodes = 2) public class BulkProcessorRetryIT extends ESIntegTestCase { private static final String INDEX_NAME = "test"; diff --git a/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java b/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java index 2b51dfea95721..dce9b1b629693 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java @@ -112,7 +112,7 @@ protected BulkShardResponse newResponseInstance(StreamInput in) throws IOExcepti } @Override - protected boolean coordinatingBytesNeedAccounted(BulkShardRequest request) { + protected boolean shouldMarkCoordinatingBytes(BulkShardRequest request) { if (request.isCoordinatingBytesAccounted()) { return false; } else { diff --git a/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java b/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java index d34b96538f831..7374e8d84079e 100644 --- a/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java +++ b/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java @@ -289,7 +289,11 @@ protected void handlePrimaryRequest(final ConcreteShardRequest request, ActionListener listener = ActionListener.runAfter(new ChannelActionListener<>(channel, transportPrimaryAction, request), releasable::close); - new AsyncPrimaryAction(request, listener, (ReplicationTask) task).run(); + try { + new AsyncPrimaryAction(request, listener, (ReplicationTask) task).run(); + } catch (RuntimeException e) { + listener.onFailure(e); + } } protected Releasable checkPrimaryLimits(final Request request) { @@ -505,11 +509,16 @@ public void runPostReplicaActions(ActionListener listener) { } protected void handleReplicaRequest(final ConcreteReplicaRequest replicaRequest, final TransportChannel channel, - final Task task) { + final Task task) { Releasable releasable = checkReplicaLimits(replicaRequest.getRequest()); ActionListener listener = ActionListener.runAfter(new ChannelActionListener<>(channel, transportReplicaAction, replicaRequest), releasable::close); - new AsyncReplicaAction(replicaRequest, listener, (ReplicationTask) task).run(); + + try { + new AsyncReplicaAction(replicaRequest, listener, (ReplicationTask) task).run(); + } catch (RuntimeException e) { + listener.onFailure(e); + } } protected Releasable checkReplicaLimits(final ReplicaRequest request) { @@ -553,27 +562,32 @@ private final class AsyncReplicaAction extends AbstractRunnable implements Actio @Override public void onResponse(Releasable releasable) { assert replica.getActiveOperationsCount() != 0 : "must perform shard operation under a permit"; - shardOperationOnReplica(replicaRequest.getRequest(), replica, ActionListener.wrap((replicaResult) -> - replicaResult.runPostReplicaActions( - ActionListener.wrap(r -> { - final ReplicaResponse response = - new ReplicaResponse(replica.getLocalCheckpoint(), replica.getLastSyncedGlobalCheckpoint()); - releasable.close(); // release shard operation lock before responding to caller - if (logger.isTraceEnabled()) { - logger.trace("action [{}] completed on shard [{}] for request [{}]", transportReplicaAction, - replicaRequest.getRequest().shardId(), - replicaRequest.getRequest()); - } - setPhase(task, "finished"); - onCompletionListener.onResponse(response); + try { + shardOperationOnReplica(replicaRequest.getRequest(), replica, ActionListener.wrap((replicaResult) -> + replicaResult.runPostReplicaActions( + ActionListener.wrap(r -> { + final ReplicaResponse response = + new ReplicaResponse(replica.getLocalCheckpoint(), replica.getLastSyncedGlobalCheckpoint()); + releasable.close(); // release shard operation lock before responding to caller + if (logger.isTraceEnabled()) { + logger.trace("action [{}] completed on shard [{}] for request [{}]", transportReplicaAction, + replicaRequest.getRequest().shardId(), + replicaRequest.getRequest()); + } + setPhase(task, "finished"); + onCompletionListener.onResponse(response); }, e -> { - Releasables.closeWhileHandlingException(releasable); // release shard operation lock before responding to caller - responseWithFailure(e); - }) - ), e -> { + Releasables.closeWhileHandlingException(releasable); // release shard operation lock before responding to caller + responseWithFailure(e); + }) + ), e -> { + Releasables.closeWhileHandlingException(releasable); // release shard operation lock before responding to caller + AsyncReplicaAction.this.onFailure(e); + })); + } catch (Exception e) { Releasables.closeWhileHandlingException(releasable); // release shard operation lock before responding to caller AsyncReplicaAction.this.onFailure(e); - })); + } } @Override diff --git a/server/src/main/java/org/elasticsearch/action/support/replication/TransportWriteAction.java b/server/src/main/java/org/elasticsearch/action/support/replication/TransportWriteAction.java index fa57bdf620ece..f24a06bbcd296 100644 --- a/server/src/main/java/org/elasticsearch/action/support/replication/TransportWriteAction.java +++ b/server/src/main/java/org/elasticsearch/action/support/replication/TransportWriteAction.java @@ -69,6 +69,8 @@ protected TransportWriteAction(Settings settings, String actionName, TransportSe ShardStateAction shardStateAction, ActionFilters actionFilters, Writeable.Reader request, Writeable.Reader 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, request, replicaRequest, ThreadPool.Names.SAME, true, forceExecutionOnPrimary); this.executor = executor; @@ -78,7 +80,7 @@ protected TransportWriteAction(Settings settings, String actionName, TransportSe @Override protected Releasable checkOperationLimits(Request request) { - if (coordinatingBytesNeedAccounted(request)) { + if (shouldMarkCoordinatingBytes(request)) { long operationSizeInBytes = primaryOperationSize(request); return writeMemoryLimits.markCoordinatingOperationStarted(operationSizeInBytes); } else { @@ -86,8 +88,8 @@ protected Releasable checkOperationLimits(Request request) { } } - protected boolean coordinatingBytesNeedAccounted(Request request) { - return false; + protected boolean shouldMarkCoordinatingBytes(Request request) { + return true; } @Override From da1540960ed855352555f3b6883bae271a6f38e3 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Thu, 11 Jun 2020 17:30:28 -0600 Subject: [PATCH 33/41] Changes --- .../elasticsearch/action/bulk/WriteMemoryLimitsIT.java | 8 ++------ .../elasticsearch/action/bulk/TransportBulkAction.java | 8 ++++++-- .../action/bulk/TransportShardBulkAction.java | 6 ------ 3 files changed, 8 insertions(+), 14 deletions(-) diff --git a/server/src/internalClusterTest/java/org/elasticsearch/action/bulk/WriteMemoryLimitsIT.java b/server/src/internalClusterTest/java/org/elasticsearch/action/bulk/WriteMemoryLimitsIT.java index f50064ba101a1..731b40da6eced 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/action/bulk/WriteMemoryLimitsIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/action/bulk/WriteMemoryLimitsIT.java @@ -159,22 +159,18 @@ public void testWriteBytesAreIncremented() throws Exception { latchBlockingReplicationSend.countDown(); IndexRequest request1 = new IndexRequest(index).source(Collections.singletonMap("key", randomAlphaOfLength(50))); - IndexRequest request2 = new IndexRequest(index).source(Collections.singletonMap("key", randomAlphaOfLength(50))); ActionFuture future1 = client(replicaName).index(request1); - ActionFuture future2 = client(replicaName).index(request2); - int newOperationSizes = request1.source().length() + request2.source().length() + - (WriteMemoryLimits.WRITE_REQUEST_BYTES_OVERHEAD * 2); + int newOperationSizes = request1.source().length() + (WriteMemoryLimits.WRITE_REQUEST_BYTES_OVERHEAD); assertEquals(operationSize + newOperationSizes, replicaWriteLimits.getCoordinatingBytes()); - assertBusy(() -> assertEquals(operationSize, replicaWriteLimits.getReplicaBytes())); + assertBusy(() -> assertEquals(operationSize + newOperationSizes, replicaWriteLimits.getReplicaBytes())); latchBlockingReplication.countDown(); successFuture.actionGet(); future1.actionGet(); - future2.actionGet(); assertEquals(0, primaryWriteLimits.getCoordinatingBytes()); assertEquals(0, primaryWriteLimits.getPrimaryBytes()); diff --git a/server/src/main/java/org/elasticsearch/action/bulk/TransportBulkAction.java b/server/src/main/java/org/elasticsearch/action/bulk/TransportBulkAction.java index d6dc52239d74c..ccdb2e80d3373 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/TransportBulkAction.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/TransportBulkAction.java @@ -160,8 +160,12 @@ public static IndexRequest getIndexWriteRequest(DocWriteRequest docWriteReque protected void doExecute(Task task, BulkRequest bulkRequest, ActionListener listener) { long indexingBytes = DocWriteRequest.writeSizeInBytes(bulkRequest.requests.stream()); final Releasable releasable = writeMemoryLimits.markCoordinatingOperationStarted(indexingBytes); - final ActionListener releasingListener = ActionListener.runAfter(listener, releasable::close); - doInternalExecute(task, bulkRequest, releasingListener); + final ActionListener 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 listener) { diff --git a/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java b/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java index dce9b1b629693..aa2f70d902045 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java @@ -507,10 +507,4 @@ private static Engine.Result performOpOnReplica(DocWriteResponse primaryResponse } return result; } - - private static long operationSizeInBytes(BulkItemRequest[] items) { - return DocWriteRequest.writeSizeInBytes(Stream.of(items).map(BulkItemRequest::request)); - } - - } From bda0bc6c6792722b92cdd61895b339d6acb1e019 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Mon, 15 Jun 2020 18:03:33 -0600 Subject: [PATCH 34/41] Changes --- .../elasticsearch/action/bulk/BulkShardRequest.java | 13 +------------ .../action/bulk/TransportShardBulkAction.java | 7 +------ 2 files changed, 2 insertions(+), 18 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/bulk/BulkShardRequest.java b/server/src/main/java/org/elasticsearch/action/bulk/BulkShardRequest.java index 5fd4af01498f1..540db58da864c 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/BulkShardRequest.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/BulkShardRequest.java @@ -31,12 +31,10 @@ import java.io.IOException; import java.util.HashSet; import java.util.Set; -import java.util.concurrent.atomic.AtomicBoolean; public class BulkShardRequest extends ReplicatedWriteRequest { - private final AtomicBoolean bytesAccounted = new AtomicBoolean(false); - private final BulkItemRequest[] items; + private BulkItemRequest[] items; public BulkShardRequest(StreamInput in) throws IOException { super(in); @@ -156,13 +154,4 @@ public void onRetry() { } } } - - public void markCoordinatingBytesAccounted() { - assert bytesAccounted.get() == false; - bytesAccounted.set(true); - } - - public boolean isCoordinatingBytesAccounted() { - return bytesAccounted.get(); - } } diff --git a/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java b/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java index aa2f70d902045..00bbb5a51a45c 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java @@ -113,12 +113,7 @@ protected BulkShardResponse newResponseInstance(StreamInput in) throws IOExcepti @Override protected boolean shouldMarkCoordinatingBytes(BulkShardRequest request) { - if (request.isCoordinatingBytesAccounted()) { - return false; - } else { - request.markCoordinatingBytesAccounted(); - return true; - } + return request.getParentTask().getNodeId().equals(clusterService.localNode().getId()) == false; } @Override From 3733dfc36e924c3cdcb11ecc2092d99ef519e0d4 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Mon, 15 Jun 2020 18:16:29 -0600 Subject: [PATCH 35/41] Compile --- .../java/org/elasticsearch/action/bulk/TransportBulkAction.java | 1 - 1 file changed, 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/action/bulk/TransportBulkAction.java b/server/src/main/java/org/elasticsearch/action/bulk/TransportBulkAction.java index e56f156ec0383..fd96f8debdecd 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/TransportBulkAction.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/TransportBulkAction.java @@ -569,7 +569,6 @@ protected void doRun() { if (task != null) { bulkShardRequest.setParentTask(nodeId, task.getId()); } - bulkShardRequest.markCoordinatingBytesAccounted(); client.executeLocally(TransportShardBulkAction.TYPE, bulkShardRequest, new ActionListener<>() { @Override public void onResponse(BulkShardResponse bulkShardResponse) { From e1470244f7420174b255eafa67279d11d357f1a0 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Mon, 15 Jun 2020 21:47:47 -0600 Subject: [PATCH 36/41] Runbefore --- .../support/replication/TransportReplicationAction.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java b/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java index 7374e8d84079e..cd67dd2b7771f 100644 --- a/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java +++ b/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java @@ -287,7 +287,7 @@ protected Releasable checkOperationLimits(final Request request) { protected void handlePrimaryRequest(final ConcreteShardRequest request, final TransportChannel channel, final Task task) { Releasable releasable = checkPrimaryLimits(request.getRequest()); ActionListener listener = - ActionListener.runAfter(new ChannelActionListener<>(channel, transportPrimaryAction, request), releasable::close); + ActionListener.runBefore(new ChannelActionListener<>(channel, transportPrimaryAction, request), releasable::close); try { new AsyncPrimaryAction(request, listener, (ReplicationTask) task).run(); @@ -512,7 +512,7 @@ protected void handleReplicaRequest(final ConcreteReplicaRequest final Task task) { Releasable releasable = checkReplicaLimits(replicaRequest.getRequest()); ActionListener listener = - ActionListener.runAfter(new ChannelActionListener<>(channel, transportReplicaAction, replicaRequest), releasable::close); + ActionListener.runBefore(new ChannelActionListener<>(channel, transportReplicaAction, replicaRequest), releasable::close); try { new AsyncReplicaAction(replicaRequest, listener, (ReplicationTask) task).run(); From 7267ba917efc089a5f866a931492dda7ee0a6ebb Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Tue, 16 Jun 2020 15:24:54 -0600 Subject: [PATCH 37/41] Changes --- .../action/bulk/TransportShardBulkAction.java | 5 ----- .../replication/TransportReplicationAction.java | 2 +- .../support/replication/TransportWriteAction.java | 11 +---------- 3 files changed, 2 insertions(+), 16 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java b/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java index 00bbb5a51a45c..083448ea2435b 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java @@ -111,11 +111,6 @@ protected BulkShardResponse newResponseInstance(StreamInput in) throws IOExcepti return new BulkShardResponse(in); } - @Override - protected boolean shouldMarkCoordinatingBytes(BulkShardRequest request) { - return request.getParentTask().getNodeId().equals(clusterService.localNode().getId()) == false; - } - @Override protected void dispatchedShardOperationOnPrimary(BulkShardRequest request, IndexShard primary, ActionListener> listener) { diff --git a/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java b/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java index cd67dd2b7771f..440018946de31 100644 --- a/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java +++ b/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java @@ -273,7 +273,7 @@ boolean isRetryableClusterBlockException(final Throwable e) { return false; } - protected void handleOperationRequest(final Request request, final TransportChannel channel, Task task) { + private void handleOperationRequest(final Request request, final TransportChannel channel, Task task) { Releasable releasable = checkOperationLimits(request); ActionListener listener = ActionListener.runAfter(new ChannelActionListener<>(channel, actionName, request), releasable::close); diff --git a/server/src/main/java/org/elasticsearch/action/support/replication/TransportWriteAction.java b/server/src/main/java/org/elasticsearch/action/support/replication/TransportWriteAction.java index f24a06bbcd296..8407dba506a29 100644 --- a/server/src/main/java/org/elasticsearch/action/support/replication/TransportWriteAction.java +++ b/server/src/main/java/org/elasticsearch/action/support/replication/TransportWriteAction.java @@ -80,16 +80,7 @@ protected TransportWriteAction(Settings settings, String actionName, TransportSe @Override protected Releasable checkOperationLimits(Request request) { - if (shouldMarkCoordinatingBytes(request)) { - long operationSizeInBytes = primaryOperationSize(request); - return writeMemoryLimits.markCoordinatingOperationStarted(operationSizeInBytes); - } else { - return () -> {}; - } - } - - protected boolean shouldMarkCoordinatingBytes(Request request) { - return true; + return writeMemoryLimits.markCoordinatingOperationStarted(primaryOperationSize(request)); } @Override From eb8fd4294c918d0448da7c00c20903a8cb16ee30 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Wed, 17 Jun 2020 18:13:50 -0600 Subject: [PATCH 38/41] Changes --- .../action/bulk/WriteMemoryLimitsIT.java | 27 ++++++++++--------- .../elasticsearch/action/DocWriteRequest.java | 17 +++--------- .../action/bulk/WriteMemoryLimits.java | 2 +- .../action/delete/DeleteRequest.java | 8 ++++++ .../action/index/IndexRequest.java | 8 ++++++ .../action/update/UpdateRequest.java | 16 +++++++++++ 6 files changed, 51 insertions(+), 27 deletions(-) diff --git a/server/src/internalClusterTest/java/org/elasticsearch/action/bulk/WriteMemoryLimitsIT.java b/server/src/internalClusterTest/java/org/elasticsearch/action/bulk/WriteMemoryLimitsIT.java index 731b40da6eced..03425157d5b2b 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/action/bulk/WriteMemoryLimitsIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/action/bulk/WriteMemoryLimitsIT.java @@ -25,6 +25,7 @@ import org.elasticsearch.action.index.IndexResponse; 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; @@ -113,18 +114,19 @@ public void testWriteBytesAreIncremented() throws Exception { connection.sendRequest(requestId, action, request, options); }); - final BulkRequest lessThan1KB = new BulkRequest(); - int totalSourceLength = 0; + final BulkRequest bulkRequest = new BulkRequest(); + int totalRequestSize = 0; for (int i = 0; i < 80; ++i) { - IndexRequest request = new IndexRequest(index).source(Collections.singletonMap("key", randomAlphaOfLength(50))); - totalSourceLength += request.source().length(); - lessThan1KB.add(request); + IndexRequest request = new IndexRequest(index).id(UUIDs.base64UUID()) + .source(Collections.singletonMap("key", randomAlphaOfLength(50))); + totalRequestSize += request.ramBytesUsed(); + bulkRequest.add(request); } - final int operationSize = totalSourceLength + WriteMemoryLimits.WRITE_REQUEST_BYTES_OVERHEAD; + final int operationSize = totalRequestSize + WriteMemoryLimits.WRITE_REQUEST_BYTES_OVERHEAD; try { - final ActionFuture successFuture = client(replicaName).bulk(lessThan1KB); + final ActionFuture successFuture = client(replicaName).bulk(bulkRequest); replicationSendPointReached.await(); WriteMemoryLimits primaryWriteLimits = internalCluster().getInstance(WriteMemoryLimits.class, primaryName); @@ -158,14 +160,15 @@ public void testWriteBytesAreIncremented() throws Exception { newActionsSendPointReached.await(); latchBlockingReplicationSend.countDown(); - IndexRequest request1 = new IndexRequest(index).source(Collections.singletonMap("key", randomAlphaOfLength(50))); + IndexRequest request = new IndexRequest(index).id(UUIDs.base64UUID()) + .source(Collections.singletonMap("key", randomAlphaOfLength(50))); - ActionFuture future1 = client(replicaName).index(request1); + ActionFuture future1 = client(replicaName).index(request); - int newOperationSizes = request1.source().length() + (WriteMemoryLimits.WRITE_REQUEST_BYTES_OVERHEAD); + long newOperationSize = request.ramBytesUsed() + WriteMemoryLimits.WRITE_REQUEST_BYTES_OVERHEAD; - assertEquals(operationSize + newOperationSizes, replicaWriteLimits.getCoordinatingBytes()); - assertBusy(() -> assertEquals(operationSize + newOperationSizes, replicaWriteLimits.getReplicaBytes())); + assertEquals(operationSize + newOperationSize, replicaWriteLimits.getCoordinatingBytes()); + assertBusy(() -> assertEquals(operationSize + newOperationSize, replicaWriteLimits.getReplicaBytes())); latchBlockingReplication.countDown(); diff --git a/server/src/main/java/org/elasticsearch/action/DocWriteRequest.java b/server/src/main/java/org/elasticsearch/action/DocWriteRequest.java index b59845a940a3b..0d88cfe7c39ea 100644 --- a/server/src/main/java/org/elasticsearch/action/DocWriteRequest.java +++ b/server/src/main/java/org/elasticsearch/action/DocWriteRequest.java @@ -18,6 +18,7 @@ */ package org.elasticsearch.action; +import org.apache.lucene.util.Accountable; import org.elasticsearch.action.delete.DeleteRequest; import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.support.IndicesOptions; @@ -39,7 +40,7 @@ * 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} */ -public interface DocWriteRequest extends IndicesRequest { +public interface DocWriteRequest extends IndicesRequest, Accountable { /** * Set the index for this request @@ -259,18 +260,6 @@ static ActionRequestValidationException validateSeqNoBasedCASParams( } static long writeSizeInBytes(Stream> requestStream) { - return requestStream.mapToLong(request -> { - if (request instanceof IndexRequest) { - if (((IndexRequest) request).source() != null) { - return ((IndexRequest) request).source().length(); - } - } else if (request instanceof UpdateRequest) { - IndexRequest doc = ((UpdateRequest) request).doc(); - if (doc != null && doc.source() != null) { - return ((UpdateRequest) request).doc().source().length(); - } - } - return 0; - }).sum(); + return requestStream.mapToLong(Accountable::ramBytesUsed).sum(); } } diff --git a/server/src/main/java/org/elasticsearch/action/bulk/WriteMemoryLimits.java b/server/src/main/java/org/elasticsearch/action/bulk/WriteMemoryLimits.java index d4dbb2d27e1dd..aa341e5866952 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/WriteMemoryLimits.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/WriteMemoryLimits.java @@ -26,7 +26,7 @@ public class WriteMemoryLimits { // A heuristic for the bytes overhead of a single write operation - public static final int WRITE_REQUEST_BYTES_OVERHEAD = 4096; + public static final int WRITE_REQUEST_BYTES_OVERHEAD = 1024; private final AtomicLong coordinatingBytes = new AtomicLong(0); private final AtomicLong primaryBytes = new AtomicLong(0); diff --git a/server/src/main/java/org/elasticsearch/action/delete/DeleteRequest.java b/server/src/main/java/org/elasticsearch/action/delete/DeleteRequest.java index 28e22addd8e6d..d588db040f90d 100644 --- a/server/src/main/java/org/elasticsearch/action/delete/DeleteRequest.java +++ b/server/src/main/java/org/elasticsearch/action/delete/DeleteRequest.java @@ -19,6 +19,7 @@ package org.elasticsearch.action.delete; +import org.apache.lucene.util.RamUsageEstimator; import org.elasticsearch.Version; import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.CompositeIndicesRequest; @@ -53,6 +54,8 @@ public class DeleteRequest extends ReplicatedWriteRequest implements DocWriteRequest, CompositeIndicesRequest { + private static final long SHALLOW_SIZE = RamUsageEstimator.shallowSizeOfInstance(DeleteRequest.class); + private static final ShardId NO_SHARD_ID = null; private String id; @@ -247,4 +250,9 @@ public void writeTo(StreamOutput out) throws IOException { public String toString() { return "delete {[" + index + "][" + id + "]}"; } + + @Override + public long ramBytesUsed() { + return SHALLOW_SIZE + (2 * id.length()); + } } diff --git a/server/src/main/java/org/elasticsearch/action/index/IndexRequest.java b/server/src/main/java/org/elasticsearch/action/index/IndexRequest.java index 635ef118d1a32..5284a1b99c287 100644 --- a/server/src/main/java/org/elasticsearch/action/index/IndexRequest.java +++ b/server/src/main/java/org/elasticsearch/action/index/IndexRequest.java @@ -19,6 +19,7 @@ package org.elasticsearch.action.index; +import org.apache.lucene.util.RamUsageEstimator; import org.elasticsearch.ElasticsearchGenerationException; import org.elasticsearch.Version; import org.elasticsearch.action.ActionRequestValidationException; @@ -76,6 +77,8 @@ */ public class IndexRequest extends ReplicatedWriteRequest implements DocWriteRequest, CompositeIndicesRequest { + private static final long SHALLOW_SIZE = RamUsageEstimator.shallowSizeOfInstance(IndexRequest.class); + /** * Max length of the source document to include into string() * @@ -676,4 +679,9 @@ public void onRetry() { public long getAutoGeneratedTimestamp() { return autoGeneratedTimestamp; } + + @Override + public long ramBytesUsed() { + return SHALLOW_SIZE + (id == null ? 0 : (id.length() * 2)) + (source == null ? 0 : source.length()); + } } diff --git a/server/src/main/java/org/elasticsearch/action/update/UpdateRequest.java b/server/src/main/java/org/elasticsearch/action/update/UpdateRequest.java index 9847ed74112e9..f3d3614218664 100644 --- a/server/src/main/java/org/elasticsearch/action/update/UpdateRequest.java +++ b/server/src/main/java/org/elasticsearch/action/update/UpdateRequest.java @@ -19,6 +19,7 @@ package org.elasticsearch.action.update; +import org.apache.lucene.util.RamUsageEstimator; import org.elasticsearch.Version; import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.DocWriteRequest; @@ -59,6 +60,9 @@ public class UpdateRequest extends InstanceShardOperationRequest implements DocWriteRequest, WriteRequest, ToXContentObject { + + private static final long SHALLOW_SIZE = RamUsageEstimator.shallowSizeOfInstance(UpdateRequest.class); + private static ObjectParser PARSER; private static final ParseField SCRIPT_FIELD = new ParseField("script"); @@ -906,4 +910,16 @@ public String toString() { res.append(", detect_noop[").append(detectNoop).append("]"); 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 + (id.length() * 2) + childRequestBytes; + } } From ab2f9c2f7405cdfae79839e4de02478fdb075e6a Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Wed, 17 Jun 2020 18:54:15 -0600 Subject: [PATCH 39/41] NPE --- .../java/org/elasticsearch/action/delete/DeleteRequest.java | 2 +- .../java/org/elasticsearch/action/update/UpdateRequest.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/delete/DeleteRequest.java b/server/src/main/java/org/elasticsearch/action/delete/DeleteRequest.java index d588db040f90d..48732707acebe 100644 --- a/server/src/main/java/org/elasticsearch/action/delete/DeleteRequest.java +++ b/server/src/main/java/org/elasticsearch/action/delete/DeleteRequest.java @@ -253,6 +253,6 @@ public String toString() { @Override public long ramBytesUsed() { - return SHALLOW_SIZE + (2 * id.length()); + return SHALLOW_SIZE + (id == null ? 0 : (2 * id.length())); } } diff --git a/server/src/main/java/org/elasticsearch/action/update/UpdateRequest.java b/server/src/main/java/org/elasticsearch/action/update/UpdateRequest.java index f3d3614218664..3326e92487089 100644 --- a/server/src/main/java/org/elasticsearch/action/update/UpdateRequest.java +++ b/server/src/main/java/org/elasticsearch/action/update/UpdateRequest.java @@ -920,6 +920,6 @@ public long ramBytesUsed() { if (upsertRequest != null) { childRequestBytes += upsertRequest.ramBytesUsed(); } - return SHALLOW_SIZE + (id.length() * 2) + childRequestBytes; + return SHALLOW_SIZE + (id == null ? 0 : (id.length() * 2)) + childRequestBytes; } } From 7f6c0334879072df1a24e311d24500fe772cf23e Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Thu, 18 Jun 2020 10:34:31 -0600 Subject: [PATCH 40/41] Changes --- .../action/bulk/WriteMemoryLimitsIT.java | 31 ++++++++++++------- .../action/bulk/BulkRequest.java | 11 ++++++- .../action/bulk/TransportBulkAction.java | 2 +- .../action/delete/DeleteRequest.java | 2 +- .../action/index/IndexRequest.java | 2 +- .../action/update/UpdateRequest.java | 2 +- 6 files changed, 34 insertions(+), 16 deletions(-) diff --git a/server/src/internalClusterTest/java/org/elasticsearch/action/bulk/WriteMemoryLimitsIT.java b/server/src/internalClusterTest/java/org/elasticsearch/action/bulk/WriteMemoryLimitsIT.java index 03425157d5b2b..46efb6d81139e 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/action/bulk/WriteMemoryLimitsIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/action/bulk/WriteMemoryLimitsIT.java @@ -22,7 +22,6 @@ 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.action.index.IndexResponse; import org.elasticsearch.cluster.metadata.IndexMetadata; import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.common.UUIDs; @@ -123,7 +122,8 @@ public void testWriteBytesAreIncremented() throws Exception { bulkRequest.add(request); } - final int operationSize = totalRequestSize + WriteMemoryLimits.WRITE_REQUEST_BYTES_OVERHEAD; + final long bulkRequestSize = bulkRequest.ramBytesUsed() + WriteMemoryLimits.WRITE_REQUEST_BYTES_OVERHEAD; + final long bulkShardRequestSize = totalRequestSize + WriteMemoryLimits.WRITE_REQUEST_BYTES_OVERHEAD; try { final ActionFuture successFuture = client(replicaName).bulk(bulkRequest); @@ -132,10 +132,10 @@ public void testWriteBytesAreIncremented() throws Exception { WriteMemoryLimits primaryWriteLimits = internalCluster().getInstance(WriteMemoryLimits.class, primaryName); WriteMemoryLimits replicaWriteLimits = internalCluster().getInstance(WriteMemoryLimits.class, replicaName); - assertEquals(operationSize, primaryWriteLimits.getCoordinatingBytes()); - assertEquals(operationSize, primaryWriteLimits.getPrimaryBytes()); + assertEquals(bulkShardRequestSize, primaryWriteLimits.getCoordinatingBytes()); + assertEquals(bulkShardRequestSize, primaryWriteLimits.getPrimaryBytes()); assertEquals(0, primaryWriteLimits.getReplicaBytes()); - assertEquals(operationSize, replicaWriteLimits.getCoordinatingBytes()); + assertEquals(bulkRequestSize, replicaWriteLimits.getCoordinatingBytes()); assertEquals(0, replicaWriteLimits.getPrimaryBytes()); assertEquals(0, replicaWriteLimits.getReplicaBytes()); @@ -162,18 +162,21 @@ public void testWriteBytesAreIncremented() throws Exception { IndexRequest request = new IndexRequest(index).id(UUIDs.base64UUID()) .source(Collections.singletonMap("key", randomAlphaOfLength(50))); + final BulkRequest secondBulkRequest = new BulkRequest(); + secondBulkRequest.add(request); - ActionFuture future1 = client(replicaName).index(request); + ActionFuture secondFuture = client(replicaName).bulk(secondBulkRequest); - long newOperationSize = request.ramBytesUsed() + WriteMemoryLimits.WRITE_REQUEST_BYTES_OVERHEAD; + final long secondBulkRequestSize = secondBulkRequest.ramBytesUsed() + WriteMemoryLimits.WRITE_REQUEST_BYTES_OVERHEAD; + final long secondBulkShardRequestSize = request.ramBytesUsed() + WriteMemoryLimits.WRITE_REQUEST_BYTES_OVERHEAD; - assertEquals(operationSize + newOperationSize, replicaWriteLimits.getCoordinatingBytes()); - assertBusy(() -> assertEquals(operationSize + newOperationSize, replicaWriteLimits.getReplicaBytes())); + assertEquals(bulkRequestSize + secondBulkRequestSize, replicaWriteLimits.getCoordinatingBytes()); + assertBusy(() -> assertEquals(bulkShardRequestSize + secondBulkShardRequestSize, replicaWriteLimits.getReplicaBytes())); latchBlockingReplication.countDown(); successFuture.actionGet(); - future1.actionGet(); + secondFuture.actionGet(); assertEquals(0, primaryWriteLimits.getCoordinatingBytes()); assertEquals(0, primaryWriteLimits.getPrimaryBytes()); @@ -182,11 +185,17 @@ public void testWriteBytesAreIncremented() throws Exception { 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) { - latchBlockingReplicationSend.countDown(); + latchBlockingReplication.countDown(); } primaryTransportService.clearAllRules(); } diff --git a/server/src/main/java/org/elasticsearch/action/bulk/BulkRequest.java b/server/src/main/java/org/elasticsearch/action/bulk/BulkRequest.java index 9484e1582d93c..2177df9b62175 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/BulkRequest.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/BulkRequest.java @@ -19,6 +19,8 @@ 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.ActionRequestValidationException; import org.elasticsearch.action.CompositeIndicesRequest; @@ -55,7 +57,9 @@ * Note that we only support refresh on the bulk request not per item. * @see org.elasticsearch.client.Client#bulk(BulkRequest) */ -public class BulkRequest extends ActionRequest implements CompositeIndicesRequest, WriteRequest { +public class BulkRequest extends ActionRequest implements CompositeIndicesRequest, WriteRequest, Accountable { + + private static final long SHALLOW_SIZE = RamUsageEstimator.shallowSizeOfInstance(BulkRequest.class); private static final int REQUEST_OVERHEAD = 50; @@ -373,4 +377,9 @@ private static String valueOrDefault(String value, String globalDefault) { } return value; } + + @Override + public long ramBytesUsed() { + return SHALLOW_SIZE + requests.stream().mapToLong(Accountable::ramBytesUsed).sum(); + } } diff --git a/server/src/main/java/org/elasticsearch/action/bulk/TransportBulkAction.java b/server/src/main/java/org/elasticsearch/action/bulk/TransportBulkAction.java index fd96f8debdecd..e443d0292f250 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/TransportBulkAction.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/TransportBulkAction.java @@ -161,7 +161,7 @@ public static IndexRequest getIndexWriteRequest(DocWriteRequest docWriteReque @Override protected void doExecute(Task task, BulkRequest bulkRequest, ActionListener listener) { - long indexingBytes = DocWriteRequest.writeSizeInBytes(bulkRequest.requests.stream()); + long indexingBytes = bulkRequest.ramBytesUsed(); final Releasable releasable = writeMemoryLimits.markCoordinatingOperationStarted(indexingBytes); final ActionListener releasingListener = ActionListener.runBefore(listener, releasable::close); try { diff --git a/server/src/main/java/org/elasticsearch/action/delete/DeleteRequest.java b/server/src/main/java/org/elasticsearch/action/delete/DeleteRequest.java index 48732707acebe..9598225ae2419 100644 --- a/server/src/main/java/org/elasticsearch/action/delete/DeleteRequest.java +++ b/server/src/main/java/org/elasticsearch/action/delete/DeleteRequest.java @@ -253,6 +253,6 @@ public String toString() { @Override public long ramBytesUsed() { - return SHALLOW_SIZE + (id == null ? 0 : (2 * id.length())); + return SHALLOW_SIZE + RamUsageEstimator.sizeOf(id); } } diff --git a/server/src/main/java/org/elasticsearch/action/index/IndexRequest.java b/server/src/main/java/org/elasticsearch/action/index/IndexRequest.java index 5284a1b99c287..38d0ae23aaead 100644 --- a/server/src/main/java/org/elasticsearch/action/index/IndexRequest.java +++ b/server/src/main/java/org/elasticsearch/action/index/IndexRequest.java @@ -682,6 +682,6 @@ public long getAutoGeneratedTimestamp() { @Override public long ramBytesUsed() { - return SHALLOW_SIZE + (id == null ? 0 : (id.length() * 2)) + (source == null ? 0 : source.length()); + return SHALLOW_SIZE + RamUsageEstimator.sizeOf(id) + (source == null ? 0 : source.ramBytesUsed()); } } diff --git a/server/src/main/java/org/elasticsearch/action/update/UpdateRequest.java b/server/src/main/java/org/elasticsearch/action/update/UpdateRequest.java index 3326e92487089..a8311c6ea0f92 100644 --- a/server/src/main/java/org/elasticsearch/action/update/UpdateRequest.java +++ b/server/src/main/java/org/elasticsearch/action/update/UpdateRequest.java @@ -920,6 +920,6 @@ public long ramBytesUsed() { if (upsertRequest != null) { childRequestBytes += upsertRequest.ramBytesUsed(); } - return SHALLOW_SIZE + (id == null ? 0 : (id.length() * 2)) + childRequestBytes; + return SHALLOW_SIZE + RamUsageEstimator.sizeOf(id) + childRequestBytes; } } From f2a6c778656ac0377b9aee7990003a52bbeba966 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Wed, 24 Jun 2020 16:33:54 -0600 Subject: [PATCH 41/41] Changes --- .../action/bulk/WriteMemoryLimitsIT.java | 17 ++++++++++------- .../elasticsearch/action/DocWriteRequest.java | 5 ----- .../action/bulk/BulkItemRequest.java | 11 ++++++++++- .../action/bulk/BulkShardRequest.java | 13 +++++++++++-- .../action/bulk/TransportShardBulkAction.java | 5 ++--- .../action/bulk/WriteMemoryLimits.java | 15 ++++++--------- .../replication/TransportReplicationAction.java | 3 ++- 7 files changed, 41 insertions(+), 28 deletions(-) diff --git a/server/src/internalClusterTest/java/org/elasticsearch/action/bulk/WriteMemoryLimitsIT.java b/server/src/internalClusterTest/java/org/elasticsearch/action/bulk/WriteMemoryLimitsIT.java index 46efb6d81139e..276d3013ed928 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/action/bulk/WriteMemoryLimitsIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/action/bulk/WriteMemoryLimitsIT.java @@ -40,6 +40,7 @@ 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 { @@ -119,11 +120,12 @@ public void testWriteBytesAreIncremented() throws Exception { 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() + WriteMemoryLimits.WRITE_REQUEST_BYTES_OVERHEAD; - final long bulkShardRequestSize = totalRequestSize + WriteMemoryLimits.WRITE_REQUEST_BYTES_OVERHEAD; + final long bulkRequestSize = bulkRequest.ramBytesUsed(); + final long bulkShardRequestSize = totalRequestSize; try { final ActionFuture successFuture = client(replicaName).bulk(bulkRequest); @@ -132,8 +134,8 @@ public void testWriteBytesAreIncremented() throws Exception { WriteMemoryLimits primaryWriteLimits = internalCluster().getInstance(WriteMemoryLimits.class, primaryName); WriteMemoryLimits replicaWriteLimits = internalCluster().getInstance(WriteMemoryLimits.class, replicaName); - assertEquals(bulkShardRequestSize, primaryWriteLimits.getCoordinatingBytes()); - assertEquals(bulkShardRequestSize, primaryWriteLimits.getPrimaryBytes()); + assertThat(primaryWriteLimits.getCoordinatingBytes(), greaterThan(bulkShardRequestSize)); + assertThat(primaryWriteLimits.getPrimaryBytes(), greaterThan(bulkShardRequestSize)); assertEquals(0, primaryWriteLimits.getReplicaBytes()); assertEquals(bulkRequestSize, replicaWriteLimits.getCoordinatingBytes()); assertEquals(0, replicaWriteLimits.getPrimaryBytes()); @@ -167,11 +169,12 @@ public void testWriteBytesAreIncremented() throws Exception { ActionFuture secondFuture = client(replicaName).bulk(secondBulkRequest); - final long secondBulkRequestSize = secondBulkRequest.ramBytesUsed() + WriteMemoryLimits.WRITE_REQUEST_BYTES_OVERHEAD; - final long secondBulkShardRequestSize = request.ramBytesUsed() + WriteMemoryLimits.WRITE_REQUEST_BYTES_OVERHEAD; + final long secondBulkRequestSize = secondBulkRequest.ramBytesUsed(); + final long secondBulkShardRequestSize = request.ramBytesUsed(); assertEquals(bulkRequestSize + secondBulkRequestSize, replicaWriteLimits.getCoordinatingBytes()); - assertBusy(() -> assertEquals(bulkShardRequestSize + secondBulkShardRequestSize, replicaWriteLimits.getReplicaBytes())); + assertBusy(() -> assertThat(replicaWriteLimits.getReplicaBytes(), + greaterThan(bulkShardRequestSize + secondBulkShardRequestSize))); latchBlockingReplication.countDown(); diff --git a/server/src/main/java/org/elasticsearch/action/DocWriteRequest.java b/server/src/main/java/org/elasticsearch/action/DocWriteRequest.java index f037ac3c93bd6..679080fb41eb2 100644 --- a/server/src/main/java/org/elasticsearch/action/DocWriteRequest.java +++ b/server/src/main/java/org/elasticsearch/action/DocWriteRequest.java @@ -32,7 +32,6 @@ import java.io.IOException; import java.util.Locale; -import java.util.stream.Stream; import static org.elasticsearch.action.ValidateActions.addValidationError; import static org.elasticsearch.index.seqno.SequenceNumbers.UNASSIGNED_PRIMARY_TERM; @@ -282,8 +281,4 @@ static ActionRequestValidationException validateSeqNoBasedCASParams( return validationException; } - - static long writeSizeInBytes(Stream> requestStream) { - return requestStream.mapToLong(Accountable::ramBytesUsed).sum(); - } } diff --git a/server/src/main/java/org/elasticsearch/action/bulk/BulkItemRequest.java b/server/src/main/java/org/elasticsearch/action/bulk/BulkItemRequest.java index 6ede31a3f05e2..9edfe2625a24c 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/BulkItemRequest.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/BulkItemRequest.java @@ -19,6 +19,8 @@ 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.common.Nullable; import org.elasticsearch.common.Strings; @@ -30,7 +32,9 @@ import java.io.IOException; 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 DocWriteRequest request; @@ -115,4 +119,9 @@ public void writeThin(StreamOutput out) throws IOException { DocWriteRequest.writeDocumentRequestThin(out, request); out.writeOptionalWriteable(primaryResponse == null ? null : primaryResponse::writeThin); } + + @Override + public long ramBytesUsed() { + return SHALLOW_SIZE + request.ramBytesUsed(); + } } diff --git a/server/src/main/java/org/elasticsearch/action/bulk/BulkShardRequest.java b/server/src/main/java/org/elasticsearch/action/bulk/BulkShardRequest.java index 498b510a51472..4b780ee8861b2 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/BulkShardRequest.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/BulkShardRequest.java @@ -19,6 +19,8 @@ package org.elasticsearch.action.bulk; +import org.apache.lucene.util.Accountable; +import org.apache.lucene.util.RamUsageEstimator; import org.elasticsearch.Version; import org.elasticsearch.action.DocWriteRequest; import org.elasticsearch.action.index.IndexRequest; @@ -32,12 +34,14 @@ import java.io.IOException; import java.util.HashSet; import java.util.Set; +import java.util.stream.Stream; -public class BulkShardRequest extends ReplicatedWriteRequest { +public class BulkShardRequest extends ReplicatedWriteRequest implements Accountable { 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 { super(in); @@ -164,4 +168,9 @@ public void onRetry() { } } } + + @Override + public long ramBytesUsed() { + return SHALLOW_SIZE + Stream.of(items).mapToLong(Accountable::ramBytesUsed).sum(); + } } diff --git a/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java b/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java index 083448ea2435b..d1c77df56b790 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java @@ -77,7 +77,6 @@ import java.util.concurrent.Executor; import java.util.function.Consumer; import java.util.function.LongSupplier; -import java.util.stream.Stream; /** Performs shard-level bulk (index, delete or update) operations */ public class TransportShardBulkAction extends TransportWriteAction { @@ -142,7 +141,7 @@ public void onTimeout(TimeValue timeout) { @Override protected long primaryOperationSize(BulkShardRequest request) { - return DocWriteRequest.writeSizeInBytes(Stream.of(request.items()).map(BulkItemRequest::request)); + return request.ramBytesUsed(); } public static void performOnPrimary( @@ -425,7 +424,7 @@ protected void dispatchedShardOperationOnReplica(BulkShardRequest request, Index @Override protected long replicaOperationSize(BulkShardRequest request) { - return DocWriteRequest.writeSizeInBytes(Stream.of(request.items()).map(BulkItemRequest::request)); + return request.ramBytesUsed(); } public static Translog.Location performOnReplica(BulkShardRequest request, IndexShard replica) throws Exception { diff --git a/server/src/main/java/org/elasticsearch/action/bulk/WriteMemoryLimits.java b/server/src/main/java/org/elasticsearch/action/bulk/WriteMemoryLimits.java index aa341e5866952..84c702f110622 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/WriteMemoryLimits.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/WriteMemoryLimits.java @@ -25,16 +25,13 @@ public class WriteMemoryLimits { - // A heuristic for the bytes overhead of a single write operation - public static final int WRITE_REQUEST_BYTES_OVERHEAD = 1024; - 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(WRITE_REQUEST_BYTES_OVERHEAD + bytes); - return () -> coordinatingBytes.getAndAdd(-(WRITE_REQUEST_BYTES_OVERHEAD + bytes)); + coordinatingBytes.addAndGet(bytes); + return () -> coordinatingBytes.getAndAdd(-bytes); } public long getCoordinatingBytes() { @@ -42,8 +39,8 @@ public long getCoordinatingBytes() { } public Releasable markPrimaryOperationStarted(long bytes) { - primaryBytes.addAndGet(WRITE_REQUEST_BYTES_OVERHEAD + bytes); - return () -> primaryBytes.getAndAdd(-(WRITE_REQUEST_BYTES_OVERHEAD + bytes)); + primaryBytes.addAndGet(bytes); + return () -> primaryBytes.getAndAdd(-bytes); } public long getPrimaryBytes() { @@ -51,8 +48,8 @@ public long getPrimaryBytes() { } public Releasable markReplicaOperationStarted(long bytes) { - replicaBytes.getAndAdd(WRITE_REQUEST_BYTES_OVERHEAD + bytes); - return () -> replicaBytes.getAndAdd(-(WRITE_REQUEST_BYTES_OVERHEAD + bytes)); + replicaBytes.getAndAdd(bytes); + return () -> replicaBytes.getAndAdd(-bytes); } public long getReplicaBytes() { diff --git a/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java b/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java index 440018946de31..1c05b596d97f0 100644 --- a/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java +++ b/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java @@ -276,7 +276,7 @@ boolean isRetryableClusterBlockException(final Throwable e) { private void handleOperationRequest(final Request request, final TransportChannel channel, Task task) { Releasable releasable = checkOperationLimits(request); ActionListener listener = - ActionListener.runAfter(new ChannelActionListener<>(channel, actionName, request), releasable::close); + ActionListener.runBefore(new ChannelActionListener<>(channel, actionName, request), releasable::close); execute(task, request, listener); } @@ -584,6 +584,7 @@ public void onResponse(Releasable releasable) { Releasables.closeWhileHandlingException(releasable); // release shard operation lock before responding to caller 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 AsyncReplicaAction.this.onFailure(e);