Hi,

See answers below.

/Jakob

On Mon, Oct 19, 2015 at 4:03 PM, Maximilian Michels <m...@apache.org> wrote:

> Hi Jakob,
>
> Thank you for reporting the bug. Could you please post your
> configuration here? In particular, could you please tell us the value
> of the following configuration variables:
>
> taskmanager.heap.mb

taskmanager.heap.mb: 2048

> taskmanager.network.numberOfBuffers
>

Default value. Not changed.


> taskmanager.memory.off-heap
>
> Default value Not changed.


> Are you running the Flink cluster in batch or streaming mode?
>
> Started in streaming mode. Running with two nodes. In the cluster.
Also, I have set the "env.java.opts: -XX:+UseConcMarkSweepGC" due to some
strange java core dumps in the G1 GC.


> Direct memory is used by Flink's network layer. My guess is that you
> have set taskmanager.heap.mb too low (it constraints the number of
> direct memory at the moment).
>
> Thank you,
> Max
>
>
> On Mon, Oct 19, 2015 at 3:24 PM, Jakob Ericsson
> <jakob.erics...@gmail.com> wrote:
> > Hello,
> >
> > We are running into a strange problem with Direct Memory buffers. From
> what
> > I know, we are not using any direct memory buffers inside our code.
> > This is pretty trivial streaming application just doing some dedupliction
> > and union some kafka streams.
> >
> > /Jakob
> >
> >
> >
> > 2015-10-19 13:27:59,064 INFO  org.apache.flink.runtime.taskmanager.Task
> > - FilterAndTransform -> (Filter, Filter) (3/4) switched to FAILED with
> > exception.
> >
> org.apache.flink.runtime.io.network.netty.exception.LocalTransportException:
> > java.lang.OutOfMemoryError: Direct buffer memory
> >         at
> >
> org.apache.flink.runtime.io.network.netty.PartitionRequestClientHandler.exceptionCaught(PartitionRequestClientHandler.java:153)
> >         at
> >
> io.netty.channel.AbstractChannelHandlerContext.invokeExceptionCaught(AbstractChannelHandlerContext.java:246)
> >         at
> >
> io.netty.channel.AbstractChannelHandlerContext.fireExceptionCaught(AbstractChannelHandlerContext.java:224)
> >         at
> >
> io.netty.channel.ChannelInboundHandlerAdapter.exceptionCaught(ChannelInboundHandlerAdapter.java:131)
> >         at
> >
> io.netty.channel.AbstractChannelHandlerContext.invokeExceptionCaught(AbstractChannelHandlerContext.java:246)
> >         at
> >
> io.netty.channel.AbstractChannelHandlerContext.fireExceptionCaught(AbstractChannelHandlerContext.java:224)
> >         at
> >
> io.netty.channel.ChannelInboundHandlerAdapter.exceptionCaught(ChannelInboundHandlerAdapter.java:131)
> >         at
> >
> io.netty.channel.AbstractChannelHandlerContext.invokeExceptionCaught(AbstractChannelHandlerContext.java:246)
> >         at
> >
> io.netty.channel.AbstractChannelHandlerContext.notifyHandlerException(AbstractChannelHandlerContext.java:737)
> >         at
> >
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:310)
> >         at
> >
> io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:294)
> >         at
> >
> io.netty.channel.DefaultChannelPipeline.fireChannelRead(DefaultChannelPipeline.java:846)
> >         at
> >
> io.netty.channel.nio.AbstractNioByteChannel$NioByteUnsafe.read(AbstractNioByteChannel.java:131)
> >         at
> >
> io.netty.channel.nio.NioEventLoop.processSelectedKey(NioEventLoop.java:511)
> >         at
> >
> io.netty.channel.nio.NioEventLoop.processSelectedKeysOptimized(NioEventLoop.java:468)
> >         at
> >
> io.netty.channel.nio.NioEventLoop.processSelectedKeys(NioEventLoop.java:382)
> >         at io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:354)
> >         at
> >
> io.netty.util.concurrent.SingleThreadEventExecutor$2.run(SingleThreadEventExecutor.java:112)
> >         at java.lang.Thread.run(Thread.java:745)
> > Caused by: io.netty.handler.codec.DecoderException:
> > java.lang.OutOfMemoryError: Direct buffer memory
> >         at
> >
> io.netty.handler.codec.ByteToMessageDecoder.channelRead(ByteToMessageDecoder.java:234)
> >         at
> >
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:308)
> >         ... 9 more
> > Caused by: java.lang.OutOfMemoryError: Direct buffer memory
> >         at java.nio.Bits.reserveMemory(Bits.java:658)
> >         at java.nio.DirectByteBuffer.<init>(DirectByteBuffer.java:123)
> >         at java.nio.ByteBuffer.allocateDirect(ByteBuffer.java:311)
> >         at
> >
> io.netty.buffer.PoolArena$DirectArena.newUnpooledChunk(PoolArena.java:651)
> >         at io.netty.buffer.PoolArena.allocateHuge(PoolArena.java:237)
> >         at io.netty.buffer.PoolArena.allocate(PoolArena.java:215)
> >         at io.netty.buffer.PoolArena.reallocate(PoolArena.java:358)
> >         at io.netty.buffer.PooledByteBuf.capacity(PooledByteBuf.java:111)
> >         at
> > io.netty.buffer.AbstractByteBuf.ensureWritable(AbstractByteBuf.java:251)
> >         at
> > io.netty.buffer.AbstractByteBuf.writeBytes(AbstractByteBuf.java:849)
> >         at
> > io.netty.buffer.AbstractByteBuf.writeBytes(AbstractByteBuf.java:841)
> >         at
> > io.netty.buffer.AbstractByteBuf.writeBytes(AbstractByteBuf.java:831)
> >         at
> >
> io.netty.handler.codec.ByteToMessageDecoder$1.cumulate(ByteToMessageDecoder.java:92)
> >         at
> >
> io.netty.handler.codec.ByteToMessageDecoder.channelRead(ByteToMessageDecoder.java:228)
> >         ... 10 more
> >
>
  • [no subject] Jakob Ericsson
    • Re: Maximilian Michels
      • Re: Jakob Ericsson
        • Re: Maximilian Michels
          • Re: Maximilian Michels
            • Re: Gyula Fóra
              • Re: Maximilian Michels
                • Re: Maximilian Michels
                • Re: Jakob Ericsson
                • Re: Maximilian Michels
                • Re: Stephan Ewen
                • Re: Jakob Ericsson

Reply via email to