You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ap...@apache.org on 2014/10/08 21:10:33 UTC

[1/5] git commit: HBASE-12199 Make TestAtomicOperation and TestEncodedSeekers faster

Repository: hbase
Updated Branches:
  refs/heads/0.98 638abb0e2 -> 890916a23
  refs/heads/branch-1 283ec5764 -> ad2064d8a
  refs/heads/master e952e7362 -> e31286bf6


HBASE-12199 Make TestAtomicOperation and TestEncodedSeekers faster

Signed-off-by: Elliott Clark <ec...@apache.org>


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

Branch: refs/heads/0.98
Commit: be64e4892f1b59da96be2c556200bf974d797b60
Parents: 638abb0
Author: manukranthk <ma...@fb.com>
Authored: Tue Oct 7 17:07:19 2014 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Wed Oct 8 11:57:47 2014 -0700

----------------------------------------------------------------------
 .../hadoop/hbase/io/encoding/TestEncodedSeekers.java     |  2 ++
 .../hadoop/hbase/regionserver/TestAtomicOperation.java   | 11 ++++++++++-
 2 files changed, 12 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/be64e489/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestEncodedSeekers.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestEncodedSeekers.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestEncodedSeekers.java
index bc87acd..c17565e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestEncodedSeekers.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestEncodedSeekers.java
@@ -31,6 +31,7 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.MediumTests;
 import org.apache.hadoop.hbase.Tag;
