Improve testTransportStatsWithException test (#28554)
This commit modifies the transport stats with exception test to remove the requirement that we calculate the published address size when comparing bytes received. This is tricky and is currently broken as we also place the address string in the transport exception, however we do not adjust the bytes for that. The solution in this commit is to just serialize the transport exception in the test and use that for the calculation.
This commit is contained in:
parent
666c4f9414
commit
16f7e00514
|
@ -30,7 +30,9 @@ import org.elasticsearch.Version;
|
||||||
import org.elasticsearch.action.ActionListenerResponseHandler;
|
import org.elasticsearch.action.ActionListenerResponseHandler;
|
||||||
import org.elasticsearch.action.support.PlainActionFuture;
|
import org.elasticsearch.action.support.PlainActionFuture;
|
||||||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||||
|
import org.elasticsearch.common.io.stream.BytesStreamOutput;
|
||||||
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
|
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
|
||||||
|
import org.elasticsearch.common.io.stream.ReleasableBytesStreamOutput;
|
||||||
import org.elasticsearch.common.io.stream.StreamInput;
|
import org.elasticsearch.common.io.stream.StreamInput;
|
||||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||||
import org.elasticsearch.common.network.NetworkService;
|
import org.elasticsearch.common.network.NetworkService;
|
||||||
|
@ -2390,6 +2392,7 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
|
||||||
serviceC.start();
|
serviceC.start();
|
||||||
serviceC.acceptIncomingRequests();
|
serviceC.acceptIncomingRequests();
|
||||||
CountDownLatch responseLatch = new CountDownLatch(1);
|
CountDownLatch responseLatch = new CountDownLatch(1);
|
||||||
|
AtomicReference<TransportException> receivedException = new AtomicReference<>(null);
|
||||||
TransportResponseHandler<TransportResponse> transportResponseHandler = new TransportResponseHandler<TransportResponse>() {
|
TransportResponseHandler<TransportResponse> transportResponseHandler = new TransportResponseHandler<TransportResponse>() {
|
||||||
@Override
|
@Override
|
||||||
public TransportResponse newInstance() {
|
public TransportResponse newInstance() {
|
||||||
|
@ -2403,6 +2406,7 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void handleException(TransportException exp) {
|
public void handleException(TransportException exp) {
|
||||||
|
receivedException.set(exp);
|
||||||
responseLatch.countDown();
|
responseLatch.countDown();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -2427,7 +2431,7 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
|
||||||
TransportRequestOptions.Type.STATE);
|
TransportRequestOptions.Type.STATE);
|
||||||
try (Transport.Connection connection = serviceC.openConnection(serviceB.getLocalNode(), builder.build())) {
|
try (Transport.Connection connection = serviceC.openConnection(serviceB.getLocalNode(), builder.build())) {
|
||||||
assertBusy(() -> { // netty for instance invokes this concurrently so we better use assert busy here
|
assertBusy(() -> { // netty for instance invokes this concurrently so we better use assert busy here
|
||||||
TransportStats transportStats = serviceC.transport.getStats(); // request has ben send
|
TransportStats transportStats = serviceC.transport.getStats(); // request has been sent
|
||||||
assertEquals(1, transportStats.getRxCount());
|
assertEquals(1, transportStats.getRxCount());
|
||||||
assertEquals(1, transportStats.getTxCount());
|
assertEquals(1, transportStats.getTxCount());
|
||||||
assertEquals(25, transportStats.getRxSize().getBytes());
|
assertEquals(25, transportStats.getRxSize().getBytes());
|
||||||
|
@ -2437,7 +2441,7 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
|
||||||
transportResponseHandler);
|
transportResponseHandler);
|
||||||
receivedLatch.await();
|
receivedLatch.await();
|
||||||
assertBusy(() -> { // netty for instance invokes this concurrently so we better use assert busy here
|
assertBusy(() -> { // netty for instance invokes this concurrently so we better use assert busy here
|
||||||
TransportStats transportStats = serviceC.transport.getStats(); // request has ben send
|
TransportStats transportStats = serviceC.transport.getStats(); // request has been sent
|
||||||
assertEquals(1, transportStats.getRxCount());
|
assertEquals(1, transportStats.getRxCount());
|
||||||
assertEquals(2, transportStats.getTxCount());
|
assertEquals(2, transportStats.getTxCount());
|
||||||
assertEquals(25, transportStats.getRxSize().getBytes());
|
assertEquals(25, transportStats.getRxSize().getBytes());
|
||||||
|
@ -2448,10 +2452,14 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
|
||||||
stats = serviceC.transport.getStats(); // exception response has been received
|
stats = serviceC.transport.getStats(); // exception response has been received
|
||||||
assertEquals(2, stats.getRxCount());
|
assertEquals(2, stats.getRxCount());
|
||||||
assertEquals(2, stats.getTxCount());
|
assertEquals(2, stats.getTxCount());
|
||||||
int addressLen = serviceB.boundAddress().publishAddress().address().getAddress().getAddress().length;
|
TransportException exception = receivedException.get();
|
||||||
// if we are bound to a IPv6 address the response address is serialized with the exception so it will be different depending
|
assertNotNull(exception);
|
||||||
// on the stack. The emphemeral port will always be in the same range
|
BytesStreamOutput streamOutput = new BytesStreamOutput();
|
||||||
assertEquals(183 + addressLen, stats.getRxSize().getBytes());
|
exception.writeTo(streamOutput);
|
||||||
|
String failedMessage = "Unexpected read bytes size. The transport exception that was received=" + exception;
|
||||||
|
// 49 bytes are the non-exception message bytes that have been received. It should include the initial
|
||||||
|
// handshake message and the header, version, etc bytes in the exception message.
|
||||||
|
assertEquals(failedMessage, 49 + streamOutput.bytes().length(), stats.getRxSize().getBytes());
|
||||||
assertEquals(91, stats.getTxSize().getBytes());
|
assertEquals(91, stats.getTxSize().getBytes());
|
||||||
} finally {
|
} finally {
|
||||||
serviceC.close();
|
serviceC.close();
|
||||||
|
|
Loading…
Reference in New Issue