Skip to content

Commit 7f612d5

Browse files
authored
Always compress based on the settings (#36522)
Currently TransportRequestOptions allows specific requests to request compression. This commit removes this and always compresses based on the settings. Additionally, it removes TransportResponseOptions as they are unused. This closes #36399.
1 parent 02d0f16 commit 7f612d5

File tree

17 files changed

+45
-163
lines changed

17 files changed

+45
-163
lines changed

server/src/main/java/org/elasticsearch/action/admin/cluster/node/tasks/get/TransportGetTaskAction.java

Lines changed: 0 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -102,7 +102,6 @@ private void runOnNodeWithTaskIfPossible(Task thisTask, GetTaskRequest request,
102102
if (request.getTimeout() != null) {
103103
builder.withTimeout(request.getTimeout());
104104
}
105-
builder.withCompress(false);
106105
DiscoveryNode node = clusterService.state().nodes().get(request.getTaskId().getNodeId());
107106
if (node == null) {
108107
// Node is no longer part of the cluster! Try and look the task up from the results index.

server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/TransportNodesSnapshotsStatus.java

Lines changed: 0 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -69,11 +69,6 @@ public TransportNodesSnapshotsStatus(ThreadPool threadPool, ClusterService clust
6969
this.snapshotShardsService = snapshotShardsService;
7070
}
7171

72-
@Override
73-
protected boolean transportCompress() {
74-
return true; // compress since the metadata can become large
75-
}
76-
7772
@Override
7873
protected NodeRequest newNodeRequest(String nodeId, Request request) {
7974
return new NodeRequest(nodeId, request);

server/src/main/java/org/elasticsearch/action/bulk/BulkAction.java

Lines changed: 1 addition & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -39,9 +39,6 @@ public BulkResponse newResponse() {
3939

4040
@Override
4141
public TransportRequestOptions transportOptions(Settings settings) {
42-
return TransportRequestOptions.builder()
43-
.withType(TransportRequestOptions.Type.BULK)
44-
.withCompress(settings.getAsBoolean("action.bulk.compress", true)
45-
).build();
42+
return TransportRequestOptions.builder().withType(TransportRequestOptions.Type.BULK).build();
4643
}
4744
}

server/src/main/java/org/elasticsearch/action/support/nodes/TransportNodesAction.java

Lines changed: 0 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -82,10 +82,6 @@ protected void doExecute(Task task, NodesRequest request, ActionListener<NodesRe
8282
new AsyncAction(task, request, listener).start();
8383
}
8484

85-
protected boolean transportCompress() {
86-
return false;
87-
}
88-
8985
/**
9086
* Map the responses into {@code nodeResponseClass} responses and {@link FailedNodeException}s.
9187
*
@@ -173,7 +169,6 @@ void start() {
173169
if (request.timeout() != null) {
174170
builder.withTimeout(request.timeout());
175171
}
176-
builder.withCompress(transportCompress());
177172
for (int i = 0; i < nodes.length; i++) {
178173
final int idx = i;
179174
final DiscoveryNode node = nodes[i];

server/src/main/java/org/elasticsearch/action/support/tasks/TransportTasksAction.java

Lines changed: 0 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -212,10 +212,6 @@ protected TasksResponse newResponse(TasksRequest request, AtomicReferenceArray r
212212
*/
213213
protected abstract void taskOperation(TasksRequest request, OperationTask task, ActionListener<TaskResponse> listener);
214214

215-
protected boolean transportCompress() {
216-
return false;
217-
}
218-
219215
private class AsyncAction {
220216

221217
private final TasksRequest request;
@@ -255,7 +251,6 @@ private void start() {
255251
if (request.getTimeout() != null) {
256252
builder.withTimeout(request.getTimeout());
257253
}
258-
builder.withCompress(transportCompress());
259254
for (int i = 0; i < nodesIds.length; i++) {
260255
final String nodeId = nodesIds[i];
261256
final int idx = i;

server/src/main/java/org/elasticsearch/cluster/coordination/PublicationTransportHandler.java

Lines changed: 6 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -78,6 +78,10 @@ public class PublicationTransportHandler {
7878
private final AtomicLong fullClusterStateReceivedCount = new AtomicLong();
7979
private final AtomicLong incompatibleClusterStateDiffReceivedCount = new AtomicLong();
8080
private final AtomicLong compatibleClusterStateDiffReceivedCount = new AtomicLong();
81+
// -> no need to put a timeout on the options here, because we want the response to eventually be received
82+
// and not log an error if it arrives after the timeout
83+
private final TransportRequestOptions stateRequestOptions = TransportRequestOptions.builder()
84+
.withType(TransportRequestOptions.Type.STATE).build();
8185

8286
public PublicationTransportHandler(TransportService transportService, NamedWriteableRegistry namedWriteableRegistry,
8387
Function<PublishRequest, PublishWithJoinResponse> handlePublishRequest,
@@ -213,7 +217,6 @@ public String toString() {
213217
@Override
214218
public void sendApplyCommit(DiscoveryNode destination, ApplyCommitRequest applyCommitRequest,
215219
ActionListener<TransportResponse.Empty> responseActionListener) {
216-
TransportRequestOptions options = TransportRequestOptions.builder().withType(TransportRequestOptions.Type.STATE).build();
217220
final String actionName;
218221
final TransportRequest transportRequest;
219222
if (Coordinator.isZen1Node(destination)) {
@@ -223,7 +226,7 @@ public void sendApplyCommit(DiscoveryNode destination, ApplyCommitRequest applyC
223226
actionName = COMMIT_STATE_ACTION_NAME;
224227
transportRequest = applyCommitRequest;
225228
}
226-
transportService.sendRequest(destination, actionName, transportRequest, options,
229+
transportService.sendRequest(destination, actionName, transportRequest, stateRequestOptions,
227230
new TransportResponseHandler<TransportResponse.Empty>() {
228231

229232
@Override
@@ -254,11 +257,6 @@ private void sendClusterStateToNode(ClusterState clusterState, BytesReference by
254257
ActionListener<PublishWithJoinResponse> responseActionListener, boolean sendDiffs,
255258
Map<Version, BytesReference> serializedStates) {
256259
try {
257-
// -> no need to put a timeout on the options here, because we want the response to eventually be received
258-
// and not log an error if it arrives after the timeout
259-
// -> no need to compress, we already compressed the bytes
260-
final TransportRequestOptions options = TransportRequestOptions.builder()
261-
.withType(TransportRequestOptions.Type.STATE).withCompress(false).build();
262260
final BytesTransportRequest request = new BytesTransportRequest(bytes, node.getVersion());
263261
final Consumer<TransportException> transportExceptionHandler = exp -> {
264262
if (sendDiffs && exp.unwrapCause() instanceof IncompatibleClusterStateVersionException) {
@@ -304,7 +302,7 @@ public String executor() {
304302
actionName = PUBLISH_STATE_ACTION_NAME;
305303
transportResponseHandler = publishWithJoinResponseHandler;
306304
}
307-
transportService.sendRequest(node, actionName, request, options, transportResponseHandler);
305+
transportService.sendRequest(node, actionName, request, stateRequestOptions, transportResponseHandler);
308306
} catch (Exception e) {
309307
logger.warn(() -> new ParameterizedMessage("error sending cluster state to {}", node), e);
310308
responseActionListener.onFailure(e);

server/src/main/java/org/elasticsearch/discovery/zen/PublishClusterStateAction.java

Lines changed: 7 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -76,6 +76,11 @@ public class PublishClusterStateAction {
7676
public static final String SEND_ACTION_NAME = "internal:discovery/zen/publish/send";
7777
public static final String COMMIT_ACTION_NAME = "internal:discovery/zen/publish/commit";
7878

79+
// -> no need to put a timeout on the options, because we want the state response to eventually be received
80+
// and not log an error if it arrives after the timeout
81+
private final TransportRequestOptions stateRequestOptions = TransportRequestOptions.builder()
82+
.withType(TransportRequestOptions.Type.STATE).build();
83+
7984
public interface IncomingClusterStateListener {
8085

8186
/**
@@ -284,14 +289,9 @@ private void sendClusterStateToNode(final ClusterState clusterState, BytesRefere
284289
final boolean sendDiffs, final Map<Version, BytesReference> serializedStates) {
285290
try {
286291

287-
// -> no need to put a timeout on the options here, because we want the response to eventually be received
288-
// and not log an error if it arrives after the timeout
289-
// -> no need to compress, we already compressed the bytes
290-
TransportRequestOptions options = TransportRequestOptions.builder()
291-
.withType(TransportRequestOptions.Type.STATE).withCompress(false).build();
292292
transportService.sendRequest(node, SEND_ACTION_NAME,
293293
new BytesTransportRequest(bytes, node.getVersion()),
294-
options,
294+
stateRequestOptions,
295295
new EmptyTransportResponseHandler(ThreadPool.Names.SAME) {
296296

297297
@Override
@@ -324,12 +324,9 @@ private void sendCommitToNode(final DiscoveryNode node, final ClusterState clust
324324
try {
325325
logger.trace("sending commit for cluster state (uuid: [{}], version [{}]) to [{}]",
326326
clusterState.stateUUID(), clusterState.version(), node);
327-
TransportRequestOptions options = TransportRequestOptions.builder().withType(TransportRequestOptions.Type.STATE).build();
328-
// no need to put a timeout on the options here, because we want the response to eventually be received
329-
// and not log an error if it arrives after the timeout
330327
transportService.sendRequest(node, COMMIT_ACTION_NAME,
331328
new CommitClusterStateRequest(clusterState.stateUUID()),
332-
options,
329+
stateRequestOptions,
333330
new EmptyTransportResponseHandler(ThreadPool.Names.SAME) {
334331

335332
@Override

server/src/main/java/org/elasticsearch/gateway/TransportNodesListGatewayMetaState.java

Lines changed: 0 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -66,11 +66,6 @@ public ActionFuture<NodesGatewayMetaState> list(String[] nodesIds, @Nullable Tim
6666
return future;
6767
}
6868

69-
@Override
70-
protected boolean transportCompress() {
71-
return true; // compress since the metadata can become large
72-
}
73-
7469
@Override
7570
protected NodeRequest newNodeRequest(String nodeId, Request request) {
7671
return new NodeRequest(nodeId);

server/src/main/java/org/elasticsearch/gateway/TransportNodesListGatewayStartedShards.java

Lines changed: 0 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -91,11 +91,6 @@ public void list(ShardId shardId, DiscoveryNode[] nodes,
9191
execute(new Request(shardId, nodes), listener);
9292
}
9393

94-
@Override
95-
protected boolean transportCompress() {
96-
return true; // this can become big...
97-
}
98-
9994
@Override
10095
protected NodeRequest newNodeRequest(String nodeId, Request request) {
10196
return new NodeRequest(nodeId, request);

server/src/main/java/org/elasticsearch/indices/recovery/RemoteRecoveryTargetHandler.java

Lines changed: 0 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -62,12 +62,10 @@ public RemoteRecoveryTargetHandler(long recoveryId, ShardId shardId, TransportSe
6262
this.recoverySettings = recoverySettings;
6363
this.onSourceThrottle = onSourceThrottle;
6464
this.translogOpsRequestOptions = TransportRequestOptions.builder()
65-
.withCompress(true)
6665
.withType(TransportRequestOptions.Type.RECOVERY)
6766
.withTimeout(recoverySettings.internalActionLongTimeout())
6867
.build();
6968
this.fileChunkRequestOptions = TransportRequestOptions.builder()
70-
.withCompress(false) // lucene files are already compressed and therefore compressing this won't really help much so
7169
// we are saving the cpu for other things
7270
.withType(TransportRequestOptions.Type.RECOVERY)
7371
.withTimeout(recoverySettings.internalActionTimeout())

0 commit comments

Comments
 (0)