You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ap...@apache.org on 2017/07/14 22:45:53 UTC

hbase git commit: HBASE-14070 Undoing the use of master's timestamp for meta updates (Sai Teja Ranuva)

Repository: hbase
Updated Branches:
  refs/heads/HBASE-14070.HLC f66976ada -> c90948c8e


HBASE-14070 Undoing the use of master's timestamp for meta updates (Sai Teja Ranuva)

Removes all instances where the timestamp is explicitly set for meta updates so that
updates are instead timestamped through the Clock API.
Co-Authored-By: Sai Teja Ranuva <sr...@uci.edu>

Change-Id: Ic45ef8b67ded802645f359e7d901fa511da93574
Signed-off-by: Apekshit Sharma <ap...@apache.org>


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

Branch: refs/heads/HBASE-14070.HLC
Commit: c90948c8e28224c303337db44944d303f853ad4d
Parents: f66976a
Author: Amit Patel <ia...@gmail.com>
Authored: Thu Jun 29 11:43:16 2017 -0700
Committer: Apekshit Sharma <ap...@apache.org>
Committed: Fri Jul 14 15:44:04 2017 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hbase/MetaTableAccessor.java  | 144 +++++--------------
 .../master/assignment/RegionStateStore.java     |   5 +-
 .../hbase/regionserver/HRegionServer.java       |   3 +-
 .../hadoop/hbase/util/HBaseFsckRepair.java      |   2 +-
 .../hadoop/hbase/TestMetaTableAccessor.java     | 101 +------------
 .../hbase/util/TestHBaseFsckReplicas.java       |   2 +-
 6 files changed, 46 insertions(+), 211 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/c90948c8/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
index de79a67..da15e5d 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
@@ -62,7 +62,6 @@ import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos;
 import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos.MutateRowsRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos.MutateRowsResponse;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.ExceptionUtil;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.PairOfSameType;
