2222import org .elasticsearch .action .StepListener ;
2323import org .elasticsearch .action .support .PlainActionFuture ;
2424import org .elasticsearch .cluster .node .DiscoveryNode ;
25- import org .elasticsearch .core .Nullable ;
26- import org .elasticsearch .core .SuppressForbidden ;
2725import org .elasticsearch .common .io .stream .BytesStreamOutput ;
2826import org .elasticsearch .common .io .stream .StreamInput ;
2927import org .elasticsearch .common .io .stream .StreamOutput ;
3634import org .elasticsearch .common .settings .Settings ;
3735import org .elasticsearch .common .transport .BoundTransportAddress ;
3836import org .elasticsearch .common .transport .TransportAddress ;
39- import org .elasticsearch .core .TimeValue ;
4037import org .elasticsearch .common .util .concurrent .AbstractRunnable ;
4138import org .elasticsearch .common .util .concurrent .ConcurrentCollections ;
39+ import org .elasticsearch .core .Nullable ;
40+ import org .elasticsearch .core .SuppressForbidden ;
41+ import org .elasticsearch .core .TimeValue ;
4242import org .elasticsearch .core .internal .io .IOUtils ;
4343import org .elasticsearch .mocksocket .MockServerSocket ;
4444import org .elasticsearch .node .Node ;
8989import static org .hamcrest .Matchers .containsString ;
9090import static org .hamcrest .Matchers .empty ;
9191import static org .hamcrest .Matchers .equalTo ;
92+ import static org .hamcrest .Matchers .greaterThan ;
9293import static org .hamcrest .Matchers .hasToString ;
9394import static org .hamcrest .Matchers .instanceOf ;
95+ import static org .hamcrest .Matchers .lessThan ;
9496import static org .hamcrest .Matchers .not ;
9597import static org .hamcrest .Matchers .notNullValue ;
9698import static org .hamcrest .Matchers .nullValue ;
@@ -639,6 +641,70 @@ public void handleException(TransportException exp) {
639641 }
640642 }
641643
644+ public void testIndexingDataCompression () throws Exception {
645+ try (MockTransportService serviceC = buildService ("TS_C" , CURRENT_VERSION , Settings .EMPTY )) {
646+ String component = "cccccccccooooooooooooooommmmmmmmmmmppppppppppprrrrrrrreeeeeeeeeessssssssiiiiiiiiiibbbbbbbbllllllllleeeeee" ;
647+ String text = component .repeat (30 );
648+ TransportRequestHandler <StringMessageRequest > handler = (request , channel , task ) -> {
649+ assertThat (text , equalTo (request .message ));
650+ try {
651+ channel .sendResponse (new StringMessageResponse ("" ));
652+ } catch (IOException e ) {
653+ logger .error ("Unexpected failure" , e );
654+ fail (e .getMessage ());
655+ }
656+ };
657+ serviceA .registerRequestHandler ("internal:sayHello" , ThreadPool .Names .GENERIC , StringMessageRequest ::new , handler );
658+ serviceC .registerRequestHandler ("internal:sayHello" , ThreadPool .Names .GENERIC , StringMessageRequest ::new , handler );
659+
660+ Settings settingsWithCompress = Settings .builder ()
661+ .put (TransportSettings .TRANSPORT_COMPRESS .getKey (), Compression .Enabled .INDEXING_DATA )
662+ .put (TransportSettings .TRANSPORT_COMPRESSION_SCHEME .getKey (),
663+ randomFrom (Compression .Scheme .DEFLATE , Compression .Scheme .LZ4 ))
664+ .build ();
665+ ConnectionProfile connectionProfile = ConnectionProfile .buildDefaultConnectionProfile (settingsWithCompress );
666+ connectToNode (serviceC , serviceA .getLocalDiscoNode (), connectionProfile );
667+ connectToNode (serviceA , serviceC .getLocalDiscoNode (), connectionProfile );
668+
669+ TransportResponseHandler <StringMessageResponse > responseHandler = new TransportResponseHandler <>() {
670+ @ Override
671+ public StringMessageResponse read (StreamInput in ) throws IOException {
672+ return new StringMessageResponse (in );
673+ }
674+
675+ @ Override
676+ public String executor () {
677+ return ThreadPool .Names .GENERIC ;
678+ }
679+
680+ @ Override
681+ public void handleResponse (StringMessageResponse response ) {
682+ }
683+
684+ @ Override
685+ public void handleException (TransportException exp ) {
686+ logger .error ("Unexpected failure" , exp );
687+ fail ("got exception instead of a response: " + exp .getMessage ());
688+ }
689+ };
690+
691+ Future <StringMessageResponse > compressed = submitRequest (serviceC , serviceA .getLocalDiscoNode (), "internal:sayHello" ,
692+ new StringMessageRequest (text , -1 , true ), responseHandler );
693+ Future <StringMessageResponse > uncompressed = submitRequest (serviceA , serviceC .getLocalDiscoNode (), "internal:sayHello" ,
694+ new StringMessageRequest (text , -1 , false ), responseHandler );
695+
696+ compressed .get ();
697+ uncompressed .get ();
698+ final long bytesLength ;
699+ try (BytesStreamOutput output = new BytesStreamOutput ()) {
700+ new StringMessageRequest (text , -1 ).writeTo (output );
701+ bytesLength = output .bytes ().length ();
702+ }
703+ assertThat (serviceA .transport ().getStats ().getRxSize ().getBytes (), lessThan (bytesLength ));
704+ assertThat (serviceC .transport ().getStats ().getRxSize ().getBytes (), greaterThan (bytesLength ));
705+ }
706+ }
707+
642708 public void testErrorMessage () throws InterruptedException {
643709 serviceA .registerRequestHandler ("internal:sayHelloException" , ThreadPool .Names .GENERIC , StringMessageRequest ::new ,
644710 (request , channel , task ) -> {
@@ -1150,14 +1216,20 @@ public void handleException(TransportException exp) {
11501216 }
11511217 }
11521218
1153- public static class StringMessageRequest extends TransportRequest {
1219+ public static class StringMessageRequest extends TransportRequest implements RawIndexingDataTransportRequest {
11541220
11551221 private String message ;
11561222 private long timeout ;
1223+ private boolean isRawIndexingData = false ;
11571224
11581225 StringMessageRequest (String message , long timeout ) {
1226+ this (message , timeout , false );
1227+ }
1228+
1229+ StringMessageRequest (String message , long timeout , boolean isRawIndexingData ) {
11591230 this .message = message ;
11601231 this .timeout = timeout ;
1232+ this .isRawIndexingData = isRawIndexingData ;
11611233 }
11621234
11631235 public StringMessageRequest (StreamInput in ) throws IOException {
@@ -1174,6 +1246,11 @@ public long timeout() {
11741246 return timeout ;
11751247 }
11761248
1249+ @ Override
1250+ public boolean isRawIndexingData () {
1251+ return isRawIndexingData ;
1252+ }
1253+
11771254 @ Override
11781255 public void writeTo (StreamOutput out ) throws IOException {
11791256 super .writeTo (out );
0 commit comments