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/11/29 23:41:16 UTC

hbase git commit: HBASE-19376 Fix more binary compatibility problems with branch-1.4 / branch-1

Repository: hbase
Updated Branches:
  refs/heads/branch-1 36a702996 -> 4c413e0c5


HBASE-19376 Fix more binary compatibility problems with branch-1.4 / branch-1


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

Branch: refs/heads/branch-1
Commit: 4c413e0c50777e1d0cbe72f8f081da96063913c0
Parents: 36a7029
Author: Andrew Purtell <ap...@apache.org>
Authored: Wed Nov 29 11:43:07 2017 -0800
Committer: Andrew Purtell <ap...@apache.org>
Committed: Wed Nov 29 14:07:36 2017 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/ClusterStatus.java  | 25 +++++++-
 .../org/apache/hadoop/hbase/client/Result.java  | 15 +++++
 .../apache/hadoop/hbase/filter/FilterList.java  |  9 +++
 .../hbase/mapreduce/TableInputFormatBase.java   | 62 +++++++++++++++++++-
 .../hbase/master/ClusterStatusPublisher.java    |  7 +--
 .../hbase/master/snapshot/SnapshotManager.java  | 13 +++-
 6 files changed, 120 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/4c413e0c/hbase-client/src/main/java/org/apache/hadoop/hbase/ClusterStatus.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ClusterStatus.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ClusterStatus.java
index 10637d6..2eb1162 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ClusterStatus.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ClusterStatus.java
@@ -20,12 +20,15 @@
 package org.apache.hadoop.hbase;
 
 import com.google.common.base.Objects;
+import com.google.common.collect.Sets;
+
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
+import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
@@ -41,7 +44,6 @@ import org.apache.hadoop.hbase.protobuf.generated.FSProtos.HBaseVersionFileConte
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType;
-import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.io.VersionedWritable;
 
 
@@ -100,6 +102,26 @@ public class ClusterStatus extends VersionedWritable {
     super();
   }
 
