hbase-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From te...@apache.org
Subject hbase git commit: HBASE-13971 Flushes stuck since 6 hours on a regionserver
Date Thu, 16 Jul 2015 23:45:32 GMT
Repository: hbase
Updated Branches:
  refs/heads/branch-1.2 2862d6847 -> 5bb840c83


HBASE-13971 Flushes stuck since 6 hours on a regionserver


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/5bb840c8
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/5bb840c8
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/5bb840c8

Branch: refs/heads/branch-1.2
Commit: 5bb840c8328e4387b30c4baef02e6f3887cad5cc
Parents: 2862d68
Author: tedyu <yuzhihong@gmail.com>
Authored: Thu Jul 16 16:45:24 2015 -0700
Committer: tedyu <yuzhihong@gmail.com>
Committed: Thu Jul 16 16:45:24 2015 -0700

----------------------------------------------------------------------
 .../hadoop/hbase/regionserver/HRegion.java       | 12 ++++++++++--
 .../java/org/apache/hadoop/hbase/wal/WALKey.java | 19 ++++++++++++++++++-
 2 files changed, 28 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/5bb840c8/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
----------------------------------------------------------------------
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
index 8f3c58d..f572d48 100644
--- 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
@@ -201,6 +201,12 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver,
Regi
 
   public static final String LOAD_CFS_ON_DEMAND_CONFIG_KEY =
       "hbase.hregion.scan.loadColumnFamiliesOnDemand";
+  
+  // in milliseconds
+  private static final String MAX_WAIT_FOR_SEQ_ID_KEY =
+      "hbase.hregion.max.wait.for.seq.id";
+
+  private static final int DEFAULT_MAX_WAIT_FOR_SEQ_ID = 60000;
 
   /**
    * This is the global default value for durability. All tables/mutations not
@@ -333,6 +339,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver,
Regi
    */
   private boolean isLoadingCfsOnDemandDefault = false;
 
+  private int maxWaitForSeqId;
   private final AtomicInteger majorInProgress = new AtomicInteger(0);
   private final AtomicInteger minorInProgress = new AtomicInteger(0);
 
@@ -668,6 +675,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver,
Regi
     this.rowLockWaitDuration = conf.getInt("hbase.rowlock.wait.duration",
                     DEFAULT_ROWLOCK_WAIT_DURATION);
 
+    maxWaitForSeqId = conf.getInt(MAX_WAIT_FOR_SEQ_ID_KEY, DEFAULT_MAX_WAIT_FOR_SEQ_ID);
     this.isLoadingCfsOnDemandDefault = conf.getBoolean(LOAD_CFS_ON_DEMAND_CONFIG_KEY, true);
     this.htableDescriptor = htd;
     this.rsServices = rsServices;
@@ -2420,7 +2428,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver,
Regi
   @VisibleForTesting
   protected long getNextSequenceId(final WAL wal) throws IOException {
     WALKey key = this.appendEmptyEdit(wal, null);
-    return key.getSequenceId();
+    return key.getSequenceId(maxWaitForSeqId);
   }
 
   //////////////////////////////////////////////////////////////////////////////
@@ -7351,7 +7359,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver,
Regi
   public static final long FIXED_OVERHEAD = ClassSize.align(
       ClassSize.OBJECT +
       ClassSize.ARRAY +
-      45 * ClassSize.REFERENCE + 2 * Bytes.SIZEOF_INT +
+      45 * ClassSize.REFERENCE + 3 * Bytes.SIZEOF_INT +
       (14 * Bytes.SIZEOF_LONG) +
       5 * Bytes.SIZEOF_BOOLEAN);
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/5bb840c8/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALKey.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALKey.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALKey.java
index e8056e4..69c2aec 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALKey.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALKey.java
@@ -30,6 +30,7 @@ import java.util.NavigableMap;
 import java.util.TreeMap;
 import java.util.UUID;
 import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
 
 
 import org.apache.hadoop.hbase.util.ByteStringer;
@@ -301,8 +302,24 @@ public class WALKey implements SequenceId, Comparable<WALKey> {
    */
   @Override
   public long getSequenceId() throws IOException {
+    return getSequenceId(-1);
+  }
+
+  /**
+   * Wait for sequence number is assigned &amp; return the assigned value
+   * @param maxWaitForSeqId maximum duration, in milliseconds, to wait for seq number to
be assigned
+   * @return long the new assigned sequence number
+   * @throws IOException
+   */
+  public long getSequenceId(int maxWaitForSeqId) throws IOException {
     try {
-      this.seqNumAssignedLatch.await();
+      if (maxWaitForSeqId < 0) {
+        this.seqNumAssignedLatch.await();
+      } else {
+        if (!this.seqNumAssignedLatch.await(maxWaitForSeqId, TimeUnit.MILLISECONDS)) {
+          throw new IOException("Timed out waiting for seq number to be assigned");
+        }
+      }
     } catch (InterruptedException ie) {
       LOG.warn("Thread interrupted waiting for next log sequence number");
       InterruptedIOException iie = new InterruptedIOException();


Mime
View raw message