You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@druid.apache.org by GitBox <gi...@apache.org> on 2019/02/13 20:15:41 UTC

[GitHub] samarthjain opened a new issue #7068: Creating zk nodes for segments to load is effectively serial

samarthjain opened a new issue #7068: Creating zk nodes for segments to load is effectively serial
URL: https://github.com/apache/incubator-druid/issues/7068
 
 
   I have been investigating why clusters have been taking a long time to download segments. After adding a few log lines and closely inspecting the code, it looks like that a zookeeper node for a segment is not created till the processing (download and memory mapping) of previous segment is complete.
   
   See the below code blocks in CuratorLoadQueuePeon
   
   In processSegmentChangeRequest(), we have this piece of block that prevents processing of new segment change requests if there is one in progress. 
   ```
   if (currentlyProcessing != null) {
        log.debug(
             "Server[%s] skipping processSegmentChangeRequest because something is currently loading[%s].",
             basePath,
             currentlyProcessing.getSegmentId()
         );
         return;
    }
   ```
   
   In non-failure conditions, `currentlyProcessing` is set to `null` only after the zk node for the segment has been deleted
   
   ```
   case NodeDeleted:
                   log.info("Watcher notified of %s deletion", watchedEvent.getPath());
                   entryRemoved(watchedEvent.getPath());
                   break;
   ```
   
   `entryRemoved()` calls `actionCompleted()` which is where we end up setting `currentlyProcessing` to null
   
   ```
   final List<LoadPeonCallback> callbacks = currentlyProcessing.getCallbacks();
         currentlyProcessing = null;
   ```
   
   Code in ZkCoordinator which does the node deletion in a callback after the download segment request is complete:
   
   ```
    request.go(
                             dataSegmentChangeHandler,
                             new DataSegmentChangeCallback()
                             {
                               boolean hasRun = false;
   
                               @Override
                               public void execute()
                               {
                                 try {
                                   if (!hasRun) {
   >>>>>>>                       curator.delete().guaranteed().forPath(path);
                                     hasRun = true;
                                   }
                                 }
                                 catch (Exception e) {
                                   try {
                                     curator.delete().guaranteed().forPath(path);
                                   }
                                   catch (Exception e1) {
                                     log.error(e1, "Failed to delete zNode[%s], but ignoring exception.", path);
                                   }
                                   log.error(e, "Exception while removing zNode[%s]", path);
                                   throw Throwables.propagate(e);
                                 }
                               }
                             }
                         
   ```
   
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org