You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by zh...@apache.org on 2016/11/22 10:30:27 UTC

[4/4] hbase git commit: HBASE-17132 Cleanup deprecated code for WAL

HBASE-17132 Cleanup deprecated code for WAL


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

Branch: refs/heads/master
Commit: 47a4e3437bb637522525c0baf3d04a11c0546980
Parents: 92b494f
Author: zhangduo <zh...@apache.org>
Authored: Tue Nov 22 10:53:34 2016 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Tue Nov 22 18:27:41 2016 +0800

----------------------------------------------------------------------
 .../hbase/coprocessor/BaseRegionObserver.java   |  17 +-
 .../hbase/coprocessor/BaseWALObserver.java      |  21 +-
 .../hbase/coprocessor/RegionObserver.java       |  53 +-
 .../hadoop/hbase/coprocessor/WALObserver.java   |  60 +-
 .../hadoop/hbase/mapreduce/HLogInputFormat.java |  83 --
 .../hadoop/hbase/mapreduce/WALPlayer.java       |   9 +-
 .../hadoop/hbase/regionserver/HRegion.java      |  20 +-
 .../regionserver/RegionCoprocessorHost.java     | 100 +--
 .../hadoop/hbase/regionserver/wal/FSHLog.java   |   8 +-
 .../hadoop/hbase/regionserver/wal/HLogKey.java  | 285 -------
 .../hbase/regionserver/wal/ReaderBase.java      |   8 +-
 .../hbase/regionserver/wal/ReplayHLogKey.java   |  54 --
 .../regionserver/wal/SequenceFileLogReader.java | 309 -------
 .../regionserver/wal/WALCoprocessorHost.java    |  40 +-
 .../security/access/AccessControlLists.java     |  54 +-
 .../HbaseObjectWritableFor96Migration.java      | 844 -------------------
 .../java/org/apache/hadoop/hbase/wal/WAL.java   |  10 +-
 .../org/apache/hadoop/hbase/wal/WALFactory.java |  51 +-
 .../org/apache/hadoop/hbase/wal/WALKey.java     |  17 +-
 .../apache/hadoop/hbase/wal/WALProvider.java    |   2 +-
 .../apache/hadoop/hbase/wal/WALSplitter.java    |  23 +-
 .../coprocessor/SampleRegionWALObserver.java    |  76 +-
 .../hbase/coprocessor/SimpleRegionObserver.java |  39 +-
 .../TestRegionObserverInterface.java            | 382 +++------
 .../hbase/coprocessor/TestWALObserver.java      | 106 +--
 .../hbase/mapreduce/TestHLogRecordReader.java   |  46 -
 .../master/TestDistributedLogSplitting.java     |  11 +-
 .../regionserver/TestFailedAppendAndSync.java   |   2 +-
 .../hadoop/hbase/regionserver/TestHRegion.java  |  33 +-
 .../hbase/regionserver/TestWALLockup.java       |  16 +-
 .../wal/FaultyProtobufLogReader.java            |  76 ++
 .../wal/FaultySequenceFileLogReader.java        |  80 --
 .../regionserver/wal/SequenceFileLogWriter.java | 239 ------
 .../wal/TestReadOldRootAndMetaEdits.java        | 161 ----
 .../TestReplicationWALEntryFilters.java         |  29 +-
 .../HbaseObjectWritableFor96Migration.java      | 816 ++++++++++++++++++
 .../security/access/TestTablePermissions.java   |   4 +-
 .../wal/TestDefaultWALProviderWithHLogKey.java  |  37 -
 .../hadoop/hbase/wal/TestFSHLogProvider.java    |   2 -
 .../apache/hadoop/hbase/wal/TestWALFactory.java |  79 +-
 .../apache/hadoop/hbase/wal/TestWALMethods.java |  27 +-
 .../apache/hadoop/hbase/wal/TestWALSplit.java   |  46 +-
 42 files changed, 1242 insertions(+), 3133 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/47a4e343/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseRegionObserver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseRegionObserver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseRegionObserver.java
index 3442b64..660c631 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseRegionObserver.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseRegionObserver.java
@@ -19,6 +19,8 @@
 
 package org.apache.hadoop.hbase.coprocessor;
 
+import com.google.common.collect.ImmutableList;
+
 import java.io.IOException;
 import java.util.List;
 import java.util.Map;
@@ -58,13 +60,10 @@ import org.apache.hadoop.hbase.regionserver.StoreFile;
 import org.apache.hadoop.hbase.regionserver.StoreFileReader;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
 import org.apache.hadoop.hbase.regionserver.querymatcher.DeleteTracker;
-import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.wal.WALKey;
 
-import com.google.common.collect.ImmutableList;
-
 /**
  * An abstract class that implements RegionObserver.
  * By extending it, you can create your own region observer without
@@ -485,12 +484,6 @@ public class BaseRegionObserver implements RegionObserver {
       HRegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException {
   }
 
-  @Override
-  public void preWALRestore(ObserverContext<RegionCoprocessorEnvironment> env, HRegionInfo info,
-      HLogKey logKey, WALEdit logEdit) throws IOException {
-    preWALRestore(env, info, (WALKey)logKey, logEdit);
-  }
-
   /**
    * Implementers should override this version of the method and leave the deprecated one as-is.
    */
@@ -500,12 +493,6 @@ public class BaseRegionObserver implements RegionObserver {
   }
 
   @Override
-  public void postWALRestore(ObserverContext<RegionCoprocessorEnvironment> env,
-      HRegionInfo info, HLogKey logKey, WALEdit logEdit) throws IOException {
-    postWALRestore(env, info, (WALKey)logKey, logEdit);
-  }
-
-  @Override
   public void preBulkLoadHFile(final ObserverContext<RegionCoprocessorEnvironment> ctx,
     List<Pair<byte[], String>> familyPaths) throws IOException {
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/47a4e343/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseWALObserver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseWALObserver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseWALObserver.java
index 1d0076a..8507db7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseWALObserver.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseWALObserver.java
@@ -21,14 +21,13 @@ package org.apache.hadoop.hbase.coprocessor;
 import java.io.IOException;
 
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.CoprocessorEnvironment;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
-import org.apache.hadoop.hbase.wal.WALKey;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
+import org.apache.hadoop.hbase.wal.WALKey;
 
 /**
  * An abstract class that implements WALObserver.
@@ -53,24 +52,12 @@ public class BaseWALObserver implements WALObserver {
     return false;
   }
 
-  @Override
-  public boolean preWALWrite(ObserverContext<WALCoprocessorEnvironment> ctx, HRegionInfo info,
-      HLogKey logKey, WALEdit logEdit) throws IOException {
-    return preWALWrite(ctx, info, (WALKey)logKey, logEdit);
-  }
-
   /**
    * Implementers should override this method and leave the deprecated version as-is.
    */
   @Override
   public void postWALWrite(ObserverContext<? extends WALCoprocessorEnvironment> ctx,
-      HRegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException { }
-
-  @Override
-  public void postWALWrite(ObserverContext<WALCoprocessorEnvironment> ctx, HRegionInfo info,
-      HLogKey logKey, WALEdit logEdit) throws IOException {
-    postWALWrite(ctx, info, (WALKey)logKey, logEdit);
-  }
+      HRegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException {}
 
   @Override
   public void preWALRoll(ObserverContext<? extends WALCoprocessorEnvironment> ctx,

http://git-wip-us.apache.org/repos/asf/hbase/blob/47a4e343/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java
index d9a509c..0d01baa 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java
@@ -19,6 +19,8 @@
 
 package org.apache.hadoop.hbase.coprocessor;
 
+import com.google.common.collect.ImmutableList;
+
 import java.io.IOException;
 import java.util.List;
 import java.util.Map;
@@ -59,13 +61,10 @@ import org.apache.hadoop.hbase.regionserver.StoreFile;
 import org.apache.hadoop.hbase.regionserver.StoreFileReader;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
 import org.apache.hadoop.hbase.regionserver.querymatcher.DeleteTracker;
-import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.wal.WALKey;
 
-import com.google.common.collect.ImmutableList;
-
 /**
  * Coprocessors implement this interface to observe and mediate client actions
  * on the region.
@@ -1318,30 +1317,6 @@ public interface RegionObserver extends Coprocessor {
       HRegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException;
 
   /**
-   * Called before a {@link org.apache.hadoop.hbase.regionserver.wal.WALEdit}
-   * replayed for this region.
-   *
-   * This method is left in place to maintain binary compatibility with older
-   * {@link RegionObserver}s. If an implementation directly overrides
-   * {@link #preWALRestore(ObserverContext, HRegionInfo, WALKey, WALEdit)} then this version
-   * won't be called at all, barring problems with the Security Manager. To work correctly
-   * in the presence of a strict Security Manager, or in the case of an implementation that
-   * relies on a parent class to implement preWALRestore, you should implement this method
-   * as a call to the non-deprecated version.
-   *
-   * Users of this method will see all edits that can be treated as HLogKey. If there are
-   * edits that can't be treated as HLogKey they won't be offered to coprocessors that rely
-   * on this method. If a coprocessor gets skipped because of this mechanism, a log message
-   * at ERROR will be generated per coprocessor on the logger for {@link CoprocessorHost} once per
-   * classloader.
-   *
-   * @deprecated use {@link #preWALRestore(ObserverContext, HRegionInfo, WALKey, WALEdit)}
-   */
-  @Deprecated
-  void preWALRestore(final ObserverContext<RegionCoprocessorEnvironment> ctx,
-      HRegionInfo info, HLogKey logKey, WALEdit logEdit) throws IOException;
-
-  /**
    * Called after a {@link org.apache.hadoop.hbase.regionserver.wal.WALEdit}
    * replayed for this region.
    */
@@ -1349,30 +1324,6 @@ public interface RegionObserver extends Coprocessor {
       HRegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException;
 
   /**
-   * Called after a {@link org.apache.hadoop.hbase.regionserver.wal.WALEdit}
-   * replayed for this region.
-   *
-   * This method is left in place to maintain binary compatibility with older
-   * {@link RegionObserver}s. If an implementation directly overrides
-   * {@link #postWALRestore(ObserverContext, HRegionInfo, WALKey, WALEdit)} then this version
-   * won't be called at all, barring problems with the Security Manager. To work correctly
-   * in the presence of a strict Security Manager, or in the case of an implementation that
-   * relies on a parent class to implement preWALRestore, you should implement this method
-   * as a call to the non-deprecated version.
-   *
-   * Users of this method will see all edits that can be treated as HLogKey. If there are
-   * edits that can't be treated as HLogKey they won't be offered to coprocessors that rely
-   * on this method. If a coprocessor gets skipped because of this mechanism, a log message
-   * at ERROR will be generated per coprocessor on the logger for {@link CoprocessorHost} once per
-   * classloader.
-   *
-   * @deprecated use {@link #postWALRestore(ObserverContext, HRegionInfo, WALKey, WALEdit)}
-   */
-  @Deprecated
-  void postWALRestore(final ObserverContext<RegionCoprocessorEnvironment> ctx,
-      HRegionInfo info, HLogKey logKey, WALEdit logEdit) throws IOException;
-
-  /**
    * Called before bulkLoadHFile. Users can create a StoreFile instance to
    * access the contents of a HFile.
    *

http://git-wip-us.apache.org/repos/asf/hbase/blob/47a4e343/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/WALObserver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/WALObserver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/WALObserver.java
index 7fd03ce..344a764 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/WALObserver.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/WALObserver.java
@@ -19,17 +19,16 @@
 
 package org.apache.hadoop.hbase.coprocessor;
 
+import java.io.IOException;
+
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.Coprocessor;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
-import org.apache.hadoop.hbase.wal.WALKey;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
-
-import java.io.IOException;
+import org.apache.hadoop.hbase.wal.WALKey;
 
 /**
  * It's provided to have a way for coprocessors to observe, rewrite,
@@ -59,31 +58,6 @@ public interface WALObserver extends Coprocessor {
       HRegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException;
 
   /**
-   * Called before a {@link org.apache.hadoop.hbase.regionserver.wal.WALEdit}
-   * is writen to WAL.
-   *
-   * This method is left in place to maintain binary compatibility with older
-   * {@link WALObserver}s. If an implementation directly overrides
-   * {@link #preWALWrite(ObserverContext, HRegionInfo, WALKey, WALEdit)} then this version
-   * won't be called at all, barring problems with the Security Manager. To work correctly
-   * in the presence of a strict Security Manager, or in the case of an implementation that
-   * relies on a parent class to implement preWALWrite, you should implement this method
-   * as a call to the non-deprecated version.
-   *
-   * Users of this method will see all edits that can be treated as HLogKey. If there are
-   * edits that can't be treated as HLogKey they won't be offered to coprocessors that rely
-   * on this method. If a coprocessor gets skipped because of this mechanism, a log message
-   * at ERROR will be generated per coprocessor on the logger for {@link CoprocessorHost} once per
-   * classloader.
-   *
-   * @return true if default behavior should be bypassed, false otherwise
-   * @deprecated use {@link #preWALWrite(ObserverContext, HRegionInfo, WALKey, WALEdit)}
-   */
-  @Deprecated
-  boolean preWALWrite(ObserverContext<WALCoprocessorEnvironment> ctx,
-      HRegionInfo info, HLogKey logKey, WALEdit logEdit) throws IOException;
-
-  /**
    * Called after a {@link org.apache.hadoop.hbase.regionserver.wal.WALEdit}
    * is writen to WAL.
    */
@@ -91,30 +65,6 @@ public interface WALObserver extends Coprocessor {
       HRegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException;
 
   /**
-   * Called after a {@link org.apache.hadoop.hbase.regionserver.wal.WALEdit}
-   * is writen to WAL.
-   *
-   * This method is left in place to maintain binary compatibility with older
-   * {@link WALObserver}s. If an implementation directly overrides
-   * {@link #postWALWrite(ObserverContext, HRegionInfo, WALKey, WALEdit)} then this version
-   * won't be called at all, barring problems with the Security Manager. To work correctly
-   * in the presence of a strict Security Manager, or in the case of an implementation that
-   * relies on a parent class to implement preWALWrite, you should implement this method
-   * as a call to the non-deprecated version.
-   *
-   * Users of this method will see all edits that can be treated as HLogKey. If there are
-   * edits that can't be treated as HLogKey they won't be offered to coprocessors that rely
-   * on this method. If a coprocessor gets skipped because of this mechanism, a log message
-   * at ERROR will be generated per coprocessor on the logger for {@link CoprocessorHost} once per
-   * classloader.
-   *
-   * @deprecated use {@link #postWALWrite(ObserverContext, HRegionInfo, WALKey, WALEdit)}
-   */
-  @Deprecated
-  void postWALWrite(ObserverContext<WALCoprocessorEnvironment> ctx,
-      HRegionInfo info, HLogKey logKey, WALEdit logEdit) throws IOException;
-
-  /**
    * Called before rolling the current WAL
    * @param oldPath the path of the current wal that we are replacing
    * @param newPath the path of the wal we are going to create

http://git-wip-us.apache.org/repos/asf/hbase/blob/47a4e343/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/HLogInputFormat.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/HLogInputFormat.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/HLogInputFormat.java
deleted file mode 100644
index e727e15..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/HLogInputFormat.java
+++ /dev/null
@@ -1,83 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.mapreduce;
-
-import java.io.IOException;
-import java.util.List;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
-import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
-import org.apache.hadoop.mapreduce.InputFormat;
-import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.JobContext;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-
-/**
- * Simple {@link InputFormat} for {@link org.apache.hadoop.hbase.wal.WAL} 
- * files.
- * @deprecated use {@link WALInputFormat}.  Remove in hadoop 3.0
- */
-@Deprecated
-@InterfaceAudience.Public
-public class HLogInputFormat extends InputFormat<HLogKey, WALEdit> {
-  private static final Log LOG = LogFactory.getLog(HLogInputFormat.class);
-  public static final String START_TIME_KEY = "hlog.start.time";
-  public static final String END_TIME_KEY = "hlog.end.time";
-
-  // Delegate to WALInputFormat for implementation.
-  private final WALInputFormat delegate = new WALInputFormat();
-
-  /**
-   * {@link RecordReader} that pulls out the legacy HLogKey format directly.
-   */
-  static class HLogKeyRecordReader extends WALInputFormat.WALRecordReader<HLogKey> {
-    @Override
-    public HLogKey getCurrentKey() throws IOException, InterruptedException {
-      if (!(currentEntry.getKey() instanceof HLogKey)) {
-        final IllegalStateException exception = new IllegalStateException(
-            "HLogInputFormat only works when given entries that have HLogKey for keys. This" +
-            " one had '" + currentEntry.getKey().getClass() + "'");
-        LOG.error("The deprecated HLogInputFormat has to work with the deprecated HLogKey class, " +
-            " but HBase internals read the wal entry using some other class." +
-            " This is a bug; please file an issue or email the developer mailing list. It is " +
-            "likely that you would not have this problem if you updated to use WALInputFormat. " +
-            "You will need the following exception details when seeking help from the HBase " +
-            "community.",
-            exception);
-        throw exception;
-      }
-      return (HLogKey)currentEntry.getKey();
-    }
-  }
-
-  @Override
-  public List<InputSplit> getSplits(JobContext context) throws IOException,
-      InterruptedException {
-    return delegate.getSplits(context, START_TIME_KEY, END_TIME_KEY);
-  }
-
-  @Override
-  public RecordReader<HLogKey, WALEdit> createRecordReader(InputSplit split,
-      TaskAttemptContext context) throws IOException, InterruptedException {
-    return new HLogKeyRecordReader();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/47a4e343/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/WALPlayer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/WALPlayer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/WALPlayer.java
index 452714b..2c67baf 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/WALPlayer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/WALPlayer.java
@@ -80,8 +80,6 @@ public class WALPlayer extends Configured implements Tool {
     Configuration.addDeprecation("hlog.bulk.output", BULK_OUTPUT_CONF_KEY);
     Configuration.addDeprecation("hlog.input.tables", TABLES_KEY);
     Configuration.addDeprecation("hlog.input.tablesmap", TABLE_MAP_KEY);
-    Configuration.addDeprecation(HLogInputFormat.START_TIME_KEY, WALInputFormat.START_TIME_KEY);
-    Configuration.addDeprecation(HLogInputFormat.END_TIME_KEY, WALInputFormat.END_TIME_KEY);
   }
 
   private final static String JOB_NAME_CONF_KEY = "mapreduce.job.name";
@@ -241,11 +239,10 @@ public class WALPlayer extends Configured implements Tool {
    * @return The newly created job.
    * @throws IOException When setting up the job fails.
    */
-  public Job createSubmittableJob(String[] args)
-  throws IOException {
+  public Job createSubmittableJob(String[] args) throws IOException {
     Configuration conf = getConf();
-    setupTime(conf, HLogInputFormat.START_TIME_KEY);
-    setupTime(conf, HLogInputFormat.END_TIME_KEY);
+    setupTime(conf, WALInputFormat.START_TIME_KEY);
+    setupTime(conf, WALInputFormat.END_TIME_KEY);
     Path inputDir = new Path(args[0]);
     String[] tables = args[1].split(",");
     String[] tableMap;

http://git-wip-us.apache.org/repos/asf/hbase/blob/47a4e343/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 6415672..6b41bc4 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
@@ -19,6 +19,13 @@ package org.apache.hadoop.hbase.regionserver;
 
 import static org.apache.hadoop.hbase.HConstants.REPLICATION_SCOPE_LOCAL;
 
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Optional;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.io.Closeables;
+
 import java.io.EOFException;
 import java.io.FileNotFoundException;
 import java.io.IOException;
@@ -137,8 +144,6 @@ import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
 import org.apache.hadoop.hbase.regionserver.throttle.CompactionThroughputControllerFactory;
 import org.apache.hadoop.hbase.regionserver.throttle.NoLimitThroughputController;
 import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
-import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
-import org.apache.hadoop.hbase.regionserver.wal.ReplayHLogKey;
 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
 import org.apache.hadoop.hbase.regionserver.wal.WALUtil;
 import org.apache.hadoop.hbase.security.User;
@@ -182,13 +187,6 @@ import org.apache.hadoop.util.StringUtils;
 import org.apache.htrace.Trace;
 import org.apache.htrace.TraceScope;
 
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Optional;
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import com.google.common.io.Closeables;
-
 
 @SuppressWarnings("deprecation")
 @InterfaceAudience.Private
@@ -3258,7 +3256,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
       long txid;
       if (replay) {
         // use wal key from the original
-        walKey = new ReplayHLogKey(this.getRegionInfo().getEncodedNameAsBytes(),
+        walKey = new WALKey(this.getRegionInfo().getEncodedNameAsBytes(),
           this.htableDescriptor.getTableName(), WALKey.NO_SEQUENCE_ID, now,
           mutation.getClusterIds(), currentNonceGroup, currentNonce, mvcc);
         walKey.setOrigLogSeqNum(batchOp.getReplaySequenceId());
@@ -3277,7 +3275,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
                 writeEntry = mvcc.begin();
               }
               // we use HLogKey here instead of WALKey directly to support legacy coprocessors.
-              walKey = new HLogKey(this.getRegionInfo().getEncodedNameAsBytes(),
+              walKey = new WALKey(this.getRegionInfo().getEncodedNameAsBytes(),
                   this.htableDescriptor.getTableName(), WALKey.NO_SEQUENCE_ID, now,
                   mutation.getClusterIds(), currentNonceGroup, currentNonce, mvcc,
                   this.getReplicationScope());

http://git-wip-us.apache.org/repos/asf/hbase/blob/47a4e343/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java
index f6388fa..2ddb1c0 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java
@@ -19,29 +19,26 @@
 
 package org.apache.hadoop.hbase.regionserver;
 
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+import com.google.protobuf.Message;
+import com.google.protobuf.Service;
+
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 import java.util.NavigableSet;
-import java.util.Optional;
 import java.util.UUID;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 import java.util.regex.Matcher;
 
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Lists;
-import com.google.protobuf.Message;
-import com.google.protobuf.Service;
-
 import org.apache.commons.collections.map.AbstractReferenceMap;
 import org.apache.commons.collections.map.ReferenceMap;
 import org.apache.commons.lang.ClassUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -53,6 +50,8 @@ import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.client.Append;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Durability;
@@ -79,13 +78,12 @@ import org.apache.hadoop.hbase.ipc.RpcServer;
 import org.apache.hadoop.hbase.regionserver.Region.Operation;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
 import org.apache.hadoop.hbase.regionserver.querymatcher.DeleteTracker;
-import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
-import org.apache.hadoop.hbase.security.User;
-import org.apache.hadoop.hbase.wal.WALKey;
 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
+import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.CoprocessorClassLoader;
 import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.wal.WALKey;
 
 /**
  * Implements the coprocessor environment and runtime support for coprocessors
@@ -114,8 +112,6 @@ public class RegionCoprocessorHost
     private Region region;
     private RegionServerServices rsServices;
     ConcurrentMap<String, Object> sharedData;
-    private final boolean useLegacyPre;
-    private final boolean useLegacyPost;
 
     /**
      * Constructor
@@ -129,14 +125,6 @@ public class RegionCoprocessorHost
       this.region = region;
       this.rsServices = services;
       this.sharedData = sharedData;
-      // Pick which version of the WAL related events we'll call.
-      // This way we avoid calling the new version on older RegionObservers so
-      // we can maintain binary compatibility.
-      // See notes in javadoc for RegionObserver
-      useLegacyPre = useLegacyMethod(impl.getClass(), "preWALRestore", ObserverContext.class,
-          HRegionInfo.class, WALKey.class, WALEdit.class);
-      useLegacyPost = useLegacyMethod(impl.getClass(), "postWALRestore", ObserverContext.class,
-          HRegionInfo.class, WALKey.class, WALEdit.class);
     }
 
     /** @return the region */
@@ -422,31 +410,6 @@ public class RegionCoprocessorHost
   }
 
   /**
-   * HBASE-4014 : This is used by coprocessor hooks which are not declared to throw exceptions.
-   *
-   * For example, {@link
-   * org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost#preOpen()} and
-   * {@link org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost#postOpen()} are such hooks.
-   *
-   * See also
-   * {@link org.apache.hadoop.hbase.master.MasterCoprocessorHost#handleCoprocessorThrowable(
-   *    CoprocessorEnvironment, Throwable)}
-   * @param env The coprocessor that threw the exception.
-   * @param e The exception that was thrown.
-   */
-  private void handleCoprocessorThrowableNoRethrow(
-      final CoprocessorEnvironment env, final Throwable e) {
-    try {
-      handleCoprocessorThrowable(env,e);
-    } catch (IOException ioe) {
-      // We cannot throw exceptions from the caller hook, so ignore.
-      LOG.warn(
-        "handleCoprocessorThrowable() threw an IOException while attempting to handle Throwable " +
-        e + ". Ignoring.",e);
-    }
-  }
-
-  /**
    * Invoked before a region open.
    *
    * @throws IOException Signals that an I/O exception has occurred.
@@ -1470,33 +1433,12 @@ public class RegionCoprocessorHost
       @Override
       public void call(RegionObserver oserver, ObserverContext<RegionCoprocessorEnvironment> ctx)
           throws IOException {
-        // Once we don't need to support the legacy call, replace RegionOperation with a version
-        // that's ObserverContext<RegionEnvironment> and avoid this cast.
-        final RegionEnvironment env = (RegionEnvironment)ctx.getEnvironment();
-        if (env.useLegacyPre) {
-          if (logKey instanceof HLogKey) {
-            oserver.preWALRestore(ctx, info, (HLogKey)logKey, logEdit);
-          } else {
-            legacyWarning(oserver.getClass(), "There are wal keys present that are not HLogKey.");
-          }
-        } else {
-          oserver.preWALRestore(ctx, info, logKey, logEdit);
-        }
+        oserver.preWALRestore(ctx, info, logKey, logEdit);
       }
     });
   }
 
   /**
-   * @return true if default behavior should be bypassed, false otherwise
-   * @deprecated use {@link #preWALRestore(HRegionInfo, WALKey, WALEdit)}; as of 2.0, remove in 3.0
-   */
-  @Deprecated
-  public boolean preWALRestore(final HRegionInfo info, final HLogKey logKey,
-      final WALEdit logEdit) throws IOException {
-    return preWALRestore(info, (WALKey)logKey, logEdit);
-  }
-
-  /**
    * @param info
    * @param logKey
    * @param logEdit
@@ -1508,32 +1450,12 @@ public class RegionCoprocessorHost
       @Override
       public void call(RegionObserver oserver, ObserverContext<RegionCoprocessorEnvironment> ctx)
           throws IOException {
-        // Once we don't need to support the legacy call, replace RegionOperation with a version
-        // that's ObserverContext<RegionEnvironment> and avoid this cast.
-        final RegionEnvironment env = (RegionEnvironment)ctx.getEnvironment();
-        if (env.useLegacyPost) {
-          if (logKey instanceof HLogKey) {
-            oserver.postWALRestore(ctx, info, (HLogKey)logKey, logEdit);
-          } else {
-            legacyWarning(oserver.getClass(), "There are wal keys present that are not HLogKey.");
-          }
-        } else {
-          oserver.postWALRestore(ctx, info, logKey, logEdit);
-        }
+        oserver.postWALRestore(ctx, info, logKey, logEdit);
       }
     });
   }
 
   /**
-   * @deprecated use {@link #postWALRestore(HRegionInfo, WALKey, WALEdit)}; as of 2.0, remove in 3.0
-   */
-  @Deprecated
-  public void postWALRestore(final HRegionInfo info, final HLogKey logKey, final WALEdit logEdit)
-      throws IOException {
-    postWALRestore(info, (WALKey)logKey, logEdit);
-  }
-
-  /**
    * @param familyPaths pairs of { CF, file path } submitted for bulk load
    * @return true if the default operation should be bypassed
    * @throws IOException

http://git-wip-us.apache.org/repos/asf/hbase/blob/47a4e343/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java
index b4f0a29..de5c588 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java
@@ -661,12 +661,8 @@ public class FSHLog extends AbstractFSWAL<Writer> {
     } finally {
       rollWriterLock.unlock();
     }
-    try {
-      if (lowReplication || writer != null && writer.getLength() > logrollsize) {
-        requestLogRoll(lowReplication);
-      }
-    } catch (IOException e) {
-      LOG.warn("Writer.getLength() failed; continuing", e);
+    if (lowReplication || writer != null && writer.getLength() > logrollsize) {
+      requestLogRoll(lowReplication);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/47a4e343/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogKey.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogKey.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogKey.java
deleted file mode 100644
index d7bf4a5..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogKey.java
+++ /dev/null
@@ -1,285 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.regionserver.wal;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.EOFException;
-import java.io.IOException;
-import java.util.Iterator;
-import java.util.List;
-import java.util.NavigableMap;
-import java.util.UUID;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.HBaseInterfaceAudience;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.wal.WALKey;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableUtils;
-
-import com.google.common.annotations.VisibleForTesting;
-
-/**
- * A Key for an entry in the change log.
- *
- * The log intermingles edits to many tables and rows, so each log entry
- * identifies the appropriate table and row.  Within a table and row, they're
- * also sorted.
- *
- * <p>Some Transactional edits (START, COMMIT, ABORT) will not have an
- * associated row.
- * @deprecated use WALKey. Deprecated as of 1.0 (HBASE-12522). Remove in 2.0
- */
-@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.REPLICATION)
-@Deprecated
-public class HLogKey extends WALKey implements Writable {
-  private static final Log LOG = LogFactory.getLog(HLogKey.class);
-
-  public HLogKey() {
-    super();
-  }
-
-  @VisibleForTesting
-  public HLogKey(final byte[] encodedRegionName, final TableName tablename, long logSeqNum,
-      final long now, UUID clusterId) {
-    super(encodedRegionName, tablename, logSeqNum, now, clusterId);
-  }
-
-  public HLogKey(final byte[] encodedRegionName, final TableName tablename) {
-    super(encodedRegionName, tablename, null);
-  }
-
-  @VisibleForTesting
-  public HLogKey(final byte[] encodedRegionName, final TableName tablename, final long now) {
-    super(encodedRegionName, tablename, now);
-  }
-
-  @VisibleForTesting
-  public HLogKey(final byte[] encodedRegionName, final TableName tablename, final long now,
-      final NavigableMap<byte[], Integer> replicationScope) {
-    super(encodedRegionName, tablename, now, replicationScope);
-  }
-
-  public HLogKey(final byte[] encodedRegionName, final TableName tablename, final long now,
-      final MultiVersionConcurrencyControl mvcc, final NavigableMap<byte[], Integer> scopes) {
-    super(encodedRegionName, tablename, now, mvcc, scopes);
-  }
-
-  /**
-   * Create the log key for writing to somewhere.
-   * We maintain the tablename mainly for debugging purposes.
-   * A regionName is always a sub-table object.
-   * <p>Used by log splitting and snapshots.
-   *
-   * @param encodedRegionName Encoded name of the region as returned by
-   * <code>HRegionInfo#getEncodedNameAsBytes()</code>.
-   * @param tablename   - name of table
-   * @param logSeqNum   - log sequence number
-   * @param now Time at which this edit was written.
-   * @param clusterIds the clusters that have consumed the change(used in Replication)
-   */
-  public HLogKey(
-      final byte[] encodedRegionName,
-      final TableName tablename,
-      long logSeqNum,
-      final long now,
-      List<UUID> clusterIds,
-      long nonceGroup,
-      long nonce,
-      MultiVersionConcurrencyControl mvcc) {
-    super(encodedRegionName, tablename, logSeqNum, now, clusterIds, nonceGroup, nonce, mvcc);
-  }
-
-  /**
-   * Create the log key for writing to somewhere.
-   * We maintain the tablename mainly for debugging purposes.
-   * A regionName is always a sub-table object.
-   * <p>Used by log splitting and snapshots.
-   *
-   * @param encodedRegionName Encoded name of the region as returned by
-   * <code>HRegionInfo#getEncodedNameAsBytes()</code>.
-   * @param tablename   - name of table
-   * @param logSeqNum   - log sequence number
-   * @param now Time at which this edit was written.
-   * @param clusterIds the clusters that have consumed the change(used in Replication)
-   * @param nonceGroup the noncegroup
-   * @param nonce      the nonce
-   * @param replicationScope the replicationScope of the non-default column families' of the region
-   */
-  public HLogKey(
-      final byte[] encodedRegionName,
-      final TableName tablename,
-      long logSeqNum,
-      final long now,
-      List<UUID> clusterIds,
-      long nonceGroup,
-      long nonce,
-      MultiVersionConcurrencyControl mvcc, NavigableMap<byte[], Integer> replicationScope) {
-    super(encodedRegionName, tablename, logSeqNum, now, clusterIds, nonceGroup, nonce, mvcc,
-        replicationScope);
-  }
-
-  /**
-   * Create the log key for writing to somewhere.
-   * We maintain the tablename mainly for debugging purposes.
-   * A regionName is always a sub-table object.
-   *
-   * @param encodedRegionName Encoded name of the region as returned by
-   * <code>HRegionInfo#getEncodedNameAsBytes()</code>.
-   * @param tablename
-   * @param now Time at which this edit was written.
-   * @param clusterIds the clusters that have consumed the change(used in Replication)
-   * @param nonceGroup
-   * @param nonce
-   */
-  public HLogKey(final byte[] encodedRegionName,
-                 final TableName tablename,
-                 final long now,
-                 List<UUID> clusterIds,
-                 long nonceGroup,
-                 long nonce,
-                 final MultiVersionConcurrencyControl mvcc) {
-    super(encodedRegionName, tablename, now, clusterIds, nonceGroup, nonce, mvcc);
-  }
-
-  /**
-   * Create the log key for writing to somewhere.
-   * We maintain the tablename mainly for debugging purposes.
-   * A regionName is always a sub-table object.
-   *
-   * @param encodedRegionName Encoded name of the region as returned by
-   * <code>HRegionInfo#getEncodedNameAsBytes()</code>.
-   * @param tablename
-   * @param logSeqNum
-   * @param nonceGroup
-   * @param nonce
-   */
-  public HLogKey(final byte [] encodedRegionName, final TableName tablename, long logSeqNum,
-      long nonceGroup, long nonce, MultiVersionConcurrencyControl mvcc) {
-    super(encodedRegionName, tablename, logSeqNum, nonceGroup, nonce, mvcc);
-  }
-
-  /**
-   * @deprecated Don't use these Writables methods. Use PB instead.
-   */
-  @Override
-  @Deprecated
-  public void write(DataOutput out) throws IOException {
-    LOG.warn("HLogKey is being serialized to writable - only expected in test code");
-    WritableUtils.writeVInt(out, VERSION.code);
-    if (compressionContext == null) {
-      Bytes.writeByteArray(out, this.encodedRegionName);
-      Bytes.writeByteArray(out, this.tablename.getName());
-    } else {
-      Compressor.writeCompressed(this.encodedRegionName, 0,
-          this.encodedRegionName.length, out,
-          compressionContext.regionDict);
-      Compressor.writeCompressed(this.tablename.getName(), 0,
-          this.tablename.getName().length, out,
-          compressionContext.tableDict);
-    }
-    out.writeLong(getSequenceId());
-    out.writeLong(this.writeTime);
-    // Don't need to write the clusters information as we are using protobufs from 0.95
-    // Writing only the first clusterId for testing the legacy read
-    Iterator<UUID> iterator = clusterIds.iterator();
-    if(iterator.hasNext()){
-      out.writeBoolean(true);
-      UUID clusterId = iterator.next();
-      out.writeLong(clusterId.getMostSignificantBits());
-      out.writeLong(clusterId.getLeastSignificantBits());
-    } else {
-      out.writeBoolean(false);
-    }
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    Version version = Version.UNVERSIONED;
-    // HLogKey was not versioned in the beginning.
-    // In order to introduce it now, we make use of the fact
-    // that encodedRegionName was written with Bytes.writeByteArray,
-    // which encodes the array length as a vint which is >= 0.
-    // Hence if the vint is >= 0 we have an old version and the vint
-    // encodes the length of encodedRegionName.
-    // If < 0 we just read the version and the next vint is the length.
-    // @see Bytes#readByteArray(DataInput)
-    serializeReplicationScope(false); // writable HLogKey does not contain scopes
-    int len = WritableUtils.readVInt(in);
-    byte[] tablenameBytes = null;
-    if (len < 0) {
-      // what we just read was the version
-      version = Version.fromCode(len);
-      // We only compress V2 of WALkey.
-      // If compression is on, the length is handled by the dictionary
-      if (compressionContext == null || !version.atLeast(Version.COMPRESSED)) {
-        len = WritableUtils.readVInt(in);
-      }
-    }
-    if (compressionContext == null || !version.atLeast(Version.COMPRESSED)) {
-      this.encodedRegionName = new byte[len];
-      in.readFully(this.encodedRegionName);
-      tablenameBytes = Bytes.readByteArray(in);
-    } else {
-      this.encodedRegionName = Compressor.readCompressed(in, compressionContext.regionDict);
-      tablenameBytes = Compressor.readCompressed(in, compressionContext.tableDict);
-    }
-
-    setSequenceId(in.readLong());
-    this.writeTime = in.readLong();
-
-    this.clusterIds.clear();
-    if (version.atLeast(Version.INITIAL)) {
-      if (in.readBoolean()) {
-        // read the older log
-        // Definitely is the originating cluster
-        clusterIds.add(new UUID(in.readLong(), in.readLong()));
-      }
-    } else {
-      try {
-        // dummy read (former byte cluster id)
-        in.readByte();
-      } catch(EOFException e) {
-        // Means it's a very old key, just continue
-        if (LOG.isTraceEnabled()) LOG.trace(e);
-      }
-    }
-    try {
-      this.tablename = TableName.valueOf(tablenameBytes);
-    } catch (IllegalArgumentException iae) {
-      if (Bytes.toString(tablenameBytes).equals(TableName.OLD_META_STR)) {
-        // It is a pre-namespace meta table edit, continue with new format.
-        LOG.info("Got an old .META. edit, continuing with new format ");
-        this.tablename = TableName.META_TABLE_NAME;
-        this.encodedRegionName = HRegionInfo.FIRST_META_REGIONINFO.getEncodedNameAsBytes();
-      } else if (Bytes.toString(tablenameBytes).equals(TableName.OLD_ROOT_STR)) {
-        this.tablename = TableName.OLD_ROOT_TABLE_NAME;
-         throw iae;
-      } else throw iae;
-    }
-    // Do not need to read the clusters information as we are using protobufs from 0.95
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/47a4e343/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ReaderBase.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ReaderBase.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ReaderBase.java
index 2114cc4..b2d6751 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ReaderBase.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ReaderBase.java
@@ -23,18 +23,18 @@ import java.io.IOException;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.io.util.LRUDictionary;
 import org.apache.hadoop.hbase.util.FSUtils;
-
 import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
 import org.apache.hadoop.hbase.wal.WAL.Entry;
+import org.apache.hadoop.hbase.wal.WALKey;
 
 @InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
 public abstract class ReaderBase implements AbstractFSWALProvider.Reader {
@@ -91,9 +91,7 @@ public abstract class ReaderBase implements AbstractFSWALProvider.Reader {
   public Entry next(Entry reuse) throws IOException {
     Entry e = reuse;
     if (e == null) {
-      // we use HLogKey here instead of WALKey directly to support legacy coprocessors,
-      // seqencefile based readers, and HLogInputFormat.
-      e = new Entry(new HLogKey(), new WALEdit());
+      e = new Entry(new WALKey(), new WALEdit());
     }
     if (compressionContext != null) {
       e.setCompressionContext(compressionContext);

http://git-wip-us.apache.org/repos/asf/hbase/blob/47a4e343/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ReplayHLogKey.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ReplayHLogKey.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ReplayHLogKey.java
deleted file mode 100644
index d5a1561..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ReplayHLogKey.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.regionserver.wal;
-
-import java.util.List;
-import java.util.UUID;
-
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl;
-
-/**
- * An HLogKey specific to WalEdits coming from replay.
- */
-@InterfaceAudience.Private
-public class ReplayHLogKey extends HLogKey {
-
-  public ReplayHLogKey(final byte [] encodedRegionName, final TableName tablename,
-      final long now, List<UUID> clusterIds, long nonceGroup, long nonce,
-      MultiVersionConcurrencyControl mvcc) {
-    super(encodedRegionName, tablename, now, clusterIds, nonceGroup, nonce, mvcc);
-  }
-
-  public ReplayHLogKey(final byte [] encodedRegionName, final TableName tablename,
-      long logSeqNum, final long now, List<UUID> clusterIds, long nonceGroup, long nonce,
-      MultiVersionConcurrencyControl mvcc) {
-    super(encodedRegionName, tablename, logSeqNum, now, clusterIds, nonceGroup, nonce, mvcc);
-  }
-
-  /**
-   * Returns the original sequence id
-   * @return long the new assigned sequence number
-   */
-  @Override
-  public long getSequenceId() {
-    return this.getOrigLogSeqNum();
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/47a4e343/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SequenceFileLogReader.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SequenceFileLogReader.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SequenceFileLogReader.java
deleted file mode 100644
index e41e1c3..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SequenceFileLogReader.java
+++ /dev/null
@@ -1,309 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.regionserver.wal;
-
-import java.io.IOException;
-import java.lang.reflect.Field;
-import java.util.NavigableMap;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.HBaseInterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.wal.WAL.Entry;
-import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
-import org.apache.hadoop.io.SequenceFile;
-import org.apache.hadoop.io.SequenceFile.Metadata;
-import org.apache.hadoop.io.Text;
-
-@InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX,
-  HBaseInterfaceAudience.CONFIG})
-public class SequenceFileLogReader extends ReaderBase {
-  private static final Log LOG = LogFactory.getLog(SequenceFileLogReader.class);
-
-  // Legacy stuff from pre-PB WAL metadata.
-  private static final Text WAL_VERSION_KEY = new Text("version");
-  // Let the version be 1.  Let absence of a version meta tag be old, version 0.
-  // Set this version '1' to be the version that introduces compression,
-  // the COMPRESSION_VERSION.
-  private static final int COMPRESSION_VERSION = 1;
-  private static final Text WAL_COMPRESSION_TYPE_KEY = new Text("compression.type");
-  private static final Text DICTIONARY_COMPRESSION_TYPE = new Text("dictionary");
-
-  /**
-   * Hack just to set the correct file length up in SequenceFile.Reader.
-   * See HADOOP-6307.  The below is all about setting the right length on the
-   * file we are reading.  fs.getFileStatus(file).getLen() is passed down to
-   * a private SequenceFile.Reader constructor.  This won't work.  Need to do
-   * the available on the stream.  The below is ugly.  It makes getPos, the
-   * first time its called, return length of the file -- i.e. tell a lie -- just
-   * so this line up in SF.Reader's constructor ends up with right answer:
-   *
-   *         this.end = in.getPos() + length;
-   *
-   */
-  private static class WALReader extends SequenceFile.Reader {
-
-    WALReader(final FileSystem fs, final Path p, final Configuration c)
-    throws IOException {
-      super(fs, p, c);
-    }
-
-    @Override
-    protected FSDataInputStream openFile(FileSystem fs, Path file,
-      int bufferSize, long length)
-    throws IOException {
-      return new WALReaderFSDataInputStream(super.openFile(fs, file,
-        bufferSize, length), length);
-    }
-
-    /**
-     * Override just so can intercept first call to getPos.
-     */
-    static class WALReaderFSDataInputStream extends FSDataInputStream {
-      private boolean firstGetPosInvocation = true;
-      private long length;
-
-      WALReaderFSDataInputStream(final FSDataInputStream is, final long l)
-      throws IOException {
-        super(is);
-        this.length = l;
-      }
-
-      // This section can be confusing.  It is specific to how HDFS works.
-      // Let me try to break it down.  This is the problem:
-      //
-      //  1. HDFS DataNodes update the NameNode about a filename's length
-      //     on block boundaries or when a file is closed. Therefore,
-      //     if an RS dies, then the NN's fs.getLength() can be out of date
-      //  2. this.in.available() would work, but it returns int &
-      //     therefore breaks for files > 2GB (happens on big clusters)
-      //  3. DFSInputStream.getFileLength() gets the actual length from the DNs
-      //  4. DFSInputStream is wrapped 2 levels deep : this.in.in
-      //
-      // So, here we adjust getPos() using getFileLength() so the
-      // SequenceFile.Reader constructor (aka: first invocation) comes out
-      // with the correct end of the file:
-      //         this.end = in.getPos() + length;
-      @Override
-      public long getPos() throws IOException {
-        if (this.firstGetPosInvocation) {
-          this.firstGetPosInvocation = false;
-          long adjust = 0;
-          HdfsDataInputStream hdfsDataInputStream = null;
-          try {
-            if (this.in.getClass().getName().endsWith("HdfsDataInputStream")
-                || this.in.getClass().getName().endsWith("DFSInputStream")) {
-              hdfsDataInputStream = (HdfsDataInputStream) this.getWrappedStream();
-              long realLength = hdfsDataInputStream.getVisibleLength();
-              assert(realLength >= this.length);
-              adjust = realLength - this.length;
-            } else {
-              LOG.info(
-                "Input stream class: " + this.in.getClass().getName() + ", not adjusting length");
-            }
-          } catch (Exception e) {
-            LOG.warn("Error while trying to get accurate file length.  "
-                + "Truncation / data loss may occur if RegionServers die.",
-              e);
-            throw new IOException(e);
-          }
-          return adjust + super.getPos();
-        }
-        return super.getPos();
-      }
-    }
-  }
-
-  // Protected for tests.
-  protected SequenceFile.Reader reader;
-  long entryStart = 0; // needed for logging exceptions
-
-  public SequenceFileLogReader() {
-    super();
-  }
-
-  @Override
-  public void close() throws IOException {
-    try {
-      if (reader != null) {
-        this.reader.close();
-        this.reader = null;
-      }
-    } catch (IOException ioe) {
-      throw addFileInfoToException(ioe);
-    }
-  }
-
-  @Override
-  public long getPosition() throws IOException {
-    return reader != null ? reader.getPosition() : 0;
-  }
-
-  @Override
-  public void reset() throws IOException {
-    // Resetting the reader lets us see newly added data if the file is being written to
-    // We also keep the same compressionContext which was previously populated for this file
-    reader = new WALReader(fs, path, conf);
-  }
-
-  @Override
-  protected String initReader(FSDataInputStream stream) throws IOException {
-    // We don't use the stream because we have to have the magic stream above.
-    if (stream != null) {
-      stream.close();
-    }
-    reset();
-    return null;
-  }
-  
-  @Override
-  protected void initAfterCompression(String cellCodecClsName) throws IOException {
-    // Nothing to do here
-  }
-
-  @Override
-  protected void initAfterCompression() throws IOException {
-    // Nothing to do here
-  }
-
-  @Override
-  protected boolean hasCompression() {
-    return isWALCompressionEnabled(reader.getMetadata());
-  }
-
-  @Override
-  protected boolean hasTagCompression() {
-    // Tag compression not supported with old SequenceFileLog Reader/Writer
-    return false;
-  }
-
-  /**
-   * Call this method after init() has been executed
-   * @return whether WAL compression is enabled
-   */
-  static boolean isWALCompressionEnabled(final Metadata metadata) {
-    // Check version is >= VERSION?
-    Text txt = metadata.get(WAL_VERSION_KEY);
-    if (txt == null || Integer.parseInt(txt.toString()) < COMPRESSION_VERSION) {
-      return false;
-    }
-    // Now check that compression type is present.  Currently only one value.
-    txt = metadata.get(WAL_COMPRESSION_TYPE_KEY);
-    return txt != null && txt.equals(DICTIONARY_COMPRESSION_TYPE);
-  }
-
-
-  /**
-   * fill in the passed entry with teh next key/value.
-   * Note that because this format deals with our legacy storage, the provided
-   * Entery MUST use an {@link HLogKey} for the key.
-   * @return boolean indicating if the contents of Entry have been filled in.
-   */
-  @Override
-  protected boolean readNext(Entry e) throws IOException {
-    try {
-      if (!(e.getKey() instanceof HLogKey)) {
-        final IllegalArgumentException exception = new IllegalArgumentException(
-            "SequenceFileLogReader only works when given entries that have HLogKey for keys. This" +
-            " one had '" + e.getKey().getClass() + "'");
-        LOG.error("We need to use the legacy SequenceFileLogReader to handle a " +
-            " pre-0.96 style WAL, but HBase internals failed to use the deprecated HLogKey class." +
-            " This is a bug; please file an issue or email the developer mailing list. You will " +
-            "need the following exception details when seeking help from the HBase community.",
-            exception);
-        throw exception;
-      }
-      boolean hasNext = this.reader.next((HLogKey)e.getKey(), e.getEdit());
-      if (!hasNext) return false;
-      // Scopes are probably in WAL edit, move to key
-      NavigableMap<byte[], Integer> scopes = e.getEdit().getAndRemoveScopes();
-      if (scopes != null) {
-        e.getKey().readOlderScopes(scopes);
-      }
-      return true;
-    } catch (IOException ioe) {
-      throw addFileInfoToException(ioe);
-    }
-  }
-
-  @Override
-  protected void seekOnFs(long pos) throws IOException {
-    try {
-      reader.seek(pos);
-    } catch (IOException ioe) {
-      throw addFileInfoToException(ioe);
-    }
-  }
-
-  protected IOException addFileInfoToException(final IOException ioe)
-  throws IOException {
-    long pos = -1;
-    try {
-      pos = getPosition();
-    } catch (IOException e) {
-      LOG.warn("Failed getting position to add to throw", e);
-    }
-
-    // See what SequenceFile.Reader thinks is the end of the file
-    long end = Long.MAX_VALUE;
-    try {
-      Field fEnd = SequenceFile.Reader.class.getDeclaredField("end");
-      fEnd.setAccessible(true);
-      end = fEnd.getLong(this.reader);
-    } catch(NoSuchFieldException nfe) {
-       /* reflection failure, keep going */
-      if (LOG.isTraceEnabled()) LOG.trace(nfe);
-    } catch(IllegalAccessException iae) {
-       /* reflection failure, keep going */
-      if (LOG.isTraceEnabled()) LOG.trace(iae);
-    } catch(Exception e) {
-       /* All other cases. Should we handle it more aggressively? */
-       LOG.warn("Unexpected exception when accessing the end field", e);
-    }
- 
-    String msg = (this.path == null? "": this.path.toString()) +
-      ", entryStart=" + entryStart + ", pos=" + pos +
-      ((end == Long.MAX_VALUE) ? "" : ", end=" + end) +
-      ", edit=" + this.edit;
-
-    // Enhance via reflection so we don't change the original class type
-    try {
-      return (IOException) ioe.getClass()
-        .getConstructor(String.class)
-        .newInstance(msg)
-        .initCause(ioe);
-    } catch(NoSuchMethodException nfe) {
-       /* reflection failure, keep going */
-      if (LOG.isTraceEnabled()) LOG.trace(nfe);
-    } catch(IllegalAccessException iae) {
-       /* reflection failure, keep going */
-      if (LOG.isTraceEnabled()) LOG.trace(iae);
-    } catch(Exception e) {
-       /* All other cases. Should we handle it more aggressively? */
-       LOG.warn("Unexpected exception when accessing the end field", e);
-    }
-    return ioe;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/47a4e343/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCoprocessorHost.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCoprocessorHost.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCoprocessorHost.java
index 87019e8..4d7e868 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCoprocessorHost.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCoprocessorHost.java
@@ -23,12 +23,15 @@ package org.apache.hadoop.hbase.regionserver.wal;
 import java.io.IOException;
 import java.util.List;
 
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Coprocessor;
 import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.coprocessor.*;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
+import org.apache.hadoop.hbase.coprocessor.ObserverContext;
+import org.apache.hadoop.hbase.coprocessor.WALCoprocessorEnvironment;
+import org.apache.hadoop.hbase.coprocessor.WALObserver;
 import org.apache.hadoop.hbase.wal.WAL;
 import org.apache.hadoop.hbase.wal.WALKey;
 
@@ -48,9 +51,6 @@ public class WALCoprocessorHost
 
     private final WAL wal;
 
-    final boolean useLegacyPre;
-    final boolean useLegacyPost;
-
     @Override
     public WAL getWAL() {
       return wal;
@@ -70,14 +70,6 @@ public class WALCoprocessorHost
         final WAL wal) {
       super(impl, priority, seq, conf);
       this.wal = wal;
-      // Pick which version of the API we'll call.
-      // This way we avoid calling the new version on older WALObservers so
-      // we can maintain binary compatibility.
-      // See notes in javadoc for WALObserver
-      useLegacyPre = useLegacyMethod(impl.getClass(), "preWALWrite", ObserverContext.class,
-          HRegionInfo.class, WALKey.class, WALEdit.class);
-      useLegacyPost = useLegacyMethod(impl.getClass(), "postWALWrite", ObserverContext.class,
-          HRegionInfo.class, WALKey.class, WALEdit.class);
     }
   }
 
@@ -131,16 +123,7 @@ public class WALCoprocessorHost
         ClassLoader cl = currentThread.getContextClassLoader();
         try {
           currentThread.setContextClassLoader(env.getClassLoader());
-          if (env.useLegacyPre) {
-            if (logKey instanceof HLogKey) {
-              observer.preWALWrite(ctx, info, (HLogKey)logKey, logEdit);
-            } else {
-              legacyWarning(observer.getClass(),
-                  "There are wal keys present that are not HLogKey.");
-            }
-          } else {
-            observer.preWALWrite(ctx, info, logKey, logEdit);
-          }
+          observer.preWALWrite(ctx, info, logKey, logEdit);
         } catch (Throwable e) {
           handleCoprocessorThrowable(env, e);
         } finally {
@@ -175,16 +158,7 @@ public class WALCoprocessorHost
         ClassLoader cl = currentThread.getContextClassLoader();
         try {
           currentThread.setContextClassLoader(env.getClassLoader());
-          if (env.useLegacyPost) {
-            if (logKey instanceof HLogKey) {
-              observer.postWALWrite(ctx, info, (HLogKey)logKey, logEdit);
-            } else {
-              legacyWarning(observer.getClass(),
-                  "There are wal keys present that are not HLogKey.");
-            }
-          } else {
-            observer.postWALWrite(ctx, info, logKey, logEdit);
-          }
+          observer.postWALWrite(ctx, info, logKey, logEdit);
         } catch (Throwable e) {
           handleCoprocessorThrowable(env, e);
         } finally {

http://git-wip-us.apache.org/repos/asf/hbase/blob/47a4e343/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlLists.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlLists.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlLists.java
index 1794464..cbdcea9 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlLists.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlLists.java
@@ -18,6 +18,10 @@
 
 package org.apache.hadoop.hbase.security.access;
 
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.ListMultimap;
+import com.google.common.collect.Lists;
+
 import java.io.ByteArrayInputStream;
 import java.io.DataInput;
 import java.io.DataInputStream;
@@ -39,7 +43,6 @@ import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.TableName;
@@ -70,10 +73,9 @@ import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.io.Text;
-
-import com.google.common.collect.ArrayListMultimap;
-import com.google.common.collect.ListMultimap;
-import com.google.common.collect.Lists;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableFactories;
+import org.apache.hadoop.io.WritableUtils;
 
 /**
  * Maintains lists of permission grants to users and groups to allow for
@@ -596,13 +598,36 @@ public class AccessControlLists {
     return ProtobufUtil.prependPBMagic(AccessControlUtil.toUserTablePermissions(perms).toByteArray());
   }
 
+  // This is part of the old HbaseObjectWritableFor96Migration.
+  private static final int LIST_CODE = 61;
+
+  private static final int WRITABLE_CODE = 14;
+
+  private static final int WRITABLE_NOT_ENCODED = 0;
+
+  private static List<TablePermission> readWritablePermissions(DataInput in, Configuration conf)
+      throws IOException, ClassNotFoundException {
+    assert WritableUtils.readVInt(in) == LIST_CODE;
+    int length = in.readInt();
+    List<TablePermission> list = new ArrayList<>(length);
+    for (int i = 0; i < length; i++) {
+      assert WritableUtils.readVInt(in) == WRITABLE_CODE;
+      assert WritableUtils.readVInt(in) == WRITABLE_NOT_ENCODED;
+      String className = Text.readString(in);
+      Class<? extends Writable> clazz = conf.getClassByName(className).asSubclass(Writable.class);
+      Writable instance = WritableFactories.newInstance(clazz, conf);
+      instance.readFields(in);
+      list.add((TablePermission) instance);
+    }
+    return list;
+  }
+
   /**
-   * Reads a set of permissions as {@link org.apache.hadoop.io.Writable} instances
-   * from the input stream.
+   * Reads a set of permissions as {@link org.apache.hadoop.io.Writable} instances from the input
+   * stream.
    */
   public static ListMultimap<String, TablePermission> readPermissions(byte[] data,
-      Configuration conf)
-          throws DeserializationException {
+      Configuration conf) throws DeserializationException {
     if (ProtobufUtil.isPBMagicPrefix(data)) {
       int pblen = ProtobufUtil.lengthOfPBMagic();
       try {
@@ -614,17 +639,18 @@ public class AccessControlLists {
         throw new DeserializationException(e);
       }
     } else {
-      ListMultimap<String,TablePermission> perms = ArrayListMultimap.create();
+      // TODO: We have to re-write non-PB data as PB encoded. Otherwise we will carry old Writables
+      // forever (here and a couple of other places).
+      ListMultimap<String, TablePermission> perms = ArrayListMultimap.create();
       try {
         DataInput in = new DataInputStream(new ByteArrayInputStream(data));
         int length = in.readInt();
-        for (int i=0; i<length; i++) {
+        for (int i = 0; i < length; i++) {
           String user = Text.readString(in);
-          List<TablePermission> userPerms =
-              (List)HbaseObjectWritableFor96Migration.readObject(in, conf);
+          List<TablePermission> userPerms = readWritablePermissions(in, conf);
           perms.putAll(user, userPerms);
         }
-      } catch (IOException e) {
+      } catch (IOException | ClassNotFoundException e) {
         throw new DeserializationException(e);
       }
       return perms;