[
https://issues.apache.org/jira/browse/STORM-834?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15106809#comment-15106809
]
Robert Joseph Evans commented on STORM-834:
-------------------------------------------
[~longtimer],
You could put in a check like that into DisruptorQueue.java.
I am not sure which version of storm you are using, but I would insert it into
the publish method
https://github.com/apache/storm/blob/master/storm-core/src/jvm/org/apache/storm/utils/DisruptorQueue.java#L496
Be aware that there are a lot of unit tests that will probably fail with this
change. Another option is to modify the code around consumeBatchToCurser to
catch the exception and print out more information about the object causing the
issues.
https://github.com/apache/storm/blob/master/storm-core/src/jvm/org/apache/storm/utils/DisruptorQueue.java#L425-L452
perhaps changing it to something like the following that adds some debug about
what the event is when an exception happens.
{code}
private void consumeBatchToCursor(long cursor, EventHandler<Object>
handler) {
for (long curr = _consumer.get() + 1; curr <= cursor; curr++) {
Object o = null;
try {
AtomicReference<Object> mo = _buffer.get(curr);
o = mo.getAndSet(null);
if (o == INTERRUPT) {
throw new InterruptedException("Disruptor processing
interrupted");
} else if (o == null) {
LOG.error("NULL found in {}:{}", this.getName(), cursor);
} else {
handler.onEvent(o, curr, curr == cursor);
if (_enableBackpressure && _cb != null &&
(_metrics.writePos() - curr + _overflowCount.get()) <= _lowWaterMark) {
try {
if (_throttleOn) {
_throttleOn = false;
_cb.lowWaterMark();
}
} catch (Exception e) {
throw new RuntimeException("Exception during
calling lowWaterMark callback!");
}
}
}
} catch (Exception e) {
throw new RuntimeException("Error while processing event ("+(o
!= null ? o.getClass() : "NULL") +") "+o, e);
}
}
_consumer.set(cursor);
}
{code}
> Class cast exception Object to Iterable
> ---------------------------------------
>
> Key: STORM-834
> URL: https://issues.apache.org/jira/browse/STORM-834
> Project: Apache Storm
> Issue Type: Bug
> Components: storm-core
> Affects Versions: 0.9.3, 0.10.0, 0.9.6
> Environment: Debian linux
> Reporter: Jason Kania
>
> This exception is happening after some hours of running. Unfortunately,
> nothing in the trace suggests the source of the error for further
> investigation.
> 2015-05-23T13:24:22.343-0400 b.s.util [ERROR] Async loop died!
> java.lang.RuntimeException: java.lang.ClassCastException: java.lang.Object
> cannot be cast to java.lang.Iterable
> at
> backtype.storm.utils.DisruptorQueue.consumeBatchToCursor(DisruptorQueue.java:128)
> ~[storm-core-0.9.3.jar:0.9.3]
> at
> backtype.storm.utils.DisruptorQueue.consumeBatchWhenAvailable(DisruptorQueue.java:99)
> ~[storm-core-0.9.3.jar:0.9.3]
> at
> backtype.storm.disruptor$consume_batch_when_available.invoke(disruptor.clj:80)
> ~[storm-core-0.9.3.jar:0.9.3]
> at
> backtype.storm.daemon.executor$fn__3441$fn__3453$fn__3500.invoke(executor.clj:748)
> ~[storm-core-0.9.3.jar:0.9.3]
> at backtype.storm.util$async_loop$fn__464.invoke(util.clj:463)
> ~[storm-core-0.9.3.jar:0.9.3]
> at clojure.lang.AFn.run(AFn.java:24) [clojure-1.5.1.jar:na]
> at java.lang.Thread.run(Thread.java:745) [na:1.7.0_75]
> Caused by: java.lang.ClassCastException: java.lang.Object cannot be cast to
> java.lang.Iterable
> at backtype.storm.util$get_iterator.invoke(util.clj:867)
> ~[storm-core-0.9.3.jar:0.9.3]
> at
> backtype.storm.daemon.executor$mk_task_receiver$fn__3364.invoke(executor.clj:397)
> ~[storm-core-0.9.3.jar:0.9.3]
> at
> backtype.storm.disruptor$clojure_handler$reify__1447.onEvent(disruptor.clj:58)
> ~[storm-core-0.9.3.jar:0.9.3]
> at
> backtype.storm.utils.DisruptorQueue.consumeBatchToCursor(DisruptorQueue.java:120)
> ~[storm-core-0.9.3.jar:0.9.3]
> ... 6 common frames omitted
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)