Skip to content

Commit c34ca2f

Browse files
committed
Only compress responses if request was compressed (elastic#36867)
This is a follow-up to some discussions around elastic#36399. Currently we have relatively confusing compression behavior where compression can be configured for requests based on transport.compress or a specific setting for a remote cluster. However, we can only compress responses based on transport.compress as we do not know where a request is coming from (currently). This commit modifies the behavior to NEVER compress responses based on settings. Instead, a response will only be compressed if the request was compressed. This commit also updates the documentation to more clearly described transport level compression.
1 parent adc20cc commit c34ca2f

File tree

4 files changed

+72
-19
lines changed

4 files changed

+72
-19
lines changed

docs/reference/modules/transport.asciidoc

Lines changed: 28 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -102,6 +102,34 @@ and ensuring that the keepalive interval is shorter than any timeout that might
102102
cause idle connections to be closed, or by setting `transport.ping_schedule` if
103103
keepalives cannot be configured.
104104

105+
[float]
106+
==== Transport Compression
107+
108+
[float]
109+
===== Request Compresssion
110+
111+
By default, the `transport.compress` setting is `false` and network-level
112+
request compression is disabled between nodes in the cluster. This default
113+
normally makes sense for local cluster communication as compression has a
114+
noticeable CPU cost and local clusters tend to be set up with fast network
115+
connections between nodes.
116+
117+
The `transport.compress` setting always configures local cluster request
118+
compression and is the fallback setting for remote cluster request compression.
119+
If you want to configure remote request compression differently than local
120+
request compression, you can set it on a per-remote cluster basis using the
121+
<<remote-cluster-settings,`cluster.remote.${cluster_alias}.transport.compress` setting>>.
122+
123+
124+
[float]
125+
===== Response Compression
126+
127+
The compression settings do not configure compression for responses. {es} will
128+
compress a response if the inbound request was compressed--even when compression
129+
is not enabled. Similarly, {es} will not compress a response if the inbound
130+
request was uncompressed--even when compression is enabled.
131+
132+
105133
[float]
106134
=== Transport Tracer
107135

server/src/main/java/org/elasticsearch/transport/TcpTransport.java

Lines changed: 2 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -141,7 +141,6 @@ public abstract class TcpTransport extends AbstractLifecycleComponent implements
141141
// this lock is here to make sure we close this transport and disconnect all the client nodes
142142
// connections while no connect operations is going on
143143
private final ReadWriteLock closeLock = new ReentrantReadWriteLock();
144-
private final boolean compressAllResponses;
145144
private volatile BoundTransportAddress boundAddress;
146145
private final String transportName;
147146

@@ -166,7 +165,6 @@ public TcpTransport(String transportName, Settings settings, Version version, T
166165
this.pageCacheRecycler = pageCacheRecycler;
167166
this.circuitBreakerService = circuitBreakerService;
168167
this.namedWriteableRegistry = namedWriteableRegistry;
169-
this.compressAllResponses = TransportSettings.TRANSPORT_COMPRESS.get(settings);
170168
this.networkService = networkService;
171169
this.transportName = transportName;
172170
this.transportLogger = new TransportLogger();
@@ -826,14 +824,13 @@ private void sendResponse(
826824
final String action,
827825
boolean compress,
828826
byte status) throws IOException {
829-
boolean compressMessage = compress || compressAllResponses;
830827

831828
status = TransportStatus.setResponse(status);
832829
ReleasableBytesStreamOutput bStream = new ReleasableBytesStreamOutput(bigArrays);
833-
CompressibleBytesOutputStream stream = new CompressibleBytesOutputStream(bStream, compressMessage);
830+
CompressibleBytesOutputStream stream = new CompressibleBytesOutputStream(bStream, compress);
834831
boolean addedReleaseListener = false;
835832
try {
836-
if (compressMessage) {
833+
if (compress) {
837834
status = TransportStatus.setCompress(status);
838835
}
839836
threadPool.getThreadContext().writeTo(stream);

server/src/test/java/org/elasticsearch/transport/TcpTransportTests.java

Lines changed: 26 additions & 12 deletions
Original file line numberDiff line numberDiff line change
@@ -23,6 +23,7 @@
2323
import org.elasticsearch.action.ActionListener;
2424
import org.elasticsearch.action.support.PlainActionFuture;
2525
import org.elasticsearch.cluster.node.DiscoveryNode;
26+
import org.elasticsearch.common.SuppressForbidden;
2627
import org.elasticsearch.common.bytes.BytesReference;
2728
import org.elasticsearch.common.compress.CompressorFactory;
2829
import org.elasticsearch.common.io.stream.BytesStreamOutput;
@@ -34,20 +35,22 @@
3435
import org.elasticsearch.common.transport.TransportAddress;
3536
import org.elasticsearch.common.util.PageCacheRecycler;
3637
import org.elasticsearch.indices.breaker.NoneCircuitBreakerService;
38+
import org.elasticsearch.tasks.TaskManager;
3739
import org.elasticsearch.test.ESTestCase;
3840
import org.elasticsearch.test.VersionUtils;
3941
import org.elasticsearch.threadpool.TestThreadPool;
4042
import org.elasticsearch.threadpool.ThreadPool;
4143

4244
import java.io.IOException;
4345
import java.io.StreamCorruptedException;
46+
import java.net.InetAddress;
4447
import java.net.InetSocketAddress;
4548
import java.util.ArrayList;
4649
import java.util.concurrent.TimeUnit;
4750
import java.util.concurrent.atomic.AtomicReference;
4851

49-
import static org.hamcrest.Matchers.equalTo;
5052
import static org.hamcrest.core.IsInstanceOf.instanceOf;
53+
import static org.mockito.Mockito.mock;
5154

5255
/** Unit tests for {@link TcpTransport} */
5356
public class TcpTransportTests extends ESTestCase {
@@ -175,11 +178,12 @@ public void testEnsureVersionCompatibility() {
175178
ise.getMessage());
176179
}
177180

178-
public void testCompressRequest() throws IOException {
181+
@SuppressForbidden(reason = "Allow accessing localhost")
182+
public void testCompressRequestAndResponse() throws IOException {
179183
final boolean compressed = randomBoolean();
180184
Req request = new Req(randomRealisticUnicodeOfLengthBetween(10, 100));
181185
ThreadPool threadPool = new TestThreadPool(TcpTransportTests.class.getName());
182-
AtomicReference<BytesReference> messageCaptor = new AtomicReference<>();
186+
AtomicReference<BytesReference> requestCaptor = new AtomicReference<>();
183187
try {
184188
TcpTransport transport = new TcpTransport("test", Settings.EMPTY, Version.CURRENT, threadPool,
185189
PageCacheRecycler.NON_RECYCLING_INSTANCE, new NoneCircuitBreakerService(), null, null) {
@@ -191,7 +195,7 @@ protected FakeServerChannel bind(String name, InetSocketAddress address) throws
191195

192196
@Override
193197
protected FakeTcpChannel initiateChannel(DiscoveryNode node) throws IOException {
194-
return new FakeTcpChannel(true, messageCaptor);
198+
return new FakeTcpChannel(false, requestCaptor);
195199
}
196200

197201
@Override
@@ -206,7 +210,7 @@ public Releasable openConnection(DiscoveryNode node, ConnectionProfile profile,
206210
int numConnections = profile.getNumConnections();
207211
ArrayList<TcpChannel> fakeChannels = new ArrayList<>(numConnections);
208212
for (int i = 0; i < numConnections; ++i) {
209-
fakeChannels.add(new FakeTcpChannel(false, messageCaptor));
213+
fakeChannels.add(new FakeTcpChannel(false, requestCaptor));
210214
}
211215
listener.onResponse(new NodeChannels(node, fakeChannels, profile, Version.CURRENT));
212216
return () -> CloseableChannel.closeChannels(fakeChannels, false);
@@ -224,11 +228,20 @@ public Releasable openConnection(DiscoveryNode node, ConnectionProfile profile,
224228
transport.openConnection(node, profileBuilder.build(), future);
225229
Transport.Connection connection = future.actionGet();
226230
connection.sendRequest(42, "foobar", request, TransportRequestOptions.EMPTY);
231+
transport.registerRequestHandler(new RequestHandlerRegistry<>("foobar", Req::new, mock(TaskManager.class),
232+
(request1, channel, task) -> channel.sendResponse(TransportResponse.Empty.INSTANCE), ThreadPool.Names.SAME,
233+
true, true));
227234

228-
BytesReference reference = messageCaptor.get();
235+
BytesReference reference = requestCaptor.get();
229236
assertNotNull(reference);
230237

231-
StreamInput streamIn = reference.streamInput();
238+
AtomicReference<BytesReference> responseCaptor = new AtomicReference<>();
239+
InetSocketAddress address = new InetSocketAddress(InetAddress.getLocalHost(), 0);
240+
FakeTcpChannel responseChannel = new FakeTcpChannel(true, address, address, responseCaptor);
241+
transport.messageReceived(reference.slice(6, reference.length() - 6), responseChannel);
242+
243+
244+
StreamInput streamIn = responseCaptor.get().streamInput();
232245
streamIn.skip(TcpHeader.MARKER_BYTES_SIZE);
233246
int len = streamIn.readInt();
234247
long requestId = streamIn.readLong();
@@ -237,17 +250,14 @@ public Releasable openConnection(DiscoveryNode node, ConnectionProfile profile,
237250
Version version = Version.fromId(streamIn.readInt());
238251
assertEquals(Version.CURRENT, version);
239252
assertEquals(compressed, TransportStatus.isCompress(status));
253+
assertFalse(TransportStatus.isRequest(status));
240254
if (compressed) {
241255
final int bytesConsumed = TcpHeader.HEADER_SIZE;
242256
streamIn = CompressorFactory.compressor(reference.slice(bytesConsumed, reference.length() - bytesConsumed))
243257
.streamInput(streamIn);
244258
}
245259
threadPool.getThreadContext().readHeaders(streamIn);
246-
assertThat(streamIn.readStringArray(), equalTo(new String[0])); // features
247-
assertEquals("foobar", streamIn.readString());
248-
Req readReq = new Req("");
249-
readReq.readFrom(streamIn);
250-
assertEquals(request.value, readReq.value);
260+
TransportResponse.Empty.INSTANCE.readFrom(streamIn);
251261

252262
} finally {
253263
ThreadPool.terminate(threadPool, 10, TimeUnit.SECONDS);
@@ -287,6 +297,10 @@ private Req(String value) {
287297
this.value = value;
288298
}
289299

300+
private Req(StreamInput in) throws IOException {
301+
value = in.readString();
302+
}
303+
290304
@Override
291305
public void readFrom(StreamInput in) throws IOException {
292306
value = in.readString();

test/framework/src/main/java/org/elasticsearch/transport/FakeTcpChannel.java

Lines changed: 16 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -28,6 +28,8 @@
2828
public class FakeTcpChannel implements TcpChannel {
2929

3030
private final boolean isServer;
31+
private final InetSocketAddress localAddress;
32+
private final InetSocketAddress remoteAddress;
3133
private final String profile;
3234
private final AtomicReference<BytesReference> messageCaptor;
3335
private final ChannelStats stats = new ChannelStats();
@@ -45,9 +47,21 @@ public FakeTcpChannel(boolean isServer, AtomicReference<BytesReference> messageC
4547
this(isServer, "profile", messageCaptor);
4648
}
4749

50+
public FakeTcpChannel(boolean isServer, InetSocketAddress localAddress, InetSocketAddress remoteAddress,
51+
AtomicReference<BytesReference> messageCaptor) {
52+
this(isServer, localAddress, remoteAddress,"profile", messageCaptor);
53+
}
54+
4855

4956
public FakeTcpChannel(boolean isServer, String profile, AtomicReference<BytesReference> messageCaptor) {
57+
this(isServer, null, null, profile, messageCaptor);
58+
}
59+
60+
public FakeTcpChannel(boolean isServer, InetSocketAddress localAddress, InetSocketAddress remoteAddress, String profile,
61+
AtomicReference<BytesReference> messageCaptor) {
5062
this.isServer = isServer;
63+
this.localAddress = localAddress;
64+
this.remoteAddress = remoteAddress;
5165
this.profile = profile;
5266
this.messageCaptor = messageCaptor;
5367
}
@@ -64,12 +78,12 @@ public String getProfile() {
6478

6579
@Override
6680
public InetSocketAddress getLocalAddress() {
67-
return null;
81+
return localAddress;
6882
}
6983

7084
@Override
7185
public InetSocketAddress getRemoteAddress() {
72-
return null;
86+
return remoteAddress;
7387
}
7488

7589
@Override

0 commit comments

Comments
 (0)