You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ec...@apache.org on 2014/12/26 20:04:33 UTC

hbase git commit: Get rid of the AbtractWAL

Repository: hbase
Updated Branches:
  refs/heads/HBASE-12259 73796eaae -> 15570dd1f


Get rid of the AbtractWAL

Summary:
With the recent refactoring of the WAL we need to get rid of the
AbstractWAL and use the WALProvider.

This diff will just remove the use of AbstractWAL

Test Plan: hbase-consensus unit tests

Reviewers: reddragon, fantasist, eclark, tedyu

Differential Revision: https://reviews.facebook.net/D30681

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/15570dd1
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/15570dd1
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/15570dd1

Branch: refs/heads/HBASE-12259
Commit: 15570dd1f1989bf9c2a52249f6039949940986cc
Parents: 73796ea
Author: Rishit Shroff <rs...@fb.com>
Authored: Tue Dec 23 10:43:22 2014 -0800
Committer: Elliott Clark <ec...@apache.org>
Committed: Fri Dec 26 11:02:56 2014 -0800

----------------------------------------------------------------------
 .../hbase/consensus/quorum/QuorumAgent.java     | 35 +++++-----
 .../hbase/regionserver/wal/AbstractWAL.java     | 69 --------------------
 2 files changed, 15 insertions(+), 89 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/15570dd1/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/QuorumAgent.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/QuorumAgent.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/QuorumAgent.java
index 8120dd2..c0fe2cb 100644
--- a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/QuorumAgent.java
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/QuorumAgent.java
@@ -1,19 +1,7 @@
 package org.apache.hadoop.hbase.consensus.quorum;
 
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.Arrays;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Random;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
-import java.util.concurrent.locks.Condition;
-import java.util.concurrent.locks.ReentrantLock;
-
+import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.util.concurrent.SettableFuture;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.client.NoLeaderForRegionException;
@@ -25,14 +13,24 @@ import org.apache.hadoop.hbase.consensus.protocol.ConsensusHost;
 import org.apache.hadoop.hbase.consensus.raft.events.ReplicateEntriesEvent;
 import org.apache.hadoop.hbase.io.hfile.Compression;
 import org.apache.hadoop.hbase.metrics.TimeStat;
-import org.apache.hadoop.hbase.regionserver.wal.AbstractWAL;
 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
 import org.apache.hadoop.hbase.util.DaemonThreadFactory;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.google.common.util.concurrent.ListenableFuture;
-import com.google.common.util.concurrent.SettableFuture;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
 
 /**
  * This is an agent that runs on the RaftQuorumContext side, and it is
@@ -248,7 +246,6 @@ public class QuorumAgent implements ConfigurationObserver {
     checkBeforeCommit();
 
     // increase the write size
-    AbstractWAL.getWriteSizeHistogram().addValue(edits.getTotalKeyValueLength());
 
     long start = System.nanoTime();
     ListenableFuture<Long> future = internalCommit(edits);
@@ -260,7 +257,6 @@ public class QuorumAgent implements ConfigurationObserver {
       double syncMicros = (System.nanoTime() - start) / 1000.0;
       getRaftQuorumContext().getConsensusMetrics().getFsSyncLatency()
         .add((long)syncMicros, TimeUnit.MICROSECONDS);
-      AbstractWAL.getSyncTimeHistogram().addValue(syncMicros);
       return seq;
     } catch (Exception e) {
       throw new IOException(e);
@@ -438,7 +434,6 @@ public class QuorumAgent implements ConfigurationObserver {
             double gsyncMicros = (System.nanoTime() - start) / 1000.0;
             getRaftQuorumContext().getConsensusMetrics().getFsGSyncLatency()
               .add((long) gsyncMicros, TimeUnit.MICROSECONDS);
-            AbstractWAL.getGSyncTimeHistogram().addValue(gsyncMicros);
           } catch (Throwable e) {
             LOG.error("Unexpected exception: ", e);
           }

http://git-wip-us.apache.org/repos/asf/hbase/blob/15570dd1/hbase-consensus/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractWAL.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractWAL.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractWAL.java
deleted file mode 100644
index ea5b5c9..0000000
--- a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractWAL.java
+++ /dev/null
@@ -1,69 +0,0 @@
-package org.apache.hadoop.hbase.regionserver.wal;
-
-import org.apache.hadoop.hbase.regionserver.metrics.PercentileMetric;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.Histogram;
-
-import java.io.IOException;
-import java.util.concurrent.ExecutionException;
-
-public abstract class AbstractWAL {
-  public static final byte [] METAFAMILY = Bytes.toBytes("METAFAMILY");
-  public static final byte [] METAROW = Bytes.toBytes("METAROW");
-
-  // For measuring size of each transaction
-  protected static Histogram writeSize = new Histogram(
-    PercentileMetric.HISTOGRAM_NUM_BUCKETS_DEFAULT,
-    PercentileMetric.HISTOGRAM_MINVALUE_DEFAULT,
-    PercentileMetric.HISTOGRAM_MAXVALUE_DEFAULT);
-
-  // For measure the sync time for each HLog.append operation;
-  protected static Histogram syncTime = new Histogram(
-    PercentileMetric.HISTOGRAM_NUM_BUCKETS_DEFAULT,
-    PercentileMetric.HISTOGRAM_MINVALUE_DEFAULT,
-    PercentileMetric.HISTOGRAM_MAXVALUE_DEFAULT);
-
-  // For measuring the internal group commit time
-  protected static Histogram gsyncTime = new Histogram(
-    PercentileMetric.HISTOGRAM_NUM_BUCKETS_DEFAULT,
-    PercentileMetric.HISTOGRAM_MINVALUE_DEFAULT,
-    PercentileMetric.HISTOGRAM_MAXVALUE_DEFAULT);
-
-  /*
-  public abstract long append(HRegionInfo info, byte [] tableName, WALEdit edits,
-                     final long now)
-    throws IOException, ExecutionException, InterruptedException;
-  */
-  public abstract long startMemStoreFlush(final byte[] regionName);
-  public abstract void completeMemStoreFlush(final byte[] regionName, final byte[] tableName,
-                                    final long logSeqId, final boolean isMetaRegion);
-  public abstract void abortMemStoreFlush(byte[] regionName);
-  public abstract long startMemStoreFlush(final byte[] regionName,
-                                 long firstSeqIdInStoresToFlush,
-                                 long firstSeqIdInStoresNotToFlush);
-
-  public abstract long obtainNextSequenceNumber()
-    throws IOException, ExecutionException, InterruptedException;
-  public abstract long getSequenceNumber();
-  public abstract void initSequenceNumber(long seqid)
-    throws IOException, ExecutionException, InterruptedException;
-
-  public abstract void close() throws IOException;
-  public abstract void closeAndDelete() throws IOException;
-  public abstract String getPath();
-
-  public static Histogram getWriteSizeHistogram() {
-    return writeSize;
-  }
-
-  public static Histogram getSyncTimeHistogram() {
-    return syncTime;
-  }
-
-  public static Histogram getGSyncTimeHistogram() {
-    return gsyncTime;
-  }
-
-  public abstract long getLastCommittedIndex();
-
-}