You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2013/05/30 19:31:16 UTC

svn commit: r1487945 - in /hbase/trunk: hbase-client/src/main/java/org/apache/hadoop/hbase/client/ hbase-common/src/main/java/org/apache/hadoop/hbase/ hbase-it/src/test/java/org/apache/hadoop/hbase/ hbase-it/src/test/java/org/apache/hadoop/hbase/util/ ...

Author: stack
Date: Thu May 30 17:31:15 2013
New Revision: 1487945

URL: http://svn.apache.org/r1487945
Log:
HBASE-8657 Miscellaneous log fixups for hbase-it; tidier logging, fix a few NPEs

Modified:
    hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientScanner.java
    hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java
    hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MetaScanner.java
    hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RetriesExhaustedWithDetailsException.java
    hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ServerCallable.java
    hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
    hbase/trunk/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestDataIngestSlowDeterministic.java
    hbase/trunk/hbase-it/src/test/java/org/apache/hadoop/hbase/util/ChaosMonkey.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedWriter.java

Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientScanner.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientScanner.java?rev=1487945&r1=1487944&r2=1487945&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientScanner.java (original)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientScanner.java Thu May 30 17:31:15 2013
@@ -93,9 +93,8 @@ public class ClientScanner extends Abstr
     public ClientScanner(final Configuration conf, final Scan scan,
       final byte[] tableName, HConnection connection) throws IOException {
       if (LOG.isDebugEnabled()) {
-        LOG.debug("Creating scanner over "
-            + Bytes.toString(tableName)
-            + " starting at key '" + Bytes.toStringBinary(scan.getStartRow()) + "'");
+        LOG.debug("Scan table=" + Bytes.toString(tableName)
+            + ", startRow=" + Bytes.toStringBinary(scan.getStartRow()));
       }
       this.scan = scan;
       this.tableName = tableName;
@@ -192,7 +191,7 @@ public class ClientScanner extends Abstr
             done) {
           close();
           if (LOG.isDebugEnabled()) {
-            LOG.debug("Finished scanning region " + this.currentRegion);
+            LOG.debug("Finished region=" + this.currentRegion);
           }
           return false;
         }

Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java?rev=1487945&r1=1487944&r2=1487945&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java (original)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java Thu May 30 17:31:15 2013
@@ -1188,14 +1188,18 @@ public class HConnectionManager {
         }
       }
       if (isNewCacheEntry) {
-        LOG.debug("Cached location for " +
+        if (LOG.isTraceEnabled()) {
+          LOG.trace("Cached location for " +
             location.getRegionInfo().getRegionNameAsString() +
             " is " + location.getHostnamePort());
+        }
       } else if (isStaleUpdate && !location.equals(oldLocation)) {
-        LOG.debug("Ignoring stale location update for "
-          + location.getRegionInfo().getRegionNameAsString() + ": "
-          + location.getHostnamePort() + " at " + location.getSeqNum() + "; local "
-          + oldLocation.getHostnamePort() + " at " + oldLocation.getSeqNum());
+        if (LOG.isTraceEnabled()) {
+          LOG.trace("Ignoring stale location update for "
+            + location.getRegionInfo().getRegionNameAsString() + ": "
+            + location.getHostnamePort() + " at " + location.getSeqNum() + "; local "
+            + oldLocation.getHostnamePort() + " at " + oldLocation.getSeqNum());
+        }
       }
     }
 