+  @Deprecated
+  public ClusterStatus(String hbaseVersion, String clusterid, List<ServerName> deadServers,
+      ServerName master) {
+    this(hbaseVersion, clusterid, new HashMap<ServerName,ServerLoad>(), deadServers, master,
+      new ArrayList<ServerName>(), new HashSet<RegionState>(), new String[0], null);
+  }
+
+  @Deprecated
+  public ClusterStatus(final String hbaseVersion, final String clusterid,
+      final Map<ServerName,ServerLoad> servers,
+      final Collection<ServerName> deadServers,
+      final ServerName master,
+      final Collection<ServerName> backupMasters,
+      final Map<String,RegionState> rit,
+      final String[ ] masterCoprocessors,
+      final Boolean balancerOn) {
+    this(hbaseVersion, clusterid, servers, deadServers, master, backupMasters, Sets.newHashSet(rit.values()),
+      masterCoprocessors, balancerOn);
+  }
+
   public ClusterStatus(final String hbaseVersion, final String clusterid,
       final Map<ServerName, ServerLoad> servers,
       final Collection<ServerName> deadServers,
@@ -109,7 +131,6 @@ public class ClusterStatus extends VersionedWritable {
       final String[] masterCoprocessors,
       final Boolean balancerOn) {
     this.hbaseVersion = hbaseVersion;
-
     this.liveServers = servers;
     this.deadServers = deadServers;
     this.master = master;

http://git-wip-us.apache.org/repos/asf/hbase/blob/4c413e0c/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Result.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Result.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Result.java
index ffd7695..d14171f 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Result.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Result.java
@@ -869,6 +869,21 @@ public class Result implements CellScannable, CellScanner {
    * @return The complete result that is formed by combining all of the partial results together
    * @throws IOException A complete result cannot be formed because the results in the partial list
    *           come from different rows
+   * @deprecated
+   */
+  @Deprecated
+  public static Result createCompleteResult(List<Result> partialResults)
+      throws IOException {
+    return createCompleteResult((Iterable<Result>)partialResults);
+  }
+
+  /**
+   * Forms a single result from the partial results in the partialResults list. This method is
+   * useful for reconstructing partial results on the client side.
+   * @param partialResults iterable of partial results
+   * @return The complete result that is formed by combining all of the partial results together
+   * @throws IOException A complete result cannot be formed because the results in the partial list
+   *           come from different rows
    */
   public static Result createCompleteResult(Iterable<Result> partialResults)
       throws IOException {

http://git-wip-us.apache.org/repos/asf/hbase/blob/4c413e0c/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java
index 8345fcf..8b0e8b2 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java
@@ -301,6 +301,15 @@ final public class FilterList extends Filter {
     return this.reversed;
   }
 
+  /**
+   * @param maxFilters ignored
+   * @deprecated
+   */
+  @Deprecated
+  public String toString(int maxFilters) {
+    return this.filterListBase.toString();
+  }
+
   @Override
   public String toString() {
     return this.filterListBase.toString();

http://git-wip-us.apache.org/repos/asf/hbase/blob/4c413e0c/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableInputFormatBase.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableInputFormatBase.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableInputFormatBase.java
index f4c2f9b..c5c00ce 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableInputFormatBase.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableInputFormatBase.java
@@ -24,7 +24,6 @@ import java.net.InetAddress;
 import java.net.InetSocketAddress;
 import java.net.UnknownHostException;
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.HashMap;
 import java.util.List;
 
@@ -32,7 +31,6 @@ import javax.naming.NamingException;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.HConstants;
@@ -437,6 +435,28 @@ extends InputFormat<ImmutableBytesWritable, Result> {
     }
     return res;
   }
+
+  /**
+   * Calculates the number of MapReduce input splits for the map tasks. The number of
+   * MapReduce input splits depends on the average region size.
+   * Make it 'public' for testing
+   * <p>
+   * Deprecated. Former functionality has been replaced by calculateAutoBalancedSplits and
+   * will function differently. Do not use.
+   * <p>
+   * @param list  The list of input splits before balance.
+   * @param context The current job context.
+   * @param average The average size of all regions .
+   * @return The list of input splits.
+   * @throws IOException
+   * @deprecated
+   */
+  @Deprecated
+  public List<InputSplit> calculateRebalancedSplits(List<InputSplit> list, JobContext context,
+      long average) throws IOException {
+    return calculateAutoBalancedSplits(list, average);
+  }
+
   /**
    * Calculates the number of MapReduce input splits for the map tasks. The number of
    * MapReduce input splits depends on the average region size.
@@ -516,6 +536,44 @@ extends InputFormat<ImmutableBytesWritable, Result> {
   }
 
   /**
+   * Deprecated. Do not use.
+   * @param start Start key of the region
+   * @param end End key of the region
+   * @param isText It determines to use text key mode or binary key mode
+   * @return The split point in the region.
+   */
+  @Deprecated
+  public static byte[] getSplitKey(byte[] start, byte[] end, boolean isText) {
+    byte upperLimitByte;
+    byte lowerLimitByte;
+    //Use text mode or binary mode.
+    if (isText) {
+      //The range of text char set in ASCII is [32,126], the lower limit is space and the upper
+      // limit is '~'.
+      upperLimitByte = '~';
+      lowerLimitByte = ' ';
+    } else {
+      upperLimitByte = -1;
+      lowerLimitByte = 0;
+    }
+    // For special case
+    // Example 1 : startkey=null, endkey="hhhqqqwww", splitKey="h"
+    // Example 2 (text key mode): startKey="ffffaaa", endKey=null, splitkey="f~~~~~~"
+    if (start.length == 0 && end.length == 0){
+      return new byte[]{(byte) ((lowerLimitByte + upperLimitByte) / 2)};
+    }
+    if (start.length == 0 && end.length != 0){
+      return new byte[]{ end[0] };
+    }
+    if (start.length != 0 && end.length == 0){
+      byte[] result =new byte[start.length];
+      result[0]=start[0];
+      return result;
+    }
+    return Bytes.split(start, end, false, 1)[1];
+  }
+
+  /**
    * @deprecated mistakenly made public in 0.98.7. scope will change to package-private
    */
   @Deprecated

http://git-wip-us.apache.org/repos/asf/hbase/blob/4c413e0c/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterStatusPublisher.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterStatusPublisher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterStatusPublisher.java
index 4d4358c..00c738b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterStatusPublisher.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterStatusPublisher.java
@@ -161,13 +161,8 @@ public class ClusterStatusPublisher extends ScheduledChore {
     //  hbase:meta server name.
     ClusterStatus cs = new ClusterStatus(VersionInfo.getVersion(),
         master.getMasterFileSystem().getClusterId().toString(),
-        null,
         sns,
-        master.getServerName(),
-        null,
-        null,
-        null,
-        null);
+        master.getServerName());
 
 
     publisher.publish(cs);

http://git-wip-us.apache.org/repos/asf/hbase/blob/4c413e0c/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java
index 3efe984..d5aaca9 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java
@@ -683,7 +683,18 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable
 
   /**
    * Restore the specified snapshot
-   * @param reqSnapshot
+   * @param reqSnapshot the snapshot to restore
+   * @throws IOException
+   */
+  @Deprecated
+  public void restoreSnapshot(SnapshotDescription reqSnapshot) throws IOException {
+    restoreSnapshot(reqSnapshot, false);
+  }
+
+  /**
+   * Restore the specified snapshot
+   * @param reqSnapshot the snapshot to restore
+   * @param restoreAcl whether or not to restore ACLs on the snapshot
    * @throws IOException
    */
   public void restoreSnapshot(SnapshotDescription reqSnapshot, boolean restoreAcl)