diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java (revision 158123) +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java (revision 158276) @@ -309,6 +309,11 @@ final long busyWaitDuration; static final long DEFAULT_BUSY_WAIT_DURATION = HConstants.DEFAULT_HBASE_RPC_TIMEOUT; + // bulkload acquires the write lock, and waiting a long time for write lock can block other + // read/update operations, so we need another param to tune the wait duration of bulkload, and it + // can be configured by table + final long bulkloadWaitDuration; + // If updating multiple rows in one call, wait longer, // i.e. waiting for busyWaitDuration * # of rows. However, // we can limit the max multiplier. @@ -617,6 +622,7 @@ this.busyWaitDuration = conf.getLong( "hbase.busy.wait.duration", DEFAULT_BUSY_WAIT_DURATION); + this.bulkloadWaitDuration = conf.getLong("hbase.bulkload.wait.duration", -1); this.maxBusyWaitMultiplier = conf.getInt("hbase.busy.wait.multiplier.max", 2); if (busyWaitDuration * maxBusyWaitMultiplier <= 0L) { throw new IllegalArgumentException("Invalid hbase.busy.wait.duration (" @@ -3816,7 +3822,7 @@ LOG.debug("bulkload: waiting for write lock"); } long startTime = EnvironmentEdgeManager.currentTimeMillis(); - startBulkRegionOperation(hasMultipleColumnFamilies(familyPaths)); + startBulkRegionOperation(hasMultipleColumnFamilies(familyPaths), true); long lockTime = EnvironmentEdgeManager.currentTimeMillis(); if (LOG.isDebugEnabled()) { LOG.debug("bulkload: get write lock after " + (lockTime - startTime) + " ms"); @@ -6243,11 +6249,22 @@ */ private void startBulkRegionOperation(boolean writeLockNeeded) throws NotServingRegionException, RegionTooBusyException, InterruptedIOException { + startBulkRegionOperation(writeLockNeeded, false); + } + + /** + * Bulkload use this api to tune the wait duration + */ + private void startBulkRegionOperation(boolean writeLockNeeded, boolean isBulkload) + throws NotServingRegionException, RegionTooBusyException, InterruptedIOException { if (this.closing.get()) { throw new NotServingRegionException(getRegionNameAsString() + " is closing"); } - if (writeLockNeeded) lock(lock.writeLock()); - else lock(lock.readLock()); + if (writeLockNeeded) { + lock(lock.writeLock(), isBulkload); + } else { + lock(lock.readLock()); + } if (this.closed.get()) { if (writeLockNeeded) lock.writeLock().unlock(); else lock.readLock().unlock(); @@ -6291,19 +6308,33 @@ private void lock(final Lock lock) throws RegionTooBusyException, InterruptedIOException { - lock(lock, 1); + lock(lock, false); } + private void lock(final Lock lock, final boolean isBulkload) throws RegionTooBusyException, + InterruptedIOException { + lock(lock, 1, isBulkload); + } + + private void lock(final Lock lock, final int multiplier) throws RegionTooBusyException, + InterruptedIOException { + lock(lock, multiplier, false); + } + /** * Try to acquire a lock. Throw RegionTooBusyException * if failed to get the lock in time. Throw InterruptedIOException * if interrupted while waiting for the lock. */ - private void lock(final Lock lock, final int multiplier) + private void lock(final Lock lock, final int multiplier, final boolean isBulkload) throws RegionTooBusyException, InterruptedIOException { try { - final long waitTime = Math.min(maxBusyWaitDuration, - busyWaitDuration * Math.min(multiplier, maxBusyWaitMultiplier)); + long generalWaitTime = + Math.min(maxBusyWaitDuration, + busyWaitDuration * Math.min(multiplier, maxBusyWaitMultiplier)); + final long waitTime = + (isBulkload && (0 < this.bulkloadWaitDuration)) ? this.bulkloadWaitDuration + : generalWaitTime; if (!lock.tryLock(waitTime, TimeUnit.MILLISECONDS)) { throw new RegionTooBusyException( "failed to get a lock in " + waitTime + " ms. " +