@@ -1388,7 +1392,7 @@ public class HConnectionManager {
                 // tries at this point is 1 or more; decrement to start from 0.
                 long pauseTime = ConnectionUtils.getPauseTime(pause, tries - 1);
                 LOG.info("getMaster attempt " + tries + " of " + numTries +
-                    " failed; retrying after sleep of " +pauseTime + ", exception=" +
+                    " failed; retrying after sleep of " + pauseTime + ", exception=" +
                   exceptionCaught);
 
                 try {
@@ -2217,10 +2221,11 @@ public class HConnectionManager {
           if (LOG.isTraceEnabled() && isRetry) {
             StringBuilder sb = new StringBuilder();
             for (Action<R> action : e.getValue().allActions()) {
-              sb.append(Bytes.toStringBinary(action.getAction().getRow())).append(';');
+              if (sb.length() > 0) sb.append(' ');
+              sb.append(Bytes.toStringBinary(action.getAction().getRow()));
             }
-            LOG.trace("Will retry requests to [" + e.getKey().getHostnamePort()
-              + "] after delay of [" + backoffTime + "] for rows [" + sb.toString() + "]");
+            LOG.trace("Attempt #" + this.curNumRetries + " against " + e.getKey().getHostnamePort()
+              + " after=" + backoffTime + "ms, row(s)=" + sb.toString());
           }
           Triple<MultiAction<R>, HRegionLocation, Future<MultiResponse>> p =
             new Triple<MultiAction<R>, HRegionLocation, Future<MultiResponse>>(
@@ -2280,11 +2285,10 @@ public class HConnectionManager {
         //  we had more than numRetries for any action
         //  In this case, we will finish the current retries but we won't start new ones.
         boolean lastRetry = false;
-        // despite its name numRetries means number of tries. So if numRetries == 1 it means we
-        //  won't retry. And we compare vs. 2 in case someone set it to zero.
+        // If hci.numTries is 1 or 0, we do not retry.
         boolean noRetry = (hci.numTries < 2);
 
-        // Analyze and resubmit until all actions are done successfully or failed after numRetries
+        // Analyze and resubmit until all actions are done successfully or failed after numTries
         while (!this.inProgress.isEmpty()) {
           // We need the original multi action to find out what actions to replay if
           //  we have a 'total' failure of the Future<MultiResponse>
@@ -2355,8 +2359,9 @@ public class HConnectionManager {
           // Retry all actions in toReplay then clear it.
           if (!noRetry && !toReplay.isEmpty()) {
             if (isTraceEnabled) {
-              LOG.trace("Retrying due to errors" + (lastRetry ? " (one last time)" : "")
-                   + ": " + retriedErrors.getDescriptionAndClear());
+              LOG.trace("Retrying #" + this.curNumRetries +
+                (lastRetry ? " (one last time)": "") + " because " +
+                retriedErrors.getDescriptionAndClear());
             }
             doRetry();
             if (lastRetry) {

Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MetaScanner.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MetaScanner.java?rev=1487945&r1=1487944&r2=1487945&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MetaScanner.java (original)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MetaScanner.java Thu May 30 17:31:15 2013
@@ -156,8 +156,8 @@ public class MetaScanner {
       int rows = Math.min(rowLimit, configuration.getInt(HConstants.HBASE_META_SCANNER_CACHING,
         HConstants.DEFAULT_HBASE_META_SCANNER_CACHING));
       scan.setCaching(rows);
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Scanning " + Bytes.toString(metaTableName) + " starting at row=" +
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("Scanning " + Bytes.toString(metaTableName) + " starting at row=" +
           Bytes.toStringBinary(startRow) + " for max=" + rowUpperLimit + " with caching=" + rows);
       }
       // Run the scan

Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RetriesExhaustedWithDetailsException.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RetriesExhaustedWithDetailsException.java?rev=1487945&r1=1487944&r2=1487945&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RetriesExhaustedWithDetailsException.java (original)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RetriesExhaustedWithDetailsException.java Thu May 30 17:31:15 2013
@@ -126,13 +126,14 @@ extends RetriesExhaustedException {
       Throwable t = this.exceptions.get(i);
       Row action = this.actions.get(i);
       String server = this.hostnameAndPort.get(i);
-      pw.append("Error");
+      pw.append("exception");
       if (this.exceptions.size() > 1) {
         pw.append(" #" + i);
       }
-      pw.append(" from [" + server + "] for ["
-        + ((action == null) ? "unknown key" : Bytes.toStringBinary(action.getRow())) + "]");
+      pw.append(" from " + server + " for "
+        + ((action == null) ? "unknown key" : Bytes.toStringBinary(action.getRow())));
       if (t != null) {
+        pw.println();
         t.printStackTrace(pw);
       }
     }

Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ServerCallable.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ServerCallable.java?rev=1487945&r1=1487944&r2=1487945&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ServerCallable.java (original)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ServerCallable.java Thu May 30 17:31:15 2013
@@ -94,6 +94,10 @@ public abstract class ServerCallable<T> 
    */
   public void prepare(final boolean reload) throws IOException {
     this.location = connection.getRegionLocation(tableName, row, reload);
+    if (this.location == null) {
+      throw new IOException("Failed to find location, tableName=" + tableName + ", row=" +
+        Bytes.toString(row) + ", reload=" + reload);
+    }
     this.stub = connection.getClient(location.getServerName());
   }
 
@@ -169,7 +173,7 @@ public abstract class ServerCallable<T> 
         prepare(tries != 0); // if called with false, check table status on ZK
         return call();
       } catch (Throwable t) {
-        LOG.warn("Call exception, tries=" + tries + ", numRetries=" + numRetries + ": " + t);
+        LOG.warn("Call exception, tries=" + tries + ", numRetries=" + numRetries, t);
 
         t = translateException(t);
         // translateException throws an exception when we should not retry, i.e. when it's the

Modified: hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java?rev=1487945&r1=1487944&r2=1487945&view=diff
==============================================================================
--- hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java (original)
+++ hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java Thu May 30 17:31:15 2013
@@ -574,7 +574,7 @@ public final class HConstants {
   /**
    * Default value of {@link #HBASE_CLIENT_RETRIES_NUMBER}.
    */
-  public static int DEFAULT_HBASE_CLIENT_RETRIES_NUMBER = 10;
+  public static int DEFAULT_HBASE_CLIENT_RETRIES_NUMBER = 20;
 
   /**
    * Parameter name for client prefetch limit, used as the maximum number of regions

Modified: hbase/trunk/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestDataIngestSlowDeterministic.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestDataIngestSlowDeterministic.java?rev=1487945&r1=1487944&r2=1487945&view=diff
==============================================================================
--- hbase/trunk/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestDataIngestSlowDeterministic.java (original)
+++ hbase/trunk/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestDataIngestSlowDeterministic.java Thu May 30 17:31:15 2013
@@ -31,13 +31,13 @@ import org.junit.experimental.categories
 /**
  * A system test which does large data ingestion and verify using {@link LoadTestTool}.
  * It performs a set of actions deterministically using ChaosMonkey, then starts killing
- * things randomly. You can configure how long should the load test run by using 
+ * things randomly. You can configure how long should the load test run by using
  * "hbase.IntegrationTestDataIngestSlowDeterministic.runtime" configuration parameter.
  */
 @Category(IntegrationTests.class)
 public class IntegrationTestDataIngestSlowDeterministic extends IngestIntegrationTestBase {
   private static final int SERVER_COUNT = 4; // number of slaves for the smallest cluster
-  private static final long DEFAULT_RUN_TIME = 30 * 60 * 1000;
+  private static final long DEFAULT_RUN_TIME = 10 * 60 * 1000;
   private static final long CHAOS_EVERY_MS = 150 * 1000; // Chaos every 2.5 minutes.
 
   private ChaosMonkey monkey;

Modified: hbase/trunk/hbase-it/src/test/java/org/apache/hadoop/hbase/util/ChaosMonkey.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-it/src/test/java/org/apache/hadoop/hbase/util/ChaosMonkey.java?rev=1487945&r1=1487944&r2=1487945&view=diff
==============================================================================
--- hbase/trunk/hbase-it/src/test/java/org/apache/hadoop/hbase/util/ChaosMonkey.java (original)
+++ hbase/trunk/hbase-it/src/test/java/org/apache/hadoop/hbase/util/ChaosMonkey.java Thu May 30 17:31:15 2013
@@ -22,13 +22,11 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
-import java.util.HashSet;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.Queue;
 import java.util.Random;
-import java.util.Set;
 
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.logging.Log;
@@ -38,8 +36,8 @@ import org.apache.hadoop.hbase.ClusterSt
 import org.apache.hadoop.hbase.HBaseCluster;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.IntegrationTestingUtility;
 import org.apache.hadoop.hbase.IntegrationTestDataIngestWithChaosMonkey;
+import org.apache.hadoop.hbase.IntegrationTestingUtility;
 import org.apache.hadoop.hbase.ServerLoad;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.Stoppable;
@@ -49,7 +47,6 @@ import org.apache.hadoop.util.ToolRunner
 
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
-import com.google.protobuf.ServiceException;
 
 /**
  * A utility to injects faults in a running cluster.
@@ -158,6 +155,7 @@ public class ChaosMonkey extends Abstrac
     /** Returns current region servers */
     protected ServerName[] getCurrentServers() throws IOException {
       Collection<ServerName> regionServers = cluster.getClusterStatus().getServers();
+      if (regionServers == null || regionServers.size() <= 0) return new ServerName [] {};
       return regionServers.toArray(new ServerName[regionServers.size()]);
     }
 

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedWriter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedWriter.java?rev=1487945&r1=1487944&r2=1487945&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedWriter.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedWriter.java Thu May 30 17:31:15 2013
@@ -173,8 +173,8 @@ public class MultiThreadedWriter extends
   }
 
   public void insert(HTable table, Put put, long keyBase) {
+    long start = System.currentTimeMillis();
     try {
-      long start = System.currentTimeMillis();
       table.put(put);
       totalOpTimeMs.addAndGet(System.currentTimeMillis() - start);
     } catch (IOException e) {
@@ -190,8 +190,8 @@ public class MultiThreadedWriter extends
         pw.flush();
         exceptionInfo = StringUtils.stringifyException(e);
       }
-      LOG.error("Failed to insert: " + keyBase + "; region information: "
-          + getRegionDebugInfoSafe(table, put.getRow()) + "; errors: "
+      LOG.error("Failed to insert: " + keyBase + " after " + (System.currentTimeMillis() - start) +
+        "ms; region information: " + getRegionDebugInfoSafe(table, put.getRow()) + "; errors: "
           + exceptionInfo);
     }
   }