Closed ctubbsii closed 2 years ago
FateIT uses ZooKeeperTestingServer, which should use PortUtils to get a free port for the server, It does not currently do that.
So, the question is whether or not SO_REUSEADDR
(used in PortUtils) would allow 2 different processes to bind to the same port (I think the answer is yes, but I'm not 100% sure). If so, then the current implementation of ZooKeeperTestingServer.getPort
could cause it to bind to another port that was already allocated using PortUtils. Is it possible that we have two (or more) ZK servers listening on the same port?
I'm going to reopen this test, because I think the modification in #2485 is not likely going to fix this. The error is that the FaTE transaction node simply does not exist in ZK. There is no problem with actually communicating with ZK on that port or any other ZK-specific issue indicated in the logs of this test that failed. SO_REUSEADDR
would not have allowed multiple ZKs to listen on the same socket. It only allows binding to recently used ports that haven't been cleaned up yet. And, the probability of selecting a port in use would have been very low, and the test has failed multiple times. So, I'm pretty confident that's not going to fix this, although it is an improvement to the test.
Confirmed that this is still a problem after the changes in #2485:
org.apache.zookeeper.KeeperException$NoNodeException: KeeperErrorCode = NoNode for /accumulo/3d3f19b8-4ff8-47f0-b33b-a87c3d0804a2/fate/tx_731d81b1c8ef162b
at org.apache.zookeeper.KeeperException.create(KeeperException.java:118)
at org.apache.zookeeper.KeeperException.create(KeeperException.java:54)
at org.apache.zookeeper.ZooKeeper.getData(ZooKeeper.java:2131)
at org.apache.accumulo.fate.zookeeper.ZooReader.lambda$getData$0(ZooReader.java:65)
at org.apache.accumulo.fate.zookeeper.ZooReader.retryLoopMutator(ZooReader.java:165)
at org.apache.accumulo.fate.zookeeper.ZooReader.retryLoop(ZooReader.java:144)
at org.apache.accumulo.fate.zookeeper.ZooReader.retryLoop(ZooReader.java:131)
at org.apache.accumulo.fate.zookeeper.ZooReader.getData(ZooReader.java:65)
at org.apache.accumulo.test.fate.zookeeper.FateIT.getTxStatus(FateIT.java:193)
at org.apache.accumulo.test.fate.zookeeper.FateIT.testTransactionStatus(FateIT.java:156)
Here are the failsafe logs for this test: FateIT-logs.tar.gz
@dlmarion are you looking at this?
@milleruntime - not at the moment. I have been working on the ExternalCompaction ITs. With Keith's help, I think I have the solution for that.
I saw this fail again 3 times after less than 4 seconds each, before succeeding on a fourth attempt. The error was the same every time:
[ERROR] org.apache.accumulo.test.fate.zookeeper.FateIT.testTransactionStatus Time elapsed: 3.175 s <<< ERROR!
org.apache.zookeeper.KeeperException$NoNodeException: KeeperErrorCode = NoNode for /accumulo/11bc8f82-60ac-4a63-bc6b-f417dc345bc4/fate/tx_445178e030cf8c96
at org.apache.zookeeper.KeeperException.create(KeeperException.java:118)
at org.apache.zookeeper.KeeperException.create(KeeperException.java:54)
at org.apache.zookeeper.ZooKeeper.getData(ZooKeeper.java:2131)
at org.apache.accumulo.fate.zookeeper.ZooReader.lambda$getData$0(ZooReader.java:65)
at org.apache.accumulo.fate.zookeeper.ZooReader.retryLoopMutator(ZooReader.java:165)
at org.apache.accumulo.fate.zookeeper.ZooReader.retryLoop(ZooReader.java:144)
at org.apache.accumulo.fate.zookeeper.ZooReader.retryLoop(ZooReader.java:131)
at org.apache.accumulo.fate.zookeeper.ZooReader.getData(ZooReader.java:65)
at org.apache.accumulo.test.fate.zookeeper.FateIT.getTxStatus(FateIT.java:200)
at org.apache.accumulo.test.fate.zookeeper.FateIT.testTransactionStatus(FateIT.java:163)
I found that this test becomes extremely stable when I do:
@@ -197,5 +197,8 @@ public class FateIT {
private static TStatus getTxStatus(ZooReaderWriter zrw, long txid)
throws KeeperException, InterruptedException {
+ zrw.getZooKeeper().sync(ZK_ROOT, null, null);
+ zrw.getZooKeeper().sync(ZK_ROOT + Constants.ZFATE, null, null);
String txdir = String.format("%s%s/tx_%016x", ZK_ROOT, Constants.ZFATE, txid);
+ zrw.getZooKeeper().sync(txdir, null, null);
return TStatus.valueOf(new String(zrw.getData(txdir), UTF_8));
}
This seems to fix the test. However, I don't know if it's enough to address any potential issues when actually running Accumulo. In other words, I don't know if this is merely a test bug that is fixed by this change, or if it's a problem with Accumulo itself.
That's interesting. We have discussed using sync
previously with ZooKeeper, but I don't think we are actually using it anywhere in Accumulo. Presumably you only need to do the third sync call from above, on txdir
, before checking the status? This might not be an issue with Fate
because it all runs in the Manager
process and the FaTE transaction runner threads are likely using a shared ZooReaderWriter
from the ServerContext
. This seems very similar to using the Java volatile
modifier.
That's interesting. We have discussed using
sync
previously with ZooKeeper, but I don't think we are actually using it anywhere in Accumulo. Presumably you only need to do the third sync call from above, ontxdir
, before checking the status?
I tried only doing the last one. It didn't exist yet (or the client wasn't aware of its existence), so it had nothing to sync on. I had to sync the parents so knowledge of the children were updated before I could sync the child nodes. It's possible I only needed to sync the parent, though... if the sync is effectively recursive... I'm not sure exactly how it works, only that some aggressive syncing did solve the test failure.
This might not be an issue with
Fate
because it all runs in theManager
process and the FaTE transaction runner threads are likely using a sharedZooReaderWriter
from theServerContext
. This seems very similar to using the Javavolatile
modifier.
In any case, even if sync isn't necessary for item 3 in this list (assuming the same ZK object stays sync'd when it switches servers transparently), which I'm not sure about, we don't have guarantees over item 2... because ZooSession can change the ZooKeeper object used.
These are part of the things I'm trying to investigate in #2298 when I encountered this. However, progress is slow on that because of the complications involving the deprecated Instance/Connector objects and their requirements to use the singleton ZooKeeper sessions, and we can't remove those until 3.0.
The code that is being tested is for metrics - timing issues may not matter in this case and the metrics code could be modified to ignore / report zero if the node cannot be found. The value / node / client should naturally re-sync on the next metrics collection cycle.
For the test code - adding a loop and waiting for the node to exist might be sufficient - and if not might be pointing to a logic error / bad assumption on the way that zookeeper is being used.
I'm cautious of adding a sync in this case - it might be masking other issues, and the concern would be that there could also be problems with locks - which should be much more sensitive to having repeatable / consistent state across clients and connections. And in those cases I think that requiring "exact" state across the cluster might point to a more fragile design rather than accepting that it is a distributed system and "things happen when they happen, deal with it (tm)" approach.
The code that is being tested is for metrics - timing issues may not matter in this case and the metrics code could be modified to ignore / report zero if the node cannot be found. The value / node / client should naturally re-sync on the next metrics collection cycle.
I'm not sure what you mean by this, FateIT.testTransactionStatus() is not testing metrics. It's testing that the status of the transaction is what we expect as it moves through the Fate lifecycle.
I believe that there is only one ZK server being used with ZooKeeperTestingServer, so the issue here is that the Fate transaction runner thread sets the state on the node and the test thread is not seeing the change. I was under the impression that sync would help when you had multiple servers, which I don't think is the case here. I do agree that maybe looping instead of sync might work.
The code that is being tested is for metrics - timing issues may not matter in this case and the metrics code could be modified to ignore / report zero if the node cannot be found. The value / node / client should naturally re-sync on the next metrics collection cycle.
I think you're referring to #2406 , which may be flaky for a similar reason as this one? The code being tested here that I've seen failing is testing that the expected state changes happen as a FaTE operation makes progress.
For the test code - adding a loop and waiting for the node to exist might be sufficient - and if not might be pointing to a logic error / bad assumption on the way that zookeeper is being used.
A loop in the test code means it could wait up to the ZK timeout (30 seconds) before it sees the update. An explicit sync will make it more stable. I think it probably makes sense to sync when checking the expected state.
I'm cautious of adding a sync in this case - it might be masking other issues, and the concern would be that there could also be problems with locks - which should be much more sensitive to having repeatable / consistent state across clients and connections. And in those cases I think that requiring "exact" state across the cluster might point to a more fragile design rather than accepting that it is a distributed system and "things happen when they happen, deal with it (tm)" approach.
I share that concern, and is why I haven't applied this fix yet. However, I'm inclined to advocate for having a sync (asking questions from ZK devs about its usage first) for the test if it looks like this is specifically a test problem. If there are other places in our code that also would benefit from a sync, then those might be pre-existing issues, which can be solved later.
I could be mis-remembering - but fateTransactionStatus is for metrics. (added 1.10?)
FWIW - I have no issue with a sync() call in test code.
I'm pretty sure we'd have to do it before each call to get the status, since the status is updated in a separate thread by a different ZK client. However, we don't need to have all the lines I had. I was told that the path parameter wasn't really used, but from my experimentation, it seemed that the path did need to exist at least. So, using one sync call to the "/" path should suffice inside the get status method.
I could be mis-remembering - but fateTransactionStatus is for metrics. (added 1.10?)
It might be used for that, but in this test, it's being used to check for expected progress through the FaTE lifecycle.
I'm pretty sure we'd have to do it before each call to get the status
Agreed, I just saw it fail when checking the status the 3rd time, which means it found the node the first two times, but not the third.
So, I changed the test method to:
private static TStatus getTxStatus(ZooReaderWriter zrw, long txid)
throws KeeperException, InterruptedException {
String txdir = String.format("%s%s/tx_%016x", ZK_ROOT, Constants.ZFATE, txid);
zrw.getZooKeeper().sync(txdir, null, null);
return TStatus.valueOf(new String(zrw.getData(txdir), UTF_8));
}
Subsequent tests both succeeded and failed. Looking at the sync
javadoc, it's asynchronous. :-(
So, I changed the test method to:
private static TStatus getTxStatus(ZooReaderWriter zrw, long txid) throws KeeperException, InterruptedException { String txdir = String.format("%s%s/tx_%016x", ZK_ROOT, Constants.ZFATE, txid); zrw.getZooKeeper().sync(txdir, null, null); return TStatus.valueOf(new String(zrw.getData(txdir), UTF_8)); }
Subsequent tests both succeeded and failed. Looking at the
sync
javadoc, it's asynchronous. :-(
Right, so, we gotta create a callback to sync the sync :smiley_cat: I forgot to mention that.
Test name(s)
Describe the failure observed
Click here to expand
``` [ERROR] Tests run: 6, Failures: 0, Errors: 6, Skipped: 0, Time elapsed: 20.824 s <<< FAILURE! - in org.apache.accumulo.test.fate.zookeeper.FateIT [ERROR] org.apache.accumulo.test.fate.zookeeper.FateIT.testTransactionStatus Time elapsed: 3.815 s <<< ERROR! org.apache.zookeeper.KeeperException$NoNodeException: KeeperErrorCode = NoNode for /accumulo/2f292b02-56cd-4f7d-aa3e-8a8b31187557/fate/tx_699cb260ffb1234d at org.apache.zookeeper.KeeperException.create(KeeperException.java:118) at org.apache.zookeeper.KeeperException.create(KeeperException.java:54) at org.apache.zookeeper.ZooKeeper.getData(ZooKeeper.java:2131) at org.apache.accumulo.fate.zookeeper.ZooReader.lambda$getData$0(ZooReader.java:65) at org.apache.accumulo.fate.zookeeper.ZooReader.retryLoopMutator(ZooReader.java:165) at org.apache.accumulo.fate.zookeeper.ZooReader.retryLoop(ZooReader.java:144) at org.apache.accumulo.fate.zookeeper.ZooReader.retryLoop(ZooReader.java:131) at org.apache.accumulo.fate.zookeeper.ZooReader.getData(ZooReader.java:65) at org.apache.accumulo.test.fate.zookeeper.FateIT.getTxStatus(FateIT.java:193) at org.apache.accumulo.test.fate.zookeeper.FateIT.testTransactionStatus(FateIT.java:156) at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.base/java.lang.reflect.Method.invoke(Method.java:566) at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59) at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56) at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) at org.junit.internal.runners.statements.FailOnTimeout$CallableStatement.call(FailOnTimeout.java:299) at org.junit.internal.runners.statements.FailOnTimeout$CallableStatement.call(FailOnTimeout.java:293) at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264) at java.base/java.lang.Thread.run(Thread.java:829) [ERROR] org.apache.accumulo.test.fate.zookeeper.FateIT.testTransactionStatus Time elapsed: 3.176 s <<< ERROR! org.apache.zookeeper.KeeperException$NoNodeException: KeeperErrorCode = NoNode for /accumulo/2f292b02-56cd-4f7d-aa3e-8a8b31187557/fate/tx_74b386e44516c806 at org.apache.zookeeper.KeeperException.create(KeeperException.java:118) at org.apache.zookeeper.KeeperException.create(KeeperException.java:54) at org.apache.zookeeper.ZooKeeper.getData(ZooKeeper.java:2131) at org.apache.accumulo.fate.zookeeper.ZooReader.lambda$getData$0(ZooReader.java:65) at org.apache.accumulo.fate.zookeeper.ZooReader.retryLoopMutator(ZooReader.java:165) at org.apache.accumulo.fate.zookeeper.ZooReader.retryLoop(ZooReader.java:144) at org.apache.accumulo.fate.zookeeper.ZooReader.retryLoop(ZooReader.java:131) at org.apache.accumulo.fate.zookeeper.ZooReader.getData(ZooReader.java:65) at org.apache.accumulo.test.fate.zookeeper.FateIT.getTxStatus(FateIT.java:193) at org.apache.accumulo.test.fate.zookeeper.FateIT.testTransactionStatus(FateIT.java:156) at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.base/java.lang.reflect.Method.invoke(Method.java:566) at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59) at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56) at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) at org.junit.internal.runners.statements.FailOnTimeout$CallableStatement.call(FailOnTimeout.java:299) at org.junit.internal.runners.statements.FailOnTimeout$CallableStatement.call(FailOnTimeout.java:293) at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264) at java.base/java.lang.Thread.run(Thread.java:829) [ERROR] org.apache.accumulo.test.fate.zookeeper.FateIT.testTransactionStatus Time elapsed: 3.184 s <<< ERROR! org.apache.zookeeper.KeeperException$NoNodeException: KeeperErrorCode = NoNode for /accumulo/2f292b02-56cd-4f7d-aa3e-8a8b31187557/fate/tx_317d2c0c773fb67f at org.apache.zookeeper.KeeperException.create(KeeperException.java:118) at org.apache.zookeeper.KeeperException.create(KeeperException.java:54) at org.apache.zookeeper.ZooKeeper.getData(ZooKeeper.java:2131) at org.apache.accumulo.fate.zookeeper.ZooReader.lambda$getData$0(ZooReader.java:65) at org.apache.accumulo.fate.zookeeper.ZooReader.retryLoopMutator(ZooReader.java:165) at org.apache.accumulo.fate.zookeeper.ZooReader.retryLoop(ZooReader.java:144) at org.apache.accumulo.fate.zookeeper.ZooReader.retryLoop(ZooReader.java:131) at org.apache.accumulo.fate.zookeeper.ZooReader.getData(ZooReader.java:65) at org.apache.accumulo.test.fate.zookeeper.FateIT.getTxStatus(FateIT.java:193) at org.apache.accumulo.test.fate.zookeeper.FateIT.testTransactionStatus(FateIT.java:156) at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.base/java.lang.reflect.Method.invoke(Method.java:566) at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59) at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56) at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) at org.junit.internal.runners.statements.FailOnTimeout$CallableStatement.call(FailOnTimeout.java:299) at org.junit.internal.runners.statements.FailOnTimeout$CallableStatement.call(FailOnTimeout.java:293) at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264) at java.base/java.lang.Thread.run(Thread.java:829) [ERROR] org.apache.accumulo.test.fate.zookeeper.FateIT.testTransactionStatus Time elapsed: 3.176 s <<< ERROR! org.apache.zookeeper.KeeperException$NoNodeException: KeeperErrorCode = NoNode for /accumulo/2f292b02-56cd-4f7d-aa3e-8a8b31187557/fate/tx_1146fc5b1d3886ef at org.apache.zookeeper.KeeperException.create(KeeperException.java:118) at org.apache.zookeeper.KeeperException.create(KeeperException.java:54) at org.apache.zookeeper.ZooKeeper.getData(ZooKeeper.java:2131) at org.apache.accumulo.fate.zookeeper.ZooReader.lambda$getData$0(ZooReader.java:65) at org.apache.accumulo.fate.zookeeper.ZooReader.retryLoopMutator(ZooReader.java:165) at org.apache.accumulo.fate.zookeeper.ZooReader.retryLoop(ZooReader.java:144) at org.apache.accumulo.fate.zookeeper.ZooReader.retryLoop(ZooReader.java:131) at org.apache.accumulo.fate.zookeeper.ZooReader.getData(ZooReader.java:65) at org.apache.accumulo.test.fate.zookeeper.FateIT.getTxStatus(FateIT.java:193) at org.apache.accumulo.test.fate.zookeeper.FateIT.testTransactionStatus(FateIT.java:156) at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.base/java.lang.reflect.Method.invoke(Method.java:566) at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59) at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56) at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) at org.junit.internal.runners.statements.FailOnTimeout$CallableStatement.call(FailOnTimeout.java:299) at org.junit.internal.runners.statements.FailOnTimeout$CallableStatement.call(FailOnTimeout.java:293) at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264) at java.base/java.lang.Thread.run(Thread.java:829) [ERROR] org.apache.accumulo.test.fate.zookeeper.FateIT.testTransactionStatus Time elapsed: 3.161 s <<< ERROR! org.apache.zookeeper.KeeperException$NoNodeException: KeeperErrorCode = NoNode for /accumulo/2f292b02-56cd-4f7d-aa3e-8a8b31187557/fate/tx_47fd229d568ce869 at org.apache.zookeeper.KeeperException.create(KeeperException.java:118) at org.apache.zookeeper.KeeperException.create(KeeperException.java:54) at org.apache.zookeeper.ZooKeeper.getData(ZooKeeper.java:2131) at org.apache.accumulo.fate.zookeeper.ZooReader.lambda$getData$0(ZooReader.java:65) at org.apache.accumulo.fate.zookeeper.ZooReader.retryLoopMutator(ZooReader.java:165) at org.apache.accumulo.fate.zookeeper.ZooReader.retryLoop(ZooReader.java:144) at org.apache.accumulo.fate.zookeeper.ZooReader.retryLoop(ZooReader.java:131) at org.apache.accumulo.fate.zookeeper.ZooReader.getData(ZooReader.java:65) at org.apache.accumulo.test.fate.zookeeper.FateIT.getTxStatus(FateIT.java:193) at org.apache.accumulo.test.fate.zookeeper.FateIT.testTransactionStatus(FateIT.java:156) at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.base/java.lang.reflect.Method.invoke(Method.java:566) at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59) at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56) at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) at org.junit.internal.runners.statements.FailOnTimeout$CallableStatement.call(FailOnTimeout.java:299) at org.junit.internal.runners.statements.FailOnTimeout$CallableStatement.call(FailOnTimeout.java:293) at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264) at java.base/java.lang.Thread.run(Thread.java:829) [ERROR] org.apache.accumulo.test.fate.zookeeper.FateIT.testTransactionStatus Time elapsed: 3.159 s <<< ERROR! org.apache.zookeeper.KeeperException$NoNodeException: KeeperErrorCode = NoNode for /accumulo/2f292b02-56cd-4f7d-aa3e-8a8b31187557/fate/tx_25f4481f2a0b9e6d at org.apache.zookeeper.KeeperException.create(KeeperException.java:118) at org.apache.zookeeper.KeeperException.create(KeeperException.java:54) at org.apache.zookeeper.ZooKeeper.getData(ZooKeeper.java:2131) at org.apache.accumulo.fate.zookeeper.ZooReader.lambda$getData$0(ZooReader.java:65) at org.apache.accumulo.fate.zookeeper.ZooReader.retryLoopMutator(ZooReader.java:165) at org.apache.accumulo.fate.zookeeper.ZooReader.retryLoop(ZooReader.java:144) at org.apache.accumulo.fate.zookeeper.ZooReader.retryLoop(ZooReader.java:131) at org.apache.accumulo.fate.zookeeper.ZooReader.getData(ZooReader.java:65) at org.apache.accumulo.test.fate.zookeeper.FateIT.getTxStatus(FateIT.java:193) at org.apache.accumulo.test.fate.zookeeper.FateIT.testTransactionStatus(FateIT.java:156) at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.base/java.lang.reflect.Method.invoke(Method.java:566) at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59) at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56) at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) at org.junit.internal.runners.statements.FailOnTimeout$CallableStatement.call(FailOnTimeout.java:299) at org.junit.internal.runners.statements.FailOnTimeout$CallableStatement.call(FailOnTimeout.java:293) at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264) at java.base/java.lang.Thread.run(Thread.java:829) ```Testing Environment:
What have you tried already? Nothing
Additional context The last changes here were for #2462 and #2470. This is not a known flaky test, in general, so I suspect those changes are related (but it's not certain).