Ignore NoNodeExists exceptions
diff --git a/src/jvm/storm/trident/topology/state/RotatingTransactionalState.java b/src/jvm/storm/trident/topology/state/RotatingTransactionalState.java
index a44e24c..2ffe680 100644
--- a/src/jvm/storm/trident/topology/state/RotatingTransactionalState.java
+++ b/src/jvm/storm/trident/topology/state/RotatingTransactionalState.java
@@ -1,5 +1,7 @@
package storm.trident.topology.state;
+import org.apache.zookeeper.KeeperException;
+
import java.util.HashSet;
import java.util.List;
import java.util.SortedMap;
@@ -87,7 +89,16 @@
SortedMap<Long, Object> toDelete = _curr.headMap(txid);
for(long tx: new HashSet<Long>(toDelete.keySet())) {
_curr.remove(tx);
- _state.delete(txPath(tx));
+ try {
+ _state.delete(txPath(tx));
+ } catch(RuntimeException e) {
+ // Ignore NoNodeExists exceptions because when sync() it may populate _curr with stale data since
+ // zookeeper reads are eventually consistent.
+ if(!(e.getCause() instanceof KeeperException.NoNodeException)) {
+ throw e;
+ }
+ }
+
}
}