You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by nk...@apache.org on 2013/06/28 17:05:23 UTC
svn commit: r1497801 - in /hbase/branches/0.95:
hbase-client/src/main/java/org/apache/hadoop/hbase/client/
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/
hbase-server/src/test/resources/
Author: nkeywal
Date: Fri Jun 28 15:05:22 2013
New Revision: 1497801
URL: http://svn.apache.org/r1497801
Log:
HBASE-6295 Possible performance improvement in client batch operations: presplit and send in background - addendum
Modified:
hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java
hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableMultiplexer.java
hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
hbase/branches/0.95/hbase-server/src/test/resources/hbase-site.xml
Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java?rev=1497801&r1=1497800&r2=1497801&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java (original)
+++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java Fri Jun 28 15:05:22 2013
@@ -410,8 +410,8 @@ class AsyncProcess<CResult> {
try {
res = callable.withoutRetries();
} catch (IOException e) {
- LOG.warn("The call to the RS failed, we don't know where we stand. regionName="
- + regionName, e);
+ LOG.warn("The call to the RS failed, we don't know where we stand. location="
+ + loc, e);
resubmitAll(initialActions, multi, loc, numAttempt + 1, e, errorsByServer);
return;
}
@@ -430,7 +430,7 @@ class AsyncProcess<CResult> {
// this a little.
decTaskCounters(regionName);
LOG.warn("The task was rejected by the pool. This is unexpected. " +
- "regionName=" + regionName, ree);
+ "location=" + loc, ree);
// We're likely to fail again, but this will increment the attempt counter, so it will
// finish.
resubmitAll(initialActions, multi, loc, numAttempt + 1, ree, errorsByServer);
@@ -515,7 +515,13 @@ class AsyncProcess<CResult> {
}
}
- submit(initialActions, toReplay, numAttempt, true, errorsByServer);
+ if (toReplay.isEmpty()) {
+ LOG.warn("Attempt #" + numAttempt + "/" + numTries + " failed for all (" +
+ initialActions.size() + ") operations on server " + location.getServerName() +
+ " NOT resubmitting, tableName=" + Bytes.toString(tableName) + ", location=" + location);
+ } else {
+ submit(initialActions, toReplay, numAttempt, true, errorsByServer);
+ }
}
/**
@@ -533,8 +539,9 @@ class AsyncProcess<CResult> {
HConnectionManager.ServerErrorTracker errorsByServer) {
if (responses == null) {
- LOG.info("Attempt #" + numAttempt + " failed for all operations on server " +
- location.getServerName() + " , trying to resubmit.");
+ LOG.info("Attempt #" + numAttempt + "/" + numTries + " failed for all operations" +
+ " on server " + location.getServerName() + " , trying to resubmit," +
+ " tableName=" + Bytes.toString(tableName) + ", location=" + location);
resubmitAll(initialActions, rsActions, location, numAttempt + 1, null, errorsByServer);
return;
}
@@ -587,31 +594,32 @@ class AsyncProcess<CResult> {
}
if (!toReplay.isEmpty()) {
- if (numAttempt > 2) {
+ long backOffTime = (errorsByServer != null ?
+ errorsByServer.calculateBackoffTime(location, pause) :
+ ConnectionUtils.getPauseTime(pause, numAttempt));
+ if (numAttempt > 3 && LOG.isDebugEnabled()) {
// We use this value to have some logs when we have multiple failures, but not too many
// logs as errors are to be expected wehn region moves, split and so on
- LOG.debug("Attempt #" + numAttempt + " failed for " + failureCount +
+ LOG.debug("Attempt #" + numAttempt + "/" + numTries + " failed for " + failureCount +
" operations on server " + location.getServerName() + ", resubmitting " +
- toReplay.size() + ", tableName=" + Bytes.toString(tableName) +
- ", last exception was: " + throwable);
+ toReplay.size() + ", tableName=" + Bytes.toString(tableName) + ", location=" +
+ location + ", last exception was: " + throwable +
+ " - sleeping " + backOffTime + " ms.");
}
- long backOffTime = (errorsByServer != null ?
- errorsByServer.calculateBackoffTime(location, pause) :
- ConnectionUtils.getPauseTime(pause, numAttempt));
try {
Thread.sleep(backOffTime);
} catch (InterruptedException e) {
LOG.warn("Not sent: " + toReplay.size() +
- " operations, tableName=" + Bytes.toString(tableName), e);
+ " operations, tableName=" + Bytes.toString(tableName) + ", location=" + location, e);
Thread.interrupted();
return;
}
submit(initialActions, toReplay, numAttempt + 1, true, errorsByServer);
} else if (failureCount != 0) {
- LOG.warn("Attempt #" + numAttempt + " failed for " + failureCount +
+ LOG.warn("Attempt #" + numAttempt + "/" + numTries + " failed for " + failureCount +
" operations on server " + location.getServerName() + " NOT resubmitting." +
- ", tableName=" + Bytes.toString(tableName));
+ ", tableName=" + Bytes.toString(tableName) + ", location=" + location);
}
}
@@ -637,16 +645,16 @@ class AsyncProcess<CResult> {
* Wait until the async does not have more than max tasks in progress.
*/
private long waitForMaximumCurrentTasks(int max) throws InterruptedIOException {
- long lastLog = 0;
+ long lastLog = EnvironmentEdgeManager.currentTimeMillis();
long currentTasksDone = this.tasksDone.get();
while ((tasksSent.get() - currentTasksDone) > max) {
long now = EnvironmentEdgeManager.currentTimeMillis();
- if (now > lastLog + 5000) {
+ if (now > lastLog + 10000) {
lastLog = now;
- LOG.info(Bytes.toString(tableName) +
- ": Waiting for the global number of tasks to be equals or less than " + max +
- ", currently it's " + this.tasksDone.get());
+ LOG.info(": Waiting for the global number of running tasks to be equals or less than "
+ + max + ", tasksSent=" + tasksSent.get() + ", tasksDone=" + tasksDone.get() +
+ ", currentTasksDone=" + currentTasksDone + ", tableName=" + Bytes.toString(tableName));
}
waitForNextTaskDone(currentTasksDone);
currentTasksDone = this.tasksDone.get();
Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java?rev=1497801&r1=1497800&r2=1497801&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java (original)
+++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java Fri Jun 28 15:05:22 2013
@@ -181,7 +181,8 @@ public class HBaseAdmin implements Abort
this.connection = connection;
this.pause = this.conf.getLong("hbase.client.pause", 1000);
- this.numRetries = this.conf.getInt("hbase.client.retries.number", 10);
+ this.numRetries = this.conf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
+ HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER);
this.retryLongerMultiplier = this.conf.getInt(
"hbase.client.retries.longer.multiplier", 10);
}
Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java?rev=1497801&r1=1497800&r2=1497801&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java (original)
+++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java Fri Jun 28 15:05:22 2013
@@ -806,6 +806,10 @@ public class HTable implements HTableInt
}
if (synchronous || ap.hasError()) {
+ if (ap.hasError() && LOG.isDebugEnabled()) {
+ LOG.debug(Bytes.toString(tableName) + ": One or more of the operations have failed -" +
+ " waiting for all operation in progress to finish (successfully or not)");
+ }
ap.waitUntilDone();
}
Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableMultiplexer.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableMultiplexer.java?rev=1497801&r1=1497800&r2=1497801&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableMultiplexer.java (original)
+++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableMultiplexer.java Fri Jun 28 15:05:22 2013
@@ -24,6 +24,7 @@ import org.apache.commons.logging.LogFac
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.exceptions.ZooKeeperConnectionException;
@@ -92,7 +93,8 @@ public class HTableMultiplexer {
LinkedBlockingQueue<PutStatus>>();
this.serverToFlushWorkerMap = new ConcurrentHashMap<HRegionLocation, HTableFlushWorker>();
this.tableNameToHTableMap = new ConcurrentSkipListMap<byte[], HTable>(Bytes.BYTES_COMPARATOR);
- this.retryNum = conf.getInt("hbase.client.retries.number", 10);
+ this.retryNum = this.conf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
+ HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER);
this.perRegionServerBufferQueueSize = perRegionServerBufferQueueSize;
}
Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java?rev=1497801&r1=1497800&r2=1497801&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java Fri Jun 28 15:05:22 2013
@@ -505,7 +505,8 @@ public class HRegionServer implements Cl
this.useHBaseChecksum = conf.getBoolean(HConstants.HBASE_CHECKSUM_VERIFICATION, false);
// Config'ed params
- this.numRetries = conf.getInt("hbase.client.retries.number", 10);
+ this.numRetries = this.conf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
+ HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER);
this.threadWakeFrequency = conf.getInt(HConstants.THREAD_WAKE_FREQUENCY, 10 * 1000);
this.msgInterval = conf.getInt("hbase.regionserver.msginterval", 3 * 1000);
Modified: hbase/branches/0.95/hbase-server/src/test/resources/hbase-site.xml
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/resources/hbase-site.xml?rev=1497801&r1=1497800&r2=1497801&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/resources/hbase-site.xml (original)
+++ hbase/branches/0.95/hbase-server/src/test/resources/hbase-site.xml Fri Jun 28 15:05:22 2013
@@ -41,11 +41,11 @@
</property>
<property>
<name>hbase.client.retries.number</name>
- <value>10</value>
+ <value>20</value>
<description>Maximum retries. Used as maximum for all retryable
operations such as fetching of the root region from root region
server, getting a cell's value, starting a row update, etc.
- Default: 10.
+ Default: 20.
</description>
</property>
<property>