Skip to content

Commit 1761546

Browse files
author
Jason Jackson
committed
Refactoring: Added helper function exceptionCauseIsInstanceOf
1 parent 43053a7 commit 1761546

File tree

2 files changed

+13
-2
lines changed

2 files changed

+13
-2
lines changed

src/jvm/backtype/storm/utils/Utils.java

Lines changed: 11 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -357,4 +357,15 @@ public static byte[] toByteArray(ByteBuffer buffer) {
357357
buffer.get(ret, 0, ret.length);
358358
return ret;
359359
}
360+
361+
public static boolean exceptionCauseIsInstanceOf(Class klass, Throwable throwable) {
362+
Throwable t = throwable;
363+
while(t != null) {
364+
if(klass.isInstance(t)) {
365+
return true;
366+
}
367+
t = t.getCause();
368+
}
369+
return false;
370+
}
360371
}

src/jvm/storm/trident/topology/state/RotatingTransactionalState.java

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -1,5 +1,6 @@
11
package storm.trident.topology.state;
22

3+
import backtype.storm.utils.Utils;
34
import org.apache.zookeeper.KeeperException;
45

56
import java.util.HashSet;
@@ -103,11 +104,10 @@ public void cleanupBefore(long txid) {
103104
} catch(RuntimeException e) {
104105
// Ignore NoNodeExists exceptions because when sync() it may populate _curr with stale data since
105106
// zookeeper reads are eventually consistent.
106-
if(!(e.getCause() instanceof KeeperException.NoNodeException)) {
107+
if(!Utils.exceptionCauseIsInstanceOf(KeeperException.NoNodeException.class, e)) {
107108
throw e;
108109
}
109110
}
110-
111111
}
112112
}
113113

0 commit comments

Comments
 (0)