+import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
@@ -142,6 +143,7 @@ public class TestEncodedSeekers {
       byte[] key = LoadTestKVGenerator.md5PrefixedKey(i).getBytes();
       for (int j = 0; j < NUM_COLS_PER_ROW; ++j) {
         Put put = new Put(key);
+        put.setDurability(Durability.ASYNC_WAL);
         byte[] col = Bytes.toBytes(String.valueOf(j));
         byte[] value = dataGenerator.generateRandomSizeValue(key, col);
         if (includeTags) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/be64e489/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java
index b22228c..257579a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java
@@ -70,7 +70,6 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
 
-
 /**
  * Testing of HRegion.incrementColumnValue, HRegion.increment,
  * and HRegion.append
@@ -237,6 +236,7 @@ public class TestAtomicOperation {
           inc.addColumn(fam1, qual1, amount);
           inc.addColumn(fam1, qual2, amount*2);
           inc.addColumn(fam2, qual3, amount*3);
+          inc.setDurability(Durability.ASYNC_WAL);
           region.increment(inc);
 
           // verify: Make sure we only see completed increments
@@ -274,6 +274,7 @@ public class TestAtomicOperation {
               a.add(fam1, qual1, val);
               a.add(fam1, qual2, val);
               a.add(fam2, qual3, val);
+              a.setDurability(Durability.ASYNC_WAL);
               region.append(a);
 
               Get g = new Get(row);
@@ -349,16 +350,20 @@ public class TestAtomicOperation {
               if (op) {
                 Put p = new Put(row, ts);
                 p.add(fam1, qual1, value1);
+                p.setDurability(Durability.ASYNC_WAL);
                 rm.add(p);
                 Delete d = new Delete(row);
                 d.deleteColumns(fam1, qual2, ts);
+                d.setDurability(Durability.ASYNC_WAL);
                 rm.add(d);
               } else {
                 Delete d = new Delete(row);
                 d.deleteColumns(fam1, qual1, ts);
+                d.setDurability(Durability.ASYNC_WAL);
                 rm.add(d);
                 Put p = new Put(row, ts);
                 p.add(fam1, qual2, value2);
+                p.setDurability(Durability.ASYNC_WAL);
                 rm.add(p);
               }
               region.mutateRow(rm);
@@ -438,15 +443,19 @@ public class TestAtomicOperation {
               if (op) {
                 Put p = new Put(row2, ts);
                 p.add(fam1, qual1, value1);
+                p.setDurability(Durability.ASYNC_WAL);
                 mrm.add(p);
                 Delete d = new Delete(row);
                 d.deleteColumns(fam1, qual1, ts);
+                d.setDurability(Durability.ASYNC_WAL);
                 mrm.add(d);
               } else {
                 Delete d = new Delete(row2);
                 d.deleteColumns(fam1, qual1, ts);
+                d.setDurability(Durability.ASYNC_WAL);
                 mrm.add(d);
                 Put p = new Put(row, ts);
+                p.setDurability(Durability.ASYNC_WAL);
                 p.add(fam1, qual1, value2);
                 mrm.add(p);
               }


[2/5] git commit: Revert "HBASE-12148 Remove TimeRangeTracker as point of contention when many threads writing a Store"

Posted by ap...@apache.org.
Revert "HBASE-12148 Remove TimeRangeTracker as point of contention when many threads writing a Store"

This reverts commit 9f2e748f0a33a5b939a3768c8aeb10432ad43b23.

Post commit consensus is this change isn't ready yet.


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

Branch: refs/heads/0.98
Commit: 00d3eec5d69a8ac735a23fc09be7b8a13c85c0cb
Parents: be64e48
Author: Andrew Purtell <ap...@apache.org>
Authored: Wed Oct 8 12:00:52 2014 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Wed Oct 8 12:00:52 2014 -0700

----------------------------------------------------------------------
 .../hbase/regionserver/TimeRangeTracker.java    | 80 ++++++++++----------
 1 file changed, 39 insertions(+), 41 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/00d3eec5/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/TimeRangeTracker.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/TimeRangeTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/TimeRangeTracker.java
index 3d48b7a..dfa0ded 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/TimeRangeTracker.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/TimeRangeTracker.java
@@ -24,7 +24,9 @@ import java.io.IOException;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.KeyValue.Type;
 import org.apache.hadoop.hbase.io.TimeRange;
+import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.io.Writable;
 
 /**
@@ -36,41 +38,30 @@ import org.apache.hadoop.io.Writable;
  */
 @InterfaceAudience.Private
 public class TimeRangeTracker implements Writable {
-  static final long INITIAL_MINIMUM_TIMESTAMP = Long.MAX_VALUE;
-  long minimumTimestamp = INITIAL_MINIMUM_TIMESTAMP;
+
+  long minimumTimestamp = -1;
   long maximumTimestamp = -1;
 
   /**
    * Default constructor.
    * Initializes TimeRange to be null
    */
-  public TimeRangeTracker() {}
+  public TimeRangeTracker() {
+
+  }
 
   /**
    * Copy Constructor
    * @param trt source TimeRangeTracker
    */
   public TimeRangeTracker(final TimeRangeTracker trt) {
-    set(trt.getMinimumTimestamp(), trt.getMaximumTimestamp());
+    this.minimumTimestamp = trt.getMinimumTimestamp();
+    this.maximumTimestamp = trt.getMaximumTimestamp();
   }
 
   public TimeRangeTracker(long minimumTimestamp, long maximumTimestamp) {
-    set(minimumTimestamp, maximumTimestamp);
-  }
-
-  private void set(final long min, final long max) {
-    this.minimumTimestamp = min;
-    this.maximumTimestamp = max;
-  }
-
-  /**
-   * @param l
-   * @return True if we initialized values
-   */
-  private boolean init(final long l) {
-    if (this.minimumTimestamp != INITIAL_MINIMUM_TIMESTAMP) return false;
-    set(l, l);
-    return true;
+    this.minimumTimestamp = minimumTimestamp;
+    this.maximumTimestamp = maximumTimestamp;
   }
 
   /**
@@ -87,30 +78,36 @@ public class TimeRangeTracker implements Writable {
   }
 
   /**
+   * Update the current TimestampRange to include the timestamp from Key.
+   * If the Key is of type DeleteColumn or DeleteFamily, it includes the
+   * entire time range from 0 to timestamp of the key.
+   * @param key
+   */
+  public void includeTimestamp(final byte[] key) {
+    includeTimestamp(Bytes.toLong(key,key.length-KeyValue.TIMESTAMP_TYPE_SIZE));
+    int type = key[key.length - 1];
+    if (type == Type.DeleteColumn.getCode() ||
+        type == Type.DeleteFamily.getCode()) {
+      includeTimestamp(0);
+    }
+  }
+
+  /**
    * If required, update the current TimestampRange to include timestamp
    * @param timestamp the timestamp value to include
    */
-  void includeTimestamp(final long timestamp) {
-    // Do test outside of synchronization block.  Synchronization in here can be problematic
-    // when many threads writing one Store -- they can all pile up trying to add in here.
-    // Happens when doing big write upload where we are hammering on one region.
-    if (timestamp < this.minimumTimestamp) {
-      synchronized (this) {
-        if (!init(timestamp)) {
-          if (timestamp < this.minimumTimestamp) {
-            this.minimumTimestamp = timestamp;
-          }
-        }
-      }
-    } else if (timestamp > this.maximumTimestamp) {
-      synchronized (this) {
-        if (!init(timestamp)) {
-          if (this.maximumTimestamp < timestamp) {
-            this.maximumTimestamp =  timestamp;
-          }
-        }
-      }
+  private synchronized void includeTimestamp(final long timestamp) {
+    if (maximumTimestamp == -1) {
+      minimumTimestamp = timestamp;
+      maximumTimestamp = timestamp;
+    }
+    else if (minimumTimestamp > timestamp) {
+      minimumTimestamp = timestamp;
+    }
+    else if (maximumTimestamp < timestamp) {
+      maximumTimestamp = timestamp;
     }
+    return;
   }
 
   /**
@@ -119,7 +116,8 @@ public class TimeRangeTracker implements Writable {
    * @return True if there is overlap, false otherwise
    */
   public synchronized boolean includesTimeRange(final TimeRange tr) {
-    return (this.minimumTimestamp < tr.getMax() && this.maximumTimestamp >= tr.getMin());
+    return (this.minimumTimestamp < tr.getMax() &&
+        this.maximumTimestamp >= tr.getMin());
   }
 
   /**


[5/5] git commit: Fix failures caused by Jacoco syncthetic methods

Posted by ap...@apache.org.
Fix failures caused by Jacoco syncthetic methods

Signed-off-by: Andrew Purtell <ap...@apache.org>


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

Branch: refs/heads/0.98
Commit: 890916a231d43b191e3433e0b18e790e109c71fc
Parents: 00d3eec
Author: manukranthk <ma...@fb.com>
Authored: Wed Oct 8 10:36:02 2014 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Wed Oct 8 12:05:37 2014 -0700

----------------------------------------------------------------------
 .../src/main/java/org/apache/hadoop/hbase/SplitLogCounters.java    | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/890916a2/hbase-server/src/main/java/org/apache/hadoop/hbase/SplitLogCounters.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/SplitLogCounters.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/SplitLogCounters.java
index 7f8069a..b9614b4 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/SplitLogCounters.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/SplitLogCounters.java
@@ -88,7 +88,7 @@ public class SplitLogCounters {
   public static void resetCounters() throws Exception {
     Class<?> cl = (new SplitLogCounters()).getClass();
     for (Field fld : cl.getDeclaredFields()) {
-      ((AtomicLong)fld.get(null)).set(0);
+      if (!fld.isSynthetic()) ((AtomicLong)fld.get(null)).set(0);
     }
   }
 }


[4/5] git commit: Fix failures caused by Jacoco syncthetic methods

Posted by ap...@apache.org.
Fix failures caused by Jacoco syncthetic methods

Signed-off-by: Andrew Purtell <ap...@apache.org>


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

Branch: refs/heads/branch-1
Commit: ad2064d8a5ff57d021852c3210a30c5f58eaa43c
Parents: 283ec57
Author: manukranthk <ma...@fb.com>
Authored: Wed Oct 8 10:36:02 2014 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Wed Oct 8 12:05:33 2014 -0700

----------------------------------------------------------------------
 .../src/main/java/org/apache/hadoop/hbase/SplitLogCounters.java    | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/ad2064d8/hbase-server/src/main/java/org/apache/hadoop/hbase/SplitLogCounters.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/SplitLogCounters.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/SplitLogCounters.java
index 7f8069a..b9614b4 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/SplitLogCounters.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/SplitLogCounters.java
@@ -88,7 +88,7 @@ public class SplitLogCounters {
   public static void resetCounters() throws Exception {
     Class<?> cl = (new SplitLogCounters()).getClass();
     for (Field fld : cl.getDeclaredFields()) {
-      ((AtomicLong)fld.get(null)).set(0);
+      if (!fld.isSynthetic()) ((AtomicLong)fld.get(null)).set(0);
     }
   }
 }


[3/5] git commit: Fix failures caused by Jacoco syncthetic methods

Posted by ap...@apache.org.
Fix failures caused by Jacoco syncthetic methods

Signed-off-by: Andrew Purtell <ap...@apache.org>


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

Branch: refs/heads/master
Commit: e31286bf621273bb8e484f95b1a94fff15df034c
Parents: e952e73
Author: manukranthk <ma...@fb.com>
Authored: Wed Oct 8 10:36:02 2014 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Wed Oct 8 12:05:14 2014 -0700

----------------------------------------------------------------------
 .../src/main/java/org/apache/hadoop/hbase/SplitLogCounters.java    | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/e31286bf/hbase-server/src/main/java/org/apache/hadoop/hbase/SplitLogCounters.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/SplitLogCounters.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/SplitLogCounters.java
index 7f8069a..b9614b4 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/SplitLogCounters.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/SplitLogCounters.java
@@ -88,7 +88,7 @@ public class SplitLogCounters {
   public static void resetCounters() throws Exception {
     Class<?> cl = (new SplitLogCounters()).getClass();
     for (Field fld : cl.getDeclaredFields()) {
-      ((AtomicLong)fld.get(null)).set(0);
+      if (!fld.isSynthetic()) ((AtomicLong)fld.get(null)).set(0);
     }
   }
 }