[ 
https://issues.apache.org/jira/browse/YARN-7168?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Jiandan Yang  updated YARN-7168:
--------------------------------
    Description: 
In our cluster, when found NodeManager frequently FullGC when decommissioning 
NodeManager, and we found the biggest object is dataQueue of DataStreamer, it 
has almost 6w DFSPacket, and every DFSPacket is about 64k, as shown below.
The root reason is that the size of dataQueue and ackQueue in DataStreamer has 
no limit when writer thread is interrupted.
DFSOutputStream#waitAndQueuePacket does not wait when writer thread is 
interrupted.
{code:java}
while (!streamerClosed && dataQueue.size() + ackQueue.size() >
              dfsClient.getConf().getWriteMaxPackets()) {
            if (firstWait) {
              Span span = Tracer.getCurrentSpan();
              if (span != null) {
                span.addTimelineAnnotation("dataQueue.wait");
              }
              firstWait = false;
            }
            try {
              dataQueue.wait();
            } catch (InterruptedException e) {
              // If we get interrupted while waiting to queue data, we still 
need to get rid
              // of the current packet. This is because we have an invariant 
that if
              // currentPacket gets full, it will get queued before the next 
writeChunk.
              //
              // Rather than wait around for space in the queue, we should 
instead try to
              // return to the caller as soon as possible, even though we 
slightly overrun
              // the MAX_PACKETS length.
              Thread.currentThread().interrupt();  
              break;
            }
          }
        } finally {
          Span span = Tracer.getCurrentSpan();
          if ((span != null) && (!firstWait)) {
            span.addTimelineAnnotation("end.wait");
          }
        }
{code}

 I know NodeManager may stop writing when interruped, but DFSOutputStream also 
could do something to avoid fullgc

!mat.jpg|memory_analysis!


  was:
In our cluster, when found NodeManager frequently FullGC when decommissioning 
NodeManager, and we found the biggest object is dataQueue of DataStreamer, it 
has almost 6w DFSPacket, and every DFSPacket is about 64k, as shown below.
The root reason is that the size of dataQueue and ackQueue in DataStreamer has 
no limit when writer thread is interrupted. I know NodeManager may stop writing 
when interruped, but DFSOutputStream also could do something to avoid fullgc

!mat.jpg|memory_analysis!



> The size of dataQueue and ackQueue in DataStreamer has no limit when writer 
> thread is interrupted
> -------------------------------------------------------------------------------------------------
>
>                 Key: YARN-7168
>                 URL: https://issues.apache.org/jira/browse/YARN-7168
>             Project: Hadoop YARN
>          Issue Type: Bug
>          Components: client
>            Reporter: Jiandan Yang 
>         Attachments: mat.jpg
>
>
> In our cluster, when found NodeManager frequently FullGC when decommissioning 
> NodeManager, and we found the biggest object is dataQueue of DataStreamer, it 
> has almost 6w DFSPacket, and every DFSPacket is about 64k, as shown below.
> The root reason is that the size of dataQueue and ackQueue in DataStreamer 
> has no limit when writer thread is interrupted.
> DFSOutputStream#waitAndQueuePacket does not wait when writer thread is 
> interrupted.
> {code:java}
> while (!streamerClosed && dataQueue.size() + ackQueue.size() >
>               dfsClient.getConf().getWriteMaxPackets()) {
>             if (firstWait) {
>               Span span = Tracer.getCurrentSpan();
>               if (span != null) {
>                 span.addTimelineAnnotation("dataQueue.wait");
>               }
>               firstWait = false;
>             }
>             try {
>               dataQueue.wait();
>             } catch (InterruptedException e) {
>               // If we get interrupted while waiting to queue data, we still 
> need to get rid
>               // of the current packet. This is because we have an invariant 
> that if
>               // currentPacket gets full, it will get queued before the next 
> writeChunk.
>               //
>               // Rather than wait around for space in the queue, we should 
> instead try to
>               // return to the caller as soon as possible, even though we 
> slightly overrun
>               // the MAX_PACKETS length.
>               Thread.currentThread().interrupt();  
>               break;
>             }
>           }
>         } finally {
>           Span span = Tracer.getCurrentSpan();
>           if ((span != null) && (!firstWait)) {
>             span.addTimelineAnnotation("end.wait");
>           }
>         }
> {code}
>  I know NodeManager may stop writing when interruped, but DFSOutputStream 
> also could do something to avoid fullgc
> !mat.jpg|memory_analysis!



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

---------------------------------------------------------------------
To unsubscribe, e-mail: yarn-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: yarn-issues-h...@hadoop.apache.org

Reply via email to