HBASE-26416 Implement a new method for region replication instead of using replay...
[hbase.git] / hbase-server / src / test / java / org / apache / hadoop / hbase / ipc / AbstractTestIPC.java
blob030b052cb1e405771f5936fcef6840ad33c42ffd
1 /*
2 * Licensed to the Apache Software Foundation (ASF) under one
3 * or more contributor license agreements. See the NOTICE file
4 * distributed with this work for additional information
5 * regarding copyright ownership. The ASF licenses this file
6 * to you under the Apache License, Version 2.0 (the
7 * "License"); you may not use this file except in compliance
8 * with the License. You may obtain a copy of the License at
10 * http://www.apache.org/licenses/LICENSE-2.0
12 * Unless required by applicable law or agreed to in writing, software
13 * distributed under the License is distributed on an "AS IS" BASIS,
14 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 * See the License for the specific language governing permissions and
16 * limitations under the License.
18 package org.apache.hadoop.hbase.ipc;
20 import static org.apache.hadoop.hbase.ipc.TestProtobufRpcServiceImpl.SERVICE;
21 import static org.apache.hadoop.hbase.ipc.TestProtobufRpcServiceImpl.newBlockingStub;
22 import static org.apache.hadoop.hbase.ipc.TestProtobufRpcServiceImpl.newStub;
23 import static org.hamcrest.MatcherAssert.assertThat;
24 import static org.hamcrest.Matchers.greaterThanOrEqualTo;
25 import static org.junit.Assert.assertEquals;
26 import static org.junit.Assert.assertFalse;
27 import static org.junit.Assert.assertNotNull;
28 import static org.junit.Assert.assertNull;
29 import static org.junit.Assert.assertThrows;
30 import static org.junit.Assert.assertTrue;
31 import static org.junit.Assert.fail;
32 import static org.mockito.ArgumentMatchers.any;
33 import static org.mockito.Mockito.spy;
34 import static org.mockito.Mockito.verify;
35 import static org.mockito.internal.verification.VerificationModeFactory.times;
37 import io.opentelemetry.api.trace.SpanKind;
38 import io.opentelemetry.api.trace.StatusCode;
39 import io.opentelemetry.sdk.testing.junit4.OpenTelemetryRule;
40 import io.opentelemetry.sdk.trace.data.SpanData;
41 import java.io.IOException;
42 import java.net.InetSocketAddress;
43 import java.util.ArrayList;
44 import java.util.List;
45 import java.util.concurrent.TimeUnit;
46 import org.apache.hadoop.conf.Configuration;
47 import org.apache.hadoop.hbase.Cell;
48 import org.apache.hadoop.hbase.CellScanner;
49 import org.apache.hadoop.hbase.CellUtil;
50 import org.apache.hadoop.hbase.DoNotRetryIOException;
51 import org.apache.hadoop.hbase.HBaseConfiguration;
52 import org.apache.hadoop.hbase.KeyValue;
53 import org.apache.hadoop.hbase.Server;
54 import org.apache.hadoop.hbase.Waiter;
55 import org.apache.hadoop.hbase.ipc.RpcServer.BlockingServiceAndInterface;
56 import org.apache.hadoop.hbase.trace.HBaseSemanticAttributes;
57 import org.apache.hadoop.hbase.util.Bytes;
58 import org.apache.hadoop.io.compress.GzipCodec;
59 import org.apache.hadoop.util.StringUtils;
60 import org.junit.Rule;
61 import org.junit.Test;
62 import org.slf4j.Logger;
63 import org.slf4j.LoggerFactory;
65 import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableList;
66 import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
67 import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
69 import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EchoRequestProto;
70 import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EchoResponseProto;
71 import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EmptyRequestProto;
72 import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EmptyResponseProto;
73 import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.PauseRequestProto;
74 import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestRpcServiceProtos.TestProtobufRpcProto.BlockingInterface;
75 import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestRpcServiceProtos.TestProtobufRpcProto.Interface;
76 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
78 /**
79 * Some basic ipc tests.
81 public abstract class AbstractTestIPC {
83 private static final Logger LOG = LoggerFactory.getLogger(AbstractTestIPC.class);
85 private static final byte[] CELL_BYTES = Bytes.toBytes("xyz");
86 private static final KeyValue CELL = new KeyValue(CELL_BYTES, CELL_BYTES, CELL_BYTES, CELL_BYTES);
88 protected static final Configuration CONF = HBaseConfiguration.create();
89 static {
90 // Set the default to be the old SimpleRpcServer. Subclasses test it and netty.
91 CONF.set(RpcServerFactory.CUSTOM_RPC_SERVER_IMPL_CONF_KEY, SimpleRpcServer.class.getName());
94 protected abstract RpcServer createRpcServer(final Server server, final String name,
95 final List<BlockingServiceAndInterface> services,
96 final InetSocketAddress bindAddress, Configuration conf,
97 RpcScheduler scheduler) throws IOException;
99 protected abstract AbstractRpcClient<?> createRpcClientNoCodec(Configuration conf);
102 @Rule
103 public OpenTelemetryRule traceRule = OpenTelemetryRule.create();
106 * Ensure we do not HAVE TO HAVE a codec.
108 @Test
109 public void testNoCodec() throws IOException, ServiceException {
110 Configuration conf = HBaseConfiguration.create();
111 RpcServer rpcServer = createRpcServer(null, "testRpcServer",
112 Lists.newArrayList(new RpcServer.BlockingServiceAndInterface(
113 SERVICE, null)), new InetSocketAddress("localhost", 0), CONF,
114 new FifoRpcScheduler(CONF, 1));
115 try (AbstractRpcClient<?> client = createRpcClientNoCodec(conf)) {
116 rpcServer.start();
117 BlockingInterface stub = newBlockingStub(client, rpcServer.getListenerAddress());
118 HBaseRpcController pcrc = new HBaseRpcControllerImpl();
119 String message = "hello";
120 assertEquals(message,
121 stub.echo(pcrc, EchoRequestProto.newBuilder().setMessage(message).build()).getMessage());
122 assertNull(pcrc.cellScanner());
123 } finally {
124 rpcServer.stop();
128 protected abstract AbstractRpcClient<?> createRpcClient(Configuration conf);
131 * It is hard to verify the compression is actually happening under the wraps. Hope that if
132 * unsupported, we'll get an exception out of some time (meantime, have to trace it manually to
133 * confirm that compression is happening down in the client and server).
135 @Test
136 public void testCompressCellBlock() throws IOException, ServiceException {
137 Configuration conf = new Configuration(HBaseConfiguration.create());
138 conf.set("hbase.client.rpc.compressor", GzipCodec.class.getCanonicalName());
139 List<Cell> cells = new ArrayList<>();
140 int count = 3;
141 for (int i = 0; i < count; i++) {
142 cells.add(CELL);
144 RpcServer rpcServer = createRpcServer(null, "testRpcServer",
145 Lists.newArrayList(new RpcServer.BlockingServiceAndInterface(
146 SERVICE, null)), new InetSocketAddress("localhost", 0), CONF,
147 new FifoRpcScheduler(CONF, 1));
149 try (AbstractRpcClient<?> client = createRpcClient(conf)) {
150 rpcServer.start();
151 BlockingInterface stub = newBlockingStub(client, rpcServer.getListenerAddress());
152 HBaseRpcController pcrc = new HBaseRpcControllerImpl(CellUtil.createCellScanner(cells));
153 String message = "hello";
154 assertEquals(message,
155 stub.echo(pcrc, EchoRequestProto.newBuilder().setMessage(message).build()).getMessage());
156 int index = 0;
157 CellScanner cellScanner = pcrc.cellScanner();
158 assertNotNull(cellScanner);
159 while (cellScanner.advance()) {
160 assertEquals(CELL, cellScanner.current());
161 index++;
163 assertEquals(count, index);
164 } finally {
165 rpcServer.stop();
169 protected abstract AbstractRpcClient<?> createRpcClientRTEDuringConnectionSetup(
170 Configuration conf) throws IOException;
172 @Test
173 public void testRTEDuringConnectionSetup() throws Exception {
174 Configuration conf = HBaseConfiguration.create();
175 RpcServer rpcServer = createRpcServer(null, "testRpcServer",
176 Lists.newArrayList(new RpcServer.BlockingServiceAndInterface(
177 SERVICE, null)), new InetSocketAddress("localhost", 0), CONF,
178 new FifoRpcScheduler(CONF, 1));
179 try (AbstractRpcClient<?> client = createRpcClientRTEDuringConnectionSetup(conf)) {
180 rpcServer.start();
181 BlockingInterface stub = newBlockingStub(client, rpcServer.getListenerAddress());
182 stub.ping(null, EmptyRequestProto.getDefaultInstance());
183 fail("Expected an exception to have been thrown!");
184 } catch (Exception e) {
185 LOG.info("Caught expected exception: " + e.toString());
186 assertTrue(e.toString(), StringUtils.stringifyException(e).contains("Injected fault"));
187 } finally {
188 rpcServer.stop();
193 * Tests that the rpc scheduler is called when requests arrive.
195 @Test
196 public void testRpcScheduler() throws IOException, ServiceException, InterruptedException {
197 RpcScheduler scheduler = spy(new FifoRpcScheduler(CONF, 1));
198 RpcServer rpcServer = createRpcServer(null, "testRpcServer",
199 Lists.newArrayList(new RpcServer.BlockingServiceAndInterface(
200 SERVICE, null)), new InetSocketAddress("localhost", 0), CONF, scheduler);
201 verify(scheduler).init(any(RpcScheduler.Context.class));
202 try (AbstractRpcClient<?> client = createRpcClient(CONF)) {
203 rpcServer.start();
204 verify(scheduler).start();
205 BlockingInterface stub = newBlockingStub(client, rpcServer.getListenerAddress());
206 EchoRequestProto param = EchoRequestProto.newBuilder().setMessage("hello").build();
207 for (int i = 0; i < 10; i++) {
208 stub.echo(null, param);
210 verify(scheduler, times(10)).dispatch(any(CallRunner.class));
211 } finally {
212 rpcServer.stop();
213 verify(scheduler).stop();
217 /** Tests that the rpc scheduler is called when requests arrive. */
218 @Test
219 public void testRpcMaxRequestSize() throws IOException, ServiceException {
220 Configuration conf = new Configuration(CONF);
221 conf.setInt(RpcServer.MAX_REQUEST_SIZE, 1000);
222 RpcServer rpcServer = createRpcServer(null, "testRpcServer",
223 Lists.newArrayList(new RpcServer.BlockingServiceAndInterface(
224 SERVICE, null)), new InetSocketAddress("localhost", 0), conf,
225 new FifoRpcScheduler(conf, 1));
226 try (AbstractRpcClient<?> client = createRpcClient(conf)) {
227 rpcServer.start();
228 BlockingInterface stub = newBlockingStub(client, rpcServer.getListenerAddress());
229 StringBuilder message = new StringBuilder(1200);
230 for (int i = 0; i < 200; i++) {
231 message.append("hello.");
233 // set total RPC size bigger than 100 bytes
234 EchoRequestProto param = EchoRequestProto.newBuilder().setMessage(message.toString()).build();
235 stub.echo(
236 new HBaseRpcControllerImpl(CellUtil.createCellScanner(ImmutableList.<Cell> of(CELL))),
237 param);
238 fail("RPC should have failed because it exceeds max request size");
239 } catch (ServiceException e) {
240 LOG.info("Caught expected exception: " + e);
241 assertTrue(e.toString(),
242 StringUtils.stringifyException(e).contains("RequestTooBigException"));
243 } finally {
244 rpcServer.stop();
249 * Tests that the RpcServer creates & dispatches CallRunner object to scheduler with non-null
250 * remoteAddress set to its Call Object
252 @Test
253 public void testRpcServerForNotNullRemoteAddressInCallObject()
254 throws IOException, ServiceException {
255 RpcServer rpcServer = createRpcServer(null, "testRpcServer",
256 Lists.newArrayList(new RpcServer.BlockingServiceAndInterface(
257 SERVICE, null)), new InetSocketAddress("localhost", 0), CONF,
258 new FifoRpcScheduler(CONF, 1));
259 InetSocketAddress localAddr = new InetSocketAddress("localhost", 0);
260 try (AbstractRpcClient<?> client = createRpcClient(CONF)) {
261 rpcServer.start();
262 BlockingInterface stub = newBlockingStub(client, rpcServer.getListenerAddress());
263 assertEquals(localAddr.getAddress().getHostAddress(),
264 stub.addr(null, EmptyRequestProto.getDefaultInstance()).getAddr());
265 } finally {
266 rpcServer.stop();
270 @Test
271 public void testRemoteError() throws IOException, ServiceException {
272 RpcServer rpcServer = createRpcServer(null, "testRpcServer",
273 Lists.newArrayList(new RpcServer.BlockingServiceAndInterface(
274 SERVICE, null)), new InetSocketAddress("localhost", 0), CONF,
275 new FifoRpcScheduler(CONF, 1));
276 try (AbstractRpcClient<?> client = createRpcClient(CONF)) {
277 rpcServer.start();
278 BlockingInterface stub = newBlockingStub(client, rpcServer.getListenerAddress());
279 stub.error(null, EmptyRequestProto.getDefaultInstance());
280 } catch (ServiceException e) {
281 LOG.info("Caught expected exception: " + e);
282 IOException ioe = ProtobufUtil.handleRemoteException(e);
283 assertTrue(ioe instanceof DoNotRetryIOException);
284 assertTrue(ioe.getMessage().contains("server error!"));
285 } finally {
286 rpcServer.stop();
290 @Test
291 public void testTimeout() throws IOException {
292 RpcServer rpcServer = createRpcServer(null, "testRpcServer",
293 Lists.newArrayList(new RpcServer.BlockingServiceAndInterface(
294 SERVICE, null)), new InetSocketAddress("localhost", 0), CONF,
295 new FifoRpcScheduler(CONF, 1));
296 try (AbstractRpcClient<?> client = createRpcClient(CONF)) {
297 rpcServer.start();
298 BlockingInterface stub = newBlockingStub(client, rpcServer.getListenerAddress());
299 HBaseRpcController pcrc = new HBaseRpcControllerImpl();
300 int ms = 1000;
301 int timeout = 100;
302 for (int i = 0; i < 10; i++) {
303 pcrc.reset();
304 pcrc.setCallTimeout(timeout);
305 long startTime = System.nanoTime();
306 try {
307 stub.pause(pcrc, PauseRequestProto.newBuilder().setMs(ms).build());
308 } catch (ServiceException e) {
309 long waitTime = (System.nanoTime() - startTime) / 1000000;
310 // expected
311 LOG.info("Caught expected exception: " + e);
312 IOException ioe = ProtobufUtil.handleRemoteException(e);
313 assertTrue(ioe.getCause() instanceof CallTimeoutException);
314 // confirm that we got exception before the actual pause.
315 assertTrue(waitTime < ms);
318 } finally {
319 rpcServer.stop();
323 protected abstract RpcServer createTestFailingRpcServer(final Server server, final String name,
324 final List<BlockingServiceAndInterface> services,
325 final InetSocketAddress bindAddress, Configuration conf,
326 RpcScheduler scheduler) throws IOException;
328 /** Tests that the connection closing is handled by the client with outstanding RPC calls */
329 @Test
330 public void testConnectionCloseWithOutstandingRPCs() throws InterruptedException, IOException {
331 Configuration conf = new Configuration(CONF);
332 RpcServer rpcServer = createTestFailingRpcServer(null, "testRpcServer",
333 Lists.newArrayList(new RpcServer.BlockingServiceAndInterface(
334 SERVICE, null)), new InetSocketAddress("localhost", 0), CONF,
335 new FifoRpcScheduler(CONF, 1));
337 try (AbstractRpcClient<?> client = createRpcClient(conf)) {
338 rpcServer.start();
339 BlockingInterface stub = newBlockingStub(client, rpcServer.getListenerAddress());
340 EchoRequestProto param = EchoRequestProto.newBuilder().setMessage("hello").build();
341 stub.echo(null, param);
342 fail("RPC should have failed because connection closed");
343 } catch (ServiceException e) {
344 LOG.info("Caught expected exception: " + e.toString());
345 } finally {
346 rpcServer.stop();
350 @Test
351 public void testAsyncEcho() throws IOException {
352 Configuration conf = HBaseConfiguration.create();
353 RpcServer rpcServer = createRpcServer(null, "testRpcServer",
354 Lists.newArrayList(new RpcServer.BlockingServiceAndInterface(
355 SERVICE, null)), new InetSocketAddress("localhost", 0), CONF,
356 new FifoRpcScheduler(CONF, 1));
357 try (AbstractRpcClient<?> client = createRpcClient(conf)) {
358 rpcServer.start();
359 Interface stub = newStub(client, rpcServer.getListenerAddress());
360 int num = 10;
361 List<HBaseRpcController> pcrcList = new ArrayList<>();
362 List<BlockingRpcCallback<EchoResponseProto>> callbackList = new ArrayList<>();
363 for (int i = 0; i < num; i++) {
364 HBaseRpcController pcrc = new HBaseRpcControllerImpl();
365 BlockingRpcCallback<EchoResponseProto> done = new BlockingRpcCallback<>();
366 stub.echo(pcrc, EchoRequestProto.newBuilder().setMessage("hello-" + i).build(), done);
367 pcrcList.add(pcrc);
368 callbackList.add(done);
370 for (int i = 0; i < num; i++) {
371 HBaseRpcController pcrc = pcrcList.get(i);
372 assertFalse(pcrc.failed());
373 assertNull(pcrc.cellScanner());
374 assertEquals("hello-" + i, callbackList.get(i).get().getMessage());
376 } finally {
377 rpcServer.stop();
381 @Test
382 public void testAsyncRemoteError() throws IOException {
383 AbstractRpcClient<?> client = createRpcClient(CONF);
384 RpcServer rpcServer = createRpcServer(null, "testRpcServer",
385 Lists.newArrayList(new RpcServer.BlockingServiceAndInterface(
386 SERVICE, null)), new InetSocketAddress("localhost", 0), CONF,
387 new FifoRpcScheduler(CONF, 1));
388 try {
389 rpcServer.start();
390 Interface stub = newStub(client, rpcServer.getListenerAddress());
391 BlockingRpcCallback<EmptyResponseProto> callback = new BlockingRpcCallback<>();
392 HBaseRpcController pcrc = new HBaseRpcControllerImpl();
393 stub.error(pcrc, EmptyRequestProto.getDefaultInstance(), callback);
394 assertNull(callback.get());
395 assertTrue(pcrc.failed());
396 LOG.info("Caught expected exception: " + pcrc.getFailed());
397 IOException ioe = ProtobufUtil.handleRemoteException(pcrc.getFailed());
398 assertTrue(ioe instanceof DoNotRetryIOException);
399 assertTrue(ioe.getMessage().contains("server error!"));
400 } finally {
401 client.close();
402 rpcServer.stop();
406 @Test
407 public void testAsyncTimeout() throws IOException {
408 RpcServer rpcServer = createRpcServer(null, "testRpcServer",
409 Lists.newArrayList(new RpcServer.BlockingServiceAndInterface(
410 SERVICE, null)), new InetSocketAddress("localhost", 0), CONF,
411 new FifoRpcScheduler(CONF, 1));
412 try (AbstractRpcClient<?> client = createRpcClient(CONF)) {
413 rpcServer.start();
414 Interface stub = newStub(client, rpcServer.getListenerAddress());
415 List<HBaseRpcController> pcrcList = new ArrayList<>();
416 List<BlockingRpcCallback<EmptyResponseProto>> callbackList = new ArrayList<>();
417 int ms = 1000;
418 int timeout = 100;
419 long startTime = System.nanoTime();
420 for (int i = 0; i < 10; i++) {
421 HBaseRpcController pcrc = new HBaseRpcControllerImpl();
422 pcrc.setCallTimeout(timeout);
423 BlockingRpcCallback<EmptyResponseProto> callback = new BlockingRpcCallback<>();
424 stub.pause(pcrc, PauseRequestProto.newBuilder().setMs(ms).build(), callback);
425 pcrcList.add(pcrc);
426 callbackList.add(callback);
428 for (BlockingRpcCallback<?> callback : callbackList) {
429 assertNull(callback.get());
431 long waitTime = (System.nanoTime() - startTime) / 1000000;
432 for (HBaseRpcController pcrc : pcrcList) {
433 assertTrue(pcrc.failed());
434 LOG.info("Caught expected exception: " + pcrc.getFailed());
435 IOException ioe = ProtobufUtil.handleRemoteException(pcrc.getFailed());
436 assertTrue(ioe.getCause() instanceof CallTimeoutException);
438 // confirm that we got exception before the actual pause.
439 assertTrue(waitTime < ms);
440 } finally {
441 rpcServer.stop();
445 private void assertSameTraceId() {
446 String traceId = traceRule.getSpans().get(0).getTraceId();
447 for (SpanData data : traceRule.getSpans()) {
448 // assert we are the same trace
449 assertEquals(traceId, data.getTraceId());
453 private SpanData waitSpan(String name) {
454 Waiter.waitFor(CONF, 1000,
455 () -> traceRule.getSpans().stream().map(SpanData::getName).anyMatch(s -> s.equals(name)));
456 return traceRule.getSpans().stream().filter(s -> s.getName().equals(name)).findFirst().get();
459 private void assertRpcAttribute(SpanData data, String methodName, InetSocketAddress addr,
460 SpanKind kind) {
461 assertEquals(SERVICE.getDescriptorForType().getName(),
462 data.getAttributes().get(HBaseSemanticAttributes.RPC_SERVICE_KEY));
463 assertEquals(methodName, data.getAttributes().get(HBaseSemanticAttributes.RPC_METHOD_KEY));
464 if (addr != null) {
465 assertEquals(
466 addr.getHostName(),
467 data.getAttributes().get(HBaseSemanticAttributes.REMOTE_HOST_KEY));
468 assertEquals(
469 addr.getPort(),
470 data.getAttributes().get(HBaseSemanticAttributes.REMOTE_PORT_KEY).intValue());
472 assertEquals(kind, data.getKind());
475 private void assertRemoteSpan() {
476 SpanData data = waitSpan("RpcServer.process");
477 assertTrue(data.getParentSpanContext().isRemote());
478 assertEquals(SpanKind.SERVER, data.getKind());
481 @Test
482 public void testTracing() throws IOException, ServiceException {
483 RpcServer rpcServer = createRpcServer(null, "testRpcServer",
484 Lists.newArrayList(new RpcServer.BlockingServiceAndInterface(SERVICE, null)),
485 new InetSocketAddress("localhost", 0), CONF, new FifoRpcScheduler(CONF, 1));
486 try (AbstractRpcClient<?> client = createRpcClient(CONF)) {
487 rpcServer.start();
488 BlockingInterface stub = newBlockingStub(client, rpcServer.getListenerAddress());
489 stub.pause(null, PauseRequestProto.newBuilder().setMs(100).build());
490 assertRpcAttribute(waitSpan("RpcClient.callMethod"), "pause", rpcServer.getListenerAddress(),
491 SpanKind.CLIENT);
492 assertRpcAttribute(waitSpan("RpcServer.callMethod"), "pause", null, SpanKind.INTERNAL);
493 assertRemoteSpan();
494 assertSameTraceId();
495 for (SpanData data : traceRule.getSpans()) {
496 assertThat(
497 TimeUnit.NANOSECONDS.toMillis(data.getEndEpochNanos() - data.getStartEpochNanos()),
498 greaterThanOrEqualTo(100L));
499 assertEquals(StatusCode.OK, data.getStatus().getStatusCode());
502 traceRule.clearSpans();
503 assertThrows(ServiceException.class,
504 () -> stub.error(null, EmptyRequestProto.getDefaultInstance()));
505 assertRpcAttribute(waitSpan("RpcClient.callMethod"), "error", rpcServer.getListenerAddress(),
506 SpanKind.CLIENT);
507 assertRpcAttribute(waitSpan("RpcServer.callMethod"), "error", null, SpanKind.INTERNAL);
508 assertRemoteSpan();
509 assertSameTraceId();
510 for (SpanData data : traceRule.getSpans()) {
511 assertEquals(StatusCode.ERROR, data.getStatus().getStatusCode());