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

Appy commented on HBASE-19290:
------------------------------

{quote}
bq. Since expectedTasksPerRS only relay on availableRSs and numTasks, I think 
it is better to move them to getAvailableRSs() and change a name?
I do not think so, and i think getAvailableRSs and calculateAvailableSplitters 
is clear.
{quote}

Yeah, they are definitely clear, but [~tianjingyun]'s suggestion to improve the 
code further was appropriate and made sense.
Here's the diff on what he was suggesting (and what i was thinking earlier).
{noformat}
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZkSplitLogWorkerCoordination.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZkSplitLogWorkerCoordination.java
index e7d15bbc16..e7fd93bf46 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZkSplitLogWorkerCoordination.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZkSplitLogWorkerCoordination.java
@@ -318,7 +318,14 @@ public class ZkSplitLogWorkerCoordination extends 
ZKListener implements
     server.getExecutorService().submit(hsh);
   }
 
-  private int getAvailableRSs() {
+  /**
+   * This function calculates how many splitters this RS should create based 
on expected average
+   * tasks per RS and the hard limit upper bound(maxConcurrentTasks) set by 
configuration. <br>
+   * At any given time, a RS allows spawn MIN(Expected Tasks/RS, Hard Upper 
Bound)
+   * @param numTasks total number of split tasks available
+   * @return number of tasks this RS can grab
+   */
+  private int getNumExpectedTasksPerRS(int numTasks) {
     // at lease one RS(itself) available
     int availableRSs = 1;
     try {
@@ -329,22 +336,17 @@ public class ZkSplitLogWorkerCoordination extends 
ZKListener implements
       // do nothing
       LOG.debug("getAvailableRegionServers got ZooKeeper exception", e);
     }
-    return availableRSs;
+    int expectedTasksPerRS = (numTasks / availableRSs) + ((numTasks % 
availableRSs == 0) ? 0 : 1);
+    return Math.max(1, expectedTasksPerRS); // at least be one
   }
 
   /**
-   * This function calculates how many splitters it could create based on 
expected average tasks per
-   * RS and the hard limit upper bound(maxConcurrentTasks) set by 
configuration. <br>
-   * At any given time, a RS allows spawn MIN(Expected Tasks/RS, Hard Upper 
Bound)
-   * @param availableRSs current total number of available regionservers
-   * @param numTasks current total number of available tasks
+   * @param expectedTasksPerRS Average number of tasks to be handled by each RS
+   * @return true if more splitters are available, otherwise false.
    */
-  private int calculateAvailableSplitters(int availableRSs, int numTasks) {
-    int expectedTasksPerRS = (numTasks / availableRSs) + ((numTasks % 
availableRSs == 0) ? 0 : 1);
-    expectedTasksPerRS = Math.max(1, expectedTasksPerRS); // at least be one
-    // calculate how many more splitters we could spawn
-    return Math.min(expectedTasksPerRS, maxConcurrentTasks)
-        - this.tasksInProgress.get();
+  private boolean areSplittersAvailable(int expectedTasksPerRS) {
+    return (Math.min(expectedTasksPerRS, maxConcurrentTasks)
+        - this.tasksInProgress.get()) > 0;
   }
 
   /**
@@ -425,11 +427,11 @@ public class ZkSplitLogWorkerCoordination extends 
ZKListener implements
         }
       }
       int numTasks = paths.size();
-      int availableRSs = getAvailableRSs();
+      int expectedTasksPerRS = getNumExpectedTasksPerRS(numTasks);
       int taskGrabbed = 0;
       for (int i = 0; i < numTasks; i++) {
         while (!shouldStop) {
-          if (this.calculateAvailableSplitters(availableRSs, numTasks) > 0) {
+          if (this.areSplittersAvailable(expectedTasksPerRS)) {
             LOG.debug("Current region server " + server.getServerName()
                 + " is ready to take more tasks, will get task list and try 
grab tasks again.");
             int idx = (i + offset) % paths.size();
{noformat}

> Reduce zk request when doing split log
> --------------------------------------
>
>                 Key: HBASE-19290
>                 URL: https://issues.apache.org/jira/browse/HBASE-19290
>             Project: HBase
>          Issue Type: Improvement
>            Reporter: binlijin
>            Assignee: binlijin
>         Attachments: HBASE-19290.master.001.patch, 
> HBASE-19290.master.002.patch, HBASE-19290.master.003.patch
>
>
> We observe once the cluster has 1000+ nodes and when hundreds of nodes abort 
> and doing split log, the split is very very slow, and we find the 
> regionserver and master wait on the zookeeper response, so we need to reduce 
> zookeeper request and pressure for big cluster.
> (1) Reduce request to rsZNode, every time calculateAvailableSplitters will 
> get rsZNode's children from zookeeper, when cluster is huge, this is heavy. 
> This patch reduce the request. 
> (2) When the regionserver has max split tasks running, it may still trying to 
> grab task and issue zookeeper request, we should sleep and wait until we can 
> grab tasks again.  



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

Reply via email to