This is a bug introduced as part of https://issues.apache.org/jira/browse/STORM-2844, which was fixing an issue in https://issues.apache.org/jira/browse/STORM-2666. It happens because you're using autocommit. I put up an error summary here https://issues.apache.org/jira/browse/STORM-2913.
2018-01-27 0:30 GMT+01:00 Alexandre Vermeerbergen <[email protected]> : > Hello, > > My feedbacks with Storm 1.2.0 RC2 : > * On the very positive side : weird performance metrics in Nimbus UI > on capacity / latency have been fixed in RC2, thanks !! > * On the neutral side : unlike in Storm 1.1.0, memory metric in Nimbus > UI called "Assigned Mem (MB)" shows 65MB if max JVM heap is defined > using worker.childopts: "-Xmx2048m" => should I fill a JIRA for this > one? > * On the very bad side, my workers' logs are filled by this new log > from storm-kafka-client: > > 2018-01-26 23:19:55.561 o.a.s.k.s.KafkaSpout > Thread-17-eventFromMainTopic-executor[13 13] [WARN] Failed to > deserialize [OffsetAndMetadata{offset=31890, metadata=''}]. Error > likely occurred because the last commit for this topic-partition was > done using an earlier version of Storm. Defaulting to behavior > compatible with earlier version > 2018-01-26 23:19:55.627 o.a.s.k.s.KafkaSpout > Thread-17-eventFromMainTopic-executor[13 13] [WARN] Failed to > deserialize [OffsetAndMetadata{offset=30605, metadata=''}]. Error > likely occurred because the last commit for this topic-partition was > done using an earlier version of Storm. Defaulting to behavior > compatible with earlier version > 2018-01-26 23:19:55.712 o.a.s.k.s.KafkaSpout > Thread-17-eventFromMainTopic-executor[13 13] [WARN] Failed to > deserialize [OffsetAndMetadata{offset=31890, metadata=''}]. Error > likely occurred because the last commit for this topic-partition was > done using an earlier version of Storm. Defaulting to behavior > compatible with earlier version > 2018-01-26 23:19:55.904 o.a.s.k.s.KafkaSpout > Thread-17-eventFromMainTopic-executor[13 13] [WARN] Failed to > deserialize [OffsetAndMetadata{offset=31890, metadata=''}]. Error > likely occurred because the last commit for this topic-partition was > done using an earlier version of Storm. Defaulting to behavior > compatible with earlier version > 2018-01-26 23:19:55.909 o.a.s.k.s.KafkaSpout > Thread-17-eventFromMainTopic-executor[13 13] [WARN] Failed to > deserialize [OffsetAndMetadata{offset=31890, metadata=''}]. Error > likely occurred because the last commit for this topic-partition was > done using an earlier version of Storm. Defaulting to behavior > compatible with earlier version > 2018-01-26 23:19:55.909 o.a.s.k.s.KafkaSpout > Thread-17-eventFromMainTopic-executor[13 13] [WARN] Failed to > deserialize [OffsetAndMetadata{offset=31890, metadata=''}]. Error > likely occurred because the last commit for this topic-partition was > done using an earlier version of Storm. Defaulting to behavior > compatible with earlier version > 2018-01-26 23:19:55.909 o.a.s.k.s.KafkaSpout > Thread-17-eventFromMainTopic-executor[13 13] [WARN] Failed to > deserialize [OffsetAndMetadata{offset=31890, metadata=''}]. Error > likely occurred because the last commit for this topic-partition was > done using an earlier version of Storm. Defaulting to behavior > compatible with earlier version > 2018-01-26 23:19:55.909 o.a.s.k.s.KafkaSpout > Thread-17-eventFromMainTopic-executor[13 13] [WARN] Failed to > deserialize [OffsetAndMetadata{offset=31890, metadata=''}]. Error > likely occurred because the last commit for this topic-partition was > done using an earlier version of Storm. Defaulting to behavior > compatible with earlier version > 2018-01-26 23:19:56.061 o.a.s.k.s.KafkaSpout > Thread-17-eventFromMainTopic-executor[13 13] [WARN] Failed to > deserialize [OffsetAndMetadata{offset=31890, metadata=''}]. Error > likely occurred because the last commit for this topic-partition was > done using an earlier version of Storm. Defaulting to behavior > compatible with earlier version > > The topology which generates this huge log has autocommit set to true. > > => Am I the only one to get that scary behavior from storm-kafka-client? > > Setup: using Kafka 0.10.2 libs, against Kafka Broker 1.0.0 > cluster (if that matters) > > This behavior is killing our performances because there are too many > WARN logs, so I'm afraid I have to vote a '-1' > > Thanks, > Alexandre Vermeerbergen > > 2018-01-26 22:11 GMT+01:00 P. Taylor Goetz <[email protected]>: > > This is a call to vote on releasing Apache Storm 1.2.0 (rc2) > > > > Full list of changes in this release: > > > > https://dist.apache.org/repos/dist/dev/storm/apache-storm-1. > 2.0-rc2/RELEASE_NOTES.html > > > > The tag/commit to be voted upon is v1.2.0: > > > > https://git-wip-us.apache.org/repos/asf?p=storm.git;a=tree;h= > 17a4645d7d65f5a7a08a50b5185c0fc52e82692f;hb=458aa1cb696097cf07d4466aa7417c > 7b89662221 > > > > The source archive being voted upon can be found here: > > > > https://dist.apache.org/repos/dist/dev/storm/apache-storm-1. > 2.0-rc2/apache-storm-1.2.0-src.tar.gz > > > > Other release files, signatures and digests can be found here: > > > > https://dist.apache.org/repos/dist/dev/storm/apache-storm-1.2.0-rc2/ > > > > The release artifacts are signed with the following key: > > > > https://git-wip-us.apache.org/repos/asf?p=storm.git;a=blob_ > plain;f=KEYS;hb=22b832708295fa2c15c4f3c70ac0d2bc6fded4bd > > > > The Nexus staging repository for this release is: > > > > https://repository.apache.org/content/repositories/orgapachestorm-1056 > > > > Please vote on releasing this package as Apache Storm 1.2.0. > > > > When voting, please list the actions taken to verify the release. > > > > This vote will be open for at least 72 hours. > > > > [ ] +1 Release this package as Apache Storm 1.2.0 > > [ ] 0 No opinion > > [ ] -1 Do not release this package because... > > > > Thanks to everyone who contributed to this release. > > > > -Taylor >
