Uploaded image for project: 'Apache Ozone'
  1. Apache Ozone
  2. HDDS-10977

Datanode down with "ratis.thirdparty.io.grpc.StatusRuntimeException: RESOURCE_EXHAUSTED"

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Open
    • Major
    • Resolution: Unresolved
    • None
    • None
    • Ozone Datanode
    • None

    Description

      Ozone DN abruptly aborted with: 
      org.apache.ratis.thirdparty.io.grpc.StatusRuntimeException: RESOURCE_EXHAUSTED

      Jun 03, 2024 5:01:33 PM org.apache.ratis.thirdparty.io.grpc.netty.NettyServerStream$TransportState deframeFailed
      WARNING: Exception processing message
      org.apache.ratis.thirdparty.io.grpc.StatusRuntimeException: RESOURCE_EXHAUSTED: gRPC message exceeds maximum size 33554432: 33554927
      	at org.apache.ratis.thirdparty.io.grpc.Status.asRuntimeException(Status.java:529)
      	at org.apache.ratis.thirdparty.io.grpc.internal.MessageDeframer.processHeader(MessageDeframer.java:392)
      	at org.apache.ratis.thirdparty.io.grpc.internal.MessageDeframer.deliver(MessageDeframer.java:272)
      	at org.apache.ratis.thirdparty.io.grpc.internal.MessageDeframer.deframe(MessageDeframer.java:178)
      	at org.apache.ratis.thirdparty.io.grpc.internal.AbstractStream$TransportState.deframe(AbstractStream.java:211)
      	at org.apache.ratis.thirdparty.io.grpc.internal.AbstractServerStream$TransportState.inboundDataReceived(AbstractServerStream.java:262)
      	at org.apache.ratis.thirdparty.io.grpc.netty.NettyServerStream$TransportState.inboundDataReceived(NettyServerStream.java:210)
      	at org.apache.ratis.thirdparty.io.grpc.netty.NettyServerHandler.onDataRead(NettyServerHandler.java:520)
      	at org.apache.ratis.thirdparty.io.grpc.netty.NettyServerHandler.access$900(NettyServerHandler.java:111)
      	at org.apache.ratis.thirdparty.io.grpc.netty.NettyServerHandler$FrameListener.onDataRead(NettyServerHandler.java:840)
      	at org.apache.ratis.thirdparty.io.netty.handler.codec.http2.DefaultHttp2ConnectionDecoder$FrameReadListener.onDataRead(DefaultHttp2ConnectionDecoder.java:307)
      	at org.apache.ratis.thirdparty.io.netty.handler.codec.http2.Http2InboundFrameLogger$1.onDataRead(Http2InboundFrameLogger.java:48)
      	at org.apache.ratis.thirdparty.io.netty.handler.codec.http2.DefaultHttp2FrameReader.readDataFrame(DefaultHttp2FrameReader.java:415)
      	at org.apache.ratis.thirdparty.io.netty.handler.codec.http2.DefaultHttp2FrameReader.processPayloadState(DefaultHttp2FrameReader.java:250)
      	at org.apache.ratis.thirdparty.io.netty.handler.codec.http2.DefaultHttp2FrameReader.readFrame(DefaultHttp2FrameReader.java:159)
      	at org.apache.ratis.thirdparty.io.netty.handler.codec.http2.Http2InboundFrameLogger.readFrame(Http2InboundFrameLogger.java:41)
      	at org.apache.ratis.thirdparty.io.netty.handler.codec.http2.DefaultHttp2ConnectionDecoder.decodeFrame(DefaultHttp2ConnectionDecoder.java:173)
      	at org.apache.ratis.thirdparty.io.netty.handler.codec.http2.Http2ConnectionHandler$FrameDecoder.decode(Http2ConnectionHandler.java:393)
      	at org.apache.ratis.thirdparty.io.netty.handler.codec.http2.Http2ConnectionHandler.decode(Http2ConnectionHandler.java:453)
      	at org.apache.ratis.thirdparty.io.netty.handler.codec.ByteToMessageDecoder.decodeRemovalReentryProtection(ByteToMessageDecoder.java:529)
      	at org.apache.ratis.thirdparty.io.netty.handler.codec.ByteToMessageDecoder.callDecode(ByteToMessageDecoder.java:468)
      	at org.apache.ratis.thirdparty.io.netty.handler.codec.ByteToMessageDecoder.channelRead(ByteToMessageDecoder.java:290)
      	at org.apache.ratis.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:444)
      	at org.apache.ratis.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:420)
      	at org.apache.ratis.thirdparty.io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:412)
      	at org.apache.ratis.thirdparty.io.netty.channel.DefaultChannelPipeline$HeadContext.channelRead(DefaultChannelPipeline.java:1410)
      	at org.apache.ratis.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:440)
      	at org.apache.ratis.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:420)
      	at org.apache.ratis.thirdparty.io.netty.channel.DefaultChannelPipeline.fireChannelRead(DefaultChannelPipeline.java:919)
      	at org.apache.ratis.thirdparty.io.netty.channel.epoll.AbstractEpollStreamChannel$EpollStreamUnsafe.epollInReady(AbstractEpollStreamChannel.java:800)
      	at org.apache.ratis.thirdparty.io.netty.channel.epoll.EpollEventLoop.processReady(EpollEventLoop.java:509)
      	at org.apache.ratis.thirdparty.io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:407)
      	at org.apache.ratis.thirdparty.io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:997)
      	at org.apache.ratis.thirdparty.io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74)
      	at org.apache.ratis.thirdparty.io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
      	at java.lang.Thread.run(Thread.java:748)
      
      OpenJDK 64-Bit Server VM warning: INFO: os::commit_memory(0x00000005e9ca0000, 1635909632, 0) failed; error='Cannot allocate memory' (errno=12) 

      Before Aborting, there were lots of CONTAINER_NOT_FOUND messages:

      2024-06-03 17:01:33,712 WARN [882ad4eb-04f9-418e-9ea6-0802b19beade-ChunkReader-11]-org.apache.hadoop.ozone.container.common.impl.HddsDispatcher: Operation: ReadChunk , Trace ID:  , Message: ContainerID 2475 does not exist , Result: CONTAINER_NOT_FOUND , StorageContainerException Occurred.
      org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException: ContainerID 2475 does not exist
              at org.apache.hadoop.ozone.container.common.impl.HddsDispatcher.dispatchRequest(HddsDispatcher.java:314)
              at org.apache.hadoop.ozone.container.common.impl.HddsDispatcher.lambda$dispatch$0(HddsDispatcher.java:192)
              at org.apache.hadoop.hdds.server.OzoneProtocolMessageDispatcher.processRequest(OzoneProtocolMessageDispatcher.java:89)
              at org.apache.hadoop.ozone.container.common.impl.HddsDispatcher.dispatch(HddsDispatcher.java:191)
              at org.apache.hadoop.ozone.container.common.transport.server.GrpcXceiverService$1.onNext(GrpcXceiverService.java:112)
              at org.apache.hadoop.ozone.container.common.transport.server.GrpcXceiverService$1.onNext(GrpcXceiverService.java:105)
              at org.apache.ratis.thirdparty.io.grpc.stub.ServerCalls$StreamingServerCallHandler$StreamingServerCallListener.onMessage(ServerCalls.java:262)
              at org.apache.ratis.thirdparty.io.grpc.ForwardingServerCallListener.onMessage(ForwardingServerCallListener.java:33)
              at org.apache.hadoop.hdds.tracing.GrpcServerInterceptor$1.onMessage(GrpcServerInterceptor.java:49)
              at org.apache.ratis.thirdparty.io.grpc.internal.ServerCallImpl$ServerStreamListenerImpl.messagesAvailableInternal(ServerCallImpl.java:329)
              at org.apache.ratis.thirdparty.io.grpc.internal.ServerCallImpl$ServerStreamListenerImpl.messagesAvailable(ServerCallImpl.java:314)
              at org.apache.ratis.thirdparty.io.grpc.internal.ServerImpl$JumpToApplicationThreadServerStreamListener$1MessagesAvailable.runInContext(ServerImpl.java:833)
              at org.apache.ratis.thirdparty.io.grpc.internal.ContextRunnable.run(ContextRunnable.java:37)
              at org.apache.ratis.thirdparty.io.grpc.internal.SerializingExecutor.run(SerializingExecutor.java:133)
              at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
              at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
              at java.lang.Thread.run(Thread.java:748) 

      Checked the count its more than 10k

      grep "Result: CONTAINER_NOT_FOUND , StorageContainerException Occurred" /var/log/hadoop-ozone/ozone-datanode.log | wc -l
      10134 

      cc: weichiu ashishk Sammi 

      Attachments

        Activity

          People

            Unassigned Unassigned
            pratyush.bhatt Pratyush Bhatt
            Votes:
            0 Vote for this issue
            Watchers:
            4 Start watching this issue

            Dates

              Created:
              Updated: