[41/50] [abbrv] hbase git commit: HBASE-19069 Do not wrap the original CompactionLifeCycleTracker when calling CP hooks

2017-10-23 Thread zhangduo
HBASE-19069 Do not wrap the original CompactionLifeCycleTracker when calling CP 
hooks


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

Branch: refs/heads/HBASE-18410
Commit: 37b29e909defecdc580112ce6cd306710d13e9e2
Parents: 81133f8
Author: zhangduo 
Authored: Mon Oct 23 21:10:44 2017 +0800
Committer: zhangduo 
Committed: Tue Oct 24 10:56:14 2017 +0800

--
 .../hadoop/hbase/regionserver/CompactSplit.java | 135 ++-
 .../TestCompactionLifeCycleTracker.java |   9 +-
 2 files changed, 80 insertions(+), 64 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hbase/blob/37b29e90/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplit.java
--
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplit.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplit.java
index b82b346..0749f85 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplit.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplit.java
@@ -237,80 +237,73 @@ public class CompactSplit implements CompactionRequester, 
PropagatingConfigurati
 }
   }
 
-  // A compaction life cycle tracker to trace the execution of all the 
compactions triggered by one
-  // request and delegate to the source CompactionLifeCycleTracker. It will 
call completed method if
-  // all the compactions are finished.
-  private static final class AggregatingCompactionLifeCycleTracker
-  implements CompactionLifeCycleTracker {
+  private interface CompactionCompleteTracker {
+
+default void completed(Store store) {
+}
+  }
+
+  private static final CompactionCompleteTracker DUMMY_COMPLETE_TRACKER =
+  new CompactionCompleteTracker() {
+  };
+
+  private static final class AggregatingCompleteTracker implements 
CompactionCompleteTracker {
 
 private final CompactionLifeCycleTracker tracker;
 
 private final AtomicInteger remaining;
 
-public AggregatingCompactionLifeCycleTracker(CompactionLifeCycleTracker 
tracker,
-int numberOfStores) {
+public AggregatingCompleteTracker(CompactionLifeCycleTracker tracker, int 
numberOfStores) {
   this.tracker = tracker;
   this.remaining = new AtomicInteger(numberOfStores);
 }
 
-private void tryCompleted() {
+@Override
+public void completed(Store store) {
   if (remaining.decrementAndGet() == 0) {
 tracker.completed();
   }
 }
-
-@Override
-public void notExecuted(Store store, String reason) {
-  tracker.notExecuted(store, reason);
-  tryCompleted();
-}
-
-@Override
-public void beforeExecution(Store store) {
-  tracker.beforeExecution(store);
-}
-
-@Override
-public void afterExecution(Store store) {
-  tracker.afterExecution(store);
-  tryCompleted();
-}
   }
 
-  private CompactionLifeCycleTracker wrap(CompactionLifeCycleTracker tracker,
+  private CompactionCompleteTracker 
getCompleteTracker(CompactionLifeCycleTracker tracker,
   IntSupplier numberOfStores) {
 if (tracker == CompactionLifeCycleTracker.DUMMY) {
   // a simple optimization to avoid creating unnecessary objects as 
usually we do not care about
   // the life cycle of a compaction.
-  return tracker;
+  return DUMMY_COMPLETE_TRACKER;
 } else {
-  return new AggregatingCompactionLifeCycleTracker(tracker, 
numberOfStores.getAsInt());
+  return new AggregatingCompleteTracker(tracker, 
numberOfStores.getAsInt());
 }
   }
 
   @Override
   public synchronized void requestCompaction(HRegion region, String why, int 
priority,
   CompactionLifeCycleTracker tracker, User user) throws IOException {
-requestCompactionInternal(region, why, priority, true,
-  wrap(tracker, () -> region.getTableDescriptor().getColumnFamilyCount()), 
user);
+requestCompactionInternal(region, why, priority, true, tracker,
+  getCompleteTracker(tracker, () -> 
region.getTableDescriptor().getColumnFamilyCount()), user);
   }
 
   @Override
   public synchronized void requestCompaction(HRegion region, HStore store, 
String why, int priority,
   CompactionLifeCycleTracker tracker, User user) throws IOException {
-requestCompactionInternal(region, store, why, priority, true, 
wrap(tracker, () -> 1), user);
+requestCompactionInternal(region, store, why, priority, true, tracker,
+  getCompleteTracker(tracker, () -> 1), user);
   }
 
   private void 

hbase git commit: HBASE-19069 Do not wrap the original CompactionLifeCycleTracker when calling CP hooks

2017-10-23 Thread zhangduo
Repository: hbase
Updated Branches:
  refs/heads/branch-2 3e0b90b94 -> a6f89f029


HBASE-19069 Do not wrap the original CompactionLifeCycleTracker when calling CP 
hooks


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

Branch: refs/heads/branch-2
Commit: a6f89f029a78a37eb84e4908b9975118e3050603
Parents: 3e0b90b
Author: zhangduo 
Authored: Mon Oct 23 21:10:44 2017 +0800
Committer: zhangduo 
Committed: Tue Oct 24 10:56:19 2017 +0800

--
 .../hadoop/hbase/regionserver/CompactSplit.java | 135 ++-
 .../TestCompactionLifeCycleTracker.java |   9 +-
 2 files changed, 80 insertions(+), 64 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hbase/blob/a6f89f02/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplit.java
--
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplit.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplit.java
index b82b346..0749f85 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplit.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplit.java
@@ -237,80 +237,73 @@ public class CompactSplit implements CompactionRequester, 
PropagatingConfigurati
 }
   }
 
-  // A compaction life cycle tracker to trace the execution of all the 
compactions triggered by one
-  // request and delegate to the source CompactionLifeCycleTracker. It will 
call completed method if
-  // all the compactions are finished.
-  private static final class AggregatingCompactionLifeCycleTracker
-  implements CompactionLifeCycleTracker {
+  private interface CompactionCompleteTracker {
+
+default void completed(Store store) {
+}
+  }
+
+  private static final CompactionCompleteTracker DUMMY_COMPLETE_TRACKER =
+  new CompactionCompleteTracker() {
+  };
+
+  private static final class AggregatingCompleteTracker implements 
CompactionCompleteTracker {
 
 private final CompactionLifeCycleTracker tracker;
 
 private final AtomicInteger remaining;
 
-public AggregatingCompactionLifeCycleTracker(CompactionLifeCycleTracker 
tracker,
-int numberOfStores) {
+public AggregatingCompleteTracker(CompactionLifeCycleTracker tracker, int 
numberOfStores) {
   this.tracker = tracker;
   this.remaining = new AtomicInteger(numberOfStores);
 }
 
-private void tryCompleted() {
+@Override
+public void completed(Store store) {
   if (remaining.decrementAndGet() == 0) {
 tracker.completed();
   }
 }
-
-@Override
-public void notExecuted(Store store, String reason) {
-  tracker.notExecuted(store, reason);
-  tryCompleted();
-}
-
-@Override
-public void beforeExecution(Store store) {
-  tracker.beforeExecution(store);
-}
-
-@Override
-public void afterExecution(Store store) {
-  tracker.afterExecution(store);
-  tryCompleted();
-}
   }
 
-  private CompactionLifeCycleTracker wrap(CompactionLifeCycleTracker tracker,
+  private CompactionCompleteTracker 
getCompleteTracker(CompactionLifeCycleTracker tracker,
   IntSupplier numberOfStores) {
 if (tracker == CompactionLifeCycleTracker.DUMMY) {
   // a simple optimization to avoid creating unnecessary objects as 
usually we do not care about
   // the life cycle of a compaction.
-  return tracker;
+  return DUMMY_COMPLETE_TRACKER;
 } else {
-  return new AggregatingCompactionLifeCycleTracker(tracker, 
numberOfStores.getAsInt());
+  return new AggregatingCompleteTracker(tracker, 
numberOfStores.getAsInt());
 }
   }
 
   @Override
   public synchronized void requestCompaction(HRegion region, String why, int 
priority,
   CompactionLifeCycleTracker tracker, User user) throws IOException {
-requestCompactionInternal(region, why, priority, true,
-  wrap(tracker, () -> region.getTableDescriptor().getColumnFamilyCount()), 
user);
+requestCompactionInternal(region, why, priority, true, tracker,
+  getCompleteTracker(tracker, () -> 
region.getTableDescriptor().getColumnFamilyCount()), user);
   }
 
   @Override
   public synchronized void requestCompaction(HRegion region, HStore store, 
String why, int priority,
   CompactionLifeCycleTracker tracker, User user) throws IOException {
-requestCompactionInternal(region, store, why, priority, true, 
wrap(tracker, () -> 1), user);
+requestCompactionInternal(region, store, why, priority, true, tracker,
+  

hbase git commit: HBASE-19069 Do not wrap the original CompactionLifeCycleTracker when calling CP hooks

2017-10-23 Thread zhangduo
Repository: hbase
Updated Branches:
  refs/heads/master 81133f89f -> 37b29e909


HBASE-19069 Do not wrap the original CompactionLifeCycleTracker when calling CP 
hooks


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

Branch: refs/heads/master
Commit: 37b29e909defecdc580112ce6cd306710d13e9e2
Parents: 81133f8
Author: zhangduo 
Authored: Mon Oct 23 21:10:44 2017 +0800
Committer: zhangduo 
Committed: Tue Oct 24 10:56:14 2017 +0800

--
 .../hadoop/hbase/regionserver/CompactSplit.java | 135 ++-
 .../TestCompactionLifeCycleTracker.java |   9 +-
 2 files changed, 80 insertions(+), 64 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hbase/blob/37b29e90/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplit.java
--
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplit.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplit.java
index b82b346..0749f85 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplit.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplit.java
@@ -237,80 +237,73 @@ public class CompactSplit implements CompactionRequester, 
PropagatingConfigurati
 }
   }
 
-  // A compaction life cycle tracker to trace the execution of all the 
compactions triggered by one
-  // request and delegate to the source CompactionLifeCycleTracker. It will 
call completed method if
-  // all the compactions are finished.
-  private static final class AggregatingCompactionLifeCycleTracker
-  implements CompactionLifeCycleTracker {
+  private interface CompactionCompleteTracker {
+
+default void completed(Store store) {
+}
+  }
+
+  private static final CompactionCompleteTracker DUMMY_COMPLETE_TRACKER =
+  new CompactionCompleteTracker() {
+  };
+
+  private static final class AggregatingCompleteTracker implements 
CompactionCompleteTracker {
 
 private final CompactionLifeCycleTracker tracker;
 
 private final AtomicInteger remaining;
 
-public AggregatingCompactionLifeCycleTracker(CompactionLifeCycleTracker 
tracker,
-int numberOfStores) {
+public AggregatingCompleteTracker(CompactionLifeCycleTracker tracker, int 
numberOfStores) {
   this.tracker = tracker;
   this.remaining = new AtomicInteger(numberOfStores);
 }
 
-private void tryCompleted() {
+@Override
+public void completed(Store store) {
   if (remaining.decrementAndGet() == 0) {
 tracker.completed();
   }
 }
-
-@Override
-public void notExecuted(Store store, String reason) {
-  tracker.notExecuted(store, reason);
-  tryCompleted();
-}
-
-@Override
-public void beforeExecution(Store store) {
-  tracker.beforeExecution(store);
-}
-
-@Override
-public void afterExecution(Store store) {
-  tracker.afterExecution(store);
-  tryCompleted();
-}
   }
 
-  private CompactionLifeCycleTracker wrap(CompactionLifeCycleTracker tracker,
+  private CompactionCompleteTracker 
getCompleteTracker(CompactionLifeCycleTracker tracker,
   IntSupplier numberOfStores) {
 if (tracker == CompactionLifeCycleTracker.DUMMY) {
   // a simple optimization to avoid creating unnecessary objects as 
usually we do not care about
   // the life cycle of a compaction.
-  return tracker;
+  return DUMMY_COMPLETE_TRACKER;
 } else {
-  return new AggregatingCompactionLifeCycleTracker(tracker, 
numberOfStores.getAsInt());
+  return new AggregatingCompleteTracker(tracker, 
numberOfStores.getAsInt());
 }
   }
 
   @Override
   public synchronized void requestCompaction(HRegion region, String why, int 
priority,
   CompactionLifeCycleTracker tracker, User user) throws IOException {
-requestCompactionInternal(region, why, priority, true,
-  wrap(tracker, () -> region.getTableDescriptor().getColumnFamilyCount()), 
user);
+requestCompactionInternal(region, why, priority, true, tracker,
+  getCompleteTracker(tracker, () -> 
region.getTableDescriptor().getColumnFamilyCount()), user);
   }
 
   @Override
   public synchronized void requestCompaction(HRegion region, HStore store, 
String why, int priority,
   CompactionLifeCycleTracker tracker, User user) throws IOException {
-requestCompactionInternal(region, store, why, priority, true, 
wrap(tracker, () -> 1), user);
+requestCompactionInternal(region, store, why, priority, true, tracker,
+