[ 
https://issues.apache.org/jira/browse/TAJO-991?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14100330#comment-14100330
 ] 

ASF GitHub Bot commented on TAJO-991:
-------------------------------------

Github user jinossy commented on a diff in the pull request:

    https://github.com/apache/tajo/pull/107#discussion_r16338685
  
    --- Diff: 
tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/TajoPullServerService.java
 ---
    @@ -312,6 +377,63 @@ public ChannelPipeline getPipeline() throws Exception {
         }
       }
     
    +
    +  Map<String, ProcessingStatus> processingStatusMap = new 
ConcurrentHashMap<String, ProcessingStatus>();
    +
    +  public void completeFileChunk(FadvisedFileRegion filePart,
    +                                   String requestUri,
    +                                   long startTime) {
    +    ProcessingStatus status = processingStatusMap.get(requestUri);
    +    if (status != null) {
    +      status.decrementRemainFiles(filePart, startTime);
    +    }
    +  }
    +
    +  class ProcessingStatus {
    +    String requestUri;
    +    int numFiles;
    +    AtomicInteger remainFiles;
    +    long startTime;
    +    long makeFileListTime;
    +    long minTime = Long.MAX_VALUE;
    +    long maxTime;
    +    int numSlowFile;
    +
    +    public ProcessingStatus(String requestUri) {
    +      this.requestUri = requestUri;
    +      this.startTime = System.currentTimeMillis();
    +    }
    +
    +    public void setNumFiles(int numFiles) {
    +      this.numFiles = numFiles;
    +      this.remainFiles = new AtomicInteger(numFiles);
    +    }
    +    public void decrementRemainFiles(FadvisedFileRegion filePart, long 
fileStartTime) {
    +      synchronized(remainFiles) {
    --- End diff --
    
    remainFiles is AtomicInteger. please remove the synchronized block. also 
synchronized(processingStatusMap)


> Running PullServer on a dedicated JVM process which separates from worker.
> --------------------------------------------------------------------------
>
>                 Key: TAJO-991
>                 URL: https://issues.apache.org/jira/browse/TAJO-991
>             Project: Tajo
>          Issue Type: Sub-task
>          Components: data shuffle
>            Reporter: Hyoungjun Kim
>            Assignee: Hyoungjun Kim
>
> Currently PullServer runs on TajoWorker. If a TajoWorker starts Garbage 
> Collector, all fetching tasks which run on other worker are paused during 
> Garbage Collection.  Running PullServer on a dedicated JVM process which 
> separates from worker is helpful avoiding stop the all cluster by GC. 
> PullServer uses small memory and doesn't share any information and status 
> with TajoWorker. 



--
This message was sent by Atlassian JIRA
(v6.2#6252)

Reply via email to