Thanks Michal, I'll see if I can reproduce the issue in a standalone test. Would be good to get that fixed
Kind regards Julien On Fri, 31 Jan 2020 at 09:09, Michal Koziorowski < michal.koziorow...@tomtom.com> wrote: > Hi Julien, > > Unfortunately I have no fix for that. We had more issues with storm 2, > mostly because of way how we are using and configured it and we ended with > staying with Storm 1.2.3. > I wonder why only we observed this issue. Like you, we also have custom > kryo serializers used. > > Best regards, > Michal > > ------------------------------ > *From:* Julien Nioche <lists.digitalpeb...@gmail.com> > *Sent:* Tuesday, 21 January 2020 12:38 > *To:* user@storm.apache.org <user@storm.apache.org>; Michal Koziorowski < > michal.koziorow...@tomtom.com> > *Subject:* Re: Kryo errors after upgrading to Storm 2.0 > > Hi, > > I am observing the same issues as Michal did back in June. My code works > fine with Storm 1.x but the workers crash constantly with Storm 2.x > > Some exceptions look like > > > > > > *com.esotericsoftware.kryo.KryoException: Buffer underflow. at > com.esotericsoftware.kryo.io.Input.require(Input.java:199) > ~[kryo-3.0.3.jar:?] at > com.esotericsoftware.kryo.io.Input.readUtf8_slow(Input.java:575) > ~[kryo-3.0.3.jar:?] at > com.esotericsoftware.kryo.io.Input.readUtf8(Input.java:553) > ~[kryo-3.0.3.jar:?] at > com.esotericsoftware.kryo.io.Input.readString(Input.java:483) > ~[kryo-3.0.3.jar:?]* > > whereas others are > > > > > > > > > > > > > *2020-01-21 11:13:39.368 o.a.s.m.n.StormServerHandler > Netty-server-localhost-6701-worker-1 [ERROR] server errors in handling the > request com.esotericsoftware.kryo.KryoException: Encountered unregistered > class ID: 95 at > com.esotericsoftware.kryo.util.DefaultClassResolver.readClass(DefaultClassResolver.java:137) > ~[kryo-3.0.3.jar:?] at > com.esotericsoftware.kryo.Kryo.readClass(Kryo.java:670) ~[kryo-3.0.3.jar:?] > at com.esotericsoftware.kryo.Kryo.readClassAndObject(Kryo.java:781) > ~[kryo-3.0.3.jar:?] at > com.esotericsoftware.kryo.serializers.MapSerializer.read(MapSerializer.java:153) > ~[kryo-3.0.3.jar:?] at > com.esotericsoftware.kryo.serializers.MapSerializer.read(MapSerializer.java:39) > ~[kryo-3.0.3.jar:?] at > com.esotericsoftware.kryo.Kryo.readClassAndObject(Kryo.java:793) > ~[kryo-3.0.3.jar:?] at > com.esotericsoftware.kryo.serializers.CollectionSerializer.read(CollectionSerializer.java:134) > ~[kryo-3.0.3.jar:?] at > com.esotericsoftware.kryo.serializers.CollectionSerializer.read(CollectionSerializer.java:40) > ~[kryo-3.0.3.jar:?] at > com.esotericsoftware.kryo.Kryo.readObject(Kryo.java:689) > ~[kryo-3.0.3.jar:?] at > org.apache.storm.serialization.KryoValuesDeserializer.deserializeFrom(KryoValuesDeserializer.java:31) > ~[storm-client-2.1.0.jar:2.1.0]* > > the class ID values are random integers. > > The tuples in my topologies contain mostly standard classes like String; > the only exception is the following class > > > https://github.com/DigitalPebble/storm-crawler/blob/2.x/core/src/main/java/com/digitalpebble/stormcrawler/Metadata.java#L41 > > for which we specify a custom serialization for Kryo. > > My configurations contain > > topology.kryo.register: > - com.digitalpebble.stormcrawler.Metadata to register the custom class. > > Any ideas/ suggestions? > > Thanks > > Julien > > > On Sat, 22 Jun 2019 at 11:01, Stig Rohde Døssing <stigdoess...@gmail.com> > wrote: > > For KryoValuesDeserializer, I see instantiation in > > > https://github.com/apache/storm/blob/dfae3c7e5a4128f821c9c8594156caf17f375d1a > /storm-client/src/jvm/org/apache/storm/messaging/netty/StormClientPipelineFactory.java#L40 > > https://github.com/apache/storm/blob/dfae3c7e5a4128f821c9c8594156caf17f375d1a > /storm-client/src/jvm/org/apache/storm/messaging/netty/StormServerPipelineFactory.java#L40 > > https://github.com/apache/storm/blob/dfae3c7e5a4128f821c9c8594156caf17f375d1a > /storm-client/src/jvm/org/apache/storm/messaging/netty/StormClientHandler.java#L35 > > https://github.com/apache/storm/blob/dfae3c7e5a4128f821c9c8594156caf17f375d1a > /storm-client/src/jvm/org/apache/storm/serialization/KryoTupleDeserializer.java#L30 > > The client and server pipeline factories are creating the deserializer as > part of creating a Netty channel, which as I understand it means only one > thread will be interacting with the deserializer. > The client handler is only being used in the client pipeline factories, > also as part of making a channel. We could probably get rid of this > instance and just share the one used in the rest of the channel pipeline. > The tuple deserializer is only used when instantiating values that go in > ThreadLocals. > > For the serializer, I see instantiation in > > > https://github.com/apache/storm/blob/dfae3c7e5a4128f821c9c8594156caf17f375d1a > /storm-client/src/jvm/org/apache/storm/messaging/netty/StormServerPipelineFactory.java#L43 > > https://github.com/apache/storm/blob/dfae3c7e5a4128f821c9c8594156caf17f375d1a/storm-client/src/jvm/org/apache/storm/serialization/KryoTupleSerializer.java > > https://github.com/apache/storm/blob/dfae3c7e5a4128f821c9c8594156caf17f375d1a/storm-client/src/jvm/org/apache/storm/messaging/netty/Server.java#L79 > > The pipeline factory should be okay for the same reason as above. > The tuple serializer is used in a ThreadLocal, and also by > ExecutorTransfer. > The serializer used by Server is only used in a synchronized block, > synchronizing on the serializer. > > The uses look mostly safe to me. I don't think we want to use ThreadLocals > in the pipeline factories, it shouldn't be necessary. > My main worry is the serializer in ExecutorTransfer. I think this can get > used both by the main executor thread, and by any user threads interacting > with the OutputCollector. For example, if you ack tuples in a different > thread from the main thread in a bolt, you will likely be using this > serializer from multiple threads. > > We could probably do the following: > * Replace the Server serializer synchronization with a ThreadLocal. > * Share the existing pipeline deserializer with the StormClientHandler > instead of creating a new one > * Make the ExecutorTransfer serializer ThreadLocal > > What do you think? > > Den tor. 20. jun. 2019 kl. 23.11 skrev Michal Koziorowski < > michal.koziorow...@tomtom.com>: > > > <https://github.com/apache/storm/blob/master/storm-client/src/jvm/org/apache/storm/messaging/DeserializingConnectionCallback.java#L39>https://github.com/apache/storm/blob/master/storm-client/src/jvm/org/apache/storm/messaging/DeserializingConnectionCallback.java#L39 > > Looks quite good for me. There should be one Kryo instance per thread. > > > But I'm less sure about other serialization / deserialization parts: > > https://github.com/apache/storm/search?p=1&q=KryoValuesDeserializer&unscoped_q=KryoValuesDeserializer > > > <https://github.com/apache/storm/search?q=KryoValuesSerializer&unscoped_q=KryoValuesSerializer>https://github.com/apache/storm/search?q=KryoValuesSerializer&unscoped_q=KryoValuesSerializer > > > What do you think about creating wrapper around Kryo*Serializers and > Kryo*Deserializers with ThreadLocal to get rid of possible future kryo > threading issues? > > > Best regards, > > Michal > > > -------- Wiadomość oryginalna ---------- > *Od*: Stig Rohde Døssing <stigdoess...@gmail.com > <stig%20rohde%20%3d%3fiso-8859-1%3fq%3fd%3df8ssing%3f%3d%20%3cstigdoess...@gmail.com%3e> > > > *Odpowiedź do*: user@storm.apache.org > *Do*: user@storm.apache.org > *Temat*: Re: Kryo errors after upgrading to Storm 2.0 > *Data*: Thu, 20 Jun 2019 21:17:23 +0200 > > I don't think we've changed Kryo stuff much since 1.0.0. Last version > upgrade was https://github.com/apache/storm/pull/1255. > > Maybe it could be a threading issue in the Netty messaging implementation > in > https://github.com/apache/storm/tree/master/storm-client/src/jvm/org/apache/storm/messaging/netty. > I know a couple of issues of that sort were fixed in 2.0.0. 2.0.0 also > upgraded Netty, so maybe there's an issue in that code somewhere. > > Den tor. 20. jun. 2019 kl. 19.10 skrev Michal Koziorowski < > michal.koziorow...@tomtom.com>: > > Hi, > > After upgrading to Storm 2.0 and running in production couple of days I > saw following error: > > server errors in handling the request > com.esotericsoftware.kryo.KryoException: Unable to find class: j, > Serialization trace: value > (org.apache.storm.metric.api.IMetricsConsumer$DataPoint) at > com.esotericsoftware.kryo.util.DefaultClassResolver.readName(DefaultClassResolver.java:156) > ~[kryo-3.0.3.jar:?] at > com.esotericsoftware.kryo.util.DefaultClassResolver.readClass(DefaultClassResolver.java:133) > ~[kryo-3.0.3.jar:?] at > com.esotericsoftware.kryo.Kryo.readClass(Kryo.java:670) ~[kryo-3.0.3.jar:?] > at com.esotericsoftware.kryo.Kryo.readClassAndObject(Kryo.java:781) > ~[kryo-3.0.3.jar:?] at > com.esotericsoftware.kryo.serializers.MapSerializer.read(MapSerializer.java:161) > ~[kryo-3.0.3.jar:?] at > com.esotericsoftware.kryo.serializers.MapSerializer.read(MapSerializer.java:39) > ~[kryo-3.0.3.jar:?] at > com.esotericsoftware.kryo.Kryo.readObject(Kryo.java:711) > ~[kryo-3.0.3.jar:?] at > com.esotericsoftware.kryo.serializers.ObjectField.read(ObjectField.java:125) > ~[kryo-3.0.3.jar:?] at > com.esotericsoftware.kryo.serializers.FieldSerializer.read(FieldSerializer.java:551) > ~[kryo-3.0.3.jar:?] at > com.esotericsoftware.kryo.Kryo.readClassAndObject(Kryo.java:793) > ~[kryo-3.0.3.jar:?] at > com.esotericsoftware.kryo.serializers.CollectionSerializer.read(CollectionSerializer.java:134) > ~[kryo-3.0.3.jar:?] at > com.esotericsoftware.kryo.serializers.CollectionSerializer.read(CollectionSerializer.java:40) > ~[kryo-3.0.3.jar:?] at > com.esotericsoftware.kryo.Kryo.readClassAndObject(Kryo.java:793) > ~[kryo-3.0.3.jar:?] at > com.esotericsoftware.kryo.serializers.CollectionSerializer.read(CollectionSerializer.java:134) > ~[kryo-3.0.3.jar:?] at > com.esotericsoftware.kryo.serializers.CollectionSerializer.read(CollectionSerializer.java:40) > ~[kryo-3.0.3.jar:?] at > com.esotericsoftware.kryo.Kryo.readObject(Kryo.java:689) > ~[kryo-3.0.3.jar:?] at > org.apache.storm.serialization.KryoValuesDeserializer.deserializeFrom(KryoValuesDeserializer.java:31) > ~[storm-client-2.0.0.jar:2.0.0] at > org.apache.storm.serialization.KryoTupleDeserializer.deserialize(KryoTupleDeserializer.java:45) > ~[storm-client-2.0.0.jar:2.0.0] at > org.apache.storm.messaging.DeserializingConnectionCallback.recv(DeserializingConnectionCallback.java:66) > ~[storm-client-2.0.0.jar:2.0.0] at > org.apache.storm.messaging.netty.Server.enqueue(Server.java:140) > ~[storm-client-2.0.0.jar:2.0.0] at > org.apache.storm.messaging.netty.Server.received(Server.java:269) > ~[storm-client-2.0.0.jar:2.0.0] at > org.apache.storm.messaging.netty.StormServerHandler.channelRead(StormServerHandler.java:51) > ~[storm-client-2.0.0.jar:2.0.0] at > org.apache.storm.shade.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:362) > [storm-shaded-deps-2.0.0.jar:2.0.0] at > org.apache.storm.shade.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:348) > [storm-shaded-deps-2.0.0.jar:2.0.0] at > org.apache.storm.shade.io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:340) > [storm-shaded-deps-2.0.0.jar:2.0.0] at > org.apache.storm.shade.io.netty.handler.codec.ByteToMessageDecoder.fireChannelRead(ByteToMessageDecoder.java:323) > [storm-shaded-deps-2.0.0.jar:2.0.0] at > org.apache.storm.shade.io.netty.handler.codec.ByteToMessageDecoder.channelRead(ByteToMessageDecoder.java:297) > [storm-shaded-deps-2.0.0.jar:2.0.0] at > org.apache.storm.shade.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:362) > [storm-shaded-deps-2.0.0.jar:2.0.0] at > org.apache.storm.shade.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:348) > [storm-shaded-deps-2.0.0.jar:2.0.0] at > org.apache.storm.shade.io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:340) > [storm-shaded-deps-2.0.0.jar:2.0.0] at > org.apache.storm.shade.io.netty.channel.DefaultChannelPipeline$HeadContext.channelRead(DefaultChannelPipeline.java:1434) > [storm-shaded-deps-2.0.0.jar:2.0.0] at > org.apache.storm.shade.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:362) > [storm-shaded-deps-2.0.0.jar:2.0.0] at > org.apache.storm.shade.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:348) > [storm-shaded-deps-2.0.0.jar:2.0.0] at > org.apache.storm.shade.io.netty.channel.DefaultChannelPipeline.fireChannelRead(DefaultChannelPipeline.java:965) > [storm-shaded-deps-2.0.0.jar:2.0.0] at > org.apache.storm.shade.io.netty.channel.nio.AbstractNioByteChannel$NioByteUnsafe.read(AbstractNioByteChannel.java:163) > [storm-shaded-deps-2.0.0.jar:2.0.0] at > org.apache.storm.shade.io.netty.channel.nio.NioEventLoop.processSelectedKey(NioEventLoop.java:644) > [storm-shaded-deps-2.0.0.jar:2.0.0] at > org.apache.storm.shade.io.netty.channel.nio.NioEventLoop.processSelectedKeysOptimized(NioEventLoop.java:579) > [storm-shaded-deps-2.0.0.jar:2.0.0] at > org.apache.storm.shade.io.netty.channel.nio.NioEventLoop.processSelectedKeys(NioEventLoop.java:496) > [storm-shaded-deps-2.0.0.jar:2.0.0] at > org.apache.storm.shade.io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:458) > [storm-shaded-deps-2.0.0.jar:2.0.0] at > org.apache.storm.shade.io.netty.util.concurrent.SingleThreadEventExecutor$5.run(SingleThreadEventExecutor.java:897) > [storm-shaded-deps-2.0.0.jar:2.0.0] at > java.lang.Thread.run(Thread.java:748) [?:1.8.0_212] Caused by: > java.lang.ClassNotFoundException: j, at > java.net.URLClassLoader.findClass(URLClassLoader.java:382) ~[?:1.8.0_212] > at java.lang.ClassLoader.loadClass(ClassLoader.java:424) ~[?:1.8.0_212] at > sun.misc.Launcher$AppClassLoader.loadClass(Launcher.java:349) > ~[?:1.8.0_212] at java.lang.ClassLoader.loadClass(ClassLoader.java:357) > ~[?:1.8.0_212] at java.lang.Class.forName0(Native Method) ~[?:1.8.0_212] at > java.lang.Class.forName(Class.java:348) ~[?:1.8.0_212] at > com.esotericsoftware.kryo.util.DefaultClassResolver.readName(DefaultClassResolver.java:154) > ~[kryo-3.0.3.jar:?] ... 40 more > > and also messages similar to one below, but with different class IDs > > server errors in handling the request > com.esotericsoftware.kryo.KryoException: Encountered unregistered class ID: > 108 Serialization trace: value > (org.apache.storm.metric.api.IMetricsConsumer$DataPoint) at > com.esotericsoftware.kryo.util.DefaultClassResolver.readClass(DefaultClassResolver.java:137) > ~[kryo-3.0.3.jar:?] at > com.esotericsoftware.kryo.Kryo.readClass(Kryo.java:670) ~[kryo-3.0.3.jar:?] > at com.esotericsoftware.kryo.Kryo.readClassAndObject(Kryo.java:781) > ~[kryo-3.0.3.jar:?] > at > com.esotericsoftware.kryo.serializers.MapSerializer.read(MapSerializer.java:161) > ~[kryo-3.0.3.jar:?] at > com.esotericsoftware.kryo.serializers.MapSerializer.read(MapSerializer.java:39) > ~[kryo-3.0.3.jar:?] at > com.esotericsoftware.kryo.Kryo.readObject(Kryo.java:711) ~[kryo-3.0.3.jar:?] > at > com.esotericsoftware.kryo.serializers.ObjectField.read(ObjectField.java:125) > ~[kryo-3.0.3.jar:?] at > com.esotericsoftware.kryo.serializers.FieldSerializer.read(FieldSerializer.java:551) > ~[kryo-3.0.3.jar:?] at > com.esotericsoftware.kryo.Kryo.readClassAndObject(Kryo.java:793) > ~[kryo-3.0.3.jar:?] > at > com.esotericsoftware.kryo.serializers.CollectionSerializer.read(CollectionSerializer.java:134) > ~[kryo-3.0.3.jar:?] at > com.esotericsoftware.kryo.serializers.CollectionSerializer.read(CollectionSerializer.java:40) > ~[kryo-3.0.3.jar:?] at > com.esotericsoftware.kryo.Kryo.readClassAndObject(Kryo.java:793) > ~[kryo-3.0.3.jar:?] > at > com.esotericsoftware.kryo.serializers.CollectionSerializer.read(CollectionSerializer.java:134) > ~[kryo-3.0.3.jar:?] at > com.esotericsoftware.kryo.serializers.CollectionSerializer.read(CollectionSerializer.java:40) > ~[kryo-3.0.3.jar:?] at > com.esotericsoftware.kryo.Kryo.readObject(Kryo.java:689) ~[kryo-3.0.3.jar:?] > at > org.apache.storm.serialization.KryoValuesDeserializer.deserializeFrom(KryoValuesDeserializer.java:31) > ~[storm-client-2.0.0.jar:2.0.0] at > org.apache.storm.serialization.KryoTupleDeserializer.deserialize(KryoTupleDeserializer.java:45) > ~[storm-client-2.0.0.jar:2.0.0] at > org.apache.storm.messaging.DeserializingConnectionCallback.recv(DeserializingConnectionCallback.java:66) > ~[storm-client-2.0.0.jar:2.0.0] at > org.apache.storm.messaging.netty.Server.enqueue(Server.java:140) > ~[storm-client-2.0.0.jar:2.0.0] at > org.apache.storm.messaging.netty.Server.received(Server.java:269) > ~[storm-client-2.0.0.jar:2.0.0] at > org.apache.storm.messaging.netty.StormServerHandler.channelRead(StormServerHandler.java:51) > ~[storm-client-2.0.0.jar:2.0.0] at > org.apache.storm.shade.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:362) > [storm-shaded-deps-2.0.0.jar:2.0.0] at > org.apache.storm.shade.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:348) > [storm-shaded-deps-2.0.0.jar:2.0.0] at > org.apache.storm.shade.io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:340) > [storm-shaded-deps-2.0.0.jar:2.0.0] at > org.apache.storm.shade.io.netty.handler.codec.ByteToMessageDecoder.fireChannelRead(ByteToMessageDecoder.java:323) > [storm-shaded-deps-2.0.0.jar:2.0.0] at > org.apache.storm.shade.io.netty.handler.codec.ByteToMessageDecoder.fireChannelRead(ByteToMessageDecoder.java:310) > [storm-shaded-deps-2.0.0.jar:2.0.0] at > org.apache.storm.shade.io.netty.handler.codec.ByteToMessageDecoder.callDecode(ByteToMessageDecoder.java:426) > [storm-shaded-deps-2.0.0.jar:2.0.0] at > org.apache.storm.shade.io.netty.handler.codec.ByteToMessageDecoder.channelRead(ByteToMessageDecoder.java:278) > [storm-shaded-deps-2.0.0.jar:2.0.0] at > org.apache.storm.shade.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:362) > [storm-shaded-deps-2.0.0.jar:2.0.0] at > org.apache.storm.shade.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:348) > [storm-shaded-deps-2.0.0.jar:2.0.0] at > org.apache.storm.shade.io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:340) > [storm-shaded-deps-2.0.0.jar:2.0.0] at > org.apache.storm.shade.io.netty.channel.DefaultChannelPipeline$HeadContext.channelRead(DefaultChannelPipeline.java:1434) > [storm-shaded-deps-2.0.0.jar:2.0.0] at > org.apache.storm.shade.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:362) > [storm-shaded-deps-2.0.0.jar:2.0.0] at > org.apache.storm.shade.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:348) > [storm-shaded-deps-2.0.0.jar:2.0.0] at > org.apache.storm.shade.io.netty.channel.DefaultChannelPipeline.fireChannelRead(DefaultChannelPipeline.java:965) > [storm-shaded-deps-2.0.0.jar:2.0.0] at > org.apache.storm.shade.io.netty.channel.nio.AbstractNioByteChannel$NioByteUnsafe.read(AbstractNioByteChannel.java:163) > [storm-shaded-deps-2.0.0.jar:2.0.0] at > org.apache.storm.shade.io.netty.channel.nio.NioEventLoop.processSelectedKey(NioEventLoop.java:644) > [storm-shaded-deps-2.0.0.jar:2.0.0] at > org.apache.storm.shade.io.netty.channel.nio.NioEventLoop.processSelectedKeysOptimized(NioEventLoop.java:579) > [storm-shaded-deps-2.0.0.jar:2.0.0] at > org.apache.storm.shade.io.netty.channel.nio.NioEventLoop.processSelectedKeys(NioEventLoop.java:496) > [storm-shaded-deps-2.0.0.jar:2.0.0] at > org.apache.storm.shade.io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:458) > [storm-shaded-deps-2.0.0.jar:2.0.0] at > org.apache.storm.shade.io.netty.util.concurrent.SingleThreadEventExecutor$5.run(SingleThreadEventExecutor.java:897) > [storm-shaded-deps-2.0.0.jar:2.0.0] at > java.lang.Thread.run(Thread.java:748) [?:1.8.0_212] > > > And also: > server errors in handling the request > com.esotericsoftware.kryo.KryoException: Buffer underflow. at > com.esotericsoftware.kryo.io.Input.require(Input.java:182) ~[kryo-3.0.3.jar:?] > at com.esotericsoftware.kryo.io.Input.readLong(Input.java:686) > ~[kryo-3.0.3.jar:?] > at org.apache.storm.tuple.MessageId.deserialize(MessageId.java:56) > ~[storm-client-2.0.0.jar:2.0.0] at > org.apache.storm.serialization.KryoTupleDeserializer.deserialize(KryoTupleDeserializer.java:44) > ~[storm-client-2.0.0.jar:2.0.0] at > org.apache.storm.messaging.DeserializingConnectionCallback.recv(DeserializingConnectionCallback.java:66) > ~[storm-client-2.0.0.jar:2.0.0] at > org.apache.storm.messaging.netty.Server.enqueue(Server.java:140) > ~[storm-client-2.0.0.jar:2.0.0] at > org.apache.storm.messaging.netty.Server.received(Server.java:269) > ~[storm-client-2.0.0.jar:2.0.0] at > org.apache.storm.messaging.netty.StormServerHandler.channelRead(StormServerHandler.java:51) > ~[storm-client-2.0.0.jar:2.0.0] at > org.apache.storm.shade.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:362) > [storm-shaded-deps-2.0.0.jar:2.0.0] at > org.apache.storm.shade.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:348) > [storm-shaded-deps-2.0.0.jar:2.0.0] at > org.apache.storm.shade.io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:340) > [storm-shaded-deps-2.0.0.jar:2.0.0] at > org.apache.storm.shade.io.netty.handler.codec.ByteToMessageDecoder.fireChannelRead(ByteToMessageDecoder.java:323) > [storm-shaded-deps-2.0.0.jar:2.0.0] at > org.apache.storm.shade.io.netty.handler.codec.ByteToMessageDecoder.channelRead(ByteToMessageDecoder.java:297) > [storm-shaded-deps-2.0.0.jar:2.0.0] at > org.apache.storm.shade.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:362) > [storm-shaded-deps-2.0.0.jar:2.0.0] at > org.apache.storm.shade.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:348) > [storm-shaded-deps-2.0.0.jar:2.0.0] at > org.apache.storm.shade.io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:340) > [storm-shaded-deps-2.0.0.jar:2.0.0] at > org.apache.storm.shade.io.netty.channel.DefaultChannelPipeline$HeadContext.channelRead(DefaultChannelPipeline.java:1434) > [storm-shaded-deps-2.0.0.jar:2.0.0] at > org.apache.storm.shade.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:362) > [storm-shaded-deps-2.0.0.jar:2.0.0] at > org.apache.storm.shade.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:348) > [storm-shaded-deps-2.0.0.jar:2.0.0] at > org.apache.storm.shade.io.netty.channel.DefaultChannelPipeline.fireChannelRead(DefaultChannelPipeline.java:965) > [storm-shaded-deps-2.0.0.jar:2.0.0] at > org.apache.storm.shade.io.netty.channel.nio.AbstractNioByteChannel$NioByteUnsafe.read(AbstractNioByteChannel.java:163) > [storm-shaded-deps-2.0.0.jar:2.0.0] at > org.apache.storm.shade.io.netty.channel.nio.NioEventLoop.processSelectedKey(NioEventLoop.java:644) > [storm-shaded-deps-2.0.0.jar:2.0.0] at > org.apache.storm.shade.io.netty.channel.nio.NioEventLoop.processSelectedKeysOptimized(NioEventLoop.java:579) > [storm-shaded-deps-2.0.0.jar:2.0.0] at > org.apache.storm.shade.io.netty.channel.nio.NioEventLoop.processSelectedKeys(NioEventLoop.java:496) > [storm-shaded-deps-2.0.0.jar:2.0.0] at > org.apache.storm.shade.io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:458) > [storm-shaded-deps-2.0.0.jar:2.0.0] at > org.apache.storm.shade.io.netty.util.concurrent.SingleThreadEventExecutor$5.run(SingleThreadEventExecutor.java:897) > [storm-shaded-deps-2.0.0.jar:2.0.0] at > java.lang.Thread.run(Thread.java:748) [?:1.8.0_212] > > After which storm process is dying. > > I'm not 100% that this is related to upgrade to 2.0 version of storm or if > anything related to kryo was touched, but I haven't seen anything similar > with previous version of storm (1.1.x) > > Have you seen anything similar? > > Best regards, > Michal > > > > -- > > *Open Source Solutions for Text Engineering* > > http://www.digitalpebble.com > http://digitalpebble.blogspot.com/ > #digitalpebble <http://twitter.com/digitalpebble> > -- *Open Source Solutions for Text Engineering* http://www.digitalpebble.com http://digitalpebble.blogspot.com/ #digitalpebble <http://twitter.com/digitalpebble>