[ https://issues.apache.org/jira/browse/HBASE-22686?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16884106#comment-16884106 ]
Andrew Purtell commented on HBASE-22686: ---------------------------------------- Ok, patches forthcoming > ZkSplitLogWorkerCoordination doesn't allow a regionserver to pick up all of > the split work it is capable of > ----------------------------------------------------------------------------------------------------------- > > Key: HBASE-22686 > URL: https://issues.apache.org/jira/browse/HBASE-22686 > Project: HBase > Issue Type: Bug > Reporter: Andrew Purtell > Assignee: Andrew Purtell > Priority: Major > > A region hosted by a crashed regionserver cannot be reassigned until the > crashed regionserver's write-ahead logs have been processed and split into > per-region recovered edits files. Reassignment of a region from a crashed > server will be held up by the distributed split work backlog. Every > regionserver runs a background daemon thread that manages the acquisition and > execution of distributed log split tasks. This thread registers a watcher on > a znode managed by the master. When the master is processing a server > shutdown or crash or cluster restart when it detects the presence of > unprocessed WAL files it will register the WAL files for processing under the > znode. One or more live regionservers will attempt to get an exclusive lock > on an entry. One of them wins, splits the WAL file, deletes the entry, then > will acquire more work or go back to sleep if the worklist is empty. A > regionserver can acquire at most a fixed number of log split tasks determined > by configuration, hbase.regionserver.wal.max.splitters (default 2). If the > number of entries/logs to process exceeds the number of available split > workers in the cluster, perhaps due to the correlated failure of a > significant subset of the fleet, then splitting work will fall behind. > Regions may remain in RIT until the backlog is cleared. > However, the regionserver side coordination logic - > ZkSplitLogWorkerCoordination - only allows a regionserver to grab one task > one at a time. Nearby javadoc says "This policy puts an upper-limit on the > number of simultaneous log splitting that could be happening in a cluster." > That upper limit will be the number of currently live regionservers. I don't > feel I understand exactly why this is necessary or appropriate because a > regionserver can handle more than one task at once and in fact the max number > of concurrent split tasks it can accept is configurable. > {code:java} > /** > * 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 numTasks current total number of available tasks > */ > private int calculateAvailableSplitters(int numTasks) { > // at lease one RS(itself) available > int availableRSs = 1; > try { > List<String> regionServers = > ZKUtil.listChildrenNoWatch(watcher, watcher.rsZNode); > availableRSs = Math.max(availableRSs, (regionServers == null) ? 0 : > regionServers.size()); > } catch (KeeperException e) { > // do nothing > LOG.debug("getAvailableRegionServers got ZooKeeper exception", e); > } > 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(); > {code} > Shouldn't this simply be: > {code:java} > private int calculateAvailableSplitters() { > return maxConcurrentTasks - tasksInProgress.get(); > } > {code} > ? > This is branch-1. -- This message was sent by Atlassian JIRA (v7.6.14#76016)