And We see the same too

Med venlig hilsen / Best regards
Lasse Nedergaard


> Den 8. feb. 2018 kl. 11.58 skrev Stavros Kontopoulos 
> <st.kontopou...@gmail.com>:
> 
> We see the same issue here (2):
> 2018-02-08 10:55:11,447 ERROR 
> org.apache.flink.runtime.webmonitor.files.StaticFileServerHandler  - Caught 
> exception
> java.io.IOException: Connection reset by peer
> 
> Stavros
> 
>> On Sat, Jan 13, 2018 at 9:59 PM, Eron Wright <eronwri...@gmail.com> wrote:
>> Hello Dongwon,
>> 
>> Flink doesn't support a 'unique host' constraint at this time; it simply 
>> accepts adequate offers without any such consideration.   Flink does support 
>> a 'host attributes' constraint to filter certain hosts, but that's not 
>> applicable here.
>> 
>> Under the hood, Flink uses a library called Netflix Fenzo to optimize 
>> placement, and a uniqueness constraint could be added by more deeply 
>> leveraging Fenzo's constraint system.   You mentioned that you're trying to 
>> make good use of your GPU resources, which could also be achieved by 
>> treating GPU as a scalar resource (similar to how memory and cores are 
>> treated).   Mesos does support that, but Fenzo may require some enhancement. 
>>   So, these are two potential ways to enhance Flink to support your 
>> scenario.  I'm happy to help; reach out to me.
>> 
>> The obvious, ugly workaround is to configure your TMs to be large enough to 
>> consume the whole host.
>> 
>> Eron
>> 
>> 
>> 
>> 
>> 
>>> On Thu, Jan 11, 2018 at 7:18 AM, Gary Yao <g...@data-artisans.com> wrote:
>>> Hi Dongwon,
>>> 
>>> I am not familiar with the deployment on DC/OS. However, Eron Wright and 
>>> Jörg
>>> Schad (cc'd), who have worked on the Mesos integration, might be able to 
>>> help
>>> you.
>>> 
>>> Best,
>>> Gary
>>> 
>>>> On Tue, Jan 9, 2018 at 10:29 AM, Dongwon Kim <eastcirc...@gmail.com> wrote:
>>>> Hi,
>>>> 
>>>> I've launched JobManager and TaskManager on DC/OS successfully.
>>>> Now I have two new issues:
>>>> 
>>>> 1) All TaskManagers are scheduled on a single node. 
>>>> - Is it intended to maximize data locality and minimize network 
>>>> communication cost?
>>>> - Is there an option in Flink to adjust the behavior of JobManager when it 
>>>> considers multiple resource offers from different Mesos agents?
>>>> - I want to schedule TaskManager processes on different GPU servers so 
>>>> that each TaskManger process can use its own GPU cards exclusively.  
>>>> - Below is a part of JobManager log that is occurring while JobManager is 
>>>> negotiating resources with the Mesos master:
>>>> 2018-01-09 07:34:54,872 INFO  
>>>> org.apache.flink.mesos.runtime.clusterframework.MesosJobManager  - 
>>>> JobManager akka.tcp://flink@dnn-g08-233:18026/user/jobmanager was granted 
>>>> leadership with leader session ID 
>>>> Some(00000000-0000-0000-0000-000000000000).
>>>> 2018-01-09 07:34:55,889 INFO  
>>>> org.apache.flink.mesos.scheduler.ConnectionMonitor            - Connecting 
>>>> to Mesos...
>>>> 2018-01-09 07:34:55,962 INFO  
>>>> org.apache.flink.mesos.runtime.clusterframework.MesosFlinkResourceManager  
>>>> - Trying to associate with JobManager leader 
>>>> akka.tcp://flink@dnn-g08-233:18026/user/jobmanager
>>>> 2018-01-09 07:34:55,977 INFO  
>>>> org.apache.flink.mesos.runtime.clusterframework.MesosFlinkResourceManager  
>>>> - Resource Manager associating with leading JobManager 
>>>> Actor[akka://flink/user/jobmanager#-1481183359] - leader session 
>>>> 00000000-0000-0000-0000-000000000000
>>>> 2018-01-09 07:34:56,479 INFO  
>>>> org.apache.flink.mesos.runtime.clusterframework.MesosFlinkResourceManager  
>>>> - Scheduling Mesos task taskmanager-00001 with (10240.0 MB, 8.0 cpus).
>>>> 2018-01-09 07:34:56,481 INFO  
>>>> org.apache.flink.mesos.runtime.clusterframework.MesosFlinkResourceManager  
>>>> - Scheduling Mesos task taskmanager-00002 with (10240.0 MB, 8.0 cpus).
>>>> 2018-01-09 07:34:56,481 INFO  
>>>> org.apache.flink.mesos.runtime.clusterframework.MesosFlinkResourceManager  
>>>> - Scheduling Mesos task taskmanager-00003 with (10240.0 MB, 8.0 cpus).
>>>> 2018-01-09 07:34:56,481 INFO  
>>>> org.apache.flink.mesos.runtime.clusterframework.MesosFlinkResourceManager  
>>>> - Scheduling Mesos task taskmanager-00004 with (10240.0 MB, 8.0 cpus).
>>>> 2018-01-09 07:34:56,481 INFO  
>>>> org.apache.flink.mesos.runtime.clusterframework.MesosFlinkResourceManager  
>>>> - Scheduling Mesos task taskmanager-00005 with (10240.0 MB, 8.0 cpus).
>>>> 2018-01-09 07:34:56,483 INFO  
>>>> org.apache.flink.mesos.scheduler.LaunchCoordinator            - Now 
>>>> gathering offers for at least 5 task(s).
>>>> 2018-01-09 07:34:56,484 INFO  
>>>> org.apache.flink.mesos.scheduler.ConnectionMonitor            - Connected 
>>>> to Mesos as framework ID 59b85b42-a4a2-4632-9578-9e480585ecdc-0004.
>>>> 2018-01-09 07:34:56,690 INFO  
>>>> org.apache.flink.mesos.scheduler.LaunchCoordinator            - Received 
>>>> offer(s) of 606170.0 MB, 234.2 cpus:
>>>> 2018-01-09 07:34:56,692 INFO  
>>>> org.apache.flink.mesos.scheduler.LaunchCoordinator            -   
>>>> 59b85b42-a4a2-4632-9578-9e480585ecdc-O2174 from 50.1.100.233 of 111186.0 
>>>> MB, 45.9 cpus for [*]
>>>> 2018-01-09 07:34:56,692 INFO  
>>>> org.apache.flink.mesos.scheduler.LaunchCoordinator            -   
>>>> 59b85b42-a4a2-4632-9578-9e480585ecdc-O2175 from 50.1.100.235 of 123506.0 
>>>> MB, 47.3 cpus for [*]
>>>> 2018-01-09 07:34:56,692 INFO  
>>>> org.apache.flink.mesos.scheduler.LaunchCoordinator            -   
>>>> 59b85b42-a4a2-4632-9578-9e480585ecdc-O2176 from 50.1.100.234 of 124530.0 
>>>> MB, 46.6 cpus for [*]
>>>> 2018-01-09 07:34:56,692 INFO  
>>>> org.apache.flink.mesos.scheduler.LaunchCoordinator            -   
>>>> 59b85b42-a4a2-4632-9578-9e480585ecdc-O2177 from 50.1.100.231 of 123474.0 
>>>> MB, 47.2 cpus for [*]
>>>> 2018-01-09 07:34:56,693 INFO  
>>>> org.apache.flink.mesos.scheduler.LaunchCoordinator            -   
>>>> 59b85b42-a4a2-4632-9578-9e480585ecdc-O2178 from 50.1.100.232 of 123474.0 
>>>> MB, 47.2 cpus for [*]
>>>> 2018-01-09 07:34:57,711 INFO  
>>>> org.apache.flink.mesos.scheduler.LaunchCoordinator            - Processing 
>>>> 5 task(s) against 5 new offer(s) plus outstanding offers.
>>>> 2018-01-09 07:34:57,726 INFO  
>>>> org.apache.flink.mesos.scheduler.LaunchCoordinator            - Resources 
>>>> considered: (note: expired offers not deducted from below)
>>>> 2018-01-09 07:34:57,727 INFO  
>>>> org.apache.flink.mesos.scheduler.LaunchCoordinator            -   
>>>> 50.1.100.234 has 124530.0 MB, 46.6 cpus
>>>> 2018-01-09 07:34:57,728 INFO  
>>>> org.apache.flink.mesos.scheduler.LaunchCoordinator            -   
>>>> 50.1.100.235 has 123506.0 MB, 47.3 cpus
>>>> 2018-01-09 07:34:57,728 INFO  
>>>> org.apache.flink.mesos.scheduler.LaunchCoordinator            -   
>>>> 50.1.100.232 has 123474.0 MB, 47.2 cpus
>>>> 2018-01-09 07:34:57,728 INFO  
>>>> org.apache.flink.mesos.scheduler.LaunchCoordinator            -   
>>>> 50.1.100.233 has 111186.0 MB, 45.9 cpus
>>>> 2018-01-09 07:34:57,728 INFO  
>>>> org.apache.flink.mesos.scheduler.LaunchCoordinator            -   
>>>> 50.1.100.231 has 123474.0 MB, 47.2 cpus
>>>> 2018-01-09 07:34:58,069 INFO  
>>>> org.apache.flink.mesos.runtime.clusterframework.MesosFlinkResourceManager  
>>>> - Launching Mesos task taskmanager-00005 on host 50.1.100.231.
>>>> 2018-01-09 07:34:58,069 INFO  
>>>> org.apache.flink.mesos.scheduler.LaunchCoordinator            - Launched 5 
>>>> task(s) on 50.1.100.231 using 1 offer(s):
>>>> 2018-01-09 07:34:58,070 INFO  
>>>> org.apache.flink.mesos.runtime.clusterframework.MesosFlinkResourceManager  
>>>> - Launching Mesos task taskmanager-00002 on host 50.1.100.231.
>>>> 2018-01-09 07:34:58,070 INFO  
>>>> org.apache.flink.mesos.runtime.clusterframework.MesosFlinkResourceManager  
>>>> - Launching Mesos task taskmanager-00003 on host 50.1.100.231.
>>>> 2018-01-09 07:34:58,070 INFO  
>>>> org.apache.flink.mesos.runtime.clusterframework.MesosFlinkResourceManager  
>>>> - Launching Mesos task taskmanager-00004 on host 50.1.100.231.
>>>> 2018-01-09 07:34:58,070 INFO  
>>>> org.apache.flink.mesos.runtime.clusterframework.MesosFlinkResourceManager  
>>>> - Launching Mesos task taskmanager-00001 on host 50.1.100.231.
>>>> 2018-01-09 07:34:58,070 INFO  
>>>> org.apache.flink.mesos.scheduler.LaunchCoordinator            -   
>>>> 59b85b42-a4a2-4632-9578-9e480585ecdc-O2177
>>>> 2018-01-09 07:34:58,071 INFO  
>>>> org.apache.flink.mesos.scheduler.LaunchCoordinator            - No longer 
>>>> gathering offers; all requests fulfilled.
>>>> 2018-01-09 07:34:58,072 INFO  com.netflix.fenzo.TaskScheduler              
>>>>                  - Expiring all leases
>>>> 2018-01-09 07:34:58,072 INFO  
>>>> org.apache.flink.mesos.scheduler.LaunchCoordinator            - Declined 
>>>> offer 59b85b42-a4a2-4632-9578-9e480585ecdc-O2176 from 50.1.100.234 of 
>>>> 124530.0 MB, 46.6 cpus.
>>>> 2018-01-09 07:34:58,073 INFO  
>>>> org.apache.flink.mesos.scheduler.LaunchCoordinator            - Declined 
>>>> offer 59b85b42-a4a2-4632-9578-9e480585ecdc-O2175 from 50.1.100.235 of 
>>>> 123506.0 MB, 47.3 cpus.
>>>> 2018-01-09 07:34:58,073 INFO  
>>>> org.apache.flink.mesos.scheduler.LaunchCoordinator            - Declined 
>>>> offer 59b85b42-a4a2-4632-9578-9e480585ecdc-O2178 from 50.1.100.232 of 
>>>> 123474.0 MB, 47.2 cpus.
>>>> 2018-01-09 07:34:58,074 INFO  
>>>> org.apache.flink.mesos.scheduler.LaunchCoordinator            - Declined 
>>>> offer 59b85b42-a4a2-4632-9578-9e480585ecdc-O2174 from 50.1.100.233 of 
>>>> 111186.0 MB, 45.9 cpus.
>>>> 2018-01-09 07:35:05,868 INFO  org.apache.flink.mesos.scheduler.TaskMonitor 
>>>>                  - Mesos task taskmanager-00005 is running.
>>>> 2018-01-09 07:35:06,103 INFO  org.apache.flink.mesos.scheduler.TaskMonitor 
>>>>                  - Mesos task taskmanager-00001 is running.
>>>> 2018-01-09 07:35:06,111 INFO  org.apache.flink.mesos.scheduler.TaskMonitor 
>>>>                  - Mesos task taskmanager-00004 is running.
>>>> 2018-01-09 07:35:06,116 INFO  org.apache.flink.mesos.scheduler.TaskMonitor 
>>>>                  - Mesos task taskmanager-00002 is running.
>>>> 2018-01-09 07:35:06,119 INFO  org.apache.flink.mesos.scheduler.TaskMonitor 
>>>>                  - Mesos task taskmanager-00003 is running.
>>>> 2018-01-09 07:35:14,377 INFO  
>>>> org.apache.flink.mesos.runtime.clusterframework.MesosFlinkResourceManager  
>>>> - TaskManager taskmanager-00003 has started.
>>>> 2018-01-09 07:35:14,380 INFO  
>>>> org.apache.flink.runtime.instance.InstanceManager             - Registered 
>>>> TaskManager at DNN-G08-231 
>>>> (akka.tcp://flink@dnn-g08-231:1027/user/taskmanager) as 
>>>> b94277c8ad550eeef5364947e4330c00. Current number of registered hosts is 1. 
>>>> Current number of alive task slots is 8.
>>>> 2018-01-09 07:35:14,389 INFO  
>>>> org.apache.flink.mesos.runtime.clusterframework.MesosFlinkResourceManager  
>>>> - TaskManager taskmanager-00004 has started.
>>>> 2018-01-09 07:35:14,389 INFO  
>>>> org.apache.flink.runtime.instance.InstanceManager             - Registered 
>>>> TaskManager at DNN-G08-231 
>>>> (akka.tcp://flink@dnn-g08-231:1033/user/taskmanager) as 
>>>> e0183a5317b331b90496049b1893c922. Current number of registered hosts is 2. 
>>>> Current number of alive task slots is 16.
>>>> 2018-01-09 07:35:14,462 INFO  
>>>> org.apache.flink.mesos.runtime.clusterframework.MesosFlinkResourceManager  
>>>> - TaskManager taskmanager-00001 has started.
>>>> 2018-01-09 07:35:14,462 INFO  
>>>> org.apache.flink.runtime.instance.InstanceManager             - Registered 
>>>> TaskManager at DNN-G08-231 
>>>> (akka.tcp://flink@dnn-g08-231:1029/user/taskmanager) as 
>>>> 8d85b49d4118514552fcad3b98fef3e2. Current number of registered hosts is 3. 
>>>> Current number of alive task slots is 24.
>>>> 2018-01-09 07:35:14,465 INFO  
>>>> org.apache.flink.mesos.runtime.clusterframework.MesosFlinkResourceManager  
>>>> - TaskManager taskmanager-00005 has started.
>>>> 2018-01-09 07:35:14,465 INFO  
>>>> org.apache.flink.runtime.instance.InstanceManager             - Registered 
>>>> TaskManager at DNN-G08-231 
>>>> (akka.tcp://flink@dnn-g08-231:1031/user/taskmanager) as 
>>>> b740607fb2e88bcfc275498bb54ed9fd. Current number of registered hosts is 4. 
>>>> Current number of alive task slots is 32.
>>>> 2018-01-09 07:35:14,560 INFO  
>>>> org.apache.flink.mesos.runtime.clusterframework.MesosFlinkResourceManager  
>>>> - TaskManager taskmanager-00002 has started.
>>>> 2018-01-09 07:35:14,560 INFO  
>>>> org.apache.flink.runtime.instance.InstanceManager             - Registered 
>>>> TaskManager at DNN-G08-231 
>>>> (akka.tcp://flink@dnn-g08-231:1025/user/taskmanager) as 
>>>> 95433440f37ea1790e7ef9309f110fe4. Current number of registered hosts is 5. 
>>>> Current number of alive task slots is 40.
>>>> 
>>>> 
>>>> 2) After the TaskManagers are started, the following lines are repeated in 
>>>> the JobManage log every second:
>>>> 2018-01-09 07:36:51,080 ERROR 
>>>> org.apache.flink.runtime.rest.handler.legacy.files.StaticFileServerHandler 
>>>>  - Caught exception
>>>> java.io.IOException: Connection reset by peer
>>>>    at sun.nio.ch.FileDispatcherImpl.read0(Native Method)
>>>>    at sun.nio.ch.SocketDispatcher.read(SocketDispatcher.java:39)
>>>>    at sun.nio.ch.IOUtil.readIntoNativeBuffer(IOUtil.java:223)
>>>>    at sun.nio.ch.IOUtil.read(IOUtil.java:192)
>>>>    at sun.nio.ch.SocketChannelImpl.read(SocketChannelImpl.java:380)
>>>>    at 
>>>> org.apache.flink.shaded.netty4.io.netty.buffer.UnpooledUnsafeDirectByteBuf.setBytes(UnpooledUnsafeDirectByteBuf.java:447)
>>>>    at 
>>>> org.apache.flink.shaded.netty4.io.netty.buffer.AbstractByteBuf.writeBytes(AbstractByteBuf.java:881)
>>>>    at 
>>>> org.apache.flink.shaded.netty4.io.netty.channel.socket.nio.NioSocketChannel.doReadBytes(NioSocketChannel.java:241)
>>>>    at 
>>>> org.apache.flink.shaded.netty4.io.netty.channel.nio.AbstractNioByteChannel$NioByteUnsafe.read(AbstractNioByteChannel.java:119)
>>>>    at 
>>>> org.apache.flink.shaded.netty4.io.netty.channel.nio.NioEventLoop.processSelectedKey(NioEventLoop.java:511)
>>>>    at 
>>>> org.apache.flink.shaded.netty4.io.netty.channel.nio.NioEventLoop.processSelectedKeysOptimized(NioEventLoop.java:468)
>>>>    at 
>>>> org.apache.flink.shaded.netty4.io.netty.channel.nio.NioEventLoop.processSelectedKeys(NioEventLoop.java:382)
>>>>    at 
>>>> org.apache.flink.shaded.netty4.io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:354)
>>>>    at 
>>>> org.apache.flink.shaded.netty4.io.netty.util.concurrent.SingleThreadEventExecutor$2.run(SingleThreadEventExecutor.java:111)
>>>>    at 
>>>> org.apache.flink.shaded.netty4.io.netty.util.concurrent.DefaultThreadFactory$DefaultRunnableDecorator.run(DefaultThreadFactory.java:137)
>>>>    at java.lang.Thread.run(Thread.java:748)
>>>> 2018-01-09 07:37:43,600 ERROR 
>>>> org.apache.flink.runtime.rest.handler.legacy.files.StaticFileServerHandler 
>>>>  - Caught exception
>>>> java.io.IOException: Connection reset by peer
>>>>    at sun.nio.ch.FileDispatcherImpl.read0(Native Method)
>>>>    at sun.nio.ch.SocketDispatcher.read(SocketDispatcher.java:39)
>>>>    at sun.nio.ch.IOUtil.readIntoNativeBuffer(IOUtil.java:223)
>>>>    at sun.nio.ch.IOUtil.read(IOUtil.java:192)
>>>>    at sun.nio.ch.SocketChannelImpl.read(SocketChannelImpl.java:380)
>>>>    at 
>>>> org.apache.flink.shaded.netty4.io.netty.buffer.UnpooledUnsafeDirectByteBuf.setBytes(UnpooledUnsafeDirectByteBuf.java:447)
>>>>    at 
>>>> org.apache.flink.shaded.netty4.io.netty.buffer.AbstractByteBuf.writeBytes(AbstractByteBuf.java:881)
>>>>    at 
>>>> org.apache.flink.shaded.netty4.io.netty.channel.socket.nio.NioSocketChannel.doReadBytes(NioSocketChannel.java:241)
>>>>    at 
>>>> org.apache.flink.shaded.netty4.io.netty.channel.nio.AbstractNioByteChannel$NioByteUnsafe.read(AbstractNioByteChannel.java:119)
>>>>    at 
>>>> org.apache.flink.shaded.netty4.io.netty.channel.nio.NioEventLoop.processSelectedKey(NioEventLoop.java:511)
>>>>    at 
>>>> org.apache.flink.shaded.netty4.io.netty.channel.nio.NioEventLoop.processSelectedKeysOptimized(NioEventLoop.java:468)
>>>>    at 
>>>> org.apache.flink.shaded.netty4.io.netty.channel.nio.NioEventLoop.processSelectedKeys(NioEventLoop.java:382)
>>>>    at 
>>>> org.apache.flink.shaded.netty4.io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:354)
>>>>    at 
>>>> org.apache.flink.shaded.netty4.io.netty.util.concurrent.SingleThreadEventExecutor$2.run(SingleThreadEventExecutor.java:111)
>>>>    at 
>>>> org.apache.flink.shaded.netty4.io.netty.util.concurrent.DefaultThreadFactory$DefaultRunnableDecorator.run(DefaultThreadFactory.java:137)
>>>>    at java.lang.Thread.run(Thread.java:748)
>>>> 2018-01-09 07:38:43,619 ERROR 
>>>> org.apache.flink.runtime.rest.handler.legacy.files.StaticFileServerHandler 
>>>>  - Caught exception
>>>> java.io.IOException: Connection reset by peer
>>>>    at sun.nio.ch.FileDispatcherImpl.read0(Native Method)
>>>>    at sun.nio.ch.SocketDispatcher.read(SocketDispatcher.java:39)
>>>>    at sun.nio.ch.IOUtil.readIntoNativeBuffer(IOUtil.java:223)
>>>>    at sun.nio.ch.IOUtil.read(IOUtil.java:192)
>>>>    at sun.nio.ch.SocketChannelImpl.read(SocketChannelImpl.java:380)
>>>>    at 
>>>> org.apache.flink.shaded.netty4.io.netty.buffer.UnpooledUnsafeDirectByteBuf.setBytes(UnpooledUnsafeDirectByteBuf.java:447)
>>>>    at 
>>>> org.apache.flink.shaded.netty4.io.netty.buffer.AbstractByteBuf.writeBytes(AbstractByteBuf.java:881)
>>>>    at 
>>>> org.apache.flink.shaded.netty4.io.netty.channel.socket.nio.NioSocketChannel.doReadBytes(NioSocketChannel.java:241)
>>>>    at 
>>>> org.apache.flink.shaded.netty4.io.netty.channel.nio.AbstractNioByteChannel$NioByteUnsafe.read(AbstractNioByteChannel.java:119)
>>>>    at 
>>>> org.apache.flink.shaded.netty4.io.netty.channel.nio.NioEventLoop.processSelectedKey(NioEventLoop.java:511)
>>>>    at 
>>>> org.apache.flink.shaded.netty4.io.netty.channel.nio.NioEventLoop.processSelectedKeysOptimized(NioEventLoop.java:468)
>>>>    at 
>>>> org.apache.flink.shaded.netty4.io.netty.channel.nio.NioEventLoop.processSelectedKeys(NioEventLoop.java:382)
>>>>    at 
>>>> org.apache.flink.shaded.netty4.io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:354)
>>>>    at 
>>>> org.apache.flink.shaded.netty4.io.netty.util.concurrent.SingleThreadEventExecutor$2.run(SingleThreadEventExecutor.java:111)
>>>>    at 
>>>> org.apache.flink.shaded.netty4.io.netty.util.concurrent.DefaultThreadFactory$DefaultRunnableDecorator.run(DefaultThreadFactory.java:137)
>>>>    at java.lang.Thread.run(Thread.java:748)
>>>> 2018-01-09 07:39:43,630 ERROR 
>>>> org.apache.flink.runtime.rest.handler.legacy.files.StaticFileServerHandler 
>>>>  - Caught exception
>>>> java.io.IOException: Connection reset by peer
>>>>    at sun.nio.ch.FileDispatcherImpl.read0(Native Method)
>>>>    at sun.nio.ch.SocketDispatcher.read(SocketDispatcher.java:39)
>>>>    at sun.nio.ch.IOUtil.readIntoNativeBuffer(IOUtil.java:223)
>>>>    at sun.nio.ch.IOUtil.read(IOUtil.java:192)
>>>>    at sun.nio.ch.SocketChannelImpl.read(SocketChannelImpl.java:380)
>>>>    at 
>>>> org.apache.flink.shaded.netty4.io.netty.buffer.UnpooledUnsafeDirectByteBuf.setBytes(UnpooledUnsafeDirectByteBuf.java:447)
>>>>    at 
>>>> org.apache.flink.shaded.netty4.io.netty.buffer.AbstractByteBuf.writeBytes(AbstractByteBuf.java:881)
>>>>    at 
>>>> org.apache.flink.shaded.netty4.io.netty.channel.socket.nio.NioSocketChannel.doReadBytes(NioSocketChannel.java:241)
>>>>    at 
>>>> org.apache.flink.shaded.netty4.io.netty.channel.nio.AbstractNioByteChannel$NioByteUnsafe.read(AbstractNioByteChannel.java:119)
>>>>    at 
>>>> org.apache.flink.shaded.netty4.io.netty.channel.nio.NioEventLoop.processSelectedKey(NioEventLoop.java:511)
>>>>    at 
>>>> org.apache.flink.shaded.netty4.io.netty.channel.nio.NioEventLoop.processSelectedKeysOptimized(NioEventLoop.java:468)
>>>>    at 
>>>> org.apache.flink.shaded.netty4.io.netty.channel.nio.NioEventLoop.processSelectedKeys(NioEventLoop.java:382)
>>>>    at 
>>>> org.apache.flink.shaded.netty4.io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:354)
>>>>    at 
>>>> org.apache.flink.shaded.netty4.io.netty.util.concurrent.SingleThreadEventExecutor$2.run(SingleThreadEventExecutor.java:111)
>>>>    at 
>>>> org.apache.flink.shaded.netty4.io.netty.util.concurrent.DefaultThreadFactory$DefaultRunnableDecorator.run(DefaultThreadFactory.java:137)
>>>>    at java.lang.Thread.run(Thread.java:748)
>>>> - Can I ignore this exception? or there's something I should fix up?
>>>> 
>>>> Best,
>>>> 
>>>> - Dongwon
>>>> 
>>> 
>> 
> 

Reply via email to