@@ -1095,8 +1094,6 @@ public class MetaTableAccessor {
       throws IOException {
     Table metaHTable = getMetaHTable(conn);
     Get get = new Get(tableName.getName()).addColumn(getTableFamily(), getStateColumn());
-    long time = EnvironmentEdgeManager.currentTime();
-    get.setTimeRange(0, time);
     Result result =
         metaHTable.get(get);
     return getTableState(result);
@@ -1300,15 +1297,7 @@ public class MetaTableAccessor {
    */
   public static Put makePutFromRegionInfo(HRegionInfo regionInfo)
     throws IOException {
-    return makePutFromRegionInfo(regionInfo, EnvironmentEdgeManager.currentTime());
-  }
-
-  /**
-   * Generates and returns a Put containing the region into for the catalog table
-   */
-  public static Put makePutFromRegionInfo(HRegionInfo regionInfo, long ts)
-    throws IOException {
-    Put put = new Put(regionInfo.getRegionName(), ts);
+    Put put = new Put(regionInfo.getRegionName());
     addRegionInfo(put, regionInfo);
     return put;
   }
@@ -1318,20 +1307,8 @@ public class MetaTableAccessor {
    * table
    */
   public static Delete makeDeleteFromRegionInfo(HRegionInfo regionInfo) {
-    long now = EnvironmentEdgeManager.currentTime();
-    return makeDeleteFromRegionInfo(regionInfo, now);
-  }
-
-  /**
-   * Generates and returns a Delete containing the region info for the catalog
-   * table
-   */
-  public static Delete makeDeleteFromRegionInfo(HRegionInfo regionInfo, long ts) {
-    if (regionInfo == null) {
-      throw new IllegalArgumentException("Can't make a delete for null region");
-    }
     Delete delete = new Delete(regionInfo.getRegionName());
-    delete.addFamily(getCatalogFamily(), ts);
+    delete.addFamily(getCatalogFamily());
     return delete;
   }
 
@@ -1460,15 +1437,14 @@ public class MetaTableAccessor {
       throws IOException {
     int absoluteIndex = replicaIndexToDeleteFrom + numReplicasToRemove;
     for (byte[] row : metaRows) {
-      long now = EnvironmentEdgeManager.currentTime();
       Delete deleteReplicaLocations = new Delete(row);
       for (int i = replicaIndexToDeleteFrom; i < absoluteIndex; i++) {
         deleteReplicaLocations.addColumns(getCatalogFamily(),
-          getServerColumn(i), now);
+          getServerColumn(i));
         deleteReplicaLocations.addColumns(getCatalogFamily(),
-          getSeqNumColumn(i), now);
+          getSeqNumColumn(i));
         deleteReplicaLocations.addColumns(getCatalogFamily(),
-          getStartCodeColumn(i), now);
+          getStartCodeColumn(i));
       }
       deleteFromMetaTable(connection, deleteReplicaLocations);
     }
@@ -1573,28 +1549,16 @@ public class MetaTableAccessor {
    * Adds a hbase:meta row for each of the specified new regions.
    * @param connection connection we're using
    * @param regionInfos region information list
-   * @throws IOException if problem connecting or updating meta
-   */
-  public static void addRegionsToMeta(Connection connection,
-                                      List<HRegionInfo> regionInfos, int regionReplication)
-    throws IOException {
-    addRegionsToMeta(connection, regionInfos, regionReplication, HConstants.LATEST_TIMESTAMP);
-  }
-  /**
-   * Adds a hbase:meta row for each of the specified new regions.
-   * @param connection connection we're using
-   * @param regionInfos region information list
    * @param regionReplication
-   * @param ts desired timestamp
    * @throws IOException if problem connecting or updating meta
    */
   public static void addRegionsToMeta(Connection connection,
-      List<HRegionInfo> regionInfos, int regionReplication, long ts)
-          throws IOException {
+      List<HRegionInfo> regionInfos, int regionReplication)
+      throws IOException {
     List<Put> puts = new ArrayList<>();
     for (HRegionInfo regionInfo : regionInfos) {
       if (RegionReplicaUtil.isDefaultReplica(regionInfo)) {
-        Put put = makePutFromRegionInfo(regionInfo, ts);
+        Put put = makePutFromRegionInfo(regionInfo);
         // Add empty locations for region replicas so that number of replicas can be cached
         // whenever the primary region is looked up from meta
         for (int i = 1; i < regionReplication; i++) {
@@ -1616,11 +1580,10 @@ public class MetaTableAccessor {
   public static void addDaughter(final Connection connection,
       final HRegionInfo regionInfo, final ServerName sn, final long openSeqNum)
       throws NotAllMetaRegionsOnlineException, IOException {
-    long now = EnvironmentEdgeManager.currentTime();
-    Put put = new Put(regionInfo.getRegionName(), now);
+    Put put = new Put(regionInfo.getRegionName());
     addRegionInfo(put, regionInfo);
     if (sn != null) {
-      addLocation(put, sn, openSeqNum, -1, regionInfo.getReplicaId());
+      addLocation(put, sn, openSeqNum, regionInfo.getReplicaId());
     }
     putToMetaTable(connection, put);
     LOG.info("Added daughter " + regionInfo.getEncodedName() +
@@ -1636,37 +1599,32 @@ public class MetaTableAccessor {
    * @param regionA
    * @param regionB
    * @param sn the location of the region
-   * @param masterSystemTime
    * @param saveBarrier true if need save replication barrier in meta, used for serial replication
    * @throws IOException
    */
   public static void mergeRegions(final Connection connection, HRegionInfo mergedRegion,
-      HRegionInfo regionA, HRegionInfo regionB, ServerName sn, int regionReplication,
-      long masterSystemTime, boolean saveBarrier)
+      HRegionInfo regionA, HRegionInfo regionB, ServerName sn, int regionReplication, boolean saveBarrier)
           throws IOException {
     Table meta = getMetaHTable(connection);
     try {
       HRegionInfo copyOfMerged = new HRegionInfo(mergedRegion);
 
-      // use the maximum of what master passed us vs local time.
-      long time = Math.max(EnvironmentEdgeManager.currentTime(), masterSystemTime);
-
       // Put for parent
-      Put putOfMerged = makePutFromRegionInfo(copyOfMerged, time);
+      Put putOfMerged = makePutFromRegionInfo(copyOfMerged);
       putOfMerged.addImmutable(HConstants.CATALOG_FAMILY, HConstants.MERGEA_QUALIFIER,
         regionA.toByteArray());
       putOfMerged.addImmutable(HConstants.CATALOG_FAMILY, HConstants.MERGEB_QUALIFIER,
         regionB.toByteArray());
 
       // Deletes for merging regions
-      Delete deleteA = makeDeleteFromRegionInfo(regionA, time);
-      Delete deleteB = makeDeleteFromRegionInfo(regionB, time);
+      Delete deleteA = makeDeleteFromRegionInfo(regionA);
+      Delete deleteB = makeDeleteFromRegionInfo(regionB);
 
       // The merged is a new region, openSeqNum = 1 is fine. ServerName may be null
       // if crash after merge happened but before we got to here.. means in-memory
       // locations of offlined merged, now-closed, regions is lost. Should be ok. We
       // assign the merged region later.
-      if (sn != null) addLocation(putOfMerged, sn, 1, -1, mergedRegion.getReplicaId());
+      if (sn != null) addLocation(putOfMerged, sn, 1, mergedRegion.getReplicaId());
 
       // Add empty locations for region replicas of the merged region so that number of replicas can
       // be cached whenever the primary region is looked up from meta
@@ -1724,8 +1682,8 @@ public class MetaTableAccessor {
       Put putA = makePutFromRegionInfo(splitA);
       Put putB = makePutFromRegionInfo(splitB);
 
-      addLocation(putA, sn, 1, -1, splitA.getReplicaId()); //new regions, openSeqNum = 1 is fine.
-      addLocation(putB, sn, 1, -1, splitB.getReplicaId());
+      addLocation(putA, sn, 1, splitA.getReplicaId()); //new regions, openSeqNum = 1 is fine.
+      addLocation(putB, sn, 1, splitB.getReplicaId());
 
       // Add empty locations for region replicas of daughters so that number of replicas can be
       // cached whenever the primary region is looked up from meta
@@ -1773,8 +1731,7 @@ public class MetaTableAccessor {
    * @param state new state
    */
   public static Put makePutFromTableState(TableState state) {
-    long time = EnvironmentEdgeManager.currentTime();
-    Put put = new Put(state.getTableName().getName(), time);
+    Put put = new Put(state.getTableName().getName());
     put.addColumn(getTableFamily(), getStateColumn(), state.convert().toByteArray());
     return put;
   }
@@ -1786,9 +1743,8 @@ public class MetaTableAccessor {
    */
   public static void deleteTableState(Connection connection, TableName table)
       throws IOException {
-    long time = EnvironmentEdgeManager.currentTime();
     Delete delete = new Delete(table.getName());
-    delete.addColumns(getTableFamily(), getStateColumn(), time);
+    delete.addColumns(getTableFamily(), getStateColumn());
     deleteFromMetaTable(connection, delete);
     LOG.info("Deleted table " + table + " state from META");
   }
@@ -1874,14 +1830,12 @@ public class MetaTableAccessor {
    * @param regionInfo region to update location of
    * @param openSeqNum the latest sequence number obtained when the region was open
    * @param sn Server name
-   * @param masterSystemTime wall clock time from master if passed in the open region RPC or -1
    * @throws IOException
    */
   public static void updateRegionLocation(Connection connection,
-                                          HRegionInfo regionInfo, ServerName sn, long openSeqNum,
-                                          long masterSystemTime)
+                                          HRegionInfo regionInfo, ServerName sn, long openSeqNum)
     throws IOException {
-    updateLocation(connection, regionInfo, sn, openSeqNum, masterSystemTime);
+    updateLocation(connection, regionInfo, sn, openSeqNum);
   }
 
   /**
@@ -1915,22 +1869,16 @@ public class MetaTableAccessor {
    * @param regionInfo region to update location of
    * @param sn Server name
    * @param openSeqNum the latest sequence number obtained when the region was open
-   * @param masterSystemTime wall clock time from master if passed in the open region RPC or -1
    * @throws IOException In particular could throw {@link java.net.ConnectException}
    * if the server is down on other end.
    */
   private static void updateLocation(final Connection connection,
-                                     HRegionInfo regionInfo, ServerName sn, long openSeqNum,
-                                     long masterSystemTime)
+                                     HRegionInfo regionInfo, ServerName sn, long openSeqNum)
     throws IOException {
-
-    // use the maximum of what master passed us vs local time.
-    long time = Math.max(EnvironmentEdgeManager.currentTime(), masterSystemTime);
-
     // region replicas are kept in the primary region's row
-    Put put = new Put(getMetaKeyForRegion(regionInfo), time);
+    Put put = new Put(getMetaKeyForRegion(regionInfo));
     addRegionInfo(put, regionInfo);
-    addLocation(put, sn, openSeqNum, time, regionInfo.getReplicaId());
+    addLocation(put, sn, openSeqNum, regionInfo.getReplicaId());
     putToMetaTable(connection, put);
     LOG.info("Updated row " + regionInfo.getRegionNameAsString() +
       " with server=" + sn);
@@ -1945,9 +1893,8 @@ public class MetaTableAccessor {
   public static void deleteRegion(Connection connection,
                                   HRegionInfo regionInfo)
     throws IOException {
-    long time = EnvironmentEdgeManager.currentTime();
     Delete delete = new Delete(regionInfo.getRegionName());
-    delete.addFamily(getCatalogFamily(), time);
+    delete.addFamily(getCatalogFamily());
     deleteFromMetaTable(connection, delete);
     LOG.info("Deleted " + regionInfo.getRegionNameAsString());
   }
@@ -1960,20 +1907,10 @@ public class MetaTableAccessor {
    */
   public static void deleteRegions(Connection connection,
                                    List<HRegionInfo> regionsInfo) throws IOException {
-    deleteRegions(connection, regionsInfo, EnvironmentEdgeManager.currentTime());
-  }
-  /**
-   * Deletes the specified regions from META.
-   * @param connection connection we're using
-   * @param regionsInfo list of regions to be deleted from META
-   * @throws IOException
-   */
-  public static void deleteRegions(Connection connection, List<HRegionInfo> regionsInfo, long ts)
-  throws IOException {
     List<Delete> deletes = new ArrayList<>(regionsInfo.size());
     for (HRegionInfo hri: regionsInfo) {
       Delete e = new Delete(hri.getRegionName());
-      e.addFamily(getCatalogFamily(), ts);
+      e.addFamily(getCatalogFamily());
       deletes.add(e);
     }
     deleteFromMetaTable(connection, deletes);
@@ -2023,15 +1960,14 @@ public class MetaTableAccessor {
   public static void overwriteRegions(Connection connection,
       List<HRegionInfo> regionInfos, int regionReplication) throws IOException {
     // use master time for delete marker and the Put
-    long now = EnvironmentEdgeManager.currentTime();
-    deleteRegions(connection, regionInfos, now);
+    deleteRegions(connection, regionInfos);
     // Why sleep? This is the easiest way to ensure that the previous deletes does not
     // eclipse the following puts, that might happen in the same ts from the server.
     // See HBASE-9906, and HBASE-9879. Once either HBASE-9879, HBASE-8770 is fixed,
     // or HBASE-9905 is fixed and meta uses seqIds, we do not need the sleep.
     //
     // HBASE-13875 uses master timestamp for the mutations. The 20ms sleep is not needed
-    addRegionsToMeta(connection, regionInfos, regionReplication, now+1);
+    addRegionsToMeta(connection, regionInfos, regionReplication);
     LOG.info("Overwritten " + regionInfos.size() + " regions to Meta");
     if (LOG.isDebugEnabled()) {
       LOG.debug("Overwritten regions: " + regionInfos);
@@ -2046,10 +1982,9 @@ public class MetaTableAccessor {
    */
   public static void deleteMergeQualifiers(Connection connection,
                                            final HRegionInfo mergedRegion) throws IOException {
-    long time = EnvironmentEdgeManager.currentTime();
     Delete delete = new Delete(mergedRegion.getRegionName());
-    delete.addColumns(getCatalogFamily(), HConstants.MERGEA_QUALIFIER, time);
-    delete.addColumns(getCatalogFamily(), HConstants.MERGEB_QUALIFIER, time);
+    delete.addColumns(getCatalogFamily(), HConstants.MERGEA_QUALIFIER);
+    delete.addColumns(getCatalogFamily(), HConstants.MERGEB_QUALIFIER);
     deleteFromMetaTable(connection, delete);
     LOG.info("Deleted references in merged region "
       + mergedRegion.getRegionNameAsString() + ", qualifier="
@@ -2064,25 +1999,20 @@ public class MetaTableAccessor {
     return p;
   }
 
-  public static Put addLocation(final Put p, final ServerName sn, long openSeqNum,
-      long time, int replicaId){
-    if (time <= 0) {
-      time = EnvironmentEdgeManager.currentTime();
-    }
-    p.addImmutable(getCatalogFamily(), getServerColumn(replicaId), time,
+  public static Put addLocation(final Put p, final ServerName sn, long openSeqNum, int replicaId){
+    p.addImmutable(getCatalogFamily(), getServerColumn(replicaId),
       Bytes.toBytes(sn.getHostAndPort()));
-    p.addImmutable(getCatalogFamily(), getStartCodeColumn(replicaId), time,
+    p.addImmutable(getCatalogFamily(), getStartCodeColumn(replicaId),
       Bytes.toBytes(sn.getStartcode()));
-    p.addImmutable(getCatalogFamily(), getSeqNumColumn(replicaId), time,
+    p.addImmutable(getCatalogFamily(), getSeqNumColumn(replicaId),
       Bytes.toBytes(openSeqNum));
     return p;
   }
 
   public static Put addEmptyLocation(final Put p, int replicaId) {
-    long now = EnvironmentEdgeManager.currentTime();
-    p.addImmutable(getCatalogFamily(), getServerColumn(replicaId), now, null);
-    p.addImmutable(getCatalogFamily(), getStartCodeColumn(replicaId), now, null);
-    p.addImmutable(getCatalogFamily(), getSeqNumColumn(replicaId), now, null);
+    p.addImmutable(getCatalogFamily(), getServerColumn(replicaId), null);
+    p.addImmutable(getCatalogFamily(), getStartCodeColumn(replicaId), null);
+    p.addImmutable(getCatalogFamily(), getSeqNumColumn(replicaId), null);
     return p;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/c90948c8/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateStore.java
index 21e0d9c..99bc632 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateStore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateStore.java
@@ -171,7 +171,7 @@ public class RegionStateStore {
     if (openSeqNum >= 0) {
       Preconditions.checkArgument(state == State.OPEN && regionLocation != null,
           "Open region should be on a server");
-      MetaTableAccessor.addLocation(put, regionLocation, openSeqNum, -1, replicaId);
+      MetaTableAccessor.addLocation(put, regionLocation, openSeqNum, replicaId);
       info.append(", openSeqNum=").append(openSeqNum);
       info.append(", regionLocation=").append(regionLocation);
     } else if (regionLocation != null && !regionLocation.equals(lastHost)) {
@@ -233,8 +233,7 @@ public class RegionStateStore {
       final HRegionInfo hriB, final ServerName serverName)  throws IOException {
     final HTableDescriptor htd = getTableDescriptor(parent.getTable());
     MetaTableAccessor.mergeRegions(master.getConnection(), parent, hriA, hriB, serverName,
-        getRegionReplication(htd), EnvironmentEdgeManager.currentTime(),
-        hasSerialReplicationScope(htd));
+      getRegionReplication(htd), hasSerialReplicationScope(htd));
   }
 
   // ============================================================================================

http://git-wip-us.apache.org/repos/asf/hbase/blob/c90948c8/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
index 8000e89..f49b944 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
@@ -2204,7 +2204,6 @@ public class HRegionServer extends HasThread implements
   public boolean reportRegionStateTransition(final RegionStateTransitionContext context) {
     TransitionCode code = context.getCode();
     long openSeqNum = context.getOpenSeqNum();
-    long masterSystemTime = context.getMasterSystemTime();
     HRegionInfo[] hris = context.getHris();
 
     if (TEST_SKIP_REPORTING_TRANSITION) {
@@ -2223,7 +2222,7 @@ public class HRegionServer extends HasThread implements
         } else {
           try {
             MetaTableAccessor.updateRegionLocation(clusterConnection,
-              hris[0], serverName, openSeqNum, masterSystemTime);
+              hris[0], serverName, openSeqNum);
           } catch (IOException e) {
             LOG.info("Failed to update meta", e);
             return false;

http://git-wip-us.apache.org/repos/asf/hbase/blob/c90948c8/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsckRepair.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsckRepair.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsckRepair.java
index 961e8a0..4e8aa5f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsckRepair.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsckRepair.java
@@ -176,7 +176,7 @@ public class HBaseFsckRepair {
         // see the additional replicas when it is asked to assign. The
         // final value of these columns will be different and will be updated
         // by the actual regionservers that start hosting the respective replicas
-        MetaTableAccessor.addLocation(put, sn, sn.getStartcode(), -1, i);
+        MetaTableAccessor.addLocation(put, sn, sn.getStartcode(), i);
       }
     }
     meta.put(put);

http://git-wip-us.apache.org/repos/asf/hbase/blob/c90948c8/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessor.java
index f921352..7444308 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessor.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessor.java
@@ -363,20 +363,20 @@ public class TestMetaTableAccessor {
 
     Table meta = MetaTableAccessor.getMetaHTable(connection);
     try {
-      MetaTableAccessor.updateRegionLocation(connection, primary, serverName0, seqNum0, -1);
+      MetaTableAccessor.updateRegionLocation(connection, primary, serverName0, seqNum0);
 
       // assert that the server, startcode and seqNum columns are there for the primary region
       assertMetaLocation(meta, primary.getRegionName(), serverName0, seqNum0, 0, true);
 
       // add replica = 1
-      MetaTableAccessor.updateRegionLocation(connection, replica1, serverName1, seqNum1, -1);
+      MetaTableAccessor.updateRegionLocation(connection, replica1, serverName1, seqNum1);
       // check whether the primary is still there
       assertMetaLocation(meta, primary.getRegionName(), serverName0, seqNum0, 0, true);
       // now check for replica 1
       assertMetaLocation(meta, primary.getRegionName(), serverName1, seqNum1, 1, true);
 
       // add replica = 1
-      MetaTableAccessor.updateRegionLocation(connection, replica100, serverName100, seqNum100, -1);
+      MetaTableAccessor.updateRegionLocation(connection, replica100, serverName100, seqNum100);
       // check whether the primary is still there
       assertMetaLocation(meta, primary.getRegionName(), serverName0, seqNum0, 0, true);
       // check whether the replica 1 is still there
@@ -482,8 +482,7 @@ public class TestMetaTableAccessor {
       List<HRegionInfo> regionInfos = Lists.newArrayList(parentA, parentB);
       MetaTableAccessor.addRegionsToMeta(connection, regionInfos, 3);
 
-      MetaTableAccessor.mergeRegions(connection, merged, parentA, parentB, serverName0, 3,
-          HConstants.LATEST_TIMESTAMP, false);
+      MetaTableAccessor.mergeRegions(connection, merged, parentA, parentB, serverName0, 3, false);
 
       assertEmptyMetaLocation(meta, merged.getRegionName(), 1);
       assertEmptyMetaLocation(meta, merged.getRegionName(), 2);
@@ -536,98 +535,6 @@ public class TestMetaTableAccessor {
     table.close();
   }
 
-  /**
-   * Tests whether maximum of masters system time versus RSs local system time is used
-   */
-  @Test
-  public void testMastersSystemTimeIsUsedInUpdateLocations() throws IOException {
-    long regionId = System.currentTimeMillis();
-    HRegionInfo regionInfo = new HRegionInfo(TableName.valueOf(name.getMethodName()),
-      HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW, false, regionId, 0);
-
-    ServerName sn = ServerName.valueOf("bar", 0, 0);
-    Table meta = MetaTableAccessor.getMetaHTable(connection);
-    try {
-      List<HRegionInfo> regionInfos = Lists.newArrayList(regionInfo);
-      MetaTableAccessor.addRegionsToMeta(connection, regionInfos, 1);
-
-      long masterSystemTime = EnvironmentEdgeManager.currentTime() + 123456789;
-      MetaTableAccessor.updateRegionLocation(connection, regionInfo, sn, 1, masterSystemTime);
-
-      Get get = new Get(regionInfo.getRegionName());
-      Result result = meta.get(get);
-      Cell serverCell = result.getColumnLatestCell(HConstants.CATALOG_FAMILY,
-          MetaTableAccessor.getServerColumn(0));
-      Cell startCodeCell = result.getColumnLatestCell(HConstants.CATALOG_FAMILY,
-        MetaTableAccessor.getStartCodeColumn(0));
-      Cell seqNumCell = result.getColumnLatestCell(HConstants.CATALOG_FAMILY,
-        MetaTableAccessor.getSeqNumColumn(0));
-      assertNotNull(serverCell);
-      assertNotNull(startCodeCell);
-      assertNotNull(seqNumCell);
-      assertTrue(serverCell.getValueLength() > 0);
-      assertTrue(startCodeCell.getValueLength() > 0);
-      assertTrue(seqNumCell.getValueLength() > 0);
-      assertEquals(masterSystemTime, serverCell.getTimestamp());
-      assertEquals(masterSystemTime, startCodeCell.getTimestamp());
-      assertEquals(masterSystemTime, seqNumCell.getTimestamp());
-    } finally {
-      meta.close();
-    }
-  }
-
-  @Test
-  public void testMastersSystemTimeIsUsedInMergeRegions() throws IOException {
-    long regionId = System.currentTimeMillis();
-    HRegionInfo regionInfoA = new HRegionInfo(TableName.valueOf(name.getMethodName()),
-      HConstants.EMPTY_START_ROW, new byte[] {'a'}, false, regionId, 0);
-    HRegionInfo regionInfoB = new HRegionInfo(TableName.valueOf(name.getMethodName()),
-      new byte[] {'a'}, HConstants.EMPTY_END_ROW, false, regionId, 0);
-    HRegionInfo mergedRegionInfo = new HRegionInfo(TableName.valueOf(name.getMethodName()),
-      HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW, false, regionId, 0);
-
-    ServerName sn = ServerName.valueOf("bar", 0, 0);
-    Table meta = MetaTableAccessor.getMetaHTable(connection);
-    try {
-      List<HRegionInfo> regionInfos = Lists.newArrayList(regionInfoA, regionInfoB);
-      MetaTableAccessor.addRegionsToMeta(connection, regionInfos, 1);
-
-      // write the serverName column with a big current time, but set the masters time as even
-      // bigger. When region merge deletes the rows for regionA and regionB, the serverName columns
-      // should not be seen by the following get
-      long serverNameTime = EnvironmentEdgeManager.currentTime()   + 100000000;
-      long masterSystemTime = EnvironmentEdgeManager.currentTime() + 123456789;
-
-      // write the serverName columns
-      MetaTableAccessor.updateRegionLocation(connection, regionInfoA, sn, 1, serverNameTime);
-
-      // assert that we have the serverName column with expected ts
-      Get get = new Get(mergedRegionInfo.getRegionName());
-      Result result = meta.get(get);
-      Cell serverCell = result.getColumnLatestCell(HConstants.CATALOG_FAMILY,
-          MetaTableAccessor.getServerColumn(0));
-      assertNotNull(serverCell);
-      assertEquals(serverNameTime, serverCell.getTimestamp());
-
-      // now merge the regions, effectively deleting the rows for region a and b.
-      MetaTableAccessor.mergeRegions(connection, mergedRegionInfo,
-        regionInfoA, regionInfoB, sn, 1, masterSystemTime, false);
-
-      result = meta.get(get);
-      serverCell = result.getColumnLatestCell(HConstants.CATALOG_FAMILY,
-          MetaTableAccessor.getServerColumn(0));
-      Cell startCodeCell = result.getColumnLatestCell(HConstants.CATALOG_FAMILY,
-        MetaTableAccessor.getStartCodeColumn(0));
-      Cell seqNumCell = result.getColumnLatestCell(HConstants.CATALOG_FAMILY,
-        MetaTableAccessor.getSeqNumColumn(0));
-      assertNull(serverCell);
-      assertNull(startCodeCell);
-      assertNull(seqNumCell);
-    } finally {
-      meta.close();
-    }
-  }
-
   public static class SpyingRpcSchedulerFactory extends SimpleRpcSchedulerFactory {
     @Override
     public RpcScheduler create(Configuration conf, PriorityFunction priority, Abortable server) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/c90948c8/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckReplicas.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckReplicas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckReplicas.java
index 3d0647e..77bf837 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckReplicas.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckReplicas.java
@@ -181,7 +181,7 @@ public class TestHBaseFsckReplicas extends BaseTestHBaseFsck {
       Collection<ServerName> var = admin.getClusterStatus().getServers();
       ServerName sn = var.toArray(new ServerName[var.size()])[0];
       //add a location with replicaId as 2 (since we already have replicas with replicaid 0 and 1)
-      MetaTableAccessor.addLocation(put, sn, sn.getStartcode(), -1, 2);
+      MetaTableAccessor.addLocation(put, sn, sn.getStartcode(), 2);
       meta.put(put);
       // assign the new replica
       HBaseFsckRepair.fixUnassigned(admin, newHri);