You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ch...@apache.org on 2017/09/28 12:30:31 UTC
[01/19] hbase git commit: HBASE-18839 Apply RegionInfo to code base
Repository: hbase
Updated Branches:
refs/heads/master 7f4c3b356 -> a11a35a11
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckTwoRS.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckTwoRS.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckTwoRS.java
index beef02b..6fa455a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckTwoRS.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckTwoRS.java
@@ -34,8 +34,24 @@ import java.util.concurrent.TimeUnit;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.*;
-import org.apache.hadoop.hbase.client.*;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.MetaTableAccessor;
+import org.apache.hadoop.hbase.MiniHBaseCluster;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.ClusterConnection;
+import org.apache.hadoop.hbase.client.Durability;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
import org.apache.hadoop.hbase.io.HFileLink;
import org.apache.hadoop.hbase.io.hfile.HFile;
@@ -43,7 +59,6 @@ import org.apache.hadoop.hbase.io.hfile.HFileContext;
import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
import org.apache.hadoop.hbase.master.RegionState;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.testclassification.MiscTests;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
@@ -57,6 +72,7 @@ import org.junit.experimental.categories.Category;
import org.junit.rules.TestName;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Multimap;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
@Ignore // Until after HBASE-14614 goes in.
@Category({MiscTests.class, LargeTests.class})
@@ -111,10 +127,10 @@ public class TestHBaseFsckTwoRS extends BaseTestHBaseFsck {
@Test(timeout=180000)
public void testFixAssignmentsWhenMETAinTransition() throws Exception {
MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
- admin.unassign(HRegionInfo.FIRST_META_REGIONINFO.getRegionName(), true);
- assignmentManager.offlineRegion(HRegionInfo.FIRST_META_REGIONINFO);
+ admin.unassign(RegionInfoBuilder.FIRST_META_REGIONINFO.getRegionName(), true);
+ assignmentManager.offlineRegion(RegionInfoBuilder.FIRST_META_REGIONINFO);
new MetaTableLocator().deleteMetaLocation(cluster.getMaster().getZooKeeper());
- assertFalse(regionStates.isRegionOnline(HRegionInfo.FIRST_META_REGIONINFO));
+ assertFalse(regionStates.isRegionOnline(RegionInfoBuilder.FIRST_META_REGIONINFO));
HBaseFsck hbck = doFsck(conf, true);
assertErrors(hbck, new HBaseFsck.ErrorReporter.ERROR_CODE[] { HBaseFsck.ErrorReporter.ERROR_CODE.UNKNOWN, HBaseFsck.ErrorReporter.ERROR_CODE.NO_META_REGION,
HBaseFsck.ErrorReporter.ERROR_CODE.NULL_META_REGION });
@@ -134,7 +150,7 @@ public class TestHBaseFsckTwoRS extends BaseTestHBaseFsck {
assertEquals(ROWKEYS.length, countRows());
// Now let's mess it up, by adding a region with a duplicate startkey
- HRegionInfo hriDupe =
+ RegionInfo hriDupe =
createRegion(tbl.getTableDescriptor(), Bytes.toBytes("A"), Bytes.toBytes("A2"));
TEST_UTIL.assignRegion(hriDupe);
@@ -172,7 +188,7 @@ public class TestHBaseFsckTwoRS extends BaseTestHBaseFsck {
assertEquals(ROWKEYS.length, countRows());
// Now let's mess it up, by adding a region with a duplicate startkey
- HRegionInfo hriDupe =
+ RegionInfo hriDupe =
createRegion(tbl.getTableDescriptor(), Bytes.toBytes("A"), Bytes.toBytes("B"));
TEST_UTIL.assignRegion(hriDupe);
@@ -221,7 +237,7 @@ public class TestHBaseFsckTwoRS extends BaseTestHBaseFsck {
assertEquals(ROWKEYS.length, countRows());
// Mess it up by creating an overlap in the metadata
- HRegionInfo hriOverlap =
+ RegionInfo hriOverlap =
createRegion(tbl.getTableDescriptor(), Bytes.toBytes("A2"), Bytes.toBytes("B"));
TEST_UTIL.assignRegion(hriOverlap);
@@ -397,11 +413,11 @@ public class TestHBaseFsckTwoRS extends BaseTestHBaseFsck {
// Mess it up by creating an overlap
MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
- HRegionInfo hriOverlap1 =
+ RegionInfo hriOverlap1 =
createRegion(tbl.getTableDescriptor(), Bytes.toBytes("A"), Bytes.toBytes("AB"));
TEST_UTIL.assignRegion(hriOverlap1);
- HRegionInfo hriOverlap2 =
+ RegionInfo hriOverlap2 =
createRegion(tbl.getTableDescriptor(), Bytes.toBytes("AB"), Bytes.toBytes("B"));
TEST_UTIL.assignRegion(hriOverlap2);
@@ -491,7 +507,7 @@ public class TestHBaseFsckTwoRS extends BaseTestHBaseFsck {
Scan scan = new Scan();
scan.setStartRow(Bytes.toBytes(tableName+",,"));
ResultScanner scanner = meta.getScanner(scan);
- HRegionInfo hri = null;
+ RegionInfo hri = null;
Result res = scanner.next();
ServerName currServer =
@@ -515,7 +531,7 @@ public class TestHBaseFsckTwoRS extends BaseTestHBaseFsck {
put.addColumn(HConstants.CATALOG_FAMILY, HConstants.STARTCODE_QUALIFIER,
Bytes.toBytes(sn.getStartcode()));
meta.put(put);
- hri = MetaTableAccessor.getHRegionInfo(res);
+ hri = MetaTableAccessor.getRegionInfo(res);
break;
}
}
@@ -555,7 +571,7 @@ public class TestHBaseFsckTwoRS extends BaseTestHBaseFsck {
// Mess it up by leaving a hole in the meta data
admin.disableTable(tableName);
deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("B"), Bytes.toBytes("C"), true,
- true, false, true, HRegionInfo.DEFAULT_REPLICA_ID);
+ true, false, true, RegionInfo.DEFAULT_REPLICA_ID);
admin.enableTable(tableName);
HBaseFsck hbck = doFsck(conf, false);
@@ -607,7 +623,7 @@ public class TestHBaseFsckTwoRS extends BaseTestHBaseFsck {
admin.enableCatalogJanitor(false);
meta = connection.getTable(TableName.META_TABLE_NAME, tableExecutorService);
HRegionLocation loc = this.connection.getRegionLocation(table, SPLITS[0], false);
- HRegionInfo hriParent = loc.getRegionInfo();
+ RegionInfo hriParent = loc.getRegionInfo();
// Split Region A just before B
this.connection.getAdmin().split(table, Bytes.toBytes("A@"));
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/OfflineMetaRebuildTestCore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/OfflineMetaRebuildTestCore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/OfflineMetaRebuildTestCore.java
index a7c4ad1..e6070dd 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/OfflineMetaRebuildTestCore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/OfflineMetaRebuildTestCore.java
@@ -31,7 +31,6 @@ import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MetaTableAccessor;
@@ -43,6 +42,8 @@ import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
import org.apache.hadoop.hbase.client.RegionLocator;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
@@ -170,7 +171,7 @@ public class OfflineMetaRebuildTestCore {
}
for (HRegionLocation e : regions) {
- HRegionInfo hri = e.getRegionInfo();
+ RegionInfo hri = e.getRegionInfo();
ServerName hsa = e.getServerName();
if (Bytes.compareTo(hri.getStartKey(), startKey) == 0
&& Bytes.compareTo(hri.getEndKey(), endKey) == 0) {
@@ -199,11 +200,14 @@ public class OfflineMetaRebuildTestCore {
dumpMeta(htd);
}
- protected HRegionInfo createRegion(Configuration conf, final Table htbl,
+ protected RegionInfo createRegion(Configuration conf, final Table htbl,
byte[] startKey, byte[] endKey) throws IOException {
Table meta = TEST_UTIL.getConnection().getTable(TableName.META_TABLE_NAME);
HTableDescriptor htd = htbl.getTableDescriptor();
- HRegionInfo hri = new HRegionInfo(htbl.getName(), startKey, endKey);
+ RegionInfo hri = RegionInfoBuilder.newBuilder(htbl.getName())
+ .setStartKey(startKey)
+ .setEndKey(endKey)
+ .build();
LOG.info("manually adding regioninfo and hdfs data: " + hri.toString());
Path rootDir = FSUtils.getRootDir(conf);
@@ -213,7 +217,7 @@ public class OfflineMetaRebuildTestCore {
fs.mkdirs(p);
Path riPath = new Path(p, HRegionFileSystem.REGION_INFO_FILE);
FSDataOutputStream out = fs.create(riPath);
- out.write(hri.toDelimitedByteArray());
+ out.write(RegionInfo.toDelimitedByteArray(hri));
out.close();
// add to meta.
@@ -230,8 +234,8 @@ public class OfflineMetaRebuildTestCore {
ResultScanner scanner = meta.getScanner(s);
List<Delete> dels = new ArrayList<>();
for (Result r : scanner) {
- HRegionInfo info =
- MetaTableAccessor.getHRegionInfo(r);
+ RegionInfo info =
+ MetaTableAccessor.getRegionInfo(r);
if(info != null && !info.getTable().getNamespaceAsString()
.equals(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR)) {
Delete d = new Delete(r.getRow());
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/FaultyFSLog.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/FaultyFSLog.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/FaultyFSLog.java
index a666279..472eadd 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/FaultyFSLog.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/FaultyFSLog.java
@@ -21,14 +21,14 @@ package org.apache.hadoop.hbase.wal;
import java.io.IOException;
-import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.regionserver.wal.FSHLog;
+import org.apache.yetus.audience.InterfaceAudience;
// imports for things that haven't moved yet
-import org.apache.hadoop.hbase.regionserver.wal.FSHLog;
/**
* This is a utility class, used by tests, which fails operation specified by FailureType enum
@@ -44,11 +44,11 @@ public class FaultyFSLog extends FSHLog {
throws IOException {
super(fs, rootDir, logName, conf);
}
-
+
public void setFailureType(FailureType fType) {
this.ft = fType;
}
-
+
@Override
public void sync(long txid) throws IOException {
if (this.ft == FailureType.SYNC) {
@@ -58,7 +58,7 @@ public class FaultyFSLog extends FSHLog {
}
@Override
- public long append(HRegionInfo info, WALKey key,
+ public long append(RegionInfo info, WALKey key,
WALEdit edits, boolean inMemstore) throws IOException {
if (this.ft == FailureType.APPEND) {
throw new IOException("append");
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/WALPerformanceEvaluation.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/WALPerformanceEvaluation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/WALPerformanceEvaluation.java
index 0204230..75aba03 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/WALPerformanceEvaluation.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/WALPerformanceEvaluation.java
@@ -18,6 +18,8 @@
package org.apache.hadoop.hbase.wal;
+import static com.codahale.metrics.MetricRegistry.name;
+
import java.io.IOException;
import java.util.HashMap;
import java.util.HashSet;
@@ -31,7 +33,6 @@ import java.util.concurrent.TimeUnit;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
-import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.conf.Configured;
import org.apache.hadoop.fs.FileStatus;
@@ -42,40 +43,40 @@ import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
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.MockRegionServerServices;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
import org.apache.hadoop.hbase.io.crypto.KeyProviderForTesting;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.LogRoller;
import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl;
+import org.apache.hadoop.hbase.regionserver.wal.SecureProtobufLogReader;
+import org.apache.hadoop.hbase.regionserver.wal.SecureProtobufLogWriter;
+import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
import org.apache.hadoop.hbase.trace.HBaseHTraceConfiguration;
import org.apache.hadoop.hbase.trace.SpanReceiverHost;
-import org.apache.hadoop.hbase.wal.WALProvider.Writer;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.Threads;
+import org.apache.hadoop.hbase.wal.WALProvider.Writer;
import org.apache.hadoop.util.Tool;
import org.apache.hadoop.util.ToolRunner;
import org.apache.htrace.Sampler;
import org.apache.htrace.Trace;
import org.apache.htrace.TraceScope;
import org.apache.htrace.impl.ProbabilitySampler;
+import org.apache.yetus.audience.InterfaceAudience;
import com.codahale.metrics.ConsoleReporter;
import com.codahale.metrics.Histogram;
import com.codahale.metrics.Meter;
-import com.codahale.metrics.MetricRegistry;
import com.codahale.metrics.MetricFilter;
+import com.codahale.metrics.MetricRegistry;
// imports for things that haven't moved from regionserver.wal yet.
-import org.apache.hadoop.hbase.regionserver.wal.SecureProtobufLogReader;
-import org.apache.hadoop.hbase.regionserver.wal.SecureProtobufLogWriter;
-import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
-
-import static com.codahale.metrics.MetricRegistry.name;
/**
* This class runs performance benchmarks for {@link WAL}.
@@ -184,7 +185,7 @@ public final class WALPerformanceEvaluation extends Configured implements Tool {
Put put = setupPut(rand, key, value, numFamilies);
WALEdit walEdit = new WALEdit();
addFamilyMapToWALEdit(put.getFamilyCellMap(), walEdit);
- HRegionInfo hri = region.getRegionInfo();
+ RegionInfo hri = region.getRegionInfo();
final WALKey logkey =
new WALKey(hri.getEncodedNameAsBytes(), hri.getTable(), now, mvcc, scopes);
wal.append(hri, logkey, walEdit, true);
@@ -347,7 +348,7 @@ public final class WALPerformanceEvaluation extends Configured implements Tool {
long putTime = runBenchmark(benchmarks, numThreads);
logBenchmarkResult("Summary: threads=" + numThreads + ", iterations=" + numIterations +
", syncInterval=" + syncInterval, numIterations * numThreads, putTime);
-
+
for (int i = 0; i < numRegions; i++) {
if (regions[i] != null) {
closeRegion(regions[i]);
@@ -449,7 +450,7 @@ public final class WALPerformanceEvaluation extends Configured implements Tool {
private static void logBenchmarkResult(String testName, long numTests, long totalTime) {
float tsec = totalTime / 1000.0f;
LOG.info(String.format("%s took %.3fs %.3fops/s", testName, tsec, numTests / tsec));
-
+
}
private void printUsageAndExit() {
@@ -493,7 +494,7 @@ public final class WALPerformanceEvaluation extends Configured implements Tool {
private HRegion openRegion(final FileSystem fs, final Path dir, final HTableDescriptor htd,
final WALFactory wals, final long whenToRoll, final LogRoller roller) throws IOException {
// Initialize HRegion
- HRegionInfo regionInfo = new HRegionInfo(htd.getTableName());
+ RegionInfo regionInfo = RegionInfoBuilder.newBuilder(htd.getTableName()).build();
// Initialize WAL
final WAL wal =
wals.getWAL(regionInfo.getEncodedNameAsBytes(), regionInfo.getTable().getNamespace());
@@ -529,7 +530,7 @@ public final class WALPerformanceEvaluation extends Configured implements Tool {
}
});
}
-
+
return HRegion.createHRegion(regionInfo, dir, getConf(), htd, wal);
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServerRunner.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServerRunner.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServerRunner.java
index 48a365d..c1e761a 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServerRunner.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServerRunner.java
@@ -20,6 +20,11 @@ package org.apache.hadoop.hbase.thrift;
import static org.apache.hadoop.hbase.util.Bytes.getBytes;
+import javax.security.auth.callback.Callback;
+import javax.security.auth.callback.UnsupportedCallbackException;
+import javax.security.sasl.AuthorizeCallback;
+import javax.security.sasl.Sasl;
+import javax.security.sasl.SaslServer;
import java.io.IOException;
import java.net.InetAddress;
import java.net.InetSocketAddress;
@@ -39,12 +44,6 @@ import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
-import javax.security.auth.callback.Callback;
-import javax.security.auth.callback.UnsupportedCallbackException;
-import javax.security.sasl.AuthorizeCallback;
-import javax.security.sasl.Sasl;
-import javax.security.sasl.SaslServer;
-
import org.apache.commons.cli.CommandLine;
import org.apache.commons.cli.Option;
import org.apache.commons.cli.OptionGroup;
@@ -55,7 +54,6 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
@@ -63,7 +61,6 @@ import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotFoundException;
-import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.Append;
import org.apache.hadoop.hbase.client.Delete;
@@ -72,6 +69,7 @@ import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Increment;
import org.apache.hadoop.hbase.client.OperationWithAttributes;
import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.RegionLocator;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
@@ -83,7 +81,6 @@ import org.apache.hadoop.hbase.filter.PrefixFilter;
import org.apache.hadoop.hbase.filter.WhileMatchFilter;
import org.apache.hadoop.hbase.security.SecurityUtil;
import org.apache.hadoop.hbase.security.UserProvider;
-import org.apache.hadoop.hbase.thrift.CallQueue.Call;
import org.apache.hadoop.hbase.thrift.generated.AlreadyExists;
import org.apache.hadoop.hbase.thrift.generated.BatchMutation;
import org.apache.hadoop.hbase.thrift.generated.ColumnDescriptor;
@@ -123,18 +120,23 @@ import org.apache.thrift.transport.TSaslServerTransport;
import org.apache.thrift.transport.TServerSocket;
import org.apache.thrift.transport.TServerTransport;
import org.apache.thrift.transport.TTransportFactory;
-
+import org.apache.yetus.audience.InterfaceAudience;
import org.eclipse.jetty.http.HttpVersion;
-import org.eclipse.jetty.server.*;
-
-import org.apache.hadoop.hbase.shaded.com.google.common.base.Joiner;
-import org.apache.hadoop.hbase.shaded.com.google.common.base.Throwables;
-import org.apache.hadoop.hbase.shaded.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.eclipse.jetty.server.HttpConfiguration;
+import org.eclipse.jetty.server.HttpConnectionFactory;
+import org.eclipse.jetty.server.SecureRequestCustomizer;
+import org.eclipse.jetty.server.Server;
+import org.eclipse.jetty.server.ServerConnector;
+import org.eclipse.jetty.server.SslConnectionFactory;
import org.eclipse.jetty.servlet.ServletContextHandler;
import org.eclipse.jetty.servlet.ServletHolder;
import org.eclipse.jetty.util.ssl.SslContextFactory;
import org.eclipse.jetty.util.thread.QueuedThreadPool;
+import org.apache.hadoop.hbase.shaded.com.google.common.base.Joiner;
+import org.apache.hadoop.hbase.shaded.com.google.common.base.Throwables;
+import org.apache.hadoop.hbase.shaded.com.google.common.util.concurrent.ThreadFactoryBuilder;
+
/**
* ThriftServerRunner - this class starts up a Thrift server which implements
* the Hbase API specified in the Hbase.thrift IDL file.
@@ -893,7 +895,7 @@ public class ThriftServerRunner implements Runnable {
List<HRegionLocation> regionLocations = locator.getAllRegionLocations();
List<TRegionInfo> results = new ArrayList<>(regionLocations.size());
for (HRegionLocation regionLocation : regionLocations) {
- HRegionInfo info = regionLocation.getRegionInfo();
+ RegionInfo info = regionLocation.getRegionInfo();
ServerName serverName = regionLocation.getServerName();
TRegionInfo region = new TRegionInfo();
region.serverName = ByteBuffer.wrap(
@@ -987,7 +989,7 @@ public class ThriftServerRunner implements Runnable {
*/
public List<TCell> getVer(ByteBuffer tableName, ByteBuffer row, byte[] family,
byte[] qualifier, int numVersions, Map<ByteBuffer, ByteBuffer> attributes) throws IOError {
-
+
Table table = null;
try {
table = getTable(tableName);
@@ -1033,7 +1035,7 @@ public class ThriftServerRunner implements Runnable {
protected List<TCell> getVerTs(ByteBuffer tableName, ByteBuffer row, byte[] family,
byte[] qualifier, long timestamp, int numVersions, Map<ByteBuffer, ByteBuffer> attributes)
throws IOError {
-
+
Table table = null;
try {
table = getTable(tableName);
@@ -1085,7 +1087,7 @@ public class ThriftServerRunner implements Runnable {
public List<TRowResult> getRowWithColumnsTs(
ByteBuffer tableName, ByteBuffer row, List<ByteBuffer> columns,
long timestamp, Map<ByteBuffer, ByteBuffer> attributes) throws IOError {
-
+
Table table = null;
try {
table = getTable(tableName);
@@ -1151,7 +1153,7 @@ public class ThriftServerRunner implements Runnable {
List<ByteBuffer> rows,
List<ByteBuffer> columns, long timestamp,
Map<ByteBuffer, ByteBuffer> attributes) throws IOError {
-
+
Table table= null;
try {
List<Get> gets = new ArrayList<>(rows.size());
@@ -1500,7 +1502,7 @@ public class ThriftServerRunner implements Runnable {
public int scannerOpenWithScan(ByteBuffer tableName, TScan tScan,
Map<ByteBuffer, ByteBuffer> attributes)
throws IOError {
-
+
Table table = null;
try {
table = getTable(tableName);
@@ -1555,7 +1557,7 @@ public class ThriftServerRunner implements Runnable {
public int scannerOpen(ByteBuffer tableName, ByteBuffer startRow,
List<ByteBuffer> columns,
Map<ByteBuffer, ByteBuffer> attributes) throws IOError {
-
+
Table table = null;
try {
table = getTable(tableName);
@@ -1585,7 +1587,7 @@ public class ThriftServerRunner implements Runnable {
ByteBuffer stopRow, List<ByteBuffer> columns,
Map<ByteBuffer, ByteBuffer> attributes)
throws IOError, TException {
-
+
Table table = null;
try {
table = getTable(tableName);
@@ -1616,7 +1618,7 @@ public class ThriftServerRunner implements Runnable {
List<ByteBuffer> columns,
Map<ByteBuffer, ByteBuffer> attributes)
throws IOError, TException {
-
+
Table table = null;
try {
table = getTable(tableName);
@@ -1648,7 +1650,7 @@ public class ThriftServerRunner implements Runnable {
public int scannerOpenTs(ByteBuffer tableName, ByteBuffer startRow,
List<ByteBuffer> columns, long timestamp,
Map<ByteBuffer, ByteBuffer> attributes) throws IOError, TException {
-
+
Table table = null;
try {
table = getTable(tableName);
@@ -1679,7 +1681,7 @@ public class ThriftServerRunner implements Runnable {
ByteBuffer stopRow, List<ByteBuffer> columns, long timestamp,
Map<ByteBuffer, ByteBuffer> attributes)
throws IOError, TException {
-
+
Table table = null;
try {
table = getTable(tableName);
@@ -1709,7 +1711,7 @@ public class ThriftServerRunner implements Runnable {
@Override
public Map<ByteBuffer, ColumnDescriptor> getColumnDescriptors(
ByteBuffer tableName) throws IOError, TException {
-
+
Table table = null;
try {
TreeMap<ByteBuffer, ColumnDescriptor> columns = new TreeMap<>();
@@ -1741,7 +1743,7 @@ public class ThriftServerRunner implements Runnable {
throw getIOError(e);
}
}
-
+
@Override
public TRegionInfo getRegionInfo(ByteBuffer searchRow) throws IOError {
try {
@@ -1755,9 +1757,9 @@ public class ThriftServerRunner implements Runnable {
}
// find region start and end keys
- HRegionInfo regionInfo = MetaTableAccessor.getHRegionInfo(startRowResult);
+ RegionInfo regionInfo = MetaTableAccessor.getRegionInfo(startRowResult);
if (regionInfo == null) {
- throw new IOException("HRegionInfo REGIONINFO was null or " +
+ throw new IOException("RegionInfo REGIONINFO was null or " +
" empty in Meta for row="
+ Bytes.toStringBinary(row));
}
@@ -1787,7 +1789,7 @@ public class ThriftServerRunner implements Runnable {
scan.setReversed(true);
scan.addFamily(family);
scan.setStartRow(row);
- Table table = getTable(tableName);
+ Table table = getTable(tableName);
try (ResultScanner scanner = table.getScanner(scan)) {
return scanner.next();
} finally{
[17/19] hbase git commit: HBASE-18839 Apply RegionInfo to code base
Posted by ch...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
index e5f1848..2fbbc3f 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
@@ -30,11 +30,9 @@ import org.apache.hadoop.hbase.CellScannable;
import org.apache.hadoop.hbase.ClusterStatus.Option;
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
-import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.Action;
import org.apache.hadoop.hbase.client.Append;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
@@ -46,6 +44,7 @@ import org.apache.hadoop.hbase.client.MasterSwitchType;
import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.RegionCoprocessorServiceExec;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.Row;
import org.apache.hadoop.hbase.client.RowMutations;
import org.apache.hadoop.hbase.client.Scan;
@@ -54,6 +53,12 @@ import org.apache.hadoop.hbase.client.replication.ReplicationSerDeHelper;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.filter.ByteArrayComparable;
import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.security.token.Token;
+import org.apache.yetus.audience.InterfaceAudience;
+
import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ClearCompactionQueuesRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionRequest;
@@ -70,7 +75,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavor
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest.RegionUpdateInfo;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WarmupRegionRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Condition;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.GetRequest;
@@ -81,6 +85,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationPr
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.MutationType;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionAction;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.CompareType;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType;
@@ -97,12 +102,9 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteTabl
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DisableTableRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DrainRegionServersRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableCatalogJanitorRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetNamespaceDescriptorRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyNamespaceRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveDrainFromRegionServersRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetCleanerChoreRunningRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableTableRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetNamespaceDescriptorRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetSchemaAlterStatusRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesRequest;
@@ -115,17 +117,19 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsNormaliz
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyNamespaceRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.OfflineRegionRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveDrainFromRegionServersRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCatalogScanRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCleanerChoreRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetBalancerRunningRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetCleanerChoreRunningRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormalizerRunningRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SplitTableRegionRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SplitTableRegionResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesRequest;
@@ -140,10 +144,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.GetRe
import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListReplicationPeersRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.RemoveReplicationPeerRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigRequest;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.hadoop.hbase.util.Pair;
-import org.apache.hadoop.security.token.Token;
/**
* Helper utility to build protocol buffer requests,
@@ -886,10 +886,10 @@ public final class RequestConverter {
* @return a protocol buffer OpenRegionRequest
*/
public static OpenRegionRequest
- buildOpenRegionRequest(ServerName server, final List<Pair<HRegionInfo,
+ buildOpenRegionRequest(ServerName server, final List<Pair<RegionInfo,
List<ServerName>>> regionOpenInfos, Boolean openForReplay) {
OpenRegionRequest.Builder builder = OpenRegionRequest.newBuilder();
- for (Pair<HRegionInfo, List<ServerName>> regionOpenInfo: regionOpenInfos) {
+ for (Pair<RegionInfo, List<ServerName>> regionOpenInfo: regionOpenInfos) {
builder.addOpenInfo(buildRegionOpenInfo(regionOpenInfo.getFirst(),
regionOpenInfo.getSecond(), openForReplay));
}
@@ -911,7 +911,7 @@ public final class RequestConverter {
* @return a protocol buffer OpenRegionRequest
*/
public static OpenRegionRequest buildOpenRegionRequest(ServerName server,
- final HRegionInfo region, List<ServerName> favoredNodes,
+ final RegionInfo region, List<ServerName> favoredNodes,
Boolean openForReplay) {
OpenRegionRequest.Builder builder = OpenRegionRequest.newBuilder();
builder.addOpenInfo(buildRegionOpenInfo(region, favoredNodes,
@@ -929,12 +929,12 @@ public final class RequestConverter {
* @return a protocol buffer UpdateFavoredNodesRequest
*/
public static UpdateFavoredNodesRequest buildUpdateFavoredNodesRequest(
- final List<Pair<HRegionInfo, List<ServerName>>> updateRegionInfos) {
+ final List<Pair<RegionInfo, List<ServerName>>> updateRegionInfos) {
UpdateFavoredNodesRequest.Builder ubuilder = UpdateFavoredNodesRequest.newBuilder();
if (updateRegionInfos != null && !updateRegionInfos.isEmpty()) {
RegionUpdateInfo.Builder builder = RegionUpdateInfo.newBuilder();
- for (Pair<HRegionInfo, List<ServerName>> pair : updateRegionInfos) {
- builder.setRegion(HRegionInfo.convert(pair.getFirst()));
+ for (Pair<RegionInfo, List<ServerName>> pair : updateRegionInfos) {
+ builder.setRegion(ProtobufUtil.toRegionInfo(pair.getFirst()));
for (ServerName server : pair.getSecond()) {
builder.addFavoredNodes(ProtobufUtil.toServerName(server));
}
@@ -950,9 +950,9 @@ public final class RequestConverter {
*
* @param regionInfo Region we are warming up
*/
- public static WarmupRegionRequest buildWarmupRegionRequest(final HRegionInfo regionInfo) {
+ public static WarmupRegionRequest buildWarmupRegionRequest(final RegionInfo regionInfo) {
WarmupRegionRequest.Builder builder = WarmupRegionRequest.newBuilder();
- builder.setRegionInfo(HRegionInfo.convert(regionInfo));
+ builder.setRegionInfo(ProtobufUtil.toRegionInfo(regionInfo));
return builder.build();
}
@@ -1184,11 +1184,11 @@ public final class RequestConverter {
return builder.build();
}
- public static SplitTableRegionRequest buildSplitTableRegionRequest(final HRegionInfo regionInfo,
+ public static SplitTableRegionRequest buildSplitTableRegionRequest(final RegionInfo regionInfo,
final byte[] splitRow, final long nonceGroup, final long nonce)
throws DeserializationException {
SplitTableRegionRequest.Builder builder = SplitTableRegionRequest.newBuilder();
- builder.setRegionInfo(HRegionInfo.convert(regionInfo));
+ builder.setRegionInfo(ProtobufUtil.toRegionInfo(regionInfo));
if (splitRow != null) {
builder.setSplitRow(UnsafeByteOperations.unsafeWrap(splitRow));
}
@@ -1611,10 +1611,10 @@ public final class RequestConverter {
* Create a RegionOpenInfo based on given region info and version of offline node
*/
public static RegionOpenInfo buildRegionOpenInfo(
- final HRegionInfo region,
+ final RegionInfo region,
final List<ServerName> favoredNodes, Boolean openForReplay) {
RegionOpenInfo.Builder builder = RegionOpenInfo.newBuilder();
- builder.setRegion(HRegionInfo.convert(region));
+ builder.setRegion(ProtobufUtil.toRegionInfo(region));
if (favoredNodes != null) {
for (ServerName server : favoredNodes) {
builder.addFavoredNodes(ProtobufUtil.toServerName(server));
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ResponseConverter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ResponseConverter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ResponseConverter.java
index 93f74d3..611eaae 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ResponseConverter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ResponseConverter.java
@@ -28,16 +28,19 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellScanner;
import org.apache.hadoop.hbase.DoNotRetryIOException;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.SingleResponse;
import org.apache.hadoop.hbase.ipc.ServerRpcController;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetServerInfoResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ServerInfo;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiRequest;
@@ -55,9 +58,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableCata
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCatalogScanResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCleanerChoreResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdResponse;
-import org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString;
-import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController;
-import org.apache.hadoop.util.StringUtils;
import edu.umd.cs.findbugs.annotations.Nullable;
@@ -237,7 +237,7 @@ public final class ResponseConverter {
* @param proto the GetOnlineRegionResponse
* @return the list of region info
*/
- public static List<HRegionInfo> getRegionInfos(final GetOnlineRegionResponse proto) {
+ public static List<RegionInfo> getRegionInfos(final GetOnlineRegionResponse proto) {
if (proto == null || proto.getRegionInfoCount() == 0) return null;
return ProtobufUtil.getRegionInfos(proto);
}
@@ -280,10 +280,10 @@ public final class ResponseConverter {
* @return the response
*/
public static GetOnlineRegionResponse buildGetOnlineRegionResponse(
- final List<HRegionInfo> regions) {
+ final List<RegionInfo> regions) {
GetOnlineRegionResponse.Builder builder = GetOnlineRegionResponse.newBuilder();
- for (HRegionInfo region: regions) {
- builder.addRegionInfo(HRegionInfo.convert(region));
+ for (RegionInfo region: regions) {
+ builder.addRegionInfo(ProtobufUtil.toRegionInfo(region));
}
return builder.build();
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java
index dada632..cddde2f 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java
@@ -17,8 +17,6 @@
*/
package org.apache.hadoop.hbase.zookeeper;
-import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
-
import java.io.EOFException;
import java.io.IOException;
import java.net.ConnectException;
@@ -27,19 +25,19 @@ import java.net.SocketException;
import java.net.SocketTimeoutException;
import java.net.UnknownHostException;
import java.util.ArrayList;
+import java.util.Collections;
import java.util.List;
import java.util.Locale;
-
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.NotAllMetaRegionsOnlineException;
import org.apache.hadoop.hbase.ServerName;
-import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.ClusterConnection;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
import org.apache.hadoop.hbase.client.RetriesExhaustedException;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
@@ -47,17 +45,20 @@ import org.apache.hadoop.hbase.ipc.FailedServerException;
import org.apache.hadoop.hbase.ipc.HBaseRpcController;
import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.zookeeper.KeeperException;
+
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.MetaRegionServer;
-import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.Pair;
-import org.apache.hadoop.ipc.RemoteException;
-import org.apache.zookeeper.KeeperException;
/**
* Utility class to perform operation (get/wait for/verify/set/delete) on znode in ZooKeeper
@@ -93,8 +94,8 @@ public class MetaTableLocator {
* @param zkw ZooKeeper watcher to be used
* @return meta table regions and their locations.
*/
- public List<Pair<HRegionInfo, ServerName>> getMetaRegionsAndLocations(ZooKeeperWatcher zkw) {
- return getMetaRegionsAndLocations(zkw, HRegionInfo.DEFAULT_REPLICA_ID);
+ public List<Pair<RegionInfo, ServerName>> getMetaRegionsAndLocations(ZooKeeperWatcher zkw) {
+ return getMetaRegionsAndLocations(zkw, RegionInfo.DEFAULT_REPLICA_ID);
}
/**
@@ -103,12 +104,12 @@ public class MetaTableLocator {
* @param replicaId
* @return meta table regions and their locations.
*/
- public List<Pair<HRegionInfo, ServerName>> getMetaRegionsAndLocations(ZooKeeperWatcher zkw,
+ public List<Pair<RegionInfo, ServerName>> getMetaRegionsAndLocations(ZooKeeperWatcher zkw,
int replicaId) {
ServerName serverName = getMetaRegionLocation(zkw, replicaId);
- List<Pair<HRegionInfo, ServerName>> list = new ArrayList<>(1);
+ List<Pair<RegionInfo, ServerName>> list = new ArrayList<>(1);
list.add(new Pair<>(RegionReplicaUtil.getRegionInfoForReplica(
- HRegionInfo.FIRST_META_REGIONINFO, replicaId), serverName));
+ RegionInfoBuilder.FIRST_META_REGIONINFO, replicaId), serverName));
return list;
}
@@ -116,8 +117,8 @@ public class MetaTableLocator {
* @param zkw ZooKeeper watcher to be used
* @return List of meta regions
*/
- public List<HRegionInfo> getMetaRegions(ZooKeeperWatcher zkw) {
- return getMetaRegions(zkw, HRegionInfo.DEFAULT_REPLICA_ID);
+ public List<RegionInfo> getMetaRegions(ZooKeeperWatcher zkw) {
+ return getMetaRegions(zkw, RegionInfo.DEFAULT_REPLICA_ID);
}
/**
@@ -126,17 +127,17 @@ public class MetaTableLocator {
* @param replicaId
* @return List of meta regions
*/
- public List<HRegionInfo> getMetaRegions(ZooKeeperWatcher zkw, int replicaId) {
- List<Pair<HRegionInfo, ServerName>> result;
+ public List<RegionInfo> getMetaRegions(ZooKeeperWatcher zkw, int replicaId) {
+ List<Pair<RegionInfo, ServerName>> result;
result = getMetaRegionsAndLocations(zkw, replicaId);
- return getListOfHRegionInfos(result);
+ return getListOfRegionInfos(result);
}
- private List<HRegionInfo> getListOfHRegionInfos(
- final List<Pair<HRegionInfo, ServerName>> pairs) {
- if (pairs == null || pairs.isEmpty()) return null;
- List<HRegionInfo> result = new ArrayList<>(pairs.size());
- for (Pair<HRegionInfo, ServerName> pair: pairs) {
+ private List<RegionInfo> getListOfRegionInfos(
+ final List<Pair<RegionInfo, ServerName>> pairs) {
+ if (pairs == null || pairs.isEmpty()) return Collections.EMPTY_LIST;
+ List<RegionInfo> result = new ArrayList<>(pairs.size());
+ for (Pair<RegionInfo, ServerName> pair: pairs) {
result.add(pair.getFirst());
}
return result;
@@ -185,7 +186,7 @@ public class MetaTableLocator {
*/
public ServerName waitMetaRegionLocation(ZooKeeperWatcher zkw, long timeout)
throws InterruptedException, NotAllMetaRegionsOnlineException {
- return waitMetaRegionLocation(zkw, HRegionInfo.DEFAULT_REPLICA_ID, timeout);
+ return waitMetaRegionLocation(zkw, RegionInfo.DEFAULT_REPLICA_ID, timeout);
}
/**
@@ -261,7 +262,7 @@ public class MetaTableLocator {
public boolean verifyMetaRegionLocation(ClusterConnection hConnection,
ZooKeeperWatcher zkw, final long timeout)
throws InterruptedException, IOException {
- return verifyMetaRegionLocation(hConnection, zkw, timeout, HRegionInfo.DEFAULT_REPLICA_ID);
+ return verifyMetaRegionLocation(hConnection, zkw, timeout, RegionInfo.DEFAULT_REPLICA_ID);
}
/**
@@ -291,7 +292,7 @@ public class MetaTableLocator {
}
return (service != null) && verifyRegionLocation(connection, service,
getMetaRegionLocation(zkw, replicaId), RegionReplicaUtil.getRegionInfoForReplica(
- HRegionInfo.FIRST_META_REGIONINFO, replicaId).getRegionName());
+ RegionInfoBuilder.FIRST_META_REGIONINFO, replicaId).getRegionName());
}
/**
@@ -425,7 +426,7 @@ public class MetaTableLocator {
*/
public static void setMetaLocation(ZooKeeperWatcher zookeeper,
ServerName serverName, RegionState.State state) throws KeeperException {
- setMetaLocation(zookeeper, serverName, HRegionInfo.DEFAULT_REPLICA_ID, state);
+ setMetaLocation(zookeeper, serverName, RegionInfo.DEFAULT_REPLICA_ID, state);
}
/**
@@ -456,7 +457,7 @@ public class MetaTableLocator {
ZKUtil.setData(zookeeper,
zookeeper.znodePaths.getZNodeForReplica(replicaId), data);
} catch(KeeperException.NoNodeException nne) {
- if (replicaId == HRegionInfo.DEFAULT_REPLICA_ID) {
+ if (replicaId == RegionInfo.DEFAULT_REPLICA_ID) {
LOG.debug("META region location doesn't exist, create it");
} else {
LOG.debug("META region location doesn't exist for replicaId=" + replicaId +
@@ -470,7 +471,7 @@ public class MetaTableLocator {
* Load the meta region state from the meta server ZNode.
*/
public static RegionState getMetaRegionState(ZooKeeperWatcher zkw) throws KeeperException {
- return getMetaRegionState(zkw, HRegionInfo.DEFAULT_REPLICA_ID);
+ return getMetaRegionState(zkw, RegionInfo.DEFAULT_REPLICA_ID);
}
/**
@@ -514,7 +515,7 @@ public class MetaTableLocator {
state = RegionState.State.OFFLINE;
}
return new RegionState(
- RegionReplicaUtil.getRegionInfoForReplica(HRegionInfo.FIRST_META_REGIONINFO, replicaId),
+ RegionReplicaUtil.getRegionInfoForReplica(RegionInfoBuilder.FIRST_META_REGIONINFO, replicaId),
state, serverName);
}
@@ -525,12 +526,12 @@ public class MetaTableLocator {
*/
public void deleteMetaLocation(ZooKeeperWatcher zookeeper)
throws KeeperException {
- deleteMetaLocation(zookeeper, HRegionInfo.DEFAULT_REPLICA_ID);
+ deleteMetaLocation(zookeeper, RegionInfo.DEFAULT_REPLICA_ID);
}
public void deleteMetaLocation(ZooKeeperWatcher zookeeper, int replicaId)
throws KeeperException {
- if (replicaId == HRegionInfo.DEFAULT_REPLICA_ID) {
+ if (replicaId == RegionInfo.DEFAULT_REPLICA_ID) {
LOG.info("Deleting hbase:meta region location in ZooKeeper");
} else {
LOG.info("Deleting hbase:meta for " + replicaId + " region location in ZooKeeper");
@@ -586,7 +587,7 @@ public class MetaTableLocator {
public ServerName blockUntilAvailable(final ZooKeeperWatcher zkw,
final long timeout)
throws InterruptedException {
- return blockUntilAvailable(zkw, HRegionInfo.DEFAULT_REPLICA_ID, timeout);
+ return blockUntilAvailable(zkw, RegionInfo.DEFAULT_REPLICA_ID, timeout);
}
/**
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java
index 471ed96..f5166e0 100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java
@@ -19,6 +19,9 @@
package org.apache.hadoop.hbase.client;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
import java.io.IOException;
import java.io.InterruptedIOException;
@@ -58,26 +61,23 @@ import org.apache.hadoop.hbase.RegionLocations;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.AsyncProcessTask.ListRowAccess;
+import org.apache.hadoop.hbase.client.AsyncProcessTask.SubmittedRows;
+import org.apache.hadoop.hbase.client.backoff.ClientBackoffPolicy;
+import org.apache.hadoop.hbase.client.backoff.ServerStatistics;
import org.apache.hadoop.hbase.client.coprocessor.Batch;
+import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
+import org.apache.hadoop.hbase.testclassification.ClientTests;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Threads;
import org.junit.Assert;
import org.junit.BeforeClass;
+import org.junit.Ignore;
import org.junit.Rule;
import org.junit.Test;
-import org.junit.Ignore;
import org.junit.experimental.categories.Category;
import org.junit.rules.TestRule;
import org.mockito.Mockito;
-import org.apache.hadoop.hbase.client.AsyncProcessTask.SubmittedRows;
-import org.apache.hadoop.hbase.client.backoff.ClientBackoffPolicy;
-import org.apache.hadoop.hbase.client.backoff.ServerStatistics;
-import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
-import org.apache.hadoop.hbase.testclassification.ClientTests;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
@Category({ClientTests.class, MediumTests.class})
public class TestAsyncProcess {
@@ -106,9 +106,9 @@ public class TestAsyncProcess {
private static final HRegionLocation loc3 = new HRegionLocation(hri3, sn2);
// Replica stuff
- private static final HRegionInfo hri1r1 = RegionReplicaUtil.getRegionInfoForReplica(hri1, 1),
- hri1r2 = RegionReplicaUtil.getRegionInfoForReplica(hri1, 2);
- private static final HRegionInfo hri2r1 = RegionReplicaUtil.getRegionInfoForReplica(hri2, 1);
+ private static final RegionInfo hri1r1 = RegionReplicaUtil.getRegionInfoForReplica(hri1, 1);
+ private static final RegionInfo hri1r2 = RegionReplicaUtil.getRegionInfoForReplica(hri1, 2);
+ private static final RegionInfo hri2r1 = RegionReplicaUtil.getRegionInfoForReplica(hri2, 1);
private static final RegionLocations hrls1 = new RegionLocations(new HRegionLocation(hri1, sn),
new HRegionLocation(hri1r1, sn2), new HRegionLocation(hri1r2, sn3));
private static final RegionLocations hrls2 = new RegionLocations(new HRegionLocation(hri2, sn2),
@@ -355,8 +355,8 @@ public class TestAsyncProcess {
private Map<ServerName, Long> customPrimarySleepMs = new HashMap<>();
private final AtomicLong replicaCalls = new AtomicLong(0);
- public void addFailures(HRegionInfo... hris) {
- for (HRegionInfo hri : hris) {
+ public void addFailures(RegionInfo... hris) {
+ for (RegionInfo hri : hris) {
failures.add(hri.getRegionName());
}
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-endpoint/src/main/java/org/apache/hadoop/hbase/client/coprocessor/AsyncAggregationClient.java
----------------------------------------------------------------------
diff --git a/hbase-endpoint/src/main/java/org/apache/hadoop/hbase/client/coprocessor/AsyncAggregationClient.java b/hbase-endpoint/src/main/java/org/apache/hadoop/hbase/client/coprocessor/AsyncAggregationClient.java
index d7589fb..51c8248 100644
--- a/hbase-endpoint/src/main/java/org/apache/hadoop/hbase/client/coprocessor/AsyncAggregationClient.java
+++ b/hbase-endpoint/src/main/java/org/apache/hadoop/hbase/client/coprocessor/AsyncAggregationClient.java
@@ -20,8 +20,6 @@ package org.apache.hadoop.hbase.client.coprocessor;
import static org.apache.hadoop.hbase.client.coprocessor.AggregationHelper.getParsedGenericInstance;
import static org.apache.hadoop.hbase.client.coprocessor.AggregationHelper.validateArgAndGetPB;
-import com.google.protobuf.Message;
-
import java.io.IOException;
import java.util.Map;
import java.util.NavigableMap;
@@ -31,11 +29,10 @@ import java.util.TreeMap;
import java.util.concurrent.CompletableFuture;
import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.RawAsyncTable;
import org.apache.hadoop.hbase.client.RawAsyncTable.CoprocessorCallback;
import org.apache.hadoop.hbase.client.RawScanResultConsumer;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.coprocessor.ColumnInterpreter;
@@ -44,6 +41,9 @@ import org.apache.hadoop.hbase.protobuf.generated.AggregateProtos.AggregateRespo
import org.apache.hadoop.hbase.protobuf.generated.AggregateProtos.AggregateService;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.ReflectionUtils;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import com.google.protobuf.Message;
/**
* This client class is for invoking the aggregate functions deployed on the Region Server side via
@@ -73,7 +73,7 @@ public class AsyncAggregationClient {
}
@Override
- public synchronized void onRegionError(HRegionInfo region, Throwable error) {
+ public synchronized void onRegionError(RegionInfo region, Throwable error) {
completeExceptionally(error);
}
@@ -82,11 +82,11 @@ public class AsyncAggregationClient {
completeExceptionally(error);
}
- protected abstract void aggregate(HRegionInfo region, AggregateResponse resp)
+ protected abstract void aggregate(RegionInfo region, AggregateResponse resp)
throws IOException;
@Override
- public synchronized void onRegionComplete(HRegionInfo region, AggregateResponse resp) {
+ public synchronized void onRegionComplete(RegionInfo region, AggregateResponse resp) {
try {
aggregate(region, resp);
} catch (IOException e) {
@@ -135,7 +135,7 @@ public class AsyncAggregationClient {
private R max;
@Override
- protected void aggregate(HRegionInfo region, AggregateResponse resp) throws IOException {
+ protected void aggregate(RegionInfo region, AggregateResponse resp) throws IOException {
if (resp.getFirstPartCount() > 0) {
R result = getCellValueFromProto(ci, resp, 0);
if (max == null || (result != null && ci.compare(max, result) < 0)) {
@@ -171,7 +171,7 @@ public class AsyncAggregationClient {
private R min;
@Override
- protected void aggregate(HRegionInfo region, AggregateResponse resp) throws IOException {
+ protected void aggregate(RegionInfo region, AggregateResponse resp) throws IOException {
if (resp.getFirstPartCount() > 0) {
R result = getCellValueFromProto(ci, resp, 0);
if (min == null || (result != null && ci.compare(min, result) > 0)) {
@@ -208,7 +208,7 @@ public class AsyncAggregationClient {
private long count;
@Override
- protected void aggregate(HRegionInfo region, AggregateResponse resp) throws IOException {
+ protected void aggregate(RegionInfo region, AggregateResponse resp) throws IOException {
count += resp.getFirstPart(0).asReadOnlyByteBuffer().getLong();
}
@@ -239,7 +239,7 @@ public class AsyncAggregationClient {
private S sum;
@Override
- protected void aggregate(HRegionInfo region, AggregateResponse resp) throws IOException {
+ protected void aggregate(RegionInfo region, AggregateResponse resp) throws IOException {
if (resp.getFirstPartCount() > 0) {
S s = getPromotedValueFromProto(ci, resp, 0);
sum = ci.add(sum, s);
@@ -276,7 +276,7 @@ public class AsyncAggregationClient {
long count = 0L;
@Override
- protected void aggregate(HRegionInfo region, AggregateResponse resp) throws IOException {
+ protected void aggregate(RegionInfo region, AggregateResponse resp) throws IOException {
if (resp.getFirstPartCount() > 0) {
sum = ci.add(sum, getPromotedValueFromProto(ci, resp, 0));
count += resp.getSecondPart().asReadOnlyByteBuffer().getLong();
@@ -315,7 +315,7 @@ public class AsyncAggregationClient {
private long count;
@Override
- protected void aggregate(HRegionInfo region, AggregateResponse resp) throws IOException {
+ protected void aggregate(RegionInfo region, AggregateResponse resp) throws IOException {
if (resp.getFirstPartCount() > 0) {
sum = ci.add(sum, getPromotedValueFromProto(ci, resp, 0));
sumSq = ci.add(sumSq, getPromotedValueFromProto(ci, resp, 1));
@@ -357,7 +357,7 @@ public class AsyncAggregationClient {
private final NavigableMap<byte[], S> map = new TreeMap<>(Bytes.BYTES_COMPARATOR);
@Override
- protected void aggregate(HRegionInfo region, AggregateResponse resp) throws IOException {
+ protected void aggregate(RegionInfo region, AggregateResponse resp) throws IOException {
if (resp.getFirstPartCount() > 0) {
map.put(region.getStartKey(), getPromotedValueFromProto(ci, resp, firstPartIndex));
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-endpoint/src/main/java/org/apache/hadoop/hbase/coprocessor/Export.java
----------------------------------------------------------------------
diff --git a/hbase-endpoint/src/main/java/org/apache/hadoop/hbase/coprocessor/Export.java b/hbase-endpoint/src/main/java/org/apache/hadoop/hbase/coprocessor/Export.java
index 9b8901e..5095752 100644
--- a/hbase-endpoint/src/main/java/org/apache/hadoop/hbase/coprocessor/Export.java
+++ b/hbase-endpoint/src/main/java/org/apache/hadoop/hbase/coprocessor/Export.java
@@ -18,9 +18,6 @@
*/
package org.apache.hadoop.hbase.coprocessor;
-import com.google.protobuf.RpcCallback;
-import com.google.protobuf.RpcController;
-import com.google.protobuf.Service;
import java.io.Closeable;
import java.io.IOException;
import java.security.PrivilegedExceptionAction;
@@ -39,12 +36,10 @@ import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CoprocessorEnvironment;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.TableName;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.yetus.audience.InterfaceStability;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Table;
@@ -64,7 +59,6 @@ import org.apache.hadoop.hbase.regionserver.RegionScanner;
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.security.UserProvider;
import org.apache.hadoop.hbase.security.token.FsDelegationToken;
-import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.hbase.util.ArrayUtils;
import org.apache.hadoop.hbase.util.ByteStringer;
import org.apache.hadoop.hbase.util.Bytes;
@@ -77,6 +71,14 @@ import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.util.GenericOptionsParser;
import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.yetus.audience.InterfaceStability;
+
+import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
+
+import com.google.protobuf.RpcCallback;
+import com.google.protobuf.RpcController;
+import com.google.protobuf.Service;
/**
* Export an HBase table. Writes content to sequence files up in HDFS. Use
@@ -179,7 +181,7 @@ public class Export extends ExportProtos.ExportService implements RegionCoproces
}
private static SequenceFile.Writer.Option getOutputPath(final Configuration conf,
- final HRegionInfo info, final ExportProtos.ExportRequest request) throws IOException {
+ final RegionInfo info, final ExportProtos.ExportRequest request) throws IOException {
Path file = new Path(request.getOutputPath(), "export-" + info.getEncodedName());
FileSystem fs = file.getFileSystem(conf);
if (fs.exists(file)) {
@@ -189,7 +191,7 @@ public class Export extends ExportProtos.ExportService implements RegionCoproces
}
private static List<SequenceFile.Writer.Option> getWriterOptions(final Configuration conf,
- final HRegionInfo info, final ExportProtos.ExportRequest request) throws IOException {
+ final RegionInfo info, final ExportProtos.ExportRequest request) throws IOException {
List<SequenceFile.Writer.Option> rval = new LinkedList<>();
rval.add(SequenceFile.Writer.keyClass(ImmutableBytesWritable.class));
rval.add(SequenceFile.Writer.valueClass(Result.class));
@@ -341,7 +343,7 @@ public class Export extends ExportProtos.ExportService implements RegionCoproces
}
}
- private Scan validateKey(final HRegionInfo region, final ExportProtos.ExportRequest request) throws IOException {
+ private Scan validateKey(final RegionInfo region, final ExportProtos.ExportRequest request) throws IOException {
Scan scan = ProtobufUtil.toScan(request.getScan());
byte[] regionStartKey = region.getStartKey();
byte[] originStartKey = scan.getStartRow();
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-examples/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestRefreshHFilesEndpoint.java
----------------------------------------------------------------------
diff --git a/hbase-examples/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestRefreshHFilesEndpoint.java b/hbase-examples/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestRefreshHFilesEndpoint.java
index 257b075..46336d5 100644
--- a/hbase-examples/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestRefreshHFilesEndpoint.java
+++ b/hbase-examples/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestRefreshHFilesEndpoint.java
@@ -18,19 +18,25 @@
package org.apache.hadoop.hbase.coprocessor.example;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.RetriesExhaustedException;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.client.TableDescriptor;
@@ -41,7 +47,6 @@ import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HStore;
import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
-import org.apache.hadoop.hbase.regionserver.Store;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSUtils;
@@ -51,13 +56,6 @@ import org.junit.After;
import org.junit.Test;
import org.junit.experimental.categories.Category;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
-
@Category(MediumTests.class)
public class TestRefreshHFilesEndpoint {
private static final Log LOG = LogFactory.getLog(TestRefreshHFilesEndpoint.class);
@@ -139,7 +137,7 @@ public class TestRefreshHFilesEndpoint {
HStoreWithFaultyRefreshHFilesAPI store;
public HRegionForRefreshHFilesEP(final Path tableDir, final WAL wal, final FileSystem fs,
- final Configuration confParam, final HRegionInfo regionInfo,
+ final Configuration confParam, final RegionInfo regionInfo,
final TableDescriptor htd, final RegionServerServices rsServices) {
super(tableDir, wal, fs, confParam, regionInfo, htd, rsServices);
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableSnapshotInputFormat.java
----------------------------------------------------------------------
diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableSnapshotInputFormat.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableSnapshotInputFormat.java
index b5306f2..8285054 100644
--- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableSnapshotInputFormat.java
+++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableSnapshotInputFormat.java
@@ -18,11 +18,17 @@
package org.apache.hadoop.hbase.mapreduce;
-import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.lang.reflect.Method;
+import java.util.ArrayList;
+import java.util.List;
+
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
@@ -34,13 +40,9 @@ import org.apache.hadoop.mapreduce.Job;
import org.apache.hadoop.mapreduce.JobContext;
import org.apache.hadoop.mapreduce.RecordReader;
import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.yetus.audience.InterfaceAudience;
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.lang.reflect.Method;
-import java.util.ArrayList;
-import java.util.List;
+import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
/**
* TableSnapshotInputFormat allows a MapReduce job to run over a table snapshot. The job
@@ -120,10 +122,18 @@ public class TableSnapshotInputFormat extends InputFormat<ImmutableBytesWritable
delegate.readFields(in);
}
+ /**
+ * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+ * Use {@link #getRegion()}
+ */
+ @Deprecated
public HRegionInfo getRegionInfo() {
return delegate.getRegionInfo();
}
+ public RegionInfo getRegion() {
+ return delegate.getRegionInfo();
+ }
}
@VisibleForTesting
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionTool.java
----------------------------------------------------------------------
diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionTool.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionTool.java
index a94f50e..aec5fa0 100644
--- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionTool.java
+++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionTool.java
@@ -38,7 +38,7 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.HDFSBlocksDistribution;
-import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.mapreduce.JobUtil;
import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil;
@@ -113,7 +113,7 @@ public class CompactionTool extends Configured implements Tool {
Path regionDir = path.getParent();
Path tableDir = regionDir.getParent();
TableDescriptor htd = FSTableDescriptors.getTableDescriptorFromFs(fs, tableDir);
- HRegionInfo hri = HRegionFileSystem.loadRegionInfoFileContent(fs, regionDir);
+ RegionInfo hri = HRegionFileSystem.loadRegionInfoFileContent(fs, regionDir);
compactStoreFiles(tableDir, htd, hri,
path.getName(), compactOnce, major);
} else if (isRegionDir(fs, path)) {
@@ -139,7 +139,7 @@ public class CompactionTool extends Configured implements Tool {
private void compactRegion(final Path tableDir, final TableDescriptor htd,
final Path regionDir, final boolean compactOnce, final boolean major)
throws IOException {
- HRegionInfo hri = HRegionFileSystem.loadRegionInfoFileContent(fs, regionDir);
+ RegionInfo hri = HRegionFileSystem.loadRegionInfoFileContent(fs, regionDir);
for (Path familyDir: FSUtils.getFamilyDirs(fs, regionDir)) {
compactStoreFiles(tableDir, htd, hri, familyDir.getName(), compactOnce, major);
}
@@ -151,7 +151,7 @@ public class CompactionTool extends Configured implements Tool {
* no more compactions are needed. Uses the Configuration settings provided.
*/
private void compactStoreFiles(final Path tableDir, final TableDescriptor htd,
- final HRegionInfo hri, final String familyName, final boolean compactOnce,
+ final RegionInfo hri, final String familyName, final boolean compactOnce,
final boolean major) throws IOException {
HStore store = getStore(conf, fs, tableDir, htd, hri, familyName, tmpDir);
LOG.info("Compact table=" + htd.getTableName() +
@@ -183,7 +183,7 @@ public class CompactionTool extends Configured implements Tool {
* the store dir to compact as source.
*/
private static HStore getStore(final Configuration conf, final FileSystem fs,
- final Path tableDir, final TableDescriptor htd, final HRegionInfo hri,
+ final Path tableDir, final TableDescriptor htd, final RegionInfo hri,
final String familyName, final Path tempDir) throws IOException {
HRegionFileSystem regionFs = new HRegionFileSystem(conf, fs, tableDir, hri) {
@Override
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/snapshot/ExportSnapshot.java
----------------------------------------------------------------------
diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/snapshot/ExportSnapshot.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/snapshot/ExportSnapshot.java
index e8ed727..5dd9bea 100644
--- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/snapshot/ExportSnapshot.java
+++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/snapshot/ExportSnapshot.java
@@ -19,9 +19,9 @@
package org.apache.hadoop.hbase.snapshot;
import java.io.BufferedInputStream;
-import java.io.FileNotFoundException;
import java.io.DataInput;
import java.io.DataOutput;
+import java.io.FileNotFoundException;
import java.io.IOException;
import java.io.InputStream;
import java.util.ArrayList;
@@ -34,7 +34,6 @@ import org.apache.commons.cli.CommandLine;
import org.apache.commons.cli.Option;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
-import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
@@ -44,18 +43,16 @@ import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.io.FileLink;
import org.apache.hadoop.hbase.io.HFileLink;
import org.apache.hadoop.hbase.io.WALLink;
+import org.apache.hadoop.hbase.io.hadoopbackport.ThrottledInputStream;
import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil;
import org.apache.hadoop.hbase.mob.MobUtils;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotFileInfo;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest;
import org.apache.hadoop.hbase.util.AbstractHBaseTool;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.HFileArchiveUtil;
@@ -64,18 +61,22 @@ import org.apache.hadoop.io.BytesWritable;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.io.NullWritable;
import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.InputSplit;
import org.apache.hadoop.mapreduce.Job;
import org.apache.hadoop.mapreduce.JobContext;
import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.hadoop.mapreduce.InputFormat;
-import org.apache.hadoop.mapreduce.InputSplit;
import org.apache.hadoop.mapreduce.RecordReader;
import org.apache.hadoop.mapreduce.TaskAttemptContext;
import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
import org.apache.hadoop.mapreduce.security.TokenCache;
-import org.apache.hadoop.hbase.io.hadoopbackport.ThrottledInputStream;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.util.Tool;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotFileInfo;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest;
/**
* Export the specified snapshot to a given FileSystem.
@@ -566,7 +567,7 @@ public class ExportSnapshot extends AbstractHBaseTool implements Tool {
SnapshotReferenceUtil.visitReferencedFiles(conf, fs, snapshotDir, snapshotDesc,
new SnapshotReferenceUtil.SnapshotVisitor() {
@Override
- public void storeFile(final HRegionInfo regionInfo, final String family,
+ public void storeFile(final RegionInfo regionInfo, final String family,
final SnapshotRegionManifest.StoreFile storeFile) throws IOException {
// for storeFile.hasReference() case, copied as part of the manifest
if (!storeFile.hasReference()) {
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java
----------------------------------------------------------------------
diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java
index b581e04..b2b9c4d 100644
--- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java
+++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java
@@ -40,11 +40,10 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeepDeletedCells;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.TableName;
@@ -53,6 +52,7 @@ import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
@@ -65,13 +65,13 @@ import org.apache.hadoop.hbase.filter.PrefixFilter;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.mapreduce.Import.KeyValueImporter;
import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
-import org.apache.hadoop.hbase.wal.WALEdit;
-import org.apache.hadoop.hbase.wal.WAL;
-import org.apache.hadoop.hbase.wal.WALKey;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.testclassification.VerySlowMapReduceTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.LauncherSecurityManager;
+import org.apache.hadoop.hbase.wal.WAL;
+import org.apache.hadoop.hbase.wal.WALEdit;
+import org.apache.hadoop.hbase.wal.WALKey;
import org.apache.hadoop.mapreduce.Mapper.Context;
import org.apache.hadoop.util.GenericOptionsParser;
import org.apache.hadoop.util.ToolRunner;
@@ -739,7 +739,7 @@ public class TestImportExport {
Table importTable = UTIL.createTable(TableName.valueOf(importTableName), FAMILYA, 3);
// Register the wal listener for the import table
- HRegionInfo region = UTIL.getHBaseCluster().getRegionServerThreads().get(0).getRegionServer()
+ RegionInfo region = UTIL.getHBaseCluster().getRegionServerThreads().get(0).getRegionServer()
.getRegions(importTable.getName()).get(0).getRegionInfo();
TableWALActionListener walListener = new TableWALActionListener(region);
WAL wal = UTIL.getMiniHBaseCluster().getRegionServer(0).getWAL(region);
@@ -773,15 +773,15 @@ public class TestImportExport {
}
/**
- * This listens to the {@link #visitLogEntryBeforeWrite(HRegionInfo, WALKey, WALEdit)} to
+ * This listens to the {@link #visitLogEntryBeforeWrite(RegionInfo, WALKey, WALEdit)} to
* identify that an entry is written to the Write Ahead Log for the given table.
*/
private static class TableWALActionListener extends WALActionsListener.Base {
- private HRegionInfo regionInfo;
+ private RegionInfo regionInfo;
private boolean isVisited = false;
- public TableWALActionListener(HRegionInfo region) {
+ public TableWALActionListener(RegionInfo region) {
this.regionInfo = region;
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java
----------------------------------------------------------------------
diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java
index 192b85d..6105a0d 100644
--- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java
+++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java
@@ -18,7 +18,11 @@
package org.apache.hadoop.hbase.replication;
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
import java.io.IOException;
import java.util.ArrayList;
@@ -37,7 +41,6 @@ import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HBaseTestingUtility;
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.KeyValue;
import org.apache.hadoop.hbase.TableName;
@@ -48,6 +51,8 @@ import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
@@ -56,11 +61,9 @@ import org.apache.hadoop.hbase.client.replication.ReplicationAdmin;
import org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl;
-import org.apache.hadoop.hbase.wal.WALEdit;
import org.apache.hadoop.hbase.replication.regionserver.Replication;
import org.apache.hadoop.hbase.replication.regionserver.ReplicationSource;
import org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceInterface;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos;
import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.testclassification.ReplicationTests;
@@ -70,6 +73,7 @@ import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.JVMClusterUtil;
import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
import org.apache.hadoop.hbase.wal.WAL;
+import org.apache.hadoop.hbase.wal.WALEdit;
import org.apache.hadoop.hbase.wal.WALKey;
import org.apache.hadoop.mapreduce.Job;
import org.junit.Before;
@@ -79,6 +83,7 @@ import org.junit.experimental.categories.Category;
import org.junit.rules.TestName;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos;
@Category({ReplicationTests.class, LargeTests.class})
public class TestReplicationSmallTests extends TestReplicationBase {
@@ -753,8 +758,10 @@ public class TestReplicationSmallTests extends TestReplicationBase {
public void testCompactionWALEdits() throws Exception {
WALProtos.CompactionDescriptor compactionDescriptor =
WALProtos.CompactionDescriptor.getDefaultInstance();
- HRegionInfo hri = new HRegionInfo(htable1.getName(),
- HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW);
+ RegionInfo hri = RegionInfoBuilder.newBuilder(htable1.getName())
+ .setStartKey(HConstants.EMPTY_START_ROW)
+ .setEndKey(HConstants.EMPTY_END_ROW)
+ .build();
WALEdit edit = WALEdit.createCompaction(hri, compactionDescriptor);
Replication.scopeWALEdits(new WALKey(), edit,
htable1.getConfiguration(), null);
@@ -822,7 +829,7 @@ public class TestReplicationSmallTests extends TestReplicationBase {
final TableName tableName = htable1.getName();
HRegion region = utility1.getMiniHBaseCluster().getRegions(tableName).get(0);
- HRegionInfo hri = region.getRegionInfo();
+ RegionInfo hri = region.getRegionInfo();
NavigableMap<byte[], Integer> scopes = new TreeMap<>(Bytes.BYTES_COMPARATOR);
for (byte[] fam : htable1.getTableDescriptor().getFamiliesKeys()) {
scopes.put(fam, 1);
@@ -989,7 +996,7 @@ public class TestReplicationSmallTests extends TestReplicationBase {
final List<Path> emptyWalPaths = new ArrayList<>();
long ts = System.currentTimeMillis();
for (int i = 0; i < numRs; i++) {
- HRegionInfo regionInfo =
+ RegionInfo regionInfo =
utility1.getHBaseCluster().getRegions(htable1.getName()).get(0).getRegionInfo();
WAL wal = utility1.getHBaseCluster().getRegionServer(i).getWAL(regionInfo);
Path currentWalPath = AbstractFSWALProvider.getCurrentFileName(wal);
@@ -1012,7 +1019,7 @@ public class TestReplicationSmallTests extends TestReplicationBase {
// roll the original wal, which enqueues a new wal behind our empty wal
for (int i = 0; i < numRs; i++) {
- HRegionInfo regionInfo =
+ RegionInfo regionInfo =
utility1.getHBaseCluster().getRegions(htable1.getName()).get(0).getRegionInfo();
WAL wal = utility1.getHBaseCluster().getRegionServer(i).getWAL(regionInfo);
wal.rollWriter(true);
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshot.java
----------------------------------------------------------------------
diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshot.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshot.java
index 2e3cb5e..7139968 100644
--- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshot.java
+++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshot.java
@@ -39,12 +39,10 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.CategoryBasedTimeout;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.testclassification.VerySlowMapReduceTests;
import org.apache.hadoop.hbase.util.Bytes;
@@ -53,13 +51,16 @@ import org.junit.After;
import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
+import org.junit.Ignore;
import org.junit.Rule;
import org.junit.Test;
-import org.junit.Ignore;
import org.junit.experimental.categories.Category;
import org.junit.rules.TestName;
import org.junit.rules.TestRule;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest;
+
/**
* Test Export Snapshot Tool
*/
@@ -138,7 +139,7 @@ public class TestExportSnapshot {
}
protected interface RegionPredicate {
- boolean evaluate(final HRegionInfo regionInfo);
+ boolean evaluate(final RegionInfo regionInfo);
}
protected RegionPredicate getBypassRegionPredicate() {
@@ -314,7 +315,7 @@ public class TestExportSnapshot {
SnapshotReferenceUtil.visitReferencedFiles(conf, fs, exportedSnapshot,
new SnapshotReferenceUtil.SnapshotVisitor() {
@Override
- public void storeFile(final HRegionInfo regionInfo, final String family,
+ public void storeFile(final RegionInfo regionInfo, final String family,
final SnapshotRegionManifest.StoreFile storeFile) throws IOException {
if (bypassregionPredicate != null && bypassregionPredicate.evaluate(regionInfo))
return;
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/snapshot/TestMobExportSnapshot.java
----------------------------------------------------------------------
diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/snapshot/TestMobExportSnapshot.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/snapshot/TestMobExportSnapshot.java
index 7407a7d..3de54ff 100644
--- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/snapshot/TestMobExportSnapshot.java
+++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/snapshot/TestMobExportSnapshot.java
@@ -20,7 +20,7 @@ package org.apache.hadoop.hbase.snapshot;
import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.mob.MobConstants;
import org.apache.hadoop.hbase.mob.MobUtils;
import org.apache.hadoop.hbase.testclassification.LargeTests;
@@ -57,7 +57,7 @@ public class TestMobExportSnapshot extends TestExportSnapshot {
protected RegionPredicate getBypassRegionPredicate() {
return new RegionPredicate() {
@Override
- public boolean evaluate(final HRegionInfo regionInfo) {
+ public boolean evaluate(final RegionInfo regionInfo) {
return MobUtils.isMobRegionInfo(regionInfo);
}
};
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/RegionsResource.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/RegionsResource.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/RegionsResource.java
index 0369b44..183262d 100644
--- a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/RegionsResource.java
+++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/RegionsResource.java
@@ -19,30 +19,28 @@
package org.apache.hadoop.hbase.rest;
-import java.io.IOException;
-import java.util.Map;
-
import javax.ws.rs.GET;
import javax.ws.rs.Produces;
import javax.ws.rs.core.CacheControl;
import javax.ws.rs.core.Context;
import javax.ws.rs.core.Response;
-import javax.ws.rs.core.UriInfo;
import javax.ws.rs.core.Response.ResponseBuilder;
+import javax.ws.rs.core.UriInfo;
+import java.io.IOException;
+import java.util.Map;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
-
import org.apache.hadoop.hbase.MetaTableAccessor;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.rest.model.TableInfoModel;
import org.apache.hadoop.hbase.rest.model.TableRegionModel;
+import org.apache.yetus.audience.InterfaceAudience;
@InterfaceAudience.Private
public class RegionsResource extends ResourceBase {
@@ -81,11 +79,11 @@ public class RegionsResource extends ResourceBase {
Connection connection = ConnectionFactory.createConnection(servlet.getConfiguration());
@SuppressWarnings("deprecation")
- Map<HRegionInfo, ServerName> regions = MetaTableAccessor
+ Map<RegionInfo, ServerName> regions = MetaTableAccessor
.allTableRegions(connection, tableName);
connection.close();
- for (Map.Entry<HRegionInfo,ServerName> e: regions.entrySet()) {
- HRegionInfo hri = e.getKey();
+ for (Map.Entry<RegionInfo,ServerName> e: regions.entrySet()) {
+ RegionInfo hri = e.getKey();
ServerName addr = e.getValue();
model.add(
new TableRegionModel(tableName.getNameAsString(), hri.getRegionId(),
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminServer.java
----------------------------------------------------------------------
diff --git a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminServer.java b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminServer.java
index cba5d3e..b13dafd 100644
--- a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminServer.java
+++ b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminServer.java
@@ -31,11 +31,10 @@ import java.util.Set;
import org.apache.commons.lang3.StringUtils;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.constraint.ConstraintException;
import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.master.LoadBalancer;
@@ -48,6 +47,8 @@ import org.apache.hadoop.hbase.master.assignment.RegionStates.RegionStateNode;
import org.apache.hadoop.hbase.master.locking.LockManager;
import org.apache.hadoop.hbase.net.Address;
import org.apache.hadoop.hbase.procedure2.LockType;
+import org.apache.yetus.audience.InterfaceAudience;
+
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Maps;
@@ -114,9 +115,9 @@ public class RSGroupAdminServer implements RSGroupAdmin {
/**
* @return List of Regions associated with this <code>server</code>.
*/
- private List<HRegionInfo> getRegions(final Address server) {
- LinkedList<HRegionInfo> regions = new LinkedList<>();
- for (Map.Entry<HRegionInfo, ServerName> el :
+ private List<RegionInfo> getRegions(final Address server) {
+ LinkedList<RegionInfo> regions = new LinkedList<>();
+ for (Map.Entry<RegionInfo, ServerName> el :
master.getAssignmentManager().getRegionStates().getRegionAssignments().entrySet()) {
if (el.getValue() == null) continue;
if (el.getValue().getAddress().equals(server)) {
@@ -131,7 +132,7 @@ public class RSGroupAdminServer implements RSGroupAdmin {
return regions;
}
- private void addRegion(final LinkedList<HRegionInfo> regions, HRegionInfo hri) {
+ private void addRegion(final LinkedList<RegionInfo> regions, RegionInfo hri) {
// If meta, move it last otherwise other unassigns fail because meta is not
// online for them to update state in. This is dodgy. Needs to be made more
// robust. See TODO below.
@@ -206,8 +207,8 @@ public class RSGroupAdminServer implements RSGroupAdmin {
for (Iterator<Address> iter = allSevers.iterator(); iter.hasNext();) {
Address rs = iter.next();
// Get regions that are associated with this server and filter regions by tables.
- List<HRegionInfo> regions = new ArrayList<>();
- for (HRegionInfo region : getRegions(rs)) {
+ List<RegionInfo> regions = new ArrayList<>();
+ for (RegionInfo region : getRegions(rs)) {
if (!tables.contains(region.getTable())) {
regions.add(region);
}
@@ -216,7 +217,7 @@ public class RSGroupAdminServer implements RSGroupAdmin {
LOG.info("Unassigning " + regions.size() +
" region(s) from " + rs + " for server move to " + targetGroupName);
if (!regions.isEmpty()) {
- for (HRegionInfo region: regions) {
+ for (RegionInfo region: regions) {
// Regions might get assigned from tables of target group so we need to filter
if (!targetGrp.containsTable(region.getTable())) {
this.master.getAssignmentManager().unassign(region);
@@ -259,7 +260,7 @@ public class RSGroupAdminServer implements RSGroupAdmin {
} catch (InterruptedException e) {
throw new IOException("Interrupted when waiting for table lock", e);
}
- for (HRegionInfo region :
+ for (RegionInfo region :
master.getAssignmentManager().getRegionStates().getRegionsOfTable(table)) {
ServerName sn = master.getAssignmentManager().getRegionStates().getRegionServerOfRegion(region);
if (!servers.contains(sn.getAddress())) {
@@ -334,7 +335,7 @@ public class RSGroupAdminServer implements RSGroupAdmin {
for (Iterator<Address> iter = editableMovedServers.iterator(); iter.hasNext();) {
Address rs = iter.next();
// Get regions that are associated with this server.
- List<HRegionInfo> regions = getRegions(rs);
+ List<RegionInfo> regions = getRegions(rs);
// Unassign regions for a server
// TODO: This is problematic especially if hbase:meta is in the mix.
@@ -345,7 +346,7 @@ public class RSGroupAdminServer implements RSGroupAdmin {
" region(s) from " + rs + " for server move to " + targetGroupName);
if (!regions.isEmpty()) {
// TODO bulk unassign or throttled unassign?
- for (HRegionInfo region: regions) {
+ for (RegionInfo region: regions) {
// Regions might get assigned from tables of target group so we need to filter
if (!targetGrp.containsTable(region.getTable())) {
this.master.getAssignmentManager().unassign(region);
@@ -426,7 +427,7 @@ public class RSGroupAdminServer implements RSGroupAdmin {
} catch (InterruptedException e) {
throw new IOException("Interrupted when waiting for table lock", e);
}
- for (HRegionInfo region :
+ for (RegionInfo region :
master.getAssignmentManager().getRegionStates().getRegionsOfTable(table)) {
master.getAssignmentManager().unassign(region);
}
@@ -517,7 +518,7 @@ public class RSGroupAdminServer implements RSGroupAdmin {
//We balance per group instead of per table
List<RegionPlan> plans = new ArrayList<>();
- for(Map.Entry<TableName, Map<ServerName, List<HRegionInfo>>> tableMap:
+ for(Map.Entry<TableName, Map<ServerName, List<RegionInfo>>> tableMap:
getRSGroupAssignmentsByTable(groupName).entrySet()) {
LOG.info("Creating partial plan for table " + tableMap.getKey() + ": "
+ tableMap.getValue());
@@ -599,7 +600,7 @@ public class RSGroupAdminServer implements RSGroupAdmin {
Map<String, RegionState> rit = Maps.newTreeMap();
AssignmentManager am = master.getAssignmentManager();
for(TableName tableName : getRSGroupInfo(groupName).getTables()) {
- for(HRegionInfo regionInfo: am.getRegionStates().getRegionsOfTable(tableName)) {
+ for(RegionInfo regionInfo: am.getRegionStates().getRegionsOfTable(tableName)) {
RegionState state = am.getRegionStates().getRegionTransitionState(regionInfo);
if(state != null) {
rit.put(regionInfo.getEncodedName(), state);
@@ -609,16 +610,16 @@ public class RSGroupAdminServer implements RSGroupAdmin {
return rit;
}
- private Map<TableName, Map<ServerName, List<HRegionInfo>>>
+ private Map<TableName, Map<ServerName, List<RegionInfo>>>
getRSGroupAssignmentsByTable(String groupName) throws IOException {
- Map<TableName, Map<ServerName, List<HRegionInfo>>> result = Maps.newHashMap();
+ Map<TableName, Map<ServerName, List<RegionInfo>>> result = Maps.newHashMap();
RSGroupInfo rsGroupInfo = getRSGroupInfo(groupName);
- Map<TableName, Map<ServerName, List<HRegionInfo>>> assignments = Maps.newHashMap();
- for(Map.Entry<HRegionInfo, ServerName> entry:
+ Map<TableName, Map<ServerName, List<RegionInfo>>> assignments = Maps.newHashMap();
+ for(Map.Entry<RegionInfo, ServerName> entry:
master.getAssignmentManager().getRegionStates().getRegionAssignments().entrySet()) {
TableName currTable = entry.getKey().getTable();
ServerName currServer = entry.getValue();
- HRegionInfo currRegion = entry.getKey();
+ RegionInfo currRegion = entry.getKey();
if (rsGroupInfo.getTables().contains(currTable)) {
assignments.putIfAbsent(currTable, new HashMap<>());
assignments.get(currTable).putIfAbsent(currServer, new ArrayList<>());
@@ -626,7 +627,7 @@ public class RSGroupAdminServer implements RSGroupAdmin {
}
}
- Map<ServerName, List<HRegionInfo>> serverMap = Maps.newHashMap();
+ Map<ServerName, List<RegionInfo>> serverMap = Maps.newHashMap();
for(ServerName serverName: master.getServerManager().getOnlineServers().keySet()) {
if(rsGroupInfo.getServers().contains(serverName.getAddress())) {
serverMap.put(serverName, Collections.emptyList());
[18/19] hbase git commit: HBASE-18839 Apply RegionInfo to code base
Posted by ch...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
index fac3ef1..7b42d3d 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
@@ -40,6 +40,7 @@ import java.util.concurrent.TimeoutException;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicReference;
import java.util.regex.Pattern;
+import java.util.stream.Collectors;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@@ -69,8 +70,6 @@ import org.apache.hadoop.hbase.TableNotDisabledException;
import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.UnknownRegionException;
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.yetus.audience.InterfaceStability;
import org.apache.hadoop.hbase.client.replication.ReplicationSerDeHelper;
import org.apache.hadoop.hbase.client.replication.TableCFs;
import org.apache.hadoop.hbase.client.security.SecurityCapability;
@@ -86,6 +85,26 @@ import org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException;
import org.apache.hadoop.hbase.replication.ReplicationException;
import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
+import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
+import org.apache.hadoop.hbase.snapshot.HBaseSnapshotException;
+import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException;
+import org.apache.hadoop.hbase.snapshot.SnapshotCreationException;
+import org.apache.hadoop.hbase.snapshot.UnknownSnapshotException;
+import org.apache.hadoop.hbase.util.Addressing;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.ForeignExceptionUtil;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
+import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
+import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.yetus.audience.InterfaceStability;
+import org.apache.zookeeper.KeeperException;
+
+import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
@@ -184,28 +203,10 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRe
import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.GetReplicationPeerConfigResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos;
-import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
-import org.apache.hadoop.hbase.snapshot.HBaseSnapshotException;
-import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException;
-import org.apache.hadoop.hbase.snapshot.SnapshotCreationException;
-import org.apache.hadoop.hbase.snapshot.UnknownSnapshotException;
-import org.apache.hadoop.hbase.util.Addressing;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.hadoop.hbase.util.ForeignExceptionUtil;
-import org.apache.hadoop.hbase.util.Pair;
-import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
-import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
-import org.apache.hadoop.ipc.RemoteException;
-import org.apache.hadoop.util.StringUtils;
-import org.apache.zookeeper.KeeperException;
-import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
import com.google.protobuf.Descriptors;
import com.google.protobuf.Message;
import com.google.protobuf.RpcController;
-import java.util.stream.Collectors;
/**
* HBaseAdmin is no longer a client API. It is marked InterfaceAudience.Private indicating that
@@ -399,12 +400,26 @@ public class HBaseAdmin implements Admin {
@Override
public List<RegionInfo> getRegions(final ServerName sn) throws IOException {
- return getOnlineRegions(sn).stream().collect(Collectors.toList());
+ AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
+ // TODO: There is no timeout on this controller. Set one!
+ HBaseRpcController controller = rpcControllerFactory.newController();
+ return ProtobufUtil.getOnlineRegions(controller, admin);
}
@Override
public List<RegionInfo> getRegions(final TableName tableName) throws IOException {
- return getTableRegions(tableName).stream().collect(Collectors.toList());
+ ZooKeeperWatcher zookeeper =
+ new ZooKeeperWatcher(conf, ZK_IDENTIFIER_PREFIX + connection.toString(),
+ new ThrowableAbortable());
+ try {
+ if (TableName.META_TABLE_NAME.equals(tableName)) {
+ return new MetaTableLocator().getMetaRegions(zookeeper);
+ } else {
+ return MetaTableAccessor.getTableRegions(connection, tableName, true);
+ }
+ } finally {
+ zookeeper.close();
+ }
}
private static class AbortProcedureFuture extends ProcedureFuture<Boolean> {
@@ -1164,13 +1179,7 @@ public class HBaseAdmin implements Admin {
@Deprecated
@Override
public List<HRegionInfo> getOnlineRegions(final ServerName sn) throws IOException {
- AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
- // TODO: There is no timeout on this controller. Set one!
- HBaseRpcController controller = rpcControllerFactory.newController();
- List<HRegionInfo> onlineRegions = ProtobufUtil.getOnlineRegions(controller, admin);
- return onlineRegions == null ? null : onlineRegions.stream()
- .map(hri -> new ImmutableHRegionInfo(hri))
- .collect(Collectors.toList());
+ return getRegions(sn).stream().map(ImmutableHRegionInfo::new).collect(Collectors.toList());
}
@Override
@@ -1185,14 +1194,14 @@ public class HBaseAdmin implements Admin {
@Override
public void flushRegion(final byte[] regionName) throws IOException {
- Pair<HRegionInfo, ServerName> regionServerPair = getRegion(regionName);
+ Pair<RegionInfo, ServerName> regionServerPair = getRegion(regionName);
if (regionServerPair == null) {
throw new IllegalArgumentException("Unknown regionname: " + Bytes.toStringBinary(regionName));
}
if (regionServerPair.getSecond() == null) {
throw new NoServerForRegionException(Bytes.toStringBinary(regionName));
}
- final HRegionInfo hRegionInfo = regionServerPair.getFirst();
+ final RegionInfo hRegionInfo = regionServerPair.getFirst();
ServerName serverName = regionServerPair.getSecond();
final AdminService.BlockingInterface admin = this.connection.getAdmin(serverName);
Callable<Void> callable = new Callable<Void>() {
@@ -1304,13 +1313,13 @@ public class HBaseAdmin implements Admin {
checkTableExists(tableName);
zookeeper = new ZooKeeperWatcher(conf, ZK_IDENTIFIER_PREFIX + connection.toString(),
new ThrowableAbortable());
- List<Pair<HRegionInfo, ServerName>> pairs;
+ List<Pair<RegionInfo, ServerName>> pairs;
if (TableName.META_TABLE_NAME.equals(tableName)) {
pairs = new MetaTableLocator().getMetaRegionsAndLocations(zookeeper);
} else {
pairs = MetaTableAccessor.getTableRegionsAndLocations(connection, tableName);
}
- for (Pair<HRegionInfo, ServerName> pair: pairs) {
+ for (Pair<RegionInfo, ServerName> pair: pairs) {
if (pair.getFirst().isOffline()) continue;
if (pair.getSecond() == null) continue;
try {
@@ -1344,7 +1353,7 @@ public class HBaseAdmin implements Admin {
*/
private void compactRegion(final byte[] regionName, final byte[] columnFamily,final boolean major)
throws IOException {
- Pair<HRegionInfo, ServerName> regionServerPair = getRegion(regionName);
+ Pair<RegionInfo, ServerName> regionServerPair = getRegion(regionName);
if (regionServerPair == null) {
throw new IllegalArgumentException("Invalid region: " + Bytes.toStringBinary(regionName));
}
@@ -1354,7 +1363,7 @@ public class HBaseAdmin implements Admin {
compact(regionServerPair.getSecond(), regionServerPair.getFirst(), major, columnFamily);
}
- private void compact(final ServerName sn, final HRegionInfo hri,
+ private void compact(final ServerName sn, final RegionInfo hri,
final boolean major, final byte [] family)
throws IOException {
final AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
@@ -1655,7 +1664,7 @@ public class HBaseAdmin implements Admin {
}
TableName tableName = null;
- Pair<HRegionInfo, ServerName> pair;
+ Pair<RegionInfo, ServerName> pair;
for(int i = 0; i < nameofRegionsToMerge.length; i++) {
pair = getRegion(nameofRegionsToMerge[i]);
@@ -1750,7 +1759,7 @@ public class HBaseAdmin implements Admin {
throws IOException {
byte[] encodedNameofRegionToSplit = HRegionInfo.isEncodedRegionName(regionName) ?
regionName : HRegionInfo.encodeRegionName(regionName).getBytes();
- Pair<HRegionInfo, ServerName> pair = getRegion(regionName);
+ Pair<RegionInfo, ServerName> pair = getRegion(regionName);
if (pair != null) {
if (pair.getFirst() != null &&
pair.getFirst().getReplicaId() != HRegionInfo.DEFAULT_REPLICA_ID) {
@@ -1762,11 +1771,10 @@ public class HBaseAdmin implements Admin {
+ Bytes.toStringBinary(encodedNameofRegionToSplit));
}
- HRegionInfo hri = pair.getFirst();
- return splitRegionAsync(hri, splitPoint);
+ return splitRegionAsync(pair.getFirst(), splitPoint);
}
- Future<Void> splitRegionAsync(HRegionInfo hri, byte[] splitPoint) throws IOException {
+ Future<Void> splitRegionAsync(RegionInfo hri, byte[] splitPoint) throws IOException {
TableName tableName = hri.getTable();
if (hri.getStartKey() != null && splitPoint != null &&
Bytes.compareTo(hri.getStartKey(), splitPoint) == 0) {
@@ -1824,7 +1832,7 @@ public class HBaseAdmin implements Admin {
checkTableExists(tableName);
zookeeper = new ZooKeeperWatcher(conf, ZK_IDENTIFIER_PREFIX + connection.toString(),
new ThrowableAbortable());
- List<Pair<HRegionInfo, ServerName>> pairs;
+ List<Pair<RegionInfo, ServerName>> pairs;
if (TableName.META_TABLE_NAME.equals(tableName)) {
pairs = new MetaTableLocator().getMetaRegionsAndLocations(zookeeper);
} else {
@@ -1833,10 +1841,10 @@ public class HBaseAdmin implements Admin {
if (splitPoint == null) {
LOG.info("SplitPoint is null, will find bestSplitPoint from Region");
}
- for (Pair<HRegionInfo, ServerName> pair: pairs) {
+ for (Pair<RegionInfo, ServerName> pair: pairs) {
// May not be a server for a particular row
if (pair.getSecond() == null) continue;
- HRegionInfo r = pair.getFirst();
+ RegionInfo r = pair.getFirst();
// check for parents
if (r.isSplitParent()) continue;
// if a split point given, only split that particular region
@@ -1854,7 +1862,7 @@ public class HBaseAdmin implements Admin {
@Override
public void splitRegion(final byte[] regionName, final byte [] splitPoint) throws IOException {
- Pair<HRegionInfo, ServerName> regionServerPair = getRegion(regionName);
+ Pair<RegionInfo, ServerName> regionServerPair = getRegion(regionName);
if (regionServerPair == null) {
throw new IllegalArgumentException("Invalid region: " + Bytes.toStringBinary(regionName));
}
@@ -1921,19 +1929,18 @@ public class HBaseAdmin implements Admin {
* Throw IllegalArgumentException if <code>regionName</code> is null.
* @throws IOException
*/
- Pair<HRegionInfo, ServerName> getRegion(final byte[] regionName) throws IOException {
+ Pair<RegionInfo, ServerName> getRegion(final byte[] regionName) throws IOException {
if (regionName == null) {
throw new IllegalArgumentException("Pass a table name or region name");
}
- Pair<HRegionInfo, ServerName> pair =
- MetaTableAccessor.getRegion(connection, regionName);
+ Pair<RegionInfo, ServerName> pair = MetaTableAccessor.getRegion(connection, regionName);
if (pair == null) {
- final AtomicReference<Pair<HRegionInfo, ServerName>> result = new AtomicReference<>(null);
+ final AtomicReference<Pair<RegionInfo, ServerName>> result = new AtomicReference<>(null);
final String encodedName = Bytes.toString(regionName);
MetaTableAccessor.Visitor visitor = new MetaTableAccessor.Visitor() {
@Override
public boolean visit(Result data) throws IOException {
- HRegionInfo info = MetaTableAccessor.getHRegionInfo(data);
+ RegionInfo info = MetaTableAccessor.getRegionInfo(data);
if (info == null) {
LOG.warn("No serialized HRegionInfo in " + data);
return true;
@@ -1979,7 +1986,7 @@ public class HBaseAdmin implements Admin {
return HRegionInfo.FIRST_META_REGIONINFO.getRegionName();
}
byte[] tmp = regionNameOrEncodedRegionName;
- Pair<HRegionInfo, ServerName> regionServerPair = getRegion(regionNameOrEncodedRegionName);
+ Pair<RegionInfo, ServerName> regionServerPair = getRegion(regionNameOrEncodedRegionName);
if (regionServerPair != null && regionServerPair.getFirst() != null) {
tmp = regionServerPair.getFirst().getRegionName();
}
@@ -2373,23 +2380,10 @@ public class HBaseAdmin implements Admin {
@Deprecated
@Override
public List<HRegionInfo> getTableRegions(final TableName tableName)
- throws IOException {
- ZooKeeperWatcher zookeeper =
- new ZooKeeperWatcher(conf, ZK_IDENTIFIER_PREFIX + connection.toString(),
- new ThrowableAbortable());
- List<HRegionInfo> regions = null;
- try {
- if (TableName.META_TABLE_NAME.equals(tableName)) {
- regions = new MetaTableLocator().getMetaRegions(zookeeper);
- } else {
- regions = MetaTableAccessor.getTableRegions(connection, tableName, true);
- }
- } finally {
- zookeeper.close();
- }
- return regions == null ? null : regions.stream()
- .map(hri -> new ImmutableHRegionInfo(hri))
- .collect(Collectors.toList());
+ throws IOException {
+ return getRegions(tableName).stream()
+ .map(ImmutableHRegionInfo::new)
+ .collect(Collectors.toList());
}
@Override
@@ -2499,7 +2493,7 @@ public class HBaseAdmin implements Admin {
@Override
public CompactionState getCompactionStateForRegion(final byte[] regionName)
throws IOException {
- final Pair<HRegionInfo, ServerName> regionServerPair = getRegion(regionName);
+ final Pair<RegionInfo, ServerName> regionServerPair = getRegion(regionName);
if (regionServerPair == null) {
throw new IllegalArgumentException("Invalid region: " + Bytes.toStringBinary(regionName));
}
@@ -3320,7 +3314,7 @@ public class HBaseAdmin implements Admin {
default:
ZooKeeperWatcher zookeeper = null;
try {
- List<Pair<HRegionInfo, ServerName>> pairs;
+ List<Pair<RegionInfo, ServerName>> pairs;
if (TableName.META_TABLE_NAME.equals(tableName)) {
zookeeper = new ZooKeeperWatcher(conf, ZK_IDENTIFIER_PREFIX + connection.toString(),
new ThrowableAbortable());
@@ -3328,7 +3322,7 @@ public class HBaseAdmin implements Admin {
} else {
pairs = MetaTableAccessor.getTableRegionsAndLocations(connection, tableName);
}
- for (Pair<HRegionInfo, ServerName> pair: pairs) {
+ for (Pair<RegionInfo, ServerName> pair: pairs) {
if (pair.getFirst().isOffline()) continue;
if (pair.getSecond() == null) continue;
final ServerName sn = pair.getSecond();
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HRegionLocator.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HRegionLocator.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HRegionLocator.java
index 3416680..71d8b6f 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HRegionLocator.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HRegionLocator.java
@@ -22,7 +22,6 @@ import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
-import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HRegionLocation;
@@ -30,9 +29,11 @@ import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.RegionLocations;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.util.Pair;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability;
-import org.apache.hadoop.hbase.util.Pair;
+
+import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
/**
* An implementation of {@link RegionLocator}. Used to view region location information for a single
@@ -84,10 +85,10 @@ public class HRegionLocator implements RegionLocator {
@Override
public List<HRegionLocation> getAllRegionLocations() throws IOException {
TableName tableName = getName();
- List<Pair<HRegionInfo, ServerName>> locations =
+ List<Pair<RegionInfo, ServerName>> locations =
MetaTableAccessor.getTableRegionsAndLocations(this.connection, tableName);
ArrayList<HRegionLocation> regions = new ArrayList<>(locations.size());
- for (Pair<HRegionInfo, ServerName> entry : locations) {
+ for (Pair<RegionInfo, ServerName> entry : locations) {
regions.add(new HRegionLocation(entry.getFirst(), entry.getSecond()));
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableMultiplexer.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableMultiplexer.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableMultiplexer.java
index 48ae8d6..3a6e3b4 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableMultiplexer.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableMultiplexer.java
@@ -19,9 +19,6 @@
*/
package org.apache.hadoop.hbase.client;
-import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
-import org.apache.hadoop.hbase.shaded.com.google.common.util.concurrent.ThreadFactoryBuilder;
-
import java.io.IOException;
import java.util.AbstractMap.SimpleEntry;
import java.util.ArrayList;
@@ -43,13 +40,15 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
-import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.hbase.shaded.com.google.common.util.concurrent.ThreadFactoryBuilder;
/**
* HTableMultiplexer provides a thread-safe non blocking PUT API across all the tables.
@@ -207,7 +206,7 @@ public class HTableMultiplexer {
LinkedBlockingQueue<PutStatus> queue = getQueue(loc);
// Generate a MultiPutStatus object and offer it into the queue
- PutStatus s = new PutStatus(loc.getRegionInfo(), put, maxAttempts);
+ PutStatus s = new PutStatus(loc.getRegion(), put, maxAttempts);
return queue.offer(s);
}
@@ -372,11 +371,11 @@ public class HTableMultiplexer {
@VisibleForTesting
static class PutStatus {
- final HRegionInfo regionInfo;
+ final RegionInfo regionInfo;
final Put put;
final int maxAttempCount;
- public PutStatus(HRegionInfo regionInfo, Put put, int maxAttempCount) {
+ public PutStatus(RegionInfo regionInfo, Put put, int maxAttempCount) {
this.regionInfo = regionInfo;
this.put = put;
this.maxAttempCount = maxAttempCount;
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ImmutableHRegionInfo.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ImmutableHRegionInfo.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ImmutableHRegionInfo.java
index 16329c8..ed00cee 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ImmutableHRegionInfo.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ImmutableHRegionInfo.java
@@ -33,7 +33,7 @@ public class ImmutableHRegionInfo extends HRegionInfo {
*
* @param other
*/
- public ImmutableHRegionInfo(HRegionInfo other) {
+ public ImmutableHRegionInfo(RegionInfo other) {
super(other);
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
index b1aa940..589796a 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
@@ -47,7 +47,6 @@ import org.apache.hadoop.hbase.AsyncMetaTableAccessor;
import org.apache.hadoop.hbase.ClusterStatus;
import org.apache.hadoop.hbase.ClusterStatus.Option;
import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.MetaTableAccessor.QueryType;
@@ -61,7 +60,6 @@ import org.apache.hadoop.hbase.TableNotDisabledException;
import org.apache.hadoop.hbase.TableNotEnabledException;
import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.UnknownRegionException;
-import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.AsyncRpcRetryingCallerFactory.AdminRequestCallerBuilder;
import org.apache.hadoop.hbase.client.AsyncRpcRetryingCallerFactory.MasterRequestCallerBuilder;
import org.apache.hadoop.hbase.client.AsyncRpcRetryingCallerFactory.ServerRequestCallerBuilder;
@@ -78,6 +76,15 @@ import org.apache.hadoop.hbase.quotas.QuotaTableUtil;
import org.apache.hadoop.hbase.replication.ReplicationException;
import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
+import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
+import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException;
+import org.apache.hadoop.hbase.snapshot.SnapshotCreationException;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.ForeignExceptionUtil;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.yetus.audience.InterfaceAudience;
+
import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback;
import org.apache.hadoop.hbase.shaded.io.netty.util.Timeout;
import org.apache.hadoop.hbase.shaded.io.netty.util.TimerTask;
@@ -240,13 +247,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.Remov
import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos;
-import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
-import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException;
-import org.apache.hadoop.hbase.snapshot.SnapshotCreationException;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.hadoop.hbase.util.ForeignExceptionUtil;
-import org.apache.hadoop.hbase.util.Pair;
import com.google.common.annotations.VisibleForTesting;
import com.google.protobuf.Message;
@@ -595,7 +595,7 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
int notDeployed = 0;
int regionCount = 0;
for (HRegionLocation location : locations) {
- HRegionInfo info = location.getRegionInfo();
+ RegionInfo info = location.getRegionInfo();
if (location.getServerName() == null) {
if (LOG.isDebugEnabled()) {
LOG.debug("Table " + tableName + " has not deployed region "
@@ -739,10 +739,10 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
}
@Override
- public CompletableFuture<List<HRegionInfo>> getOnlineRegions(ServerName serverName) {
- return this.<List<HRegionInfo>> newAdminCaller()
+ public CompletableFuture<List<RegionInfo>> getOnlineRegions(ServerName serverName) {
+ return this.<List<RegionInfo>> newAdminCaller()
.action((controller, stub) -> this
- .<GetOnlineRegionRequest, GetOnlineRegionResponse, List<HRegionInfo>> adminCall(
+ .<GetOnlineRegionRequest, GetOnlineRegionResponse, List<RegionInfo>> adminCall(
controller, stub, RequestConverter.buildGetOnlineRegionRequest(),
(s, c, req, done) -> s.getOnlineRegion(c, req, done),
resp -> ProtobufUtil.getRegionInfos(resp)))
@@ -750,7 +750,7 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
}
@Override
- public CompletableFuture<List<HRegionInfo>> getTableRegions(TableName tableName) {
+ public CompletableFuture<List<RegionInfo>> getTableRegions(TableName tableName) {
if (tableName.equals(META_TABLE_NAME)) {
return connection.getLocator().getRegionLocation(tableName, null, null, operationTimeoutNs)
.thenApply(loc -> Arrays.asList(loc.getRegionInfo()));
@@ -807,7 +807,7 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
return;
}
- HRegionInfo regionInfo = location.getRegionInfo();
+ RegionInfo regionInfo = location.getRegionInfo();
this.<Void> newAdminCaller()
.serverName(serverName)
.action(
@@ -973,7 +973,7 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
/**
* Compact the region at specific region server.
*/
- private CompletableFuture<Void> compact(final ServerName sn, final HRegionInfo hri,
+ private CompletableFuture<Void> compact(final ServerName sn, final RegionInfo hri,
final boolean major, Optional<byte[]> columnFamily) {
return this
.<Void> newAdminCaller()
@@ -987,8 +987,8 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
private byte[] toEncodeRegionName(byte[] regionName) {
try {
- return HRegionInfo.isEncodedRegionName(regionName) ? regionName
- : Bytes.toBytes(HRegionInfo.encodeRegionName(regionName));
+ return RegionInfo.isEncodedRegionName(regionName) ? regionName
+ : Bytes.toBytes(RegionInfo.encodeRegionName(regionName));
} catch (IOException e) {
return regionName;
}
@@ -1002,8 +1002,8 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
result.completeExceptionally(err);
return;
}
- HRegionInfo regionInfo = location.getRegionInfo();
- if (regionInfo.getReplicaId() != HRegionInfo.DEFAULT_REPLICA_ID) {
+ RegionInfo regionInfo = location.getRegionInfo();
+ if (regionInfo.getReplicaId() != RegionInfo.DEFAULT_REPLICA_ID) {
result.completeExceptionally(new IllegalArgumentException(
"Can't invoke merge on non-default regions directly"));
return;
@@ -1138,14 +1138,14 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
if (results != null && !results.isEmpty()) {
List<CompletableFuture<Void>> splitFutures = new ArrayList<>();
for (Result r : results) {
- if (r.isEmpty() || MetaTableAccessor.getHRegionInfo(r) == null) continue;
+ if (r.isEmpty() || MetaTableAccessor.getRegionInfo(r) == null) continue;
RegionLocations rl = MetaTableAccessor.getRegionLocations(r);
if (rl != null) {
for (HRegionLocation h : rl.getRegionLocations()) {
if (h != null && h.getServerName() != null) {
- HRegionInfo hri = h.getRegionInfo();
+ RegionInfo hri = h.getRegion();
if (hri == null || hri.isSplitParent()
- || hri.getReplicaId() != HRegionInfo.DEFAULT_REPLICA_ID)
+ || hri.getReplicaId() != RegionInfo.DEFAULT_REPLICA_ID)
continue;
splitFutures.add(split(hri, Optional.empty()));
}
@@ -1202,8 +1202,8 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
CompletableFuture<Void> future = new CompletableFuture<>();
getRegionLocation(regionName).whenComplete(
(location, err) -> {
- HRegionInfo regionInfo = location.getRegionInfo();
- if (regionInfo.getReplicaId() != HRegionInfo.DEFAULT_REPLICA_ID) {
+ RegionInfo regionInfo = location.getRegionInfo();
+ if (regionInfo.getReplicaId() != RegionInfo.DEFAULT_REPLICA_ID) {
future.completeExceptionally(new IllegalArgumentException(
"Can't split replicas directly. "
+ "Replicas are auto-split when their primary is split."));
@@ -1226,7 +1226,7 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
return future;
}
- private CompletableFuture<Void> split(final HRegionInfo hri,
+ private CompletableFuture<Void> split(final RegionInfo hri,
Optional<byte[]> splitPoint) {
if (hri.getStartKey() != null && splitPoint.isPresent()
&& Bytes.compareTo(hri.getStartKey(), splitPoint.get()) == 0) {
@@ -2051,7 +2051,7 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
}
try {
CompletableFuture<Optional<HRegionLocation>> future;
- if (HRegionInfo.isEncodedRegionName(regionNameOrEncodedRegionName)) {
+ if (RegionInfo.isEncodedRegionName(regionNameOrEncodedRegionName)) {
future = AsyncMetaTableAccessor.getRegionLocationWithEncodedName(metaTable,
regionNameOrEncodedRegionName);
} else {
@@ -2087,19 +2087,19 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
* @param regionNameOrEncodedRegionName
* @return region info, wrapped by a {@link CompletableFuture}
*/
- private CompletableFuture<HRegionInfo> getRegionInfo(byte[] regionNameOrEncodedRegionName) {
+ private CompletableFuture<RegionInfo> getRegionInfo(byte[] regionNameOrEncodedRegionName) {
if (regionNameOrEncodedRegionName == null) {
return failedFuture(new IllegalArgumentException("Passed region name can't be null"));
}
if (Bytes.equals(regionNameOrEncodedRegionName,
- HRegionInfo.FIRST_META_REGIONINFO.getRegionName())
+ RegionInfoBuilder.FIRST_META_REGIONINFO.getRegionName())
|| Bytes.equals(regionNameOrEncodedRegionName,
- HRegionInfo.FIRST_META_REGIONINFO.getEncodedNameAsBytes())) {
- return CompletableFuture.completedFuture(HRegionInfo.FIRST_META_REGIONINFO);
+ RegionInfoBuilder.FIRST_META_REGIONINFO.getEncodedNameAsBytes())) {
+ return CompletableFuture.completedFuture(RegionInfoBuilder.FIRST_META_REGIONINFO);
}
- CompletableFuture<HRegionInfo> future = new CompletableFuture<>();
+ CompletableFuture<RegionInfo> future = new CompletableFuture<>();
getRegionLocation(regionNameOrEncodedRegionName).whenComplete((location, err) -> {
if (err != null) {
future.completeExceptionally(err);
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncTable.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncTable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncTable.java
index 4b3d44b..cd0226b 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncTable.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncTable.java
@@ -17,16 +17,15 @@
*/
package org.apache.hadoop.hbase.client;
-import com.google.protobuf.RpcCallback;
-import com.google.protobuf.RpcChannel;
-import com.google.protobuf.RpcController;
-
import java.util.concurrent.CompletableFuture;
import java.util.function.Function;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.yetus.audience.InterfaceAudience;
+import com.google.protobuf.RpcCallback;
+import com.google.protobuf.RpcChannel;
+import com.google.protobuf.RpcController;
+
/**
* A low level asynchronous table.
* <p>
@@ -134,10 +133,10 @@ public interface RawAsyncTable extends AsyncTableBase {
* As the locating itself also takes some time, the implementation may want to send rpc calls on
* the fly, which means we do not know how many regions we have when we get the return value of
* the rpc calls, so we need an {@link #onComplete()} which is used to tell you that we have
- * passed all the return values to you(through the {@link #onRegionComplete(HRegionInfo, Object)}
- * or {@link #onRegionError(HRegionInfo, Throwable)} calls), i.e, there will be no
- * {@link #onRegionComplete(HRegionInfo, Object)} or
- * {@link #onRegionError(HRegionInfo, Throwable)} calls in the future.
+ * passed all the return values to you(through the {@link #onRegionComplete(RegionInfo, Object)}
+ * or {@link #onRegionError(RegionInfo, Throwable)} calls), i.e, there will be no
+ * {@link #onRegionComplete(RegionInfo, Object)} or
+ * {@link #onRegionError(RegionInfo, Throwable)} calls in the future.
* <p>
* Here is a pseudo code to describe a typical implementation of a range coprocessor service
* method to help you better understand how the {@link CoprocessorCallback} will be called. The
@@ -179,18 +178,18 @@ public interface RawAsyncTable extends AsyncTableBase {
* @param region the region that the response belongs to
* @param resp the response of the coprocessor call
*/
- void onRegionComplete(HRegionInfo region, R resp);
+ void onRegionComplete(RegionInfo region, R resp);
/**
* @param region the region that the error belongs to
* @param error the response error of the coprocessor call
*/
- void onRegionError(HRegionInfo region, Throwable error);
+ void onRegionError(RegionInfo region, Throwable error);
/**
* Indicate that all responses of the regions have been notified by calling
- * {@link #onRegionComplete(HRegionInfo, Object)} or
- * {@link #onRegionError(HRegionInfo, Throwable)}.
+ * {@link #onRegionComplete(RegionInfo, Object)} or
+ * {@link #onRegionError(RegionInfo, Throwable)}.
*/
void onComplete();
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncTableImpl.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncTableImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncTableImpl.java
index 735ff30..722ee26 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncTableImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncTableImpl.java
@@ -23,8 +23,6 @@ import static org.apache.hadoop.hbase.HConstants.EMPTY_START_ROW;
import static org.apache.hadoop.hbase.client.ConnectionUtils.checkHasFamilies;
import static org.apache.hadoop.hbase.client.ConnectionUtils.isEmptyStopRow;
-import com.google.protobuf.RpcChannel;
-
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
@@ -38,13 +36,15 @@ import java.util.function.Function;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.CompareOperator;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.TableName;
-import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.AsyncRpcRetryingCallerFactory.SingleRequestCallerBuilder;
import org.apache.hadoop.hbase.filter.BinaryComparator;
import org.apache.hadoop.hbase.ipc.HBaseRpcController;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.ReflectionUtils;
+import org.apache.yetus.audience.InterfaceAudience;
+
import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
@@ -58,8 +58,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateRequ
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionAction;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.CompareType;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.ReflectionUtils;
+
+import com.google.protobuf.RpcChannel;
/**
* The implementation of RawAsyncTable.
@@ -439,7 +439,7 @@ class RawAsyncTableImpl implements RawAsyncTable {
}
private <S, R> CompletableFuture<R> coprocessorService(Function<RpcChannel, S> stubMaker,
- CoprocessorCallable<S, R> callable, HRegionInfo region, byte[] row) {
+ CoprocessorCallable<S, R> callable, RegionInfo region, byte[] row) {
RegionCoprocessorRpcChannelImpl channel = new RegionCoprocessorRpcChannelImpl(conn, tableName,
region, row, rpcTimeoutNs, operationTimeoutNs);
S stub = stubMaker.apply(channel);
@@ -461,7 +461,7 @@ class RawAsyncTableImpl implements RawAsyncTable {
return coprocessorService(stubMaker, callable, null, row);
}
- private boolean locateFinished(HRegionInfo region, byte[] endKey, boolean endKeyInclusive) {
+ private boolean locateFinished(RegionInfo region, byte[] endKey, boolean endKeyInclusive) {
if (isEmptyStopRow(endKey)) {
if (isEmptyStopRow(region.getEndKey())) {
return true;
@@ -488,7 +488,7 @@ class RawAsyncTableImpl implements RawAsyncTable {
return;
}
unfinishedRequest.incrementAndGet();
- HRegionInfo region = loc.getRegionInfo();
+ RegionInfo region = loc.getRegionInfo();
if (locateFinished(region, endKey, endKeyInclusive)) {
locateFinished.set(true);
} else {
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionCoprocessorRpcChannelImpl.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionCoprocessorRpcChannelImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionCoprocessorRpcChannelImpl.java
index ee30c16..d7684c2 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionCoprocessorRpcChannelImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionCoprocessorRpcChannelImpl.java
@@ -17,27 +17,27 @@
*/
package org.apache.hadoop.hbase.client;
-import com.google.protobuf.Descriptors.MethodDescriptor;
-import com.google.protobuf.Message;
-import com.google.protobuf.RpcCallback;
-import com.google.protobuf.RpcChannel;
-import com.google.protobuf.RpcController;
-
import java.io.IOException;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.TimeUnit;
import org.apache.hadoop.hbase.DoNotRetryIOException;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.TableName;
-import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils;
import org.apache.hadoop.hbase.ipc.HBaseRpcController;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
+
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceResponse;
-import org.apache.hadoop.hbase.util.Bytes;
+
+import com.google.protobuf.Descriptors.MethodDescriptor;
+import com.google.protobuf.Message;
+import com.google.protobuf.RpcCallback;
+import com.google.protobuf.RpcChannel;
+import com.google.protobuf.RpcController;
/**
* The implementation of a region based coprocessor rpc channel.
@@ -49,7 +49,7 @@ class RegionCoprocessorRpcChannelImpl implements RpcChannel {
private final TableName tableName;
- private final HRegionInfo region;
+ private final RegionInfo region;
private final byte[] row;
@@ -57,7 +57,7 @@ class RegionCoprocessorRpcChannelImpl implements RpcChannel {
private final long operationTimeoutNs;
- RegionCoprocessorRpcChannelImpl(AsyncConnectionImpl conn, TableName tableName, HRegionInfo region,
+ RegionCoprocessorRpcChannelImpl(AsyncConnectionImpl conn, TableName tableName, RegionInfo region,
byte[] row, long rpcTimeoutNs, long operationTimeoutNs) {
this.conn = conn;
this.tableName = tableName;
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfo.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfo.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfo.java
index 3646722..e9716e9 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfo.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfo.java
@@ -18,17 +18,6 @@
*/
package org.apache.hadoop.hbase.client;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.exceptions.DeserializationException;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.MD5Hash;
-import org.apache.hadoop.io.DataInputBuffer;
-import org.apache.hadoop.util.StringUtils;
-import org.apache.yetus.audience.InterfaceAudience;
-
import java.io.DataInputStream;
import java.io.IOException;
import java.util.ArrayList;
@@ -36,9 +25,21 @@ import java.util.Arrays;
import java.util.Comparator;
import java.util.List;
import java.util.stream.Collectors;
+
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.util.ByteArrayHashKey;
+import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.HashKey;
import org.apache.hadoop.hbase.util.JenkinsHash;
+import org.apache.hadoop.hbase.util.MD5Hash;
+import org.apache.hadoop.io.DataInputBuffer;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
/**
* Information about a region. A region is a range of keys in the whole keyspace
@@ -436,7 +437,7 @@ public interface RegionInfo {
* @see #parseFrom(byte[])
*/
static byte [] toByteArray(RegionInfo ri) {
- byte [] bytes = ProtobufUtil.toProtoRegionInfo(ri).toByteArray();
+ byte [] bytes = ProtobufUtil.toRegionInfo(ri).toByteArray();
return ProtobufUtil.prependPBMagic(bytes);
}
@@ -691,7 +692,7 @@ public interface RegionInfo {
* @throws IOException
*/
static byte [] toDelimitedByteArray(RegionInfo ri) throws IOException {
- return ProtobufUtil.toDelimitedByteArray(ProtobufUtil.toProtoRegionInfo(ri));
+ return ProtobufUtil.toDelimitedByteArray(ProtobufUtil.toRegionInfo(ri));
}
/**
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfoBuilder.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfoBuilder.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfoBuilder.java
index a76767d..7d5c476 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfoBuilder.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfoBuilder.java
@@ -18,16 +18,17 @@
*/
package org.apache.hadoop.hbase.client;
+import java.util.Arrays;
+
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.util.ArrayUtils;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.yetus.audience.InterfaceAudience;
-import java.util.Arrays;
-
@InterfaceAudience.Private
public class RegionInfoBuilder {
private static final Log LOG = LogFactory.getLog(RegionInfoBuilder.class);
@@ -52,7 +53,15 @@ public class RegionInfoBuilder {
public static final RegionInfo FIRST_META_REGIONINFO =
new MutableRegionInfo(1L, TableName.META_TABLE_NAME, RegionInfo.DEFAULT_REPLICA_ID);
- private MutableRegionInfo content = null;
+ private final TableName tableName;
+ private byte[] startKey = HConstants.EMPTY_START_ROW;
+ private byte[] endKey = HConstants.EMPTY_END_ROW;
+ private long regionId = System.currentTimeMillis();
+ private int replicaId = RegionInfo.DEFAULT_REPLICA_ID;
+ private boolean offLine = false;
+ private boolean split = false;
+ private byte[] regionName = null;
+ private String encodedName = null;
public static RegionInfoBuilder newBuilder(TableName tableName) {
return new RegionInfoBuilder(tableName);
@@ -63,52 +72,54 @@ public class RegionInfoBuilder {
}
private RegionInfoBuilder(TableName tableName) {
- this.content = new MutableRegionInfo(tableName);
+ this.tableName = tableName;
}
private RegionInfoBuilder(RegionInfo regionInfo) {
- this.content = new MutableRegionInfo(regionInfo);
+ this.tableName = regionInfo.getTable();
+ this.startKey = regionInfo.getStartKey();
+ this.endKey = regionInfo.getEndKey();
+ this.offLine = regionInfo.isOffline();
+ this.split = regionInfo.isSplit();
+ this.regionId = regionInfo.getRegionId();
+ this.replicaId = regionInfo.getReplicaId();
+ this.regionName = regionInfo.getRegionName();
+ this.encodedName = regionInfo.getEncodedName();
}
public RegionInfoBuilder setStartKey(byte[] startKey) {
- content.setStartKey(startKey);
+ this.startKey = startKey;
return this;
}
public RegionInfoBuilder setEndKey(byte[] endKey) {
- content.setEndKey(endKey);
+ this.endKey = endKey;
return this;
}
public RegionInfoBuilder setRegionId(long regionId) {
- content.setRegionId(regionId);
+ this.regionId = regionId;
return this;
}
public RegionInfoBuilder setReplicaId(int replicaId) {
- content.setReplicaId(replicaId);
+ this.replicaId = replicaId;
return this;
}
- public RegionInfoBuilder setSplit(boolean isSplit) {
- content.setSplit(isSplit);
+ public RegionInfoBuilder setSplit(boolean split) {
+ this.split = split;
return this;
}
- public RegionInfoBuilder setOffline(boolean isOffline) {
- content.setOffline(isOffline);
+ public RegionInfoBuilder setOffline(boolean offLine) {
+ this.offLine = offLine;
return this;
}
public RegionInfo build() {
- RegionInfo ri = new MutableRegionInfo(content);
- // Run a late check that we are not creating default meta region.
- if (ri.getTable().equals(TableName.META_TABLE_NAME) &&
- ri.getReplicaId() == RegionInfo.DEFAULT_REPLICA_ID) {
- throw new IllegalArgumentException("Cannot create the default meta region; " +
- "use static define FIRST_META_REGIONINFO");
- }
- return new MutableRegionInfo(content);
+ return new MutableRegionInfo(tableName, startKey, endKey, split,
+ regionId, replicaId, offLine, regionName, encodedName);
}
/**
@@ -144,26 +155,49 @@ public class RegionInfoBuilder {
// but now table state is kept up in zookeeper as of 0.90.0 HBase.
private boolean offLine = false;
private boolean split = false;
- private long regionId = -1;
- private int replicaId = RegionInfo.DEFAULT_REPLICA_ID;
- private transient byte [] regionName = HConstants.EMPTY_BYTE_ARRAY;
- private byte [] startKey = HConstants.EMPTY_BYTE_ARRAY;
- private byte [] endKey = HConstants.EMPTY_BYTE_ARRAY;
- private int hashCode = -1;
- private String encodedName;
- private byte [] encodedNameAsBytes;
- // Current TableName
- private TableName tableName;
-
- private void setHashCode() {
- int result = Arrays.hashCode(this.regionName);
- result ^= this.regionId;
- result ^= Arrays.hashCode(this.startKey);
- result ^= Arrays.hashCode(this.endKey);
- result ^= Boolean.valueOf(this.offLine).hashCode();
- result ^= Arrays.hashCode(this.tableName.getName());
- result ^= this.replicaId;
- this.hashCode = result;
+ private final long regionId;
+ private final int replicaId;
+ private final byte[] regionName;
+ private final byte[] startKey;
+ private final byte[] endKey;
+ private final int hashCode;
+ private final String encodedName;
+ private final byte[] encodedNameAsBytes;
+ private final TableName tableName;
+
+ private static int generateHashCode(final TableName tableName, final byte[] startKey,
+ final byte[] endKey, final long regionId,
+ final int replicaId, boolean offLine, byte[] regionName) {
+ int result = Arrays.hashCode(regionName);
+ result ^= regionId;
+ result ^= Arrays.hashCode(checkStartKey(startKey));
+ result ^= Arrays.hashCode(checkEndKey(endKey));
+ result ^= Boolean.valueOf(offLine).hashCode();
+ result ^= Arrays.hashCode(tableName.getName());
+ result ^= replicaId;
+ return result;
+ }
+
+ private static byte[] checkStartKey(byte[] startKey) {
+ return startKey == null? HConstants.EMPTY_START_ROW: startKey;
+ }
+
+ private static byte[] checkEndKey(byte[] endKey) {
+ return endKey == null? HConstants.EMPTY_END_ROW: endKey;
+ }
+
+ private static TableName checkTableName(TableName tableName) {
+ if (tableName == null) {
+ throw new IllegalArgumentException("TableName cannot be null");
+ }
+ return tableName;
+ }
+
+ private static int checkReplicaId(int regionId) {
+ if (regionId > MAX_REPLICA_ID) {
+ throw new IllegalArgumentException("ReplicaId cannot be greater than" + MAX_REPLICA_ID);
+ }
+ return regionId;
}
/**
@@ -171,162 +205,57 @@ public class RegionInfoBuilder {
* first meta regions
*/
private MutableRegionInfo(long regionId, TableName tableName, int replicaId) {
- // This constructor is currently private for making hbase:meta region only.
- super();
- this.regionId = regionId;
- this.tableName = tableName;
- this.replicaId = replicaId;
- // Note: First Meta region replicas names are in old format so we pass false here.
- this.regionName =
- RegionInfo.createRegionName(tableName, null, regionId, replicaId, false);
- setHashCode();
- }
-
- MutableRegionInfo(final TableName tableName) {
- this(tableName, null, null);
+ this(tableName,
+ HConstants.EMPTY_START_ROW,
+ HConstants.EMPTY_END_ROW,
+ false,
+ regionId,
+ replicaId,
+ false,
+ RegionInfo.createRegionName(tableName, null, regionId, replicaId, false));
}
- /**
- * Construct MutableRegionInfo with explicit parameters
- *
- * @param tableName the table name
- * @param startKey first key in region
- * @param endKey end of key range
- * @throws IllegalArgumentException
- */
- MutableRegionInfo(final TableName tableName, final byte[] startKey, final byte[] endKey)
- throws IllegalArgumentException {
- this(tableName, startKey, endKey, false);
- }
-
- /**
- * Construct MutableRegionInfo with explicit parameters
- *
- * @param tableName the table descriptor
- * @param startKey first key in region
- * @param endKey end of key range
- * @param split true if this region has split and we have daughter regions
- * regions that may or may not hold references to this region.
- * @throws IllegalArgumentException
- */
- MutableRegionInfo(final TableName tableName, final byte[] startKey, final byte[] endKey,
- final boolean split)
- throws IllegalArgumentException {
- this(tableName, startKey, endKey, split, System.currentTimeMillis());
- }
-
- /**
- * Construct MutableRegionInfo with explicit parameters
- *
- * @param tableName the table descriptor
- * @param startKey first key in region
- * @param endKey end of key range
- * @param split true if this region has split and we have daughter regions
- * regions that may or may not hold references to this region.
- * @param regionid Region id to use.
- * @throws IllegalArgumentException
- */
MutableRegionInfo(final TableName tableName, final byte[] startKey,
- final byte[] endKey, final boolean split, final long regionid)
- throws IllegalArgumentException {
- this(tableName, startKey, endKey, split, regionid, RegionInfo.DEFAULT_REPLICA_ID);
+ final byte[] endKey, final boolean split, final long regionId,
+ final int replicaId, boolean offLine, byte[] regionName) {
+ this(checkTableName(tableName),
+ checkStartKey(startKey),
+ checkEndKey(endKey),
+ split, regionId,
+ checkReplicaId(replicaId),
+ offLine,
+ regionName,
+ RegionInfo.encodeRegionName(regionName));
}
- /**
- * Construct MutableRegionInfo with explicit parameters
- *
- * @param tableName the table descriptor
- * @param startKey first key in region
- * @param endKey end of key range
- * @param split true if this region has split and we have daughter regions
- * regions that may or may not hold references to this region.
- * @param regionid Region id to use.
- * @param replicaId the replicaId to use
- * @throws IllegalArgumentException
- */
MutableRegionInfo(final TableName tableName, final byte[] startKey,
- final byte[] endKey, final boolean split, final long regionid,
- final int replicaId)
- throws IllegalArgumentException {
- super();
- if (tableName == null) {
- throw new IllegalArgumentException("TableName cannot be null");
- }
- this.tableName = tableName;
- this.offLine = false;
- this.regionId = regionid;
- this.replicaId = replicaId;
- if (this.replicaId > MAX_REPLICA_ID) {
- throw new IllegalArgumentException("ReplicaId cannot be greater than" + MAX_REPLICA_ID);
- }
-
- this.regionName = RegionInfo.createRegionName(this.tableName, startKey, regionId, replicaId,
- !this.tableName.equals(TableName.META_TABLE_NAME));
-
+ final byte[] endKey, final boolean split, final long regionId,
+ final int replicaId, boolean offLine, byte[] regionName, String encodedName) {
+ this.tableName = checkTableName(tableName);
+ this.startKey = checkStartKey(startKey);
+ this.endKey = checkEndKey(endKey);
this.split = split;
- this.endKey = endKey == null? HConstants.EMPTY_END_ROW: endKey.clone();
- this.startKey = startKey == null?
- HConstants.EMPTY_START_ROW: startKey.clone();
- this.tableName = tableName;
- setHashCode();
- }
-
- /**
- * Construct MutableRegionInfo.
- * Only for RegionInfoBuilder to use.
- * @param other
- */
- MutableRegionInfo(MutableRegionInfo other, boolean isMetaRegion) {
- super();
- if (other.getTable() == null) {
- throw new IllegalArgumentException("TableName cannot be null");
- }
- this.tableName = other.getTable();
- this.offLine = other.isOffline();
- this.regionId = other.getRegionId();
- this.replicaId = other.getReplicaId();
- if (this.replicaId > MAX_REPLICA_ID) {
- throw new IllegalArgumentException("ReplicaId cannot be greater than" + MAX_REPLICA_ID);
- }
-
- if(isMetaRegion) {
- // Note: First Meta region replicas names are in old format
- this.regionName = RegionInfo.createRegionName(
- other.getTable(), null, other.getRegionId(),
- other.getReplicaId(), false);
+ this.regionId = regionId;
+ this.replicaId = checkReplicaId(replicaId);
+ this.offLine = offLine;
+ if (ArrayUtils.isEmpty(regionName)) {
+ this.regionName = RegionInfo.createRegionName(this.tableName, this.startKey, this.regionId, this.replicaId,
+ !this.tableName.equals(TableName.META_TABLE_NAME));
+ this.encodedName = RegionInfo.encodeRegionName(this.regionName);
} else {
- this.regionName = RegionInfo.createRegionName(
- other.getTable(), other.getStartKey(), other.getRegionId(),
- other.getReplicaId(), true);
+ this.regionName = regionName;
+ this.encodedName = encodedName;
}
-
- this.split = other.isSplit();
- this.endKey = other.getEndKey() == null? HConstants.EMPTY_END_ROW: other.getEndKey().clone();
- this.startKey = other.getStartKey() == null?
- HConstants.EMPTY_START_ROW: other.getStartKey().clone();
- this.tableName = other.getTable();
- setHashCode();
- }
-
- /**
- * Construct a copy of RegionInfo as MutableRegionInfo.
- * Only for RegionInfoBuilder to use.
- * @param regionInfo
- */
- MutableRegionInfo(RegionInfo regionInfo) {
- super();
- this.endKey = regionInfo.getEndKey();
- this.offLine = regionInfo.isOffline();
- this.regionId = regionInfo.getRegionId();
- this.regionName = regionInfo.getRegionName();
- this.split = regionInfo.isSplit();
- this.startKey = regionInfo.getStartKey();
- this.hashCode = regionInfo.hashCode();
- this.encodedName = regionInfo.getEncodedName();
- this.tableName = regionInfo.getTable();
- this.replicaId = regionInfo.getReplicaId();
+ this.hashCode = generateHashCode(
+ this.tableName,
+ this.startKey,
+ this.endKey,
+ this.regionId,
+ this.replicaId,
+ this.offLine,
+ this.regionName);
+ this.encodedNameAsBytes = Bytes.toBytes(this.encodedName);
}
-
/**
* @return Return a short, printable name for this region
* (usually encoded name) for us logging.
@@ -342,15 +271,6 @@ public class RegionInfoBuilder {
return regionId;
}
- /**
- * set region id.
- * @param regionId
- * @return MutableRegionInfo
- */
- public MutableRegionInfo setRegionId(long regionId) {
- this.regionId = regionId;
- return this;
- }
/**
* @return the regionName as an array of bytes.
@@ -362,16 +282,6 @@ public class RegionInfoBuilder {
}
/**
- * set region name.
- * @param regionName
- * @return MutableRegionInfo
- */
- public MutableRegionInfo setRegionName(byte[] regionName) {
- this.regionName = regionName;
- return this;
- }
-
- /**
* @return Region name as a String for use in logging, etc.
*/
@Override
@@ -389,18 +299,12 @@ public class RegionInfoBuilder {
/** @return the encoded region name */
@Override
- public synchronized String getEncodedName() {
- if (this.encodedName == null) {
- this.encodedName = RegionInfo.encodeRegionName(this.regionName);
- }
+ public String getEncodedName() {
return this.encodedName;
}
@Override
- public synchronized byte [] getEncodedNameAsBytes() {
- if (this.encodedNameAsBytes == null) {
- this.encodedNameAsBytes = Bytes.toBytes(getEncodedName());
- }
+ public byte [] getEncodedNameAsBytes() {
return this.encodedNameAsBytes;
}
@@ -410,14 +314,6 @@ public class RegionInfoBuilder {
return startKey;
}
- /**
- * @param startKey
- * @return MutableRegionInfo
- */
- public MutableRegionInfo setStartKey(byte[] startKey) {
- this.startKey = startKey;
- return this;
- }
/** @return the endKey */
@Override
@@ -426,26 +322,11 @@ public class RegionInfoBuilder {
}
/**
- * @param endKey
- * @return MutableRegionInfo
- */
- public MutableRegionInfo setEndKey(byte[] endKey) {
- this.endKey = endKey;
- return this;
- }
-
- /**
* Get current table name of the region
* @return TableName
*/
@Override
public TableName getTable() {
- // This method name should be getTableName but there was already a method getTableName
- // that returned a byte array. It is unfortunate given everywhere else, getTableName returns
- // a TableName instance.
- if (tableName == null || tableName.getName().length == 0) {
- tableName = RegionInfo.getTable(getRegionName());
- }
return this.tableName;
}
@@ -560,11 +441,6 @@ public class RegionInfoBuilder {
return replicaId;
}
- public MutableRegionInfo setReplicaId(int replicaId) {
- this.replicaId = replicaId;
- return this;
- }
-
/**
* @see java.lang.Object#toString()
*/
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionReplicaUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionReplicaUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionReplicaUtil.java
index 2c1d478..d444c82 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionReplicaUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionReplicaUtil.java
@@ -21,9 +21,8 @@ package org.apache.hadoop.hbase.client;
import java.util.Collection;
import java.util.Iterator;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
/**
* Utility methods which contain the logic for regions and replicas.
@@ -50,30 +49,6 @@ public class RegionReplicaUtil {
static final int DEFAULT_REPLICA_ID = 0;
/**
- * Returns the HRegionInfo for the given replicaId. HRegionInfo's correspond to
- * a range of a table, but more than one "instance" of the same range can be
- * deployed which are differentiated by the replicaId.
- * @param replicaId the replicaId to use
- * @return an HRegionInfo object corresponding to the same range (table, start and
- * end key), but for the given replicaId.
- */
- @Deprecated // Deprecated for HBase-2.0.0, use #getRegionInfoForReplica
- public static HRegionInfo getRegionInfoForReplica(HRegionInfo regionInfo, int replicaId) {
- if (regionInfo.getReplicaId() == replicaId) {
- return regionInfo;
- }
- HRegionInfo replicaInfo;
- if (regionInfo.isMetaRegion()) {
- replicaInfo = new HRegionInfo(regionInfo.getRegionId(), regionInfo.getTable(), replicaId);
- } else {
- replicaInfo = new HRegionInfo(regionInfo.getTable(), regionInfo.getStartKey(),
- regionInfo.getEndKey(), regionInfo.isSplit(), regionInfo.getRegionId(), replicaId);
- }
- replicaInfo.setOffline(regionInfo.isOffline());
- return replicaInfo;
- }
-
- /**
* Returns the RegionInfo for the given replicaId.
* RegionInfo's correspond to a range of a table, but more than one
* "instance" of the same range can be deployed which are differentiated by
@@ -87,31 +62,33 @@ public class RegionReplicaUtil {
if (regionInfo.getReplicaId() == replicaId) {
return regionInfo;
}
- RegionInfoBuilder replicaInfo;
- RegionInfo ri;
+
if (regionInfo.isMetaRegion()) {
- ri = RegionInfoBuilder.FIRST_META_REGIONINFO;
+ return RegionInfoBuilder.newBuilder(regionInfo.getTable())
+ .setRegionId(regionInfo.getRegionId())
+ .setReplicaId(replicaId)
+ .setOffline(regionInfo.isOffline())
+ .build();
} else {
- replicaInfo = RegionInfoBuilder.newBuilder(regionInfo.getTable())
+ return RegionInfoBuilder.newBuilder(regionInfo.getTable())
.setStartKey(regionInfo.getStartKey())
.setEndKey(regionInfo.getEndKey())
.setSplit(regionInfo.isSplit())
.setRegionId(regionInfo.getRegionId())
- .setReplicaId(replicaId);
- replicaInfo.setOffline(regionInfo.isOffline());
- ri = replicaInfo.build();
+ .setReplicaId(replicaId)
+ .setOffline(regionInfo.isOffline())
+ .build();
}
- return ri;
}
/**
- * Returns the HRegionInfo for the default replicaId (0). HRegionInfo's correspond to
+ * Returns the RegionInfo for the default replicaId (0). RegionInfo's correspond to
* a range of a table, but more than one "instance" of the same range can be
* deployed which are differentiated by the replicaId.
- * @return an HRegionInfo object corresponding to the same range (table, start and
+ * @return an RegionInfo object corresponding to the same range (table, start and
* end key), but for the default replicaId.
*/
- public static HRegionInfo getRegionInfoForDefaultReplica(HRegionInfo regionInfo) {
+ public static RegionInfo getRegionInfoForDefaultReplica(RegionInfo regionInfo) {
return getRegionInfoForReplica(regionInfo, DEFAULT_REPLICA_ID);
}
@@ -121,7 +98,7 @@ public class RegionReplicaUtil {
}
/** @return true if this region is a default replica for the region */
- public static boolean isDefaultReplica(HRegionInfo hri) {
+ public static boolean isDefaultReplica(RegionInfo hri) {
return hri.getReplicaId() == DEFAULT_REPLICA_ID;
}
@@ -129,22 +106,22 @@ public class RegionReplicaUtil {
* Removes the non-default replicas from the passed regions collection
* @param regions
*/
- public static void removeNonDefaultRegions(Collection<HRegionInfo> regions) {
- Iterator<HRegionInfo> iterator = regions.iterator();
+ public static void removeNonDefaultRegions(Collection<RegionInfo> regions) {
+ Iterator<RegionInfo> iterator = regions.iterator();
while (iterator.hasNext()) {
- HRegionInfo hri = iterator.next();
+ RegionInfo hri = iterator.next();
if (!RegionReplicaUtil.isDefaultReplica(hri)) {
iterator.remove();
}
}
}
- public static boolean isReplicasForSameRegion(HRegionInfo regionInfoA, HRegionInfo regionInfoB) {
+ public static boolean isReplicasForSameRegion(RegionInfo regionInfoA, RegionInfo regionInfoB) {
return compareRegionInfosWithoutReplicaId(regionInfoA, regionInfoB) == 0;
}
- private static int compareRegionInfosWithoutReplicaId(HRegionInfo regionInfoA,
- HRegionInfo regionInfoB) {
+ private static int compareRegionInfosWithoutReplicaId(RegionInfo regionInfoA,
+ RegionInfo regionInfoB) {
int result = regionInfoA.getTable().compareTo(regionInfoB.getTable());
if (result != 0) {
return result;
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZooKeeperRegistry.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZooKeeperRegistry.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZooKeeperRegistry.java
index bbd726b..746382f 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZooKeeperRegistry.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZooKeeperRegistry.java
@@ -22,14 +22,13 @@ import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.RegionLocations;
import org.apache.hadoop.hbase.ServerName;
-import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
+import org.apache.yetus.audience.InterfaceAudience;
import org.apache.zookeeper.KeeperException;
/**
@@ -76,8 +75,8 @@ class ZooKeeperRegistry implements Registry {
HRegionLocation[] locs = new HRegionLocation[servers.size()];
int i = 0;
for (ServerName server : servers) {
- HRegionInfo h = RegionReplicaUtil.getRegionInfoForReplica(
- HRegionInfo.FIRST_META_REGIONINFO, i);
+ RegionInfo h = RegionReplicaUtil.getRegionInfoForReplica(
+ RegionInfoBuilder.FIRST_META_REGIONINFO, i);
if (server == null) locs[i++] = null;
else locs[i++] = new HRegionLocation(h, server, 0);
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-client/src/main/java/org/apache/hadoop/hbase/master/RegionState.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/master/RegionState.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/master/RegionState.java
index dedc906..7598067 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/master/RegionState.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/master/RegionState.java
@@ -19,10 +19,12 @@ package org.apache.hadoop.hbase.master;
import java.util.Date;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability;
+
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos;
/**
@@ -161,27 +163,27 @@ public class RegionState {
}
private final long stamp;
- private final HRegionInfo hri;
+ private final RegionInfo hri;
private final ServerName serverName;
private final State state;
// The duration of region in transition
private long ritDuration;
- public RegionState(HRegionInfo region, State state) {
+ public RegionState(RegionInfo region, State state) {
this(region, state, System.currentTimeMillis(), null);
}
- public RegionState(HRegionInfo region,
+ public RegionState(RegionInfo region,
State state, ServerName serverName) {
this(region, state, System.currentTimeMillis(), serverName);
}
- public RegionState(HRegionInfo region,
+ public RegionState(RegionInfo region,
State state, long stamp, ServerName serverName) {
this(region, state, stamp, serverName, 0);
}
- public RegionState(HRegionInfo region, State state, long stamp, ServerName serverName,
+ public RegionState(RegionInfo region, State state, long stamp, ServerName serverName,
long ritDuration) {
this.hri = region;
this.state = state;
@@ -198,7 +200,7 @@ public class RegionState {
return stamp;
}
- public HRegionInfo getRegion() {
+ public RegionInfo getRegion() {
return hri;
}
@@ -381,7 +383,7 @@ public class RegionState {
*/
public ClusterStatusProtos.RegionState convert() {
ClusterStatusProtos.RegionState.Builder regionState = ClusterStatusProtos.RegionState.newBuilder();
- regionState.setRegionInfo(HRegionInfo.convert(hri));
+ regionState.setRegionInfo(ProtobufUtil.toRegionInfo(hri));
regionState.setState(state.convert());
regionState.setStamp(getStamp());
return regionState.build();
@@ -393,7 +395,7 @@ public class RegionState {
* @return the RegionState
*/
public static RegionState convert(ClusterStatusProtos.RegionState proto) {
- return new RegionState(HRegionInfo.convert(proto.getRegionInfo()),
+ return new RegionState(ProtobufUtil.toRegionInfo(proto.getRegionInfo()),
State.convert(proto.getState()), proto.getStamp(), null);
}
@@ -407,7 +409,8 @@ public class RegionState {
return false;
}
RegionState tmp = (RegionState)obj;
- return tmp.hri.equals(hri) && tmp.state == state
+
+ return RegionInfo.COMPARATOR.compare(tmp.hri, hri) == 0 && tmp.state == state
&& ((serverName != null && serverName.equals(tmp.serverName))
|| (tmp.serverName == null && serverName == null));
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
index 3c9738e..1de57c5 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
@@ -25,6 +25,7 @@ import java.lang.reflect.Method;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.Collection;
+import java.util.Collections;
import java.util.EnumSet;
import java.util.HashMap;
import java.util.List;
@@ -56,7 +57,6 @@ import org.apache.hadoop.hbase.ExtendedCellBuilderFactory;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseIOException;
import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.ServerLoad;
@@ -1721,14 +1721,14 @@ public final class ProtobufUtil {
* @return the retrieved region info
* @throws IOException
*/
- public static HRegionInfo getRegionInfo(final RpcController controller,
+ public static org.apache.hadoop.hbase.client.RegionInfo getRegionInfo(final RpcController controller,
final AdminService.BlockingInterface admin, final byte[] regionName) throws IOException {
try {
GetRegionInfoRequest request =
RequestConverter.buildGetRegionInfoRequest(regionName);
GetRegionInfoResponse response =
admin.getRegionInfo(controller, request);
- return HRegionInfo.convert(response.getRegionInfo());
+ return toRegionInfo(response.getRegionInfo());
} catch (ServiceException se) {
throw getRemoteException(se);
}
@@ -1787,7 +1787,7 @@ public final class ProtobufUtil {
*
*/
public static void warmupRegion(final RpcController controller,
- final AdminService.BlockingInterface admin, final HRegionInfo regionInfo) throws IOException {
+ final AdminService.BlockingInterface admin, final org.apache.hadoop.hbase.client.RegionInfo regionInfo) throws IOException {
try {
WarmupRegionRequest warmupRegionRequest =
@@ -1806,7 +1806,7 @@ public final class ProtobufUtil {
* @throws IOException
*/
public static void openRegion(final RpcController controller,
- final AdminService.BlockingInterface admin, ServerName server, final HRegionInfo region)
+ final AdminService.BlockingInterface admin, ServerName server, final org.apache.hadoop.hbase.client.RegionInfo region)
throws IOException {
OpenRegionRequest request =
RequestConverter.buildOpenRegionRequest(server, region, null, null);
@@ -1825,7 +1825,7 @@ public final class ProtobufUtil {
* @return a list of online region info
* @throws IOException
*/
- public static List<HRegionInfo> getOnlineRegions(final AdminService.BlockingInterface admin)
+ public static List<org.apache.hadoop.hbase.client.RegionInfo> getOnlineRegions(final AdminService.BlockingInterface admin)
throws IOException {
return getOnlineRegions(null, admin);
}
@@ -1835,7 +1835,7 @@ public final class ProtobufUtil {
* server using admin protocol.
* @return a list of online region info
*/
- public static List<HRegionInfo> getOnlineRegions(final RpcController controller,
+ public static List<org.apache.hadoop.hbase.client.RegionInfo> getOnlineRegions(final RpcController controller,
final AdminService.BlockingInterface admin)
throws IOException {
GetOnlineRegionRequest request = RequestConverter.buildGetOnlineRegionRequest();
@@ -1852,13 +1852,13 @@ public final class ProtobufUtil {
* Get the list of region info from a GetOnlineRegionResponse
*
* @param proto the GetOnlineRegionResponse
- * @return the list of region info or null if <code>proto</code> is null
+ * @return the list of region info or empty if <code>proto</code> is null
*/
- public static List<HRegionInfo> getRegionInfos(final GetOnlineRegionResponse proto) {
- if (proto == null) return null;
- List<HRegionInfo> regionInfos = new ArrayList<>(proto.getRegionInfoList().size());
+ public static List<org.apache.hadoop.hbase.client.RegionInfo> getRegionInfos(final GetOnlineRegionResponse proto) {
+ if (proto == null) return Collections.EMPTY_LIST;
+ List<org.apache.hadoop.hbase.client.RegionInfo> regionInfos = new ArrayList<>(proto.getRegionInfoList().size());
for (RegionInfo regionInfo: proto.getRegionInfoList()) {
- regionInfos.add(HRegionInfo.convert(regionInfo));
+ regionInfos.add(toRegionInfo(regionInfo));
}
return regionInfos;
}
@@ -1950,7 +1950,7 @@ public final class ProtobufUtil {
RegionSpecifierType type = regionSpecifier.getType();
switch (type) {
case REGION_NAME:
- return HRegionInfo.encodeRegionName(value.toByteArray());
+ return org.apache.hadoop.hbase.client.RegionInfo.encodeRegionName(value.toByteArray());
case ENCODED_REGION_NAME:
return value.toStringUtf8();
default:
@@ -2074,12 +2074,14 @@ public final class ProtobufUtil {
return b.build();
}
- public static CompactionDescriptor toCompactionDescriptor(HRegionInfo info, byte[] family,
+ public static CompactionDescriptor toCompactionDescriptor(
+ org.apache.hadoop.hbase.client.RegionInfo info, byte[] family,
List<Path> inputPaths, List<Path> outputPaths, Path storeDir) {
return toCompactionDescriptor(info, null, family, inputPaths, outputPaths, storeDir);
}
- public static CompactionDescriptor toCompactionDescriptor(HRegionInfo info, byte[] regionName,
+ public static CompactionDescriptor toCompactionDescriptor(
+ org.apache.hadoop.hbase.client.RegionInfo info, byte[] regionName,
byte[] family, List<Path> inputPaths, List<Path> outputPaths, Path storeDir) {
// compaction descriptor contains relative paths.
// input / output paths are relative to the store dir
@@ -2100,7 +2102,7 @@ public final class ProtobufUtil {
return builder.build();
}
- public static FlushDescriptor toFlushDescriptor(FlushAction action, HRegionInfo hri,
+ public static FlushDescriptor toFlushDescriptor(FlushAction action, org.apache.hadoop.hbase.client.RegionInfo hri,
long flushSeqId, Map<byte[], List<Path>> committedFiles) {
FlushDescriptor.Builder desc = FlushDescriptor.newBuilder()
.setAction(action)
@@ -2125,7 +2127,7 @@ public final class ProtobufUtil {
}
public static RegionEventDescriptor toRegionEventDescriptor(
- EventType eventType, HRegionInfo hri, long seqId, ServerName server,
+ EventType eventType, org.apache.hadoop.hbase.client.RegionInfo hri, long seqId, ServerName server,
Map<byte[], List<Path>> storeFiles) {
final byte[] tableNameAsBytes = hri.getTable().getName();
final byte[] encodedNameAsBytes = hri.getEncodedNameAsBytes();
@@ -3315,7 +3317,7 @@ public final class ProtobufUtil {
* @param info the RegionInfo to convert
* @return the converted Proto RegionInfo
*/
- public static HBaseProtos.RegionInfo toProtoRegionInfo(final org.apache.hadoop.hbase.client.RegionInfo info) {
+ public static HBaseProtos.RegionInfo toRegionInfo(final org.apache.hadoop.hbase.client.RegionInfo info) {
if (info == null) return null;
HBaseProtos.RegionInfo.Builder builder = HBaseProtos.RegionInfo.newBuilder();
builder.setTableName(ProtobufUtil.toProtoTableName(info.getTable()));
[12/19] hbase git commit: HBASE-18839 Apply RegionInfo to code base
Posted by ch...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java
index ca73ff7..a05ad67 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java
@@ -43,10 +43,10 @@ import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseIOException;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HDFSBlocksDistribution;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerLoad;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
import org.apache.hadoop.hbase.master.LoadBalancer;
import org.apache.hadoop.hbase.master.MasterServices;
@@ -71,7 +71,7 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
protected static final int MIN_SERVER_BALANCE = 2;
private volatile boolean stopped = false;
- private static final List<HRegionInfo> EMPTY_REGION_LIST = new ArrayList<>(0);
+ private static final List<RegionInfo> EMPTY_REGION_LIST = new ArrayList<>(0);
static final Predicate<ServerLoad> IDLE_SERVER_PREDICATOR
= load -> load.getNumberOfRegions() == 0;
@@ -127,7 +127,7 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
boolean multiServersPerHost = false; // whether or not any host has more than one server
ArrayList<String> tables;
- HRegionInfo[] regions;
+ RegionInfo[] regions;
Deque<BalancerRegionLoad>[] regionLoads;
private RegionLocationFinder regionFinder;
@@ -160,7 +160,7 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
Map<String, Integer> hostsToIndex;
Map<String, Integer> racksToIndex;
Map<String, Integer> tablesToIndex;
- Map<HRegionInfo, Integer> regionsToIndex;
+ Map<RegionInfo, Integer> regionsToIndex;
float[] localityPerServer;
int numServers;
@@ -170,7 +170,7 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
int numRegions;
int numMovedRegions = 0; //num moved regions from the initial configuration
- Map<ServerName, List<HRegionInfo>> clusterState;
+ Map<ServerName, List<RegionInfo>> clusterState;
protected final RackManager rackManager;
// Maps region -> rackIndex -> locality of region on rack
@@ -179,7 +179,7 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
private int[][] regionsToMostLocalEntities;
protected Cluster(
- Map<ServerName, List<HRegionInfo>> clusterState,
+ Map<ServerName, List<RegionInfo>> clusterState,
Map<String, Deque<BalancerRegionLoad>> loads,
RegionLocationFinder regionFinder,
RackManager rackManager) {
@@ -188,8 +188,8 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
@SuppressWarnings("unchecked")
protected Cluster(
- Collection<HRegionInfo> unassignedRegions,
- Map<ServerName, List<HRegionInfo>> clusterState,
+ Collection<RegionInfo> unassignedRegions,
+ Map<ServerName, List<RegionInfo>> clusterState,
Map<String, Deque<BalancerRegionLoad>> loads,
RegionLocationFinder regionFinder,
RackManager rackManager) {
@@ -247,7 +247,7 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
}
// Count how many regions there are.
- for (Entry<ServerName, List<HRegionInfo>> entry : clusterState.entrySet()) {
+ for (Entry<ServerName, List<RegionInfo>> entry : clusterState.entrySet()) {
numRegions += entry.getValue().size();
}
numRegions += unassignedRegions.size();
@@ -256,7 +256,7 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
servers = new ServerName[numServers];
serversPerHost = new int[numHosts][];
serversPerRack = new int[numRacks][];
- regions = new HRegionInfo[numRegions];
+ regions = new RegionInfo[numRegions];
regionIndexToServerIndex = new int[numRegions];
initialRegionIndexToServerIndex = new int[numRegions];
regionIndexToTableIndex = new int[numRegions];
@@ -279,7 +279,7 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
int tableIndex = 0, regionIndex = 0, regionPerServerIndex = 0;
- for (Entry<ServerName, List<HRegionInfo>> entry : clusterState.entrySet()) {
+ for (Entry<ServerName, List<RegionInfo>> entry : clusterState.entrySet()) {
if (entry.getKey() == null) {
LOG.warn("SERVERNAME IS NULL, skipping " + entry.getValue());
continue;
@@ -314,7 +314,7 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
racks[entry.getValue()] = entry.getKey();
}
- for (Entry<ServerName, List<HRegionInfo>> entry : clusterState.entrySet()) {
+ for (Entry<ServerName, List<RegionInfo>> entry : clusterState.entrySet()) {
int serverIndex = serversToIndex.get(entry.getKey().getHostAndPort());
regionPerServerIndex = 0;
@@ -324,14 +324,14 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
int rackIndex = racksToIndex.get(this.rackManager.getRack(entry.getKey()));
serverIndexToRackIndex[serverIndex] = rackIndex;
- for (HRegionInfo region : entry.getValue()) {
+ for (RegionInfo region : entry.getValue()) {
registerRegion(region, regionIndex, serverIndex, loads, regionFinder);
regionsPerServer[serverIndex][regionPerServerIndex++] = regionIndex;
regionIndex++;
}
}
- for (HRegionInfo region : unassignedRegions) {
+ for (RegionInfo region : unassignedRegions) {
registerRegion(region, regionIndex, -1, loads, regionFinder);
regionIndex++;
}
@@ -378,12 +378,12 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
}
for (int i = 0; i < regions.length; i ++) {
- HRegionInfo info = regions[i];
+ RegionInfo info = regions[i];
if (RegionReplicaUtil.isDefaultReplica(info)) {
regionIndexToPrimaryIndex[i] = i;
} else {
hasRegionReplicas = true;
- HRegionInfo primaryInfo = RegionReplicaUtil.getRegionInfoForDefaultReplica(info);
+ RegionInfo primaryInfo = RegionReplicaUtil.getRegionInfoForDefaultReplica(info);
regionIndexToPrimaryIndex[i] = regionsToIndex.getOrDefault(primaryInfo, -1);
}
}
@@ -453,7 +453,7 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
}
/** Helper for Cluster constructor to handle a region */
- private void registerRegion(HRegionInfo region, int regionIndex,
+ private void registerRegion(RegionInfo region, int regionIndex,
int serverIndex, Map<String, Deque<BalancerRegionLoad>> loads,
RegionLocationFinder regionFinder) {
String tableName = region.getTable().getNameAsString();
@@ -731,7 +731,7 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
* of the region in question
* @return true or false
*/
- boolean wouldLowerAvailability(HRegionInfo regionInfo, ServerName serverName) {
+ boolean wouldLowerAvailability(RegionInfo regionInfo, ServerName serverName) {
if (!serversToIndex.containsKey(serverName.getHostAndPort())) {
return false; // safeguard against race between cluster.servers and servers from LB method args
}
@@ -783,7 +783,7 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
return false;
}
- void doAssignRegion(HRegionInfo regionInfo, ServerName serverName) {
+ void doAssignRegion(RegionInfo regionInfo, ServerName serverName) {
if (!serversToIndex.containsKey(serverName.getHostAndPort())) {
return;
}
@@ -1047,20 +1047,20 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
* Check if a region belongs to some system table.
* If so, the primary replica may be expected to be put on the master regionserver.
*/
- public boolean shouldBeOnMaster(HRegionInfo region) {
+ public boolean shouldBeOnMaster(RegionInfo region) {
return this.onlySystemTablesOnMaster && region.isSystemTable();
}
/**
* Balance the regions that should be on master regionserver.
*/
- protected List<RegionPlan> balanceMasterRegions(Map<ServerName, List<HRegionInfo>> clusterMap) {
+ protected List<RegionPlan> balanceMasterRegions(Map<ServerName, List<RegionInfo>> clusterMap) {
if (masterServerName == null || clusterMap == null || clusterMap.size() <= 1) return null;
List<RegionPlan> plans = null;
- List<HRegionInfo> regions = clusterMap.get(masterServerName);
+ List<RegionInfo> regions = clusterMap.get(masterServerName);
if (regions != null) {
Iterator<ServerName> keyIt = null;
- for (HRegionInfo region: regions) {
+ for (RegionInfo region: regions) {
if (shouldBeOnMaster(region)) continue;
// Find a non-master regionserver to host the region
@@ -1083,9 +1083,9 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
plans.add(plan);
}
}
- for (Map.Entry<ServerName, List<HRegionInfo>> server: clusterMap.entrySet()) {
+ for (Map.Entry<ServerName, List<RegionInfo>> server: clusterMap.entrySet()) {
if (masterServerName.equals(server.getKey())) continue;
- for (HRegionInfo region: server.getValue()) {
+ for (RegionInfo region: server.getValue()) {
if (!shouldBeOnMaster(region)) continue;
// Move this region to the master regionserver
@@ -1103,16 +1103,16 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
* If master is configured to carry system tables only, in here is
* where we figure what to assign it.
*/
- protected Map<ServerName, List<HRegionInfo>> assignMasterSystemRegions(
- Collection<HRegionInfo> regions, List<ServerName> servers) {
+ protected Map<ServerName, List<RegionInfo>> assignMasterSystemRegions(
+ Collection<RegionInfo> regions, List<ServerName> servers) {
if (servers == null || regions == null || regions.isEmpty()) {
return null;
}
- Map<ServerName, List<HRegionInfo>> assignments = new TreeMap<>();
+ Map<ServerName, List<RegionInfo>> assignments = new TreeMap<>();
if (this.onlySystemTablesOnMaster) {
if (masterServerName != null && servers.contains(masterServerName)) {
assignments.put(masterServerName, new ArrayList<>());
- for (HRegionInfo region : regions) {
+ for (RegionInfo region : regions) {
if (shouldBeOnMaster(region)) {
assignments.get(masterServerName).add(region);
}
@@ -1136,7 +1136,7 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
}
@Override
- public void setClusterLoad(Map<TableName, Map<ServerName, List<HRegionInfo>>> clusterLoad){
+ public void setClusterLoad(Map<TableName, Map<ServerName, List<RegionInfo>>> clusterLoad){
}
@@ -1169,7 +1169,7 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
int floor = (int) Math.floor(average * (1 - slop));
int ceiling = (int) Math.ceil(average * (1 + slop));
if (!(cs.getMaxLoad() > ceiling || cs.getMinLoad() < floor)) {
- NavigableMap<ServerAndLoad, List<HRegionInfo>> serversByLoad = cs.getServersByLoad();
+ NavigableMap<ServerAndLoad, List<RegionInfo>> serversByLoad = cs.getServersByLoad();
if (LOG.isTraceEnabled()) {
// If nothing to balance, then don't say anything unless trace-level logging.
LOG.trace("Skipping load balancing because balanced cluster; " +
@@ -1212,15 +1212,15 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
* assignment is possible (ie. no regions or no servers)
*/
@Override
- public Map<ServerName, List<HRegionInfo>> roundRobinAssignment(List<HRegionInfo> regions,
+ public Map<ServerName, List<RegionInfo>> roundRobinAssignment(List<RegionInfo> regions,
List<ServerName> servers) throws HBaseIOException {
metricsBalancer.incrMiscInvocations();
- Map<ServerName, List<HRegionInfo>> assignments = assignMasterSystemRegions(regions, servers);
+ Map<ServerName, List<RegionInfo>> assignments = assignMasterSystemRegions(regions, servers);
if (assignments != null && !assignments.isEmpty()) {
servers = new ArrayList<>(servers);
// Guarantee not to put other regions on master
servers.remove(masterServerName);
- List<HRegionInfo> masterRegions = assignments.get(masterServerName);
+ List<RegionInfo> masterRegions = assignments.get(masterServerName);
if (!masterRegions.isEmpty()) {
regions = new ArrayList<>(regions);
regions.removeAll(masterRegions);
@@ -1248,20 +1248,20 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
}
Cluster cluster = createCluster(servers, regions, false);
- List<HRegionInfo> unassignedRegions = new ArrayList<>();
+ List<RegionInfo> unassignedRegions = new ArrayList<>();
roundRobinAssignment(cluster, regions, unassignedRegions,
servers, assignments);
- List<HRegionInfo> lastFewRegions = new ArrayList<>();
+ List<RegionInfo> lastFewRegions = new ArrayList<>();
// assign the remaining by going through the list and try to assign to servers one-by-one
int serverIdx = RANDOM.nextInt(numServers);
- for (HRegionInfo region : unassignedRegions) {
+ for (RegionInfo region : unassignedRegions) {
boolean assigned = false;
for (int j = 0; j < numServers; j++) { // try all servers one by one
ServerName serverName = servers.get((j + serverIdx) % numServers);
if (!cluster.wouldLowerAvailability(region, serverName)) {
- List<HRegionInfo> serverRegions =
+ List<RegionInfo> serverRegions =
assignments.computeIfAbsent(serverName, k -> new ArrayList<>());
serverRegions.add(region);
cluster.doAssignRegion(region, serverName);
@@ -1276,10 +1276,10 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
}
// just sprinkle the rest of the regions on random regionservers. The balanceCluster will
// make it optimal later. we can end up with this if numReplicas > numServers.
- for (HRegionInfo region : lastFewRegions) {
+ for (RegionInfo region : lastFewRegions) {
int i = RANDOM.nextInt(numServers);
ServerName server = servers.get(i);
- List<HRegionInfo> serverRegions = assignments.computeIfAbsent(server, k -> new ArrayList<>());
+ List<RegionInfo> serverRegions = assignments.computeIfAbsent(server, k -> new ArrayList<>());
serverRegions.add(region);
cluster.doAssignRegion(region, server);
}
@@ -1287,7 +1287,7 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
}
protected Cluster createCluster(List<ServerName> servers,
- Collection<HRegionInfo> regions, boolean forceRefresh) {
+ Collection<RegionInfo> regions, boolean forceRefresh) {
if (forceRefresh && useRegionFinder) {
regionFinder.refreshAndWait(regions);
}
@@ -1295,7 +1295,7 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
// a cluster out of it. Note that we might have replicas already assigned to some servers
// earlier. So we want to get the snapshot to see those assignments, but this will only contain
// replicas of the regions that are passed (for performance).
- Map<ServerName, List<HRegionInfo>> clusterState = getRegionAssignmentsByServer(regions);
+ Map<ServerName, List<RegionInfo>> clusterState = getRegionAssignmentsByServer(regions);
for (ServerName server : servers) {
if (!clusterState.containsKey(server)) {
@@ -1315,7 +1315,7 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
* Used to assign a single region to a random server.
*/
@Override
- public ServerName randomAssignment(HRegionInfo regionInfo, List<ServerName> servers)
+ public ServerName randomAssignment(RegionInfo regionInfo, List<ServerName> servers)
throws HBaseIOException {
metricsBalancer.incrMiscInvocations();
if (servers != null && servers.contains(masterServerName)) {
@@ -1343,7 +1343,7 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
}
final List<ServerName> finalServers = idleServers.isEmpty() ?
servers : idleServers;
- List<HRegionInfo> regions = Lists.newArrayList(regionInfo);
+ List<RegionInfo> regions = Lists.newArrayList(regionInfo);
Cluster cluster = createCluster(finalServers, regions, false);
return randomAssignment(cluster, regionInfo, finalServers);
}
@@ -1366,16 +1366,16 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
* @return map of servers and regions to be assigned to them
*/
@Override
- public Map<ServerName, List<HRegionInfo>> retainAssignment(Map<HRegionInfo, ServerName> regions,
+ public Map<ServerName, List<RegionInfo>> retainAssignment(Map<RegionInfo, ServerName> regions,
List<ServerName> servers) throws HBaseIOException {
// Update metrics
metricsBalancer.incrMiscInvocations();
- Map<ServerName, List<HRegionInfo>> assignments = assignMasterSystemRegions(regions.keySet(), servers);
+ Map<ServerName, List<RegionInfo>> assignments = assignMasterSystemRegions(regions.keySet(), servers);
if (assignments != null && !assignments.isEmpty()) {
servers = new ArrayList<>(servers);
// Guarantee not to put other regions on master
servers.remove(masterServerName);
- List<HRegionInfo> masterRegions = assignments.get(masterServerName);
+ List<RegionInfo> masterRegions = assignments.get(masterServerName);
regions = regions.entrySet().stream().filter(e -> !masterRegions.contains(e.getKey()))
.collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
}
@@ -1416,8 +1416,8 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
Cluster cluster = createCluster(servers, regions.keySet(), true);
- for (Map.Entry<HRegionInfo, ServerName> entry : regions.entrySet()) {
- HRegionInfo region = entry.getKey();
+ for (Map.Entry<RegionInfo, ServerName> entry : regions.entrySet()) {
+ RegionInfo region = entry.getKey();
ServerName oldServerName = entry.getValue();
List<ServerName> localServers = new ArrayList<>();
if (oldServerName != null) {
@@ -1477,11 +1477,11 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
}
@Override
- public void regionOnline(HRegionInfo regionInfo, ServerName sn) {
+ public void regionOnline(RegionInfo regionInfo, ServerName sn) {
}
@Override
- public void regionOffline(HRegionInfo regionInfo) {
+ public void regionOffline(RegionInfo regionInfo) {
}
@Override
@@ -1498,7 +1498,7 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
/**
* Used to assign a single region to a random server.
*/
- private ServerName randomAssignment(Cluster cluster, HRegionInfo regionInfo,
+ private ServerName randomAssignment(Cluster cluster, RegionInfo regionInfo,
List<ServerName> servers) {
int numServers = servers.size(); // servers is not null, numServers > 1
ServerName sn = null;
@@ -1517,9 +1517,9 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
/**
* Round robin a list of regions to a list of servers
*/
- private void roundRobinAssignment(Cluster cluster, List<HRegionInfo> regions,
- List<HRegionInfo> unassignedRegions, List<ServerName> servers,
- Map<ServerName, List<HRegionInfo>> assignments) {
+ private void roundRobinAssignment(Cluster cluster, List<RegionInfo> regions,
+ List<RegionInfo> unassignedRegions, List<ServerName> servers,
+ Map<ServerName, List<RegionInfo>> assignments) {
int numServers = servers.size();
int numRegions = regions.size();
@@ -1532,9 +1532,9 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
for (int j = 0; j < numServers; j++) {
ServerName server = servers.get((j + serverIdx) % numServers);
- List<HRegionInfo> serverRegions = new ArrayList<>(max);
+ List<RegionInfo> serverRegions = new ArrayList<>(max);
for (int i = regionIdx; i < numRegions; i += numServers) {
- HRegionInfo region = regions.get(i % numRegions);
+ RegionInfo region = regions.get(i % numRegions);
if (cluster.wouldLowerAvailability(region, server)) {
unassignedRegions.add(region);
} else {
@@ -1547,8 +1547,8 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
}
}
- protected Map<ServerName, List<HRegionInfo>> getRegionAssignmentsByServer(
- Collection<HRegionInfo> regions) {
+ protected Map<ServerName, List<RegionInfo>> getRegionAssignmentsByServer(
+ Collection<RegionInfo> regions) {
if (this.services != null && this.services.getAssignmentManager() != null) {
return this.services.getAssignmentManager().getSnapShotOfAssignment(regions);
} else {
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/ClusterLoadState.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/ClusterLoadState.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/ClusterLoadState.java
index 8c337bd..a783a07 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/ClusterLoadState.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/ClusterLoadState.java
@@ -22,27 +22,27 @@ import java.util.Map;
import java.util.NavigableMap;
import java.util.TreeMap;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.client.RegionInfo;
/**
* Class used to hold the current state of the cluster and how balanced it is.
*/
public class ClusterLoadState {
- private final Map<ServerName, List<HRegionInfo>> clusterState;
- private final NavigableMap<ServerAndLoad, List<HRegionInfo>> serversByLoad;
+ private final Map<ServerName, List<RegionInfo>> clusterState;
+ private final NavigableMap<ServerAndLoad, List<RegionInfo>> serversByLoad;
private boolean emptyRegionServerPresent = false;
private int numRegions = 0;
private int numServers = 0;
- public ClusterLoadState(Map<ServerName, List<HRegionInfo>> clusterState) {
+ public ClusterLoadState(Map<ServerName, List<RegionInfo>> clusterState) {
this.numRegions = 0;
this.numServers = clusterState.size();
this.clusterState = clusterState;
serversByLoad = new TreeMap<>();
// Iterate so we can count regions as we build the map
- for (Map.Entry<ServerName, List<HRegionInfo>> server : clusterState.entrySet()) {
- List<HRegionInfo> regions = server.getValue();
+ for (Map.Entry<ServerName, List<RegionInfo>> server : clusterState.entrySet()) {
+ List<RegionInfo> regions = server.getValue();
int sz = regions.size();
if (sz == 0) emptyRegionServerPresent = true;
numRegions += sz;
@@ -50,11 +50,11 @@ public class ClusterLoadState {
}
}
- Map<ServerName, List<HRegionInfo>> getClusterState() {
+ Map<ServerName, List<RegionInfo>> getClusterState() {
return clusterState;
}
- NavigableMap<ServerAndLoad, List<HRegionInfo>> getServersByLoad() {
+ NavigableMap<ServerAndLoad, List<RegionInfo>> getServersByLoad() {
return serversByLoad;
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/FavoredStochasticBalancer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/FavoredStochasticBalancer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/FavoredStochasticBalancer.java
index 8dc2a85..a2fe9a2 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/FavoredStochasticBalancer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/FavoredStochasticBalancer.java
@@ -17,11 +17,11 @@
*/
package org.apache.hadoop.hbase.master.balancer;
+import static org.apache.hadoop.hbase.ServerName.NON_STARTCODE;
import static org.apache.hadoop.hbase.favored.FavoredNodeAssignmentHelper.FAVORED_NODES_NUM;
import static org.apache.hadoop.hbase.favored.FavoredNodesPlan.Position.PRIMARY;
import static org.apache.hadoop.hbase.favored.FavoredNodesPlan.Position.SECONDARY;
import static org.apache.hadoop.hbase.favored.FavoredNodesPlan.Position.TERTIARY;
-import static org.apache.hadoop.hbase.ServerName.NON_STARTCODE;
import java.io.IOException;
import java.util.ArrayList;
@@ -35,9 +35,9 @@ import java.util.Set;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HBaseIOException;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerLoad;
import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.favored.FavoredNodeAssignmentHelper;
import org.apache.hadoop.hbase.favored.FavoredNodesManager;
import org.apache.hadoop.hbase.favored.FavoredNodesPlan;
@@ -107,20 +107,20 @@ public class FavoredStochasticBalancer extends StochasticLoadBalancer implements
* secondary and tertiary as per favored nodes constraints.
*/
@Override
- public Map<ServerName, List<HRegionInfo>> roundRobinAssignment(List<HRegionInfo> regions,
+ public Map<ServerName, List<RegionInfo>> roundRobinAssignment(List<RegionInfo> regions,
List<ServerName> servers) throws HBaseIOException {
metricsBalancer.incrMiscInvocations();
- Set<HRegionInfo> regionSet = Sets.newHashSet(regions);
- Map<ServerName, List<HRegionInfo>> assignmentMap = assignMasterSystemRegions(regions, servers);
+ Set<RegionInfo> regionSet = Sets.newHashSet(regions);
+ Map<ServerName, List<RegionInfo>> assignmentMap = assignMasterSystemRegions(regions, servers);
if (assignmentMap != null && !assignmentMap.isEmpty()) {
servers = new ArrayList<>(servers);
// Guarantee not to put other regions on master
servers.remove(masterServerName);
- List<HRegionInfo> masterRegions = assignmentMap.get(masterServerName);
+ List<RegionInfo> masterRegions = assignmentMap.get(masterServerName);
if (!masterRegions.isEmpty()) {
- for (HRegionInfo region: masterRegions) {
+ for (RegionInfo region: masterRegions) {
regionSet.remove(region);
}
}
@@ -135,18 +135,18 @@ public class FavoredStochasticBalancer extends StochasticLoadBalancer implements
new FavoredNodeAssignmentHelper(servers, fnm.getRackManager());
helper.initialize();
- Set<HRegionInfo> systemRegions = FavoredNodesManager.filterNonFNApplicableRegions(regionSet);
+ Set<RegionInfo> systemRegions = FavoredNodesManager.filterNonFNApplicableRegions(regionSet);
regionSet.removeAll(systemRegions);
// Assign all system regions
- Map<ServerName, List<HRegionInfo>> systemAssignments =
+ Map<ServerName, List<RegionInfo>> systemAssignments =
super.roundRobinAssignment(Lists.newArrayList(systemRegions), servers);
// Segregate favored and non-favored nodes regions and assign accordingly.
- Pair<Map<ServerName,List<HRegionInfo>>, List<HRegionInfo>> segregatedRegions =
+ Pair<Map<ServerName,List<RegionInfo>>, List<RegionInfo>> segregatedRegions =
segregateRegionsAndAssignRegionsWithFavoredNodes(regionSet, servers);
- Map<ServerName, List<HRegionInfo>> regionsWithFavoredNodesMap = segregatedRegions.getFirst();
- Map<ServerName, List<HRegionInfo>> regionsWithoutFN =
+ Map<ServerName, List<RegionInfo>> regionsWithFavoredNodesMap = segregatedRegions.getFirst();
+ Map<ServerName, List<RegionInfo>> regionsWithoutFN =
generateFNForRegionsWithoutFN(helper, segregatedRegions.getSecond());
// merge the assignment maps
@@ -161,16 +161,16 @@ public class FavoredStochasticBalancer extends StochasticLoadBalancer implements
return assignmentMap;
}
- private void mergeAssignmentMaps(Map<ServerName, List<HRegionInfo>> assignmentMap,
- Map<ServerName, List<HRegionInfo>> otherAssignments) {
+ private void mergeAssignmentMaps(Map<ServerName, List<RegionInfo>> assignmentMap,
+ Map<ServerName, List<RegionInfo>> otherAssignments) {
if (otherAssignments == null || otherAssignments.isEmpty()) {
return;
}
- for (Entry<ServerName, List<HRegionInfo>> entry : otherAssignments.entrySet()) {
+ for (Entry<ServerName, List<RegionInfo>> entry : otherAssignments.entrySet()) {
ServerName sn = entry.getKey();
- List<HRegionInfo> regionsList = entry.getValue();
+ List<RegionInfo> regionsList = entry.getValue();
if (assignmentMap.get(sn) == null) {
assignmentMap.put(sn, Lists.newArrayList(regionsList));
} else {
@@ -179,11 +179,11 @@ public class FavoredStochasticBalancer extends StochasticLoadBalancer implements
}
}
- private Map<ServerName, List<HRegionInfo>> generateFNForRegionsWithoutFN(
- FavoredNodeAssignmentHelper helper, List<HRegionInfo> regions) throws IOException {
+ private Map<ServerName, List<RegionInfo>> generateFNForRegionsWithoutFN(
+ FavoredNodeAssignmentHelper helper, List<RegionInfo> regions) throws IOException {
- Map<ServerName, List<HRegionInfo>> assignmentMap = Maps.newHashMap();
- Map<HRegionInfo, List<ServerName>> regionsNoFNMap;
+ Map<ServerName, List<RegionInfo>> assignmentMap = Maps.newHashMap();
+ Map<RegionInfo, List<ServerName>> regionsNoFNMap;
if (regions.size() > 0) {
regionsNoFNMap = helper.generateFavoredNodesRoundRobin(assignmentMap, regions);
@@ -196,16 +196,16 @@ public class FavoredStochasticBalancer extends StochasticLoadBalancer implements
* Return a pair - one with assignments when favored nodes are present and another with regions
* without favored nodes.
*/
- private Pair<Map<ServerName, List<HRegionInfo>>, List<HRegionInfo>>
- segregateRegionsAndAssignRegionsWithFavoredNodes(Collection<HRegionInfo> regions,
+ private Pair<Map<ServerName, List<RegionInfo>>, List<RegionInfo>>
+ segregateRegionsAndAssignRegionsWithFavoredNodes(Collection<RegionInfo> regions,
List<ServerName> onlineServers) throws HBaseIOException {
// Since we expect FN to be present most of the time, lets create map with same size
- Map<ServerName, List<HRegionInfo>> assignmentMapForFavoredNodes =
+ Map<ServerName, List<RegionInfo>> assignmentMapForFavoredNodes =
new HashMap<>(onlineServers.size());
- List<HRegionInfo> regionsWithNoFavoredNodes = new ArrayList<>();
+ List<RegionInfo> regionsWithNoFavoredNodes = new ArrayList<>();
- for (HRegionInfo region : regions) {
+ for (RegionInfo region : regions) {
List<ServerName> favoredNodes = fnm.getFavoredNodes(region);
ServerName primaryHost = null;
ServerName secondaryHost = null;
@@ -235,10 +235,10 @@ public class FavoredStochasticBalancer extends StochasticLoadBalancer implements
return new Pair<>(assignmentMapForFavoredNodes, regionsWithNoFavoredNodes);
}
- private void addRegionToMap(Map<ServerName, List<HRegionInfo>> assignmentMapForFavoredNodes,
- HRegionInfo region, ServerName host) {
+ private void addRegionToMap(Map<ServerName, List<RegionInfo>> assignmentMapForFavoredNodes,
+ RegionInfo region, ServerName host) {
- List<HRegionInfo> regionsOnServer;
+ List<RegionInfo> regionsOnServer;
if ((regionsOnServer = assignmentMapForFavoredNodes.get(host)) == null) {
regionsOnServer = Lists.newArrayList();
assignmentMapForFavoredNodes.put(host, regionsOnServer);
@@ -265,7 +265,7 @@ public class FavoredStochasticBalancer extends StochasticLoadBalancer implements
* available (in that order).
*/
private void assignRegionToAvailableFavoredNode(
- Map<ServerName, List<HRegionInfo>> assignmentMapForFavoredNodes, HRegionInfo region,
+ Map<ServerName, List<RegionInfo>> assignmentMapForFavoredNodes, RegionInfo region,
ServerName primaryHost, ServerName secondaryHost, ServerName tertiaryHost) {
if (primaryHost != null) {
@@ -304,7 +304,7 @@ public class FavoredStochasticBalancer extends StochasticLoadBalancer implements
* destination. If we can't generate anything, lets fallback.
*/
@Override
- public ServerName randomAssignment(HRegionInfo regionInfo, List<ServerName> servers)
+ public ServerName randomAssignment(RegionInfo regionInfo, List<ServerName> servers)
throws HBaseIOException {
if (servers != null && servers.contains(masterServerName)) {
@@ -354,9 +354,9 @@ public class FavoredStochasticBalancer extends StochasticLoadBalancer implements
return destination;
}
- private void updateFavoredNodesForRegion(HRegionInfo regionInfo, List<ServerName> newFavoredNodes)
+ private void updateFavoredNodesForRegion(RegionInfo regionInfo, List<ServerName> newFavoredNodes)
throws IOException {
- Map<HRegionInfo, List<ServerName>> regionFNMap = Maps.newHashMap();
+ Map<RegionInfo, List<ServerName>> regionFNMap = Maps.newHashMap();
regionFNMap.put(regionInfo, newFavoredNodes);
fnm.updateFavoredNodes(regionFNMap);
}
@@ -365,11 +365,11 @@ public class FavoredStochasticBalancer extends StochasticLoadBalancer implements
* Reuse BaseLoadBalancer's retainAssignment, but generate favored nodes when its missing.
*/
@Override
- public Map<ServerName, List<HRegionInfo>> retainAssignment(Map<HRegionInfo, ServerName> regions,
+ public Map<ServerName, List<RegionInfo>> retainAssignment(Map<RegionInfo, ServerName> regions,
List<ServerName> servers) throws HBaseIOException {
- Map<ServerName, List<HRegionInfo>> assignmentMap = Maps.newHashMap();
- Map<ServerName, List<HRegionInfo>> result = super.retainAssignment(regions, servers);
+ Map<ServerName, List<RegionInfo>> assignmentMap = Maps.newHashMap();
+ Map<ServerName, List<RegionInfo>> result = super.retainAssignment(regions, servers);
if (result == null || result.isEmpty()) {
LOG.warn("Nothing to assign to, probably no servers or no regions");
return null;
@@ -386,15 +386,15 @@ public class FavoredStochasticBalancer extends StochasticLoadBalancer implements
helper.initialize();
LOG.debug("Generating favored nodes for regions missing them.");
- Map<HRegionInfo, List<ServerName>> regionFNMap = Maps.newHashMap();
+ Map<RegionInfo, List<ServerName>> regionFNMap = Maps.newHashMap();
try {
- for (Entry<ServerName, List<HRegionInfo>> entry : result.entrySet()) {
+ for (Entry<ServerName, List<RegionInfo>> entry : result.entrySet()) {
ServerName sn = entry.getKey();
ServerName primary = ServerName.valueOf(sn.getHostname(), sn.getPort(), NON_STARTCODE);
- for (HRegionInfo hri : entry.getValue()) {
+ for (RegionInfo hri : entry.getValue()) {
if (FavoredNodesManager.isFavoredNodeApplicable(hri)) {
List<ServerName> favoredNodes = fnm.getFavoredNodes(hri);
@@ -472,7 +472,7 @@ public class FavoredStochasticBalancer extends StochasticLoadBalancer implements
}
}
- public synchronized List<ServerName> getFavoredNodes(HRegionInfo regionInfo) {
+ public synchronized List<ServerName> getFavoredNodes(RegionInfo regionInfo) {
return this.fnm.getFavoredNodes(regionInfo);
}
@@ -487,10 +487,10 @@ public class FavoredStochasticBalancer extends StochasticLoadBalancer implements
* randomly. This would give us better distribution over a period of time after enough splits.
*/
@Override
- public void generateFavoredNodesForDaughter(List<ServerName> servers, HRegionInfo parent,
- HRegionInfo regionA, HRegionInfo regionB) throws IOException {
+ public void generateFavoredNodesForDaughter(List<ServerName> servers, RegionInfo parent,
+ RegionInfo regionA, RegionInfo regionB) throws IOException {
- Map<HRegionInfo, List<ServerName>> result = new HashMap<>();
+ Map<RegionInfo, List<ServerName>> result = new HashMap<>();
FavoredNodeAssignmentHelper helper = new FavoredNodeAssignmentHelper(servers, rackManager);
helper.initialize();
@@ -542,8 +542,8 @@ public class FavoredStochasticBalancer extends StochasticLoadBalancer implements
* keep it simple.
*/
@Override
- public void generateFavoredNodesForMergedRegion(HRegionInfo merged, HRegionInfo regionA,
- HRegionInfo regionB) throws IOException {
+ public void generateFavoredNodesForMergedRegion(RegionInfo merged, RegionInfo regionA,
+ RegionInfo regionB) throws IOException {
updateFavoredNodesForRegion(merged, fnm.getFavoredNodes(regionA));
}
@@ -572,7 +572,7 @@ public class FavoredStochasticBalancer extends StochasticLoadBalancer implements
return Cluster.NullAction;
}
- HRegionInfo hri = cluster.regions[thisRegion];
+ RegionInfo hri = cluster.regions[thisRegion];
List<ServerName> favoredNodes = fnm.getFavoredNodes(hri);
int otherServer;
if (favoredNodes == null) {
@@ -628,7 +628,7 @@ public class FavoredStochasticBalancer extends StochasticLoadBalancer implements
cluster.sortServersByRegionCount();
int thisServer = pickMostLoadedServer(cluster);
int thisRegion = pickRandomRegion(cluster, thisServer, 0);
- HRegionInfo hri = cluster.regions[thisRegion];
+ RegionInfo hri = cluster.regions[thisRegion];
int otherServer;
List<ServerName> favoredNodes = fnm.getFavoredNodes(hri);
if (favoredNodes == null) {
@@ -693,20 +693,20 @@ public class FavoredStochasticBalancer extends StochasticLoadBalancer implements
* implementation. For the misplaced regions, we assign a bogus server to it and AM takes care.
*/
@Override
- public List<RegionPlan> balanceCluster(Map<ServerName, List<HRegionInfo>> clusterState) {
+ public List<RegionPlan> balanceCluster(Map<ServerName, List<RegionInfo>> clusterState) {
if (this.services != null) {
List<RegionPlan> regionPlans = Lists.newArrayList();
- Map<ServerName, List<HRegionInfo>> correctAssignments = new HashMap<>();
+ Map<ServerName, List<RegionInfo>> correctAssignments = new HashMap<>();
int misplacedRegions = 0;
- for (Entry<ServerName, List<HRegionInfo>> entry : clusterState.entrySet()) {
+ for (Entry<ServerName, List<RegionInfo>> entry : clusterState.entrySet()) {
ServerName current = entry.getKey();
- List<HRegionInfo> regions = Lists.newArrayList();
+ List<RegionInfo> regions = Lists.newArrayList();
correctAssignments.put(current, regions);
- for (HRegionInfo hri : entry.getValue()) {
+ for (RegionInfo hri : entry.getValue()) {
List<ServerName> favoredNodes = fnm.getFavoredNodes(hri);
if (FavoredNodesPlan.getFavoredServerPosition(favoredNodes, current) != null ||
!FavoredNodesManager.isFavoredNodeApplicable(hri)) {
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/RegionInfoComparator.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/RegionInfoComparator.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/RegionInfoComparator.java
index 51d8c88..911b70b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/RegionInfoComparator.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/RegionInfoComparator.java
@@ -19,7 +19,7 @@ package org.apache.hadoop.hbase.master.balancer;
import java.util.Comparator;
-import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfo;
/**
* The following comparator assumes that RegionId from HRegionInfo can represent
@@ -27,9 +27,9 @@ import org.apache.hadoop.hbase.HRegionInfo;
* comparator is used in balanceCluster() to account for the out-of-band regions
* which were assigned to the server after some other region server crashed.
*/
-class RegionInfoComparator implements Comparator<HRegionInfo> {
+class RegionInfoComparator implements Comparator<RegionInfo> {
@Override
- public int compare(HRegionInfo l, HRegionInfo r) {
+ public int compare(RegionInfo l, RegionInfo r) {
long diff = r.getRegionId() - l.getRegionId();
if (diff < 0) return -1;
if (diff > 0) return 1;
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/RegionLocationFinder.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/RegionLocationFinder.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/RegionLocationFinder.java
index 6f56454..f5502cc 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/RegionLocationFinder.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/RegionLocationFinder.java
@@ -33,15 +33,15 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.ClusterStatus;
import org.apache.hadoop.hbase.HDFSBlocksDistribution;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.TableDescriptor;
-import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
import org.apache.hadoop.hbase.master.MasterServices;
+import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.common.cache.CacheBuilder;
import org.apache.hadoop.hbase.shaded.com.google.common.cache.CacheLoader;
@@ -71,10 +71,10 @@ class RegionLocationFinder {
// Do not scheduleFullRefresh at master startup
private long lastFullRefresh = EnvironmentEdgeManager.currentTime();
- private CacheLoader<HRegionInfo, HDFSBlocksDistribution> loader =
- new CacheLoader<HRegionInfo, HDFSBlocksDistribution>() {
+ private CacheLoader<RegionInfo, HDFSBlocksDistribution> loader =
+ new CacheLoader<RegionInfo, HDFSBlocksDistribution>() {
- public ListenableFuture<HDFSBlocksDistribution> reload(final HRegionInfo hri,
+ public ListenableFuture<HDFSBlocksDistribution> reload(final RegionInfo hri,
HDFSBlocksDistribution oldValue) throws Exception {
return executor.submit(new Callable<HDFSBlocksDistribution>() {
@Override
@@ -85,13 +85,13 @@ class RegionLocationFinder {
}
@Override
- public HDFSBlocksDistribution load(HRegionInfo key) throws Exception {
+ public HDFSBlocksDistribution load(RegionInfo key) throws Exception {
return internalGetTopBlockLocation(key);
}
};
// The cache for where regions are located.
- private LoadingCache<HRegionInfo, HDFSBlocksDistribution> cache = null;
+ private LoadingCache<RegionInfo, HDFSBlocksDistribution> cache = null;
RegionLocationFinder() {
this.cache = createCache();
@@ -109,7 +109,7 @@ class RegionLocationFinder {
* @param time time to cache the locations
* @return A new Cache.
*/
- private LoadingCache<HRegionInfo, HDFSBlocksDistribution> createCache() {
+ private LoadingCache<RegionInfo, HDFSBlocksDistribution> createCache() {
return CacheBuilder.newBuilder()
.expireAfterWrite(CACHE_TIME, TimeUnit.MILLISECONDS)
.build(loader);
@@ -155,14 +155,14 @@ class RegionLocationFinder {
// TODO: Should this refresh all the regions or only the ones assigned?
boolean includesUserTables = false;
- for (final HRegionInfo hri : am.getAssignedRegions()) {
+ for (final RegionInfo hri : am.getAssignedRegions()) {
cache.refresh(hri);
includesUserTables = includesUserTables || !hri.isSystemTable();
}
return includesUserTables;
}
- protected List<ServerName> getTopBlockLocations(HRegionInfo region) {
+ protected List<ServerName> getTopBlockLocations(RegionInfo region) {
List<String> topHosts = getBlockDistribution(region).getTopHosts();
return mapHostNameToServerName(topHosts);
}
@@ -171,7 +171,7 @@ class RegionLocationFinder {
* Returns an ordered list of hosts which have better locality for this region
* than the current host.
*/
- protected List<ServerName> getTopBlockLocations(HRegionInfo region, String currentHost) {
+ protected List<ServerName> getTopBlockLocations(RegionInfo region, String currentHost) {
HDFSBlocksDistribution blocksDistribution = getBlockDistribution(region);
List<String> topHosts = new ArrayList<>();
for (String host : blocksDistribution.getTopHosts()) {
@@ -192,7 +192,7 @@ class RegionLocationFinder {
* @param region region
* @return ordered list of hosts holding blocks of the specified region
*/
- protected HDFSBlocksDistribution internalGetTopBlockLocation(HRegionInfo region) {
+ protected HDFSBlocksDistribution internalGetTopBlockLocation(RegionInfo region) {
try {
TableDescriptor tableDescriptor = getTableDescriptor(region.getTable());
if (tableDescriptor != null) {
@@ -272,7 +272,7 @@ class RegionLocationFinder {
return topServerNames;
}
- public HDFSBlocksDistribution getBlockDistribution(HRegionInfo hri) {
+ public HDFSBlocksDistribution getBlockDistribution(RegionInfo hri) {
HDFSBlocksDistribution blockDistbn = null;
try {
if (cache.asMap().containsKey(hri)) {
@@ -294,7 +294,7 @@ class RegionLocationFinder {
}
private ListenableFuture<HDFSBlocksDistribution> asyncGetBlockDistribution(
- HRegionInfo hri) {
+ RegionInfo hri) {
try {
return loader.reload(hri, EMPTY_BLOCK_DISTRIBUTION);
} catch (Exception e) {
@@ -302,13 +302,13 @@ class RegionLocationFinder {
}
}
- public void refreshAndWait(Collection<HRegionInfo> hris) {
+ public void refreshAndWait(Collection<RegionInfo> hris) {
ArrayList<ListenableFuture<HDFSBlocksDistribution>> regionLocationFutures = new ArrayList<>(hris.size());
- for (HRegionInfo hregionInfo : hris) {
+ for (RegionInfo hregionInfo : hris) {
regionLocationFutures.add(asyncGetBlockDistribution(hregionInfo));
}
int index = 0;
- for (HRegionInfo hregionInfo : hris) {
+ for (RegionInfo hregionInfo : hris) {
ListenableFuture<HDFSBlocksDistribution> future = regionLocationFutures
.get(index);
try {
@@ -325,7 +325,7 @@ class RegionLocationFinder {
}
// For test
- LoadingCache<HRegionInfo, HDFSBlocksDistribution> getCache() {
+ LoadingCache<RegionInfo, HDFSBlocksDistribution> getCache() {
return cache;
}
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/SimpleLoadBalancer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/SimpleLoadBalancer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/SimpleLoadBalancer.java
index b6160c6..e356942 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/SimpleLoadBalancer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/SimpleLoadBalancer.java
@@ -33,12 +33,12 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseIOException;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.master.RegionPlan;
import org.apache.hadoop.hbase.util.Pair;
+import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.MinMaxPriorityQueue;
@@ -77,9 +77,9 @@ public class SimpleLoadBalancer extends BaseLoadBalancer {
private int nextRegionForUnload;
private int numRegionsAdded;
- private List<HRegionInfo> hriList;
+ private List<RegionInfo> hriList;
- public BalanceInfo(int nextRegionForUnload, int numRegionsAdded, List<HRegionInfo> hriList) {
+ public BalanceInfo(int nextRegionForUnload, int numRegionsAdded, List<RegionInfo> hriList) {
this.nextRegionForUnload = nextRegionForUnload;
this.numRegionsAdded = numRegionsAdded;
this.hriList = hriList;
@@ -97,7 +97,7 @@ public class SimpleLoadBalancer extends BaseLoadBalancer {
this.numRegionsAdded = numAdded;
}
- List<HRegionInfo> getHriList() {
+ List<RegionInfo> getHriList() {
return hriList;
}
@@ -107,11 +107,11 @@ public class SimpleLoadBalancer extends BaseLoadBalancer {
}
- public void setClusterLoad(Map<TableName, Map<ServerName, List<HRegionInfo>>> clusterLoad){
+ public void setClusterLoad(Map<TableName, Map<ServerName, List<RegionInfo>>> clusterLoad){
serverLoadList = new ArrayList<>();
float sum = 0;
- for(Map.Entry<TableName, Map<ServerName, List<HRegionInfo>>> clusterEntry : clusterLoad.entrySet()){
- for(Map.Entry<ServerName, List<HRegionInfo>> entry : clusterEntry.getValue().entrySet()){
+ for(Map.Entry<TableName, Map<ServerName, List<RegionInfo>>> clusterEntry : clusterLoad.entrySet()){
+ for(Map.Entry<ServerName, List<RegionInfo>> entry : clusterEntry.getValue().entrySet()){
if(entry.getKey().equals(masterServerName)) continue; // we shouldn't include master as potential assignee
serverLoadList.add(new ServerAndLoad(entry.getKey(), entry.getValue().size()));
sum += entry.getValue().size();
@@ -245,7 +245,7 @@ public class SimpleLoadBalancer extends BaseLoadBalancer {
*/
@Override
public List<RegionPlan> balanceCluster(
- Map<ServerName, List<HRegionInfo>> clusterMap) {
+ Map<ServerName, List<RegionInfo>> clusterMap) {
List<RegionPlan> regionsToReturn = balanceMasterRegions(clusterMap);
if (regionsToReturn != null || clusterMap == null || clusterMap.size() <= 1) {
return regionsToReturn;
@@ -267,7 +267,7 @@ public class SimpleLoadBalancer extends BaseLoadBalancer {
ClusterLoadState cs = new ClusterLoadState(clusterMap);
int numServers = cs.getNumServers();
- NavigableMap<ServerAndLoad, List<HRegionInfo>> serversByLoad = cs.getServersByLoad();
+ NavigableMap<ServerAndLoad, List<RegionInfo>> serversByLoad = cs.getServersByLoad();
int numRegions = cs.getNumRegions();
float average = cs.getLoadAverage();
int max = (int)Math.ceil(average);
@@ -291,7 +291,7 @@ public class SimpleLoadBalancer extends BaseLoadBalancer {
// flag used to fetch regions from head and tail of list, alternately
boolean fetchFromTail = false;
Map<ServerName, BalanceInfo> serverBalanceInfo = new TreeMap<>();
- for (Map.Entry<ServerAndLoad, List<HRegionInfo>> server:
+ for (Map.Entry<ServerAndLoad, List<RegionInfo>> server:
serversByLoad.descendingMap().entrySet()) {
ServerAndLoad sal = server.getKey();
int load = sal.getLoad();
@@ -300,14 +300,14 @@ public class SimpleLoadBalancer extends BaseLoadBalancer {
continue;
}
serversOverloaded++;
- List<HRegionInfo> regions = server.getValue();
+ List<RegionInfo> regions = server.getValue();
int numToOffload = Math.min(load - max, regions.size());
// account for the out-of-band regions which were assigned to this server
// after some other region server crashed
Collections.sort(regions, riComparator);
int numTaken = 0;
for (int i = 0; i <= numToOffload; ) {
- HRegionInfo hri = regions.get(i); // fetch from head
+ RegionInfo hri = regions.get(i); // fetch from head
if (fetchFromTail) {
hri = regions.get(regions.size() - 1 - i);
}
@@ -330,7 +330,7 @@ public class SimpleLoadBalancer extends BaseLoadBalancer {
Map<ServerName, Integer> underloadedServers = new HashMap<>();
int maxToTake = numRegions - min;
- for (Map.Entry<ServerAndLoad, List<HRegionInfo>> server:
+ for (Map.Entry<ServerAndLoad, List<RegionInfo>> server:
serversByLoad.entrySet()) {
if (maxToTake == 0) break; // no more to take
int load = server.getKey().getLoad();
@@ -378,14 +378,14 @@ public class SimpleLoadBalancer extends BaseLoadBalancer {
// If we need more to fill min, grab one from each most loaded until enough
if (neededRegions != 0) {
// Walk down most loaded, grabbing one from each until we get enough
- for (Map.Entry<ServerAndLoad, List<HRegionInfo>> server :
+ for (Map.Entry<ServerAndLoad, List<RegionInfo>> server :
serversByLoad.descendingMap().entrySet()) {
BalanceInfo balanceInfo =
serverBalanceInfo.get(server.getKey().getServerName());
int idx =
balanceInfo == null ? 0 : balanceInfo.getNextRegionForUnload();
if (idx >= server.getValue().size()) break;
- HRegionInfo region = server.getValue().get(idx);
+ RegionInfo region = server.getValue().get(idx);
if (region.isMetaRegion()) continue; // Don't move meta regions.
regionsToMove.add(new RegionPlan(region, server.getKey().getServerName(), null));
balanceInfo.setNumRegionsAdded(balanceInfo.getNumRegionsAdded() - 1);
@@ -402,7 +402,7 @@ public class SimpleLoadBalancer extends BaseLoadBalancer {
// Assign each underloaded up to the min, then if leftovers, assign to max
// Walk down least loaded, assigning to each to fill up to min
- for (Map.Entry<ServerAndLoad, List<HRegionInfo>> server :
+ for (Map.Entry<ServerAndLoad, List<RegionInfo>> server :
serversByLoad.entrySet()) {
int regionCount = server.getKey().getLoad();
if (regionCount >= min) break;
@@ -434,7 +434,7 @@ public class SimpleLoadBalancer extends BaseLoadBalancer {
", numServers=" + numServers + ", serversOverloaded=" + serversOverloaded +
", serversUnderloaded=" + serversUnderloaded);
StringBuilder sb = new StringBuilder();
- for (Map.Entry<ServerName, List<HRegionInfo>> e: clusterMap.entrySet()) {
+ for (Map.Entry<ServerName, List<RegionInfo>> e: clusterMap.entrySet()) {
if (sb.length() > 0) sb.append(", ");
sb.append(e.getKey().toString());
sb.append(" ");
@@ -481,10 +481,10 @@ public class SimpleLoadBalancer extends BaseLoadBalancer {
BalanceInfo balanceInfo = serverBalanceInfo.get(serverload.getServerName());
setLoad(serverLoadList, i, balanceInfo.getNumRegionsAdded());
if (balanceInfo.getHriList().size() + balanceInfo.getNumRegionsAdded() == max) {
- HRegionInfo hriToPlan;
+ RegionInfo hriToPlan;
if (balanceInfo.getHriList().isEmpty()) {
LOG.debug("During balanceOverall, we found " + serverload.getServerName()
- + " has no HRegionInfo, no operation needed");
+ + " has no RegionInfo, no operation needed");
continue;
} else if (balanceInfo.getNextRegionForUnload() >= balanceInfo.getHriList().size()) {
continue;
@@ -587,7 +587,7 @@ public class SimpleLoadBalancer extends BaseLoadBalancer {
@Override
public List<RegionPlan> balanceCluster(TableName tableName,
- Map<ServerName, List<HRegionInfo>> clusterState) throws HBaseIOException {
+ Map<ServerName, List<RegionInfo>> clusterState) throws HBaseIOException {
LOG.debug("Start Generate Balance plan for table: " + tableName);
return balanceCluster(clusterState);
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.java
index 957c182..258f68a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.java
@@ -17,8 +17,6 @@
*/
package org.apache.hadoop.hbase.master.balancer;
-import com.google.common.annotations.VisibleForTesting;
-
import java.util.ArrayDeque;
import java.util.ArrayList;
import java.util.Arrays;
@@ -38,11 +36,11 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.ClusterStatus;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.RegionLoad;
import org.apache.hadoop.hbase.ServerLoad;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.master.RegionPlan;
import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer.Cluster.Action;
@@ -58,6 +56,8 @@ import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.common.base.Optional;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
+import com.google.common.annotations.VisibleForTesting;
+
/**
* <p>This is a best effort load balancer. Given a Cost function F(C) => x It will
* randomly try and mutate the cluster to Cprime. If F(Cprime) < F(C) then the
@@ -315,7 +315,7 @@ public class StochasticLoadBalancer extends BaseLoadBalancer {
@Override
public synchronized List<RegionPlan> balanceCluster(TableName tableName, Map<ServerName,
- List<HRegionInfo>> clusterState) {
+ List<RegionInfo>> clusterState) {
this.tableName = tableName;
return balanceCluster(clusterState);
}
@@ -332,7 +332,7 @@ public class StochasticLoadBalancer extends BaseLoadBalancer {
*/
@Override
public synchronized List<RegionPlan> balanceCluster(Map<ServerName,
- List<HRegionInfo>> clusterState) {
+ List<RegionInfo>> clusterState) {
List<RegionPlan> plans = balanceMasterRegions(clusterState);
if (plans != null || clusterState == null || clusterState.size() <= 1) {
return plans;
@@ -504,7 +504,7 @@ public class StochasticLoadBalancer extends BaseLoadBalancer {
int newServerIndex = cluster.regionIndexToServerIndex[regionIndex];
if (initialServerIndex != newServerIndex) {
- HRegionInfo region = cluster.regions[regionIndex];
+ RegionInfo region = cluster.regions[regionIndex];
ServerName initialServer = cluster.servers[initialServerIndex];
ServerName newServer = cluster.servers[newServerIndex];
@@ -624,7 +624,7 @@ public class StochasticLoadBalancer extends BaseLoadBalancer {
* @param server index of the server
* @param chanceOfNoSwap Chance that this will decide to try a move rather
* than a swap.
- * @return a random {@link HRegionInfo} or null if an asymmetrical move is
+ * @return a random {@link RegionInfo} or null if an asymmetrical move is
* suggested.
*/
protected int pickRandomRegion(Cluster cluster, int server, double chanceOfNoSwap) {
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/locking/LockManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/locking/LockManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/locking/LockManager.java
index 9870add..883d659 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/locking/LockManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/locking/LockManager.java
@@ -24,13 +24,14 @@ import java.util.concurrent.TimeUnit;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.TableName;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.procedure2.LockType;
-import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.hbase.util.NonceKey;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
/**
* Functions to acquire lock on table/namespace/regions.
@@ -60,7 +61,7 @@ public final class LockManager {
return new MasterLock(tableName, type, description);
}
- public MasterLock createMasterLock(final HRegionInfo[] regionInfos, final String description) {
+ public MasterLock createMasterLock(final RegionInfo[] regionInfos, final String description) {
return new MasterLock(regionInfos, description);
}
@@ -81,7 +82,7 @@ public final class LockManager {
public class MasterLock {
private final String namespace;
private final TableName tableName;
- private final HRegionInfo[] regionInfos;
+ private final RegionInfo[] regionInfos;
private final LockType type;
private final String description;
@@ -105,7 +106,7 @@ public final class LockManager {
this.description = description;
}
- public MasterLock(final HRegionInfo[] regionInfos, final String description) {
+ public MasterLock(final RegionInfo[] regionInfos, final String description) {
this.namespace = null;
this.tableName = null;
this.regionInfos = regionInfos;
@@ -229,7 +230,7 @@ public final class LockManager {
/**
* @throws IllegalArgumentException if all regions are not from same table.
*/
- public long requestRegionsLock(final HRegionInfo[] regionInfos, final String description,
+ public long requestRegionsLock(final RegionInfo[] regionInfos, final String description,
final NonceKey nonceKey)
throws IllegalArgumentException, IOException {
master.getMasterCoprocessorHost().preRequestLock(null, null, regionInfos,
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/locking/LockProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/locking/LockProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/locking/LockProcedure.java
index ccb3d48..c9b8ef9 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/locking/LockProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/locking/LockProcedure.java
@@ -19,12 +19,16 @@
package org.apache.hadoop.hbase.master.locking;
+import java.io.IOException;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.TableName;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
import org.apache.hadoop.hbase.master.procedure.TableProcedureInterface;
import org.apache.hadoop.hbase.procedure2.LockType;
@@ -32,16 +36,13 @@ import org.apache.hadoop.hbase.procedure2.Procedure;
import org.apache.hadoop.hbase.procedure2.ProcedureEvent;
import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
+import org.apache.yetus.audience.InterfaceAudience;
+
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockProcedureData;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos;
-import java.io.IOException;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicLong;
-
/**
* Procedure to allow blessed clients and external admin tools to take our internal Schema locks
* used by the procedure framework isolating procedures doing creates/deletes etc. on
@@ -68,7 +69,7 @@ public final class LockProcedure extends Procedure<MasterProcedureEnv>
private String namespace;
private TableName tableName;
- private HRegionInfo[] regionInfos;
+ private RegionInfo[] regionInfos;
private LockType type;
// underlying namespace/table/region lock.
private LockInterface lock;
@@ -160,12 +161,12 @@ public final class LockProcedure extends Procedure<MasterProcedureEnv>
* Useful for locks acquired locally from master process.
* @throws IllegalArgumentException if all regions are not from same table.
*/
- public LockProcedure(final Configuration conf, final HRegionInfo[] regionInfos,
+ public LockProcedure(final Configuration conf, final RegionInfo[] regionInfos,
final LockType type, final String description, final CountDownLatch lockAcquireLatch)
throws IllegalArgumentException {
this(conf, type, description, lockAcquireLatch);
- // Build HRegionInfo from region names.
+ // Build RegionInfo from region names.
if (regionInfos.length == 0) {
throw new IllegalArgumentException("No regions specified for region lock");
}
@@ -269,7 +270,7 @@ public final class LockProcedure extends Procedure<MasterProcedureEnv>
.setDescription(description);
if (regionInfos != null) {
for (int i = 0; i < regionInfos.length; ++i) {
- builder.addRegionInfo(HRegionInfo.convert(regionInfos[i]));
+ builder.addRegionInfo(ProtobufUtil.toRegionInfo(regionInfos[i]));
}
} else if (namespace != null) {
builder.setNamespace(namespace);
@@ -289,9 +290,9 @@ public final class LockProcedure extends Procedure<MasterProcedureEnv>
type = LockType.valueOf(state.getLockType().name());
description = state.getDescription();
if (state.getRegionInfoCount() > 0) {
- regionInfos = new HRegionInfo[state.getRegionInfoCount()];
+ regionInfos = new RegionInfo[state.getRegionInfoCount()];
for (int i = 0; i < state.getRegionInfoCount(); ++i) {
- regionInfos[i] = HRegionInfo.convert(state.getRegionInfo(i));
+ regionInfos[i] = ProtobufUtil.toRegionInfo(state.getRegionInfo(i));
}
} else if (state.hasNamespace()) {
namespace = state.getNamespace();
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/MergeNormalizationPlan.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/MergeNormalizationPlan.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/MergeNormalizationPlan.java
index ec63e2d..b6602b1 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/MergeNormalizationPlan.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/MergeNormalizationPlan.java
@@ -18,14 +18,13 @@
*/
package org.apache.hadoop.hbase.master.normalizer;
+import java.io.IOException;
+
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.master.normalizer.NormalizationPlan.PlanType;
-
-import java.io.IOException;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.yetus.audience.InterfaceAudience;
/**
* Normalization plan to merge regions (smallest region in the table with its smallest neighbor).
@@ -34,10 +33,10 @@ import java.io.IOException;
public class MergeNormalizationPlan implements NormalizationPlan {
private static final Log LOG = LogFactory.getLog(MergeNormalizationPlan.class.getName());
- private final HRegionInfo firstRegion;
- private final HRegionInfo secondRegion;
+ private final RegionInfo firstRegion;
+ private final RegionInfo secondRegion;
- public MergeNormalizationPlan(HRegionInfo firstRegion, HRegionInfo secondRegion) {
+ public MergeNormalizationPlan(RegionInfo firstRegion, RegionInfo secondRegion) {
this.firstRegion = firstRegion;
this.secondRegion = secondRegion;
}
@@ -47,11 +46,11 @@ public class MergeNormalizationPlan implements NormalizationPlan {
return PlanType.MERGE;
}
- HRegionInfo getFirstRegion() {
+ RegionInfo getFirstRegion() {
return firstRegion;
}
- HRegionInfo getSecondRegion() {
+ RegionInfo getSecondRegion() {
return secondRegion;
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/RegionNormalizer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/RegionNormalizer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/RegionNormalizer.java
index fad8db6..74edd26 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/RegionNormalizer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/RegionNormalizer.java
@@ -21,12 +21,12 @@ package org.apache.hadoop.hbase.master.normalizer;
import java.util.List;
import org.apache.hadoop.hbase.HBaseIOException;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.TableName;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.master.MasterRpcServices;
import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.master.normalizer.NormalizationPlan.PlanType;
+import org.apache.yetus.audience.InterfaceAudience;
/**
* Performs "normalization" of regions on the cluster, making sure that suboptimal
@@ -68,8 +68,8 @@ public interface RegionNormalizer {
* @param hri the region which is involved in the plan
* @param type type of plan
*/
- void planSkipped(HRegionInfo hri, PlanType type);
-
+ void planSkipped(RegionInfo hri, PlanType type);
+
/**
* @param type type of plan for which skipped count is to be returned
* @return the count of plans of specified type which were skipped
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SimpleRegionNormalizer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SimpleRegionNormalizer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SimpleRegionNormalizer.java
index 0e2c7dd..8190f27 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SimpleRegionNormalizer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SimpleRegionNormalizer.java
@@ -26,15 +26,16 @@ import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HBaseIOException;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.RegionLoad;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
-import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.MasterSwitchType;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.master.MasterRpcServices;
import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.master.normalizer.NormalizationPlan.PlanType;
+import org.apache.yetus.audience.InterfaceAudience;
+
import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
/**
@@ -80,7 +81,7 @@ public class SimpleRegionNormalizer implements RegionNormalizer {
}
@Override
- public void planSkipped(HRegionInfo hri, PlanType type) {
+ public void planSkipped(RegionInfo hri, PlanType type) {
skippedCount[type.ordinal()]++;
}
@@ -119,7 +120,7 @@ public class SimpleRegionNormalizer implements RegionNormalizer {
}
List<NormalizationPlan> plans = new ArrayList<>();
- List<HRegionInfo> tableRegions = masterServices.getAssignmentManager().getRegionStates().
+ List<RegionInfo> tableRegions = masterServices.getAssignmentManager().getRegionStates().
getRegionsOfTable(table);
//TODO: should we make min number of regions a config param?
@@ -137,7 +138,7 @@ public class SimpleRegionNormalizer implements RegionNormalizer {
int acutalRegionCnt = 0;
for (int i = 0; i < tableRegions.size(); i++) {
- HRegionInfo hri = tableRegions.get(i);
+ RegionInfo hri = tableRegions.get(i);
long regionSize = getRegionSize(hri);
if (regionSize > 0) {
acutalRegionCnt++;
@@ -165,7 +166,7 @@ public class SimpleRegionNormalizer implements RegionNormalizer {
LOG.debug("Unable to determine whether split is enabled", e);
}
while (candidateIdx < tableRegions.size()) {
- HRegionInfo hri = tableRegions.get(candidateIdx);
+ RegionInfo hri = tableRegions.get(candidateIdx);
long regionSize = getRegionSize(hri);
// if the region is > 2 times larger than average, we split it, split
// is more high priority normalization action than merge.
@@ -180,7 +181,7 @@ public class SimpleRegionNormalizer implements RegionNormalizer {
break;
}
if (mergeEnabled) {
- HRegionInfo hri2 = tableRegions.get(candidateIdx+1);
+ RegionInfo hri2 = tableRegions.get(candidateIdx+1);
long regionSize2 = getRegionSize(hri2);
if (regionSize >= 0 && regionSize2 >= 0 && regionSize + regionSize2 < avgRegionSize) {
LOG.info("Table " + table + ", small region size: " + regionSize
@@ -201,7 +202,7 @@ public class SimpleRegionNormalizer implements RegionNormalizer {
return plans;
}
- private long getRegionSize(HRegionInfo hri) {
+ private long getRegionSize(RegionInfo hri) {
ServerName sn = masterServices.getAssignmentManager().getRegionStates().
getRegionServerOfRegion(hri);
RegionLoad regionLoad = masterServices.getServerManager().getLoad(sn).
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SplitNormalizationPlan.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SplitNormalizationPlan.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SplitNormalizationPlan.java
index a5912f6..9217143 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SplitNormalizationPlan.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SplitNormalizationPlan.java
@@ -18,15 +18,14 @@
*/
package org.apache.hadoop.hbase.master.normalizer;
+import java.io.IOException;
+import java.util.Arrays;
+
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.master.normalizer.NormalizationPlan.PlanType;
-
-import java.io.IOException;
-import java.util.Arrays;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.yetus.audience.InterfaceAudience;
/**
* Normalization plan to split region.
@@ -35,10 +34,10 @@ import java.util.Arrays;
public class SplitNormalizationPlan implements NormalizationPlan {
private static final Log LOG = LogFactory.getLog(SplitNormalizationPlan.class.getName());
- private HRegionInfo regionInfo;
+ private RegionInfo regionInfo;
private byte[] splitPoint;
- public SplitNormalizationPlan(HRegionInfo regionInfo, byte[] splitPoint) {
+ public SplitNormalizationPlan(RegionInfo regionInfo, byte[] splitPoint) {
this.regionInfo = regionInfo;
this.splitPoint = splitPoint;
}
@@ -48,11 +47,11 @@ public class SplitNormalizationPlan implements NormalizationPlan {
return PlanType.SPLIT;
}
- public HRegionInfo getRegionInfo() {
+ public RegionInfo getRegionInfo() {
return regionInfo;
}
- public void setRegionInfo(HRegionInfo regionInfo) {
+ public void setRegionInfo(RegionInfo regionInfo) {
this.regionInfo = regionInfo;
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AbstractStateMachineRegionProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AbstractStateMachineRegionProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AbstractStateMachineRegionProcedure.java
index 3b3d326..e711ca0 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AbstractStateMachineRegionProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AbstractStateMachineRegionProcedure.java
@@ -19,12 +19,15 @@
package org.apache.hadoop.hbase.master.procedure;
import java.io.IOException;
-import org.apache.hadoop.hbase.HRegionInfo;
+
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotFoundException;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
/**
@@ -35,11 +38,11 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
@InterfaceAudience.Private
public abstract class AbstractStateMachineRegionProcedure<TState>
extends AbstractStateMachineTableProcedure<TState> {
- private HRegionInfo hri;
+ private RegionInfo hri;
private volatile boolean lock = false;
public AbstractStateMachineRegionProcedure(final MasterProcedureEnv env,
- final HRegionInfo hri) {
+ final RegionInfo hri) {
super(env);
this.hri = hri;
}
@@ -50,16 +53,16 @@ public abstract class AbstractStateMachineRegionProcedure<TState>
}
/**
- * @return The HRegionInfo of the region we are operating on.
+ * @return The RegionInfo of the region we are operating on.
*/
- protected HRegionInfo getRegion() {
+ protected RegionInfo getRegion() {
return this.hri;
}
/**
* Used when deserializing. Otherwise, DON'T TOUCH IT!
*/
- protected void setRegion(final HRegionInfo hri) {
+ protected void setRegion(final RegionInfo hri) {
this.hri = hri;
}
@@ -124,13 +127,13 @@ public abstract class AbstractStateMachineRegionProcedure<TState>
protected void serializeStateData(ProcedureStateSerializer serializer)
throws IOException {
super.serializeStateData(serializer);
- serializer.serialize(HRegionInfo.convert(getRegion()));
+ serializer.serialize(ProtobufUtil.toRegionInfo(getRegion()));
}
@Override
protected void deserializeStateData(ProcedureStateSerializer serializer)
throws IOException {
super.deserializeStateData(serializer);
- this.hri = HRegionInfo.convert(serializer.deserialize(HBaseProtos.RegionInfo.class));
+ this.hri = ProtobufUtil.toRegionInfo(serializer.deserialize(HBaseProtos.RegionInfo.class));
}
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AddColumnFamilyProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AddColumnFamilyProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AddColumnFamilyProcedure.java
index 176ff3d..8351524 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AddColumnFamilyProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AddColumnFamilyProcedure.java
@@ -23,15 +23,16 @@ import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.InvalidFamilyOperationException;
import org.apache.hadoop.hbase.TableName;
-import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
+import org.apache.yetus.audience.InterfaceAudience;
+
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.AddColumnFamilyState;
@@ -48,7 +49,7 @@ public class AddColumnFamilyProcedure
private TableDescriptor unmodifiedTableDescriptor;
private ColumnFamilyDescriptor cfDescriptor;
- private List<HRegionInfo> regionInfoList;
+ private List<RegionInfo> regionInfoList;
private Boolean traceEnabled;
public AddColumnFamilyProcedure() {
@@ -347,7 +348,7 @@ public class AddColumnFamilyProcedure
}
}
- private List<HRegionInfo> getRegionInfoList(final MasterProcedureEnv env) throws IOException {
+ private List<RegionInfo> getRegionInfoList(final MasterProcedureEnv env) throws IOException {
if (regionInfoList == null) {
regionInfoList = env.getAssignmentManager().getRegionStates()
.getRegionsOfTable(getTableName());
[15/19] hbase git commit: HBASE-18839 Apply RegionInfo to code base
Posted by ch...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/favored/FavoredNodeLoadBalancer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/favored/FavoredNodeLoadBalancer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/favored/FavoredNodeLoadBalancer.java
index 9c5c180..68e5e89 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/favored/FavoredNodeLoadBalancer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/favored/FavoredNodeLoadBalancer.java
@@ -29,23 +29,26 @@ import java.util.List;
import java.util.Map;
import java.util.Set;
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.Maps;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
-import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.favored.FavoredNodesPlan.Position;
import org.apache.hadoop.hbase.HBaseIOException;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerLoad;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.master.*;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.favored.FavoredNodesPlan.Position;
+import org.apache.hadoop.hbase.master.RackManager;
+import org.apache.hadoop.hbase.master.RegionPlan;
+import org.apache.hadoop.hbase.master.ServerManager;
+import org.apache.hadoop.hbase.master.SnapshotOfRegionAssignmentFromMeta;
import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer;
import org.apache.hadoop.hbase.util.Pair;
+import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.Maps;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Sets;
/**
@@ -85,7 +88,7 @@ public class FavoredNodeLoadBalancer extends BaseLoadBalancer implements Favored
}
@Override
- public List<RegionPlan> balanceCluster(Map<ServerName, List<HRegionInfo>> clusterState) {
+ public List<RegionPlan> balanceCluster(Map<ServerName, List<RegionInfo>> clusterState) {
//TODO. Look at is whether Stochastic loadbalancer can be integrated with this
List<RegionPlan> plans = new ArrayList<>();
//perform a scan of the meta to get the latest updates (if any)
@@ -105,13 +108,13 @@ public class FavoredNodeLoadBalancer extends BaseLoadBalancer implements Favored
// FindBugs complains about useless store! serverNameToServerNameWithoutCode.put(sn, s);
serverNameWithoutCodeToServerName.put(s, sn);
}
- for (Map.Entry<ServerName, List<HRegionInfo>> entry : clusterState.entrySet()) {
+ for (Map.Entry<ServerName, List<RegionInfo>> entry : clusterState.entrySet()) {
ServerName currentServer = entry.getKey();
//get a server without the startcode for the currentServer
ServerName currentServerWithoutStartCode = ServerName.valueOf(currentServer.getHostname(),
currentServer.getPort(), ServerName.NON_STARTCODE);
- List<HRegionInfo> list = entry.getValue();
- for (HRegionInfo region : list) {
+ List<RegionInfo> list = entry.getValue();
+ for (RegionInfo region : list) {
if(!FavoredNodesManager.isFavoredNodeApplicable(region)) {
continue;
}
@@ -157,9 +160,9 @@ public class FavoredNodeLoadBalancer extends BaseLoadBalancer implements Favored
}
@Override
- public Map<ServerName, List<HRegionInfo>> roundRobinAssignment(List<HRegionInfo> regions,
+ public Map<ServerName, List<RegionInfo>> roundRobinAssignment(List<RegionInfo> regions,
List<ServerName> servers) throws HBaseIOException {
- Map<ServerName, List<HRegionInfo>> assignmentMap;
+ Map<ServerName, List<RegionInfo>> assignmentMap;
try {
FavoredNodeAssignmentHelper assignmentHelper =
new FavoredNodeAssignmentHelper(servers, rackManager);
@@ -183,10 +186,10 @@ public class FavoredNodeLoadBalancer extends BaseLoadBalancer implements Favored
// need to come up with favored nodes assignments for them. The corner case
// in (1) above is that all the nodes are unavailable and in that case, we
// will note that this region doesn't have favored nodes.
- Pair<Map<ServerName,List<HRegionInfo>>, List<HRegionInfo>> segregatedRegions =
+ Pair<Map<ServerName,List<RegionInfo>>, List<RegionInfo>> segregatedRegions =
segregateRegionsAndAssignRegionsWithFavoredNodes(regions, servers);
- Map<ServerName,List<HRegionInfo>> regionsWithFavoredNodesMap = segregatedRegions.getFirst();
- List<HRegionInfo> regionsWithNoFavoredNodes = segregatedRegions.getSecond();
+ Map<ServerName,List<RegionInfo>> regionsWithFavoredNodesMap = segregatedRegions.getFirst();
+ List<RegionInfo> regionsWithNoFavoredNodes = segregatedRegions.getSecond();
assignmentMap = new HashMap<>();
roundRobinAssignmentImpl(assignmentHelper, assignmentMap, regionsWithNoFavoredNodes,
servers);
@@ -201,7 +204,7 @@ public class FavoredNodeLoadBalancer extends BaseLoadBalancer implements Favored
}
@Override
- public ServerName randomAssignment(HRegionInfo regionInfo, List<ServerName> servers)
+ public ServerName randomAssignment(RegionInfo regionInfo, List<ServerName> servers)
throws HBaseIOException {
try {
FavoredNodeAssignmentHelper assignmentHelper =
@@ -224,9 +227,9 @@ public class FavoredNodeLoadBalancer extends BaseLoadBalancer implements Favored
}
}
}
- List<HRegionInfo> regions = new ArrayList<>(1);
+ List<RegionInfo> regions = new ArrayList<>(1);
regions.add(regionInfo);
- Map<HRegionInfo, ServerName> primaryRSMap = new HashMap<>(1);
+ Map<RegionInfo, ServerName> primaryRSMap = new HashMap<>(1);
primaryRSMap.put(regionInfo, primary);
assignSecondaryAndTertiaryNodesForRegion(assignmentHelper, regions, primaryRSMap);
return primary;
@@ -237,12 +240,12 @@ public class FavoredNodeLoadBalancer extends BaseLoadBalancer implements Favored
}
}
- private Pair<Map<ServerName, List<HRegionInfo>>, List<HRegionInfo>>
- segregateRegionsAndAssignRegionsWithFavoredNodes(List<HRegionInfo> regions,
+ private Pair<Map<ServerName, List<RegionInfo>>, List<RegionInfo>>
+ segregateRegionsAndAssignRegionsWithFavoredNodes(List<RegionInfo> regions,
List<ServerName> availableServers) {
- Map<ServerName, List<HRegionInfo>> assignmentMapForFavoredNodes = new HashMap<>(regions.size() / 2);
- List<HRegionInfo> regionsWithNoFavoredNodes = new ArrayList<>(regions.size()/2);
- for (HRegionInfo region : regions) {
+ Map<ServerName, List<RegionInfo>> assignmentMapForFavoredNodes = new HashMap<>(regions.size() / 2);
+ List<RegionInfo> regionsWithNoFavoredNodes = new ArrayList<>(regions.size()/2);
+ for (RegionInfo region : regions) {
List<ServerName> favoredNodes = fnm.getFavoredNodes(region);
ServerName primaryHost = null;
ServerName secondaryHost = null;
@@ -286,7 +289,7 @@ public class FavoredNodeLoadBalancer extends BaseLoadBalancer implements Favored
}
private void assignRegionToAvailableFavoredNode(Map<ServerName,
- List<HRegionInfo>> assignmentMapForFavoredNodes, HRegionInfo region, ServerName primaryHost,
+ List<RegionInfo>> assignmentMapForFavoredNodes, RegionInfo region, ServerName primaryHost,
ServerName secondaryHost, ServerName tertiaryHost) {
if (primaryHost != null) {
addRegionToMap(assignmentMapForFavoredNodes, region, primaryHost);
@@ -309,9 +312,9 @@ public class FavoredNodeLoadBalancer extends BaseLoadBalancer implements Favored
}
}
- private void addRegionToMap(Map<ServerName, List<HRegionInfo>> assignmentMapForFavoredNodes,
- HRegionInfo region, ServerName host) {
- List<HRegionInfo> regionsOnServer = null;
+ private void addRegionToMap(Map<ServerName, List<RegionInfo>> assignmentMapForFavoredNodes,
+ RegionInfo region, ServerName host) {
+ List<RegionInfo> regionsOnServer = null;
if ((regionsOnServer = assignmentMapForFavoredNodes.get(host)) == null) {
regionsOnServer = new ArrayList<>();
assignmentMapForFavoredNodes.put(host, regionsOnServer);
@@ -319,14 +322,14 @@ public class FavoredNodeLoadBalancer extends BaseLoadBalancer implements Favored
regionsOnServer.add(region);
}
- public synchronized List<ServerName> getFavoredNodes(HRegionInfo regionInfo) {
+ public synchronized List<ServerName> getFavoredNodes(RegionInfo regionInfo) {
return this.fnm.getFavoredNodes(regionInfo);
}
private void roundRobinAssignmentImpl(FavoredNodeAssignmentHelper assignmentHelper,
- Map<ServerName, List<HRegionInfo>> assignmentMap,
- List<HRegionInfo> regions, List<ServerName> servers) throws IOException {
- Map<HRegionInfo, ServerName> primaryRSMap = new HashMap<>();
+ Map<ServerName, List<RegionInfo>> assignmentMap,
+ List<RegionInfo> regions, List<ServerName> servers) throws IOException {
+ Map<RegionInfo, ServerName> primaryRSMap = new HashMap<>();
// figure the primary RSs
assignmentHelper.placePrimaryRSAsRoundRobin(assignmentMap, primaryRSMap, regions);
assignSecondaryAndTertiaryNodesForRegion(assignmentHelper, regions, primaryRSMap);
@@ -334,14 +337,14 @@ public class FavoredNodeLoadBalancer extends BaseLoadBalancer implements Favored
private void assignSecondaryAndTertiaryNodesForRegion(
FavoredNodeAssignmentHelper assignmentHelper,
- List<HRegionInfo> regions, Map<HRegionInfo, ServerName> primaryRSMap) throws IOException {
+ List<RegionInfo> regions, Map<RegionInfo, ServerName> primaryRSMap) throws IOException {
// figure the secondary and tertiary RSs
- Map<HRegionInfo, ServerName[]> secondaryAndTertiaryRSMap =
+ Map<RegionInfo, ServerName[]> secondaryAndTertiaryRSMap =
assignmentHelper.placeSecondaryAndTertiaryRS(primaryRSMap);
- Map<HRegionInfo, List<ServerName>> regionFNMap = Maps.newHashMap();
+ Map<RegionInfo, List<ServerName>> regionFNMap = Maps.newHashMap();
// now record all the assignments so that we can serve queries later
- for (HRegionInfo region : regions) {
+ for (RegionInfo region : regions) {
// Store the favored nodes without startCode for the ServerName objects
// We don't care about the startcode; but only the hostname really
List<ServerName> favoredNodesForRegion = new ArrayList<>(3);
@@ -371,10 +374,10 @@ public class FavoredNodeLoadBalancer extends BaseLoadBalancer implements Favored
* randomly. This would give us better distribution over a period of time after enough splits.
*/
@Override
- public void generateFavoredNodesForDaughter(List<ServerName> servers, HRegionInfo parent,
- HRegionInfo regionA, HRegionInfo regionB) throws IOException {
+ public void generateFavoredNodesForDaughter(List<ServerName> servers, RegionInfo parent,
+ RegionInfo regionA, RegionInfo regionB) throws IOException {
- Map<HRegionInfo, List<ServerName>> result = new HashMap<>();
+ Map<RegionInfo, List<ServerName>> result = new HashMap<>();
FavoredNodeAssignmentHelper helper = new FavoredNodeAssignmentHelper(servers, rackManager);
helper.initialize();
@@ -426,16 +429,16 @@ public class FavoredNodeLoadBalancer extends BaseLoadBalancer implements Favored
* keep it simple.
*/
@Override
- public void generateFavoredNodesForMergedRegion(HRegionInfo merged, HRegionInfo regionA,
- HRegionInfo regionB) throws IOException {
- Map<HRegionInfo, List<ServerName>> regionFNMap = Maps.newHashMap();
+ public void generateFavoredNodesForMergedRegion(RegionInfo merged, RegionInfo regionA,
+ RegionInfo regionB) throws IOException {
+ Map<RegionInfo, List<ServerName>> regionFNMap = Maps.newHashMap();
regionFNMap.put(merged, getFavoredNodes(regionA));
fnm.updateFavoredNodes(regionFNMap);
}
@Override
public List<RegionPlan> balanceCluster(TableName tableName,
- Map<ServerName, List<HRegionInfo>> clusterState) throws HBaseIOException {
+ Map<ServerName, List<RegionInfo>> clusterState) throws HBaseIOException {
return balanceCluster(clusterState);
}
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/favored/FavoredNodesManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/favored/FavoredNodesManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/favored/FavoredNodesManager.java
index b1a30ac..1491fae 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/favored/FavoredNodesManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/favored/FavoredNodesManager.java
@@ -18,11 +18,11 @@
*/
package org.apache.hadoop.hbase.favored;
+import static org.apache.hadoop.hbase.ServerName.NON_STARTCODE;
import static org.apache.hadoop.hbase.favored.FavoredNodeAssignmentHelper.FAVORED_NODES_NUM;
import static org.apache.hadoop.hbase.favored.FavoredNodesPlan.Position.PRIMARY;
import static org.apache.hadoop.hbase.favored.FavoredNodesPlan.Position.SECONDARY;
import static org.apache.hadoop.hbase.favored.FavoredNodesPlan.Position.TERTIARY;
-import static org.apache.hadoop.hbase.ServerName.NON_STARTCODE;
import java.io.IOException;
import java.util.ArrayList;
@@ -34,17 +34,17 @@ import java.util.Set;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
-import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseIOException;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.master.RackManager;
import org.apache.hadoop.hbase.master.SnapshotOfRegionAssignmentFromMeta;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.net.NetUtils;
+import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Maps;
@@ -66,9 +66,9 @@ public class FavoredNodesManager {
private static final Log LOG = LogFactory.getLog(FavoredNodesManager.class);
private FavoredNodesPlan globalFavoredNodesAssignmentPlan;
- private Map<ServerName, List<HRegionInfo>> primaryRSToRegionMap;
- private Map<ServerName, List<HRegionInfo>> secondaryRSToRegionMap;
- private Map<ServerName, List<HRegionInfo>> teritiaryRSToRegionMap;
+ private Map<ServerName, List<RegionInfo>> primaryRSToRegionMap;
+ private Map<ServerName, List<RegionInfo>> secondaryRSToRegionMap;
+ private Map<ServerName, List<RegionInfo>> teritiaryRSToRegionMap;
private MasterServices masterServices;
private RackManager rackManager;
@@ -108,7 +108,7 @@ public class FavoredNodesManager {
return dnPort;
}
- public synchronized List<ServerName> getFavoredNodes(HRegionInfo regionInfo) {
+ public synchronized List<ServerName> getFavoredNodes(RegionInfo regionInfo) {
return this.globalFavoredNodesAssignmentPlan.getFavoredNodes(regionInfo);
}
@@ -116,7 +116,7 @@ public class FavoredNodesManager {
* Favored nodes are not applicable for system tables. We will use this to check before
* we apply any favored nodes logic on a region.
*/
- public static boolean isFavoredNodeApplicable(HRegionInfo regionInfo) {
+ public static boolean isFavoredNodeApplicable(RegionInfo regionInfo) {
return !regionInfo.isSystemTable();
}
@@ -126,9 +126,9 @@ public class FavoredNodesManager {
* @param regions - collection of regions
* @return set of regions for which favored nodes is not applicable
*/
- public static Set<HRegionInfo> filterNonFNApplicableRegions(Collection<HRegionInfo> regions) {
- Set<HRegionInfo> fnRegions = Sets.newHashSet();
- for (HRegionInfo regionInfo : regions) {
+ public static Set<RegionInfo> filterNonFNApplicableRegions(Collection<RegionInfo> regions) {
+ Set<RegionInfo> fnRegions = Sets.newHashSet();
+ for (RegionInfo regionInfo : regions) {
if (!isFavoredNodeApplicable(regionInfo)) {
fnRegions.add(regionInfo);
}
@@ -141,7 +141,7 @@ public class FavoredNodesManager {
* sending the region server port, we use the datanode port. This helps in centralizing the DN
* port logic in Master. The RS uses the port from the favored node list as hints.
*/
- public synchronized List<ServerName> getFavoredNodesWithDNPort(HRegionInfo regionInfo) {
+ public synchronized List<ServerName> getFavoredNodesWithDNPort(RegionInfo regionInfo) {
if (getFavoredNodes(regionInfo) == null) {
return null;
}
@@ -154,12 +154,12 @@ public class FavoredNodesManager {
return fnWithDNPort;
}
- public synchronized void updateFavoredNodes(Map<HRegionInfo, List<ServerName>> regionFNMap)
+ public synchronized void updateFavoredNodes(Map<RegionInfo, List<ServerName>> regionFNMap)
throws IOException {
- Map<HRegionInfo, List<ServerName>> regionToFavoredNodes = new HashMap<>();
- for (Map.Entry<HRegionInfo, List<ServerName>> entry : regionFNMap.entrySet()) {
- HRegionInfo regionInfo = entry.getKey();
+ Map<RegionInfo, List<ServerName>> regionToFavoredNodes = new HashMap<>();
+ for (Map.Entry<RegionInfo, List<ServerName>> entry : regionFNMap.entrySet()) {
+ RegionInfo regionInfo = entry.getKey();
List<ServerName> servers = entry.getValue();
/*
@@ -199,18 +199,18 @@ public class FavoredNodesManager {
masterServices.getConnection());
deleteFavoredNodesForRegions(regionToFavoredNodes.keySet());
- for (Map.Entry<HRegionInfo, List<ServerName>> entry : regionToFavoredNodes.entrySet()) {
- HRegionInfo regionInfo = entry.getKey();
+ for (Map.Entry<RegionInfo, List<ServerName>> entry : regionToFavoredNodes.entrySet()) {
+ RegionInfo regionInfo = entry.getKey();
List<ServerName> serversWithNoStartCodes = entry.getValue();
globalFavoredNodesAssignmentPlan.updateFavoredNodesMap(regionInfo, serversWithNoStartCodes);
addToReplicaLoad(regionInfo, serversWithNoStartCodes);
}
}
- private synchronized void addToReplicaLoad(HRegionInfo hri, List<ServerName> servers) {
+ private synchronized void addToReplicaLoad(RegionInfo hri, List<ServerName> servers) {
ServerName serverToUse = ServerName.valueOf(servers.get(PRIMARY.ordinal()).getHostAndPort(),
NON_STARTCODE);
- List<HRegionInfo> regionList = primaryRSToRegionMap.get(serverToUse);
+ List<RegionInfo> regionList = primaryRSToRegionMap.get(serverToUse);
if (regionList == null) {
regionList = new ArrayList<>();
}
@@ -269,8 +269,8 @@ public class FavoredNodesManager {
return result;
}
- public synchronized void deleteFavoredNodesForRegions(Collection<HRegionInfo> regionInfoList) {
- for (HRegionInfo hri : regionInfoList) {
+ public synchronized void deleteFavoredNodesForRegions(Collection<RegionInfo> regionInfoList) {
+ for (RegionInfo hri : regionInfoList) {
List<ServerName> favNodes = getFavoredNodes(hri);
if (favNodes != null) {
if (primaryRSToRegionMap.containsKey(favNodes.get(PRIMARY.ordinal()))) {
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/favored/FavoredNodesPlan.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/favored/FavoredNodesPlan.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/favored/FavoredNodesPlan.java
index 1ac72b8..ac8281e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/favored/FavoredNodesPlan.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/favored/FavoredNodesPlan.java
@@ -22,9 +22,9 @@ import java.util.List;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.yetus.audience.InterfaceAudience;
/**
* This class contains the mapping information between each region name and
@@ -54,7 +54,7 @@ public class FavoredNodesPlan {
* @param region
* @param servers
*/
- public void updateFavoredNodesMap(HRegionInfo region, List<ServerName> servers) {
+ public void updateFavoredNodesMap(RegionInfo region, List<ServerName> servers) {
if (region == null || servers == null || servers.isEmpty()) {
return;
}
@@ -66,7 +66,7 @@ public class FavoredNodesPlan {
* @param region region
* @return the list of favored region server for this region based on the plan
*/
- public List<ServerName> removeFavoredNodes(HRegionInfo region) {
+ public List<ServerName> removeFavoredNodes(RegionInfo region) {
return favoredNodesMap.remove(region.getRegionNameAsString());
}
@@ -74,7 +74,7 @@ public class FavoredNodesPlan {
* @param region
* @return the list of favored region server for this region based on the plan
*/
- public List<ServerName> getFavoredNodes(HRegionInfo region) {
+ public List<ServerName> getFavoredNodes(RegionInfo region) {
return favoredNodesMap.get(region.getRegionNameAsString());
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/favored/FavoredNodesPromoter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/favored/FavoredNodesPromoter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/favored/FavoredNodesPromoter.java
index 14b9840..95160a6 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/favored/FavoredNodesPromoter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/favored/FavoredNodesPromoter.java
@@ -21,7 +21,7 @@ import java.io.IOException;
import java.util.List;
import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.ServerName;
@InterfaceAudience.Private
@@ -31,8 +31,8 @@ public interface FavoredNodesPromoter {
String FAVORED_ALWAYS_ASSIGN_REGIONS = "hbase.favored.assignment.always.assign";
void generateFavoredNodesForDaughter(List<ServerName> servers,
- HRegionInfo parent, HRegionInfo hriA, HRegionInfo hriB) throws IOException;
+ RegionInfo parent, RegionInfo hriA, RegionInfo hriB) throws IOException;
- void generateFavoredNodesForMergedRegion(HRegionInfo merged, HRegionInfo hriA,
- HRegionInfo hriB) throws IOException;
+ void generateFavoredNodesForMergedRegion(RegionInfo merged, RegionInfo hriA,
+ RegionInfo hriB) throws IOException;
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/io/HFileLink.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/HFileLink.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/HFileLink.java
index d2c9ff3..ee10386 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/HFileLink.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/HFileLink.java
@@ -24,19 +24,20 @@ import java.util.regex.Pattern;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
-import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
import org.apache.hadoop.hbase.mob.MobConstants;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.HFileArchiveUtil;
import org.apache.hadoop.hbase.util.Pair;
+import org.apache.yetus.audience.InterfaceAudience;
/**
* HFileLink describes a link to an hfile.
@@ -73,14 +74,14 @@ public class HFileLink extends FileLink {
public static final String LINK_NAME_REGEX =
String.format("(?:(?:%s=)?)%s=%s-%s",
TableName.VALID_NAMESPACE_REGEX, TableName.VALID_TABLE_QUALIFIER_REGEX,
- HRegionInfo.ENCODED_REGION_NAME_REGEX, StoreFileInfo.HFILE_NAME_REGEX);
+ RegionInfoBuilder.ENCODED_REGION_NAME_REGEX, StoreFileInfo.HFILE_NAME_REGEX);
/** Define the HFile Link name parser in the form of: table=region-hfile */
//made package private for testing
static final Pattern LINK_NAME_PATTERN =
Pattern.compile(String.format("^(?:(%s)(?:\\=))?(%s)=(%s)-(%s)$",
TableName.VALID_NAMESPACE_REGEX, TableName.VALID_TABLE_QUALIFIER_REGEX,
- HRegionInfo.ENCODED_REGION_NAME_REGEX, StoreFileInfo.HFILE_NAME_REGEX));
+ RegionInfoBuilder.ENCODED_REGION_NAME_REGEX, StoreFileInfo.HFILE_NAME_REGEX));
/**
* The pattern should be used for hfile and reference links
@@ -89,7 +90,7 @@ public class HFileLink extends FileLink {
private static final Pattern REF_OR_HFILE_LINK_PATTERN =
Pattern.compile(String.format("^(?:(%s)(?:=))?(%s)=(%s)-(.+)$",
TableName.VALID_NAMESPACE_REGEX, TableName.VALID_TABLE_QUALIFIER_REGEX,
- HRegionInfo.ENCODED_REGION_NAME_REGEX));
+ RegionInfoBuilder.ENCODED_REGION_NAME_REGEX));
private final Path archivePath;
private final Path originPath;
@@ -285,7 +286,7 @@ public class HFileLink extends FileLink {
* @param hfileName - Linked HFile name
* @return file name of the HFile Link
*/
- public static String createHFileLinkName(final HRegionInfo hfileRegionInfo,
+ public static String createHFileLinkName(final RegionInfo hfileRegionInfo,
final String hfileName) {
return createHFileLinkName(hfileRegionInfo.getTable(),
hfileRegionInfo.getEncodedName(), hfileName);
@@ -322,7 +323,7 @@ public class HFileLink extends FileLink {
* @throws IOException on file or parent directory creation failure
*/
public static boolean create(final Configuration conf, final FileSystem fs,
- final Path dstFamilyPath, final HRegionInfo hfileRegionInfo,
+ final Path dstFamilyPath, final RegionInfo hfileRegionInfo,
final String hfileName) throws IOException {
return create(conf, fs, dstFamilyPath, hfileRegionInfo, hfileName, true);
}
@@ -343,7 +344,7 @@ public class HFileLink extends FileLink {
* @throws IOException on file or parent directory creation failure
*/
public static boolean create(final Configuration conf, final FileSystem fs,
- final Path dstFamilyPath, final HRegionInfo hfileRegionInfo,
+ final Path dstFamilyPath, final RegionInfo hfileRegionInfo,
final String hfileName, final boolean createBackRef) throws IOException {
TableName linkedTable = hfileRegionInfo.getTable();
String linkedRegion = hfileRegionInfo.getEncodedName();
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentListener.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentListener.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentListener.java
index 52d1b1b..84a7042 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentListener.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentListener.java
@@ -17,10 +17,9 @@
*/
package org.apache.hadoop.hbase.master;
-
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.yetus.audience.InterfaceAudience;
/**
* Get notification of assignment events. The invocations are inline
@@ -33,11 +32,11 @@ public interface AssignmentListener {
* @param regionInfo The opened region.
* @param serverName The remote servers name.
*/
- void regionOpened(final HRegionInfo regionInfo, final ServerName serverName);
+ void regionOpened(final RegionInfo regionInfo, final ServerName serverName);
/**
* The region was closed on the region server.
* @param regionInfo The closed region.
*/
- void regionClosed(final HRegionInfo regionInfo);
+ void regionClosed(final RegionInfo regionInfo);
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentVerificationReport.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentVerificationReport.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentVerificationReport.java
index 9140132..ccbfadc 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentVerificationReport.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentVerificationReport.java
@@ -28,12 +28,12 @@ import java.util.Set;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.hbase.favored.FavoredNodeAssignmentHelper;
-import org.apache.hadoop.hbase.favored.FavoredNodesPlan;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.favored.FavoredNodeAssignmentHelper;
+import org.apache.hadoop.hbase.favored.FavoredNodesPlan;
+import org.apache.yetus.audience.InterfaceAudience;
/**
* Helper class that is used by {@link RegionPlacementMaintainer} to print
* information for favored nodes
@@ -51,13 +51,13 @@ public class AssignmentVerificationReport {
private int totalRegions = 0;
private int totalRegionServers = 0;
// for unassigned regions
- private List<HRegionInfo> unAssignedRegionsList = new ArrayList<>();
+ private List<RegionInfo> unAssignedRegionsList = new ArrayList<>();
// For regions without valid favored nodes
- private List<HRegionInfo> regionsWithoutValidFavoredNodes = new ArrayList<>();
+ private List<RegionInfo> regionsWithoutValidFavoredNodes = new ArrayList<>();
// For regions not running on the favored nodes
- private List<HRegionInfo> nonFavoredAssignedRegionList = new ArrayList<>();
+ private List<RegionInfo> nonFavoredAssignedRegionList = new ArrayList<>();
// For regions running on the favored nodes
private int totalFavoredAssignments = 0;
@@ -91,7 +91,7 @@ public class AssignmentVerificationReport {
this.tableName = tableName;
// Get all the regions for this table
- List<HRegionInfo> regionInfoList =
+ List<RegionInfo> regionInfoList =
snapshot.getTableToRegionMap().get(tableName);
// Get the total region num for the current table
this.totalRegions = regionInfoList.size();
@@ -99,7 +99,7 @@ public class AssignmentVerificationReport {
// Get the existing assignment plan
FavoredNodesPlan favoredNodesAssignment = snapshot.getExistingAssignmentPlan();
// Get the region to region server mapping
- Map<HRegionInfo, ServerName> currentAssignment =
+ Map<RegionInfo, ServerName> currentAssignment =
snapshot.getRegionToRegionServerMap();
// Initialize the server to its hosing region counter map
Map<ServerName, Integer> serverToHostingRegionCounterMap = new HashMap<>();
@@ -109,7 +109,7 @@ public class AssignmentVerificationReport {
// Check the favored nodes and its locality information
// Also keep tracker of the most loaded and least loaded region servers
- for (HRegionInfo region : regionInfoList) {
+ for (RegionInfo region : regionInfoList) {
try {
ServerName currentRS = currentAssignment.get(region);
// Handle unassigned regions
@@ -317,7 +317,7 @@ public class AssignmentVerificationReport {
// Set the table name
this.tableName = tableName;
// Get all the regions for this table
- List<HRegionInfo> regionInfoList = snapshot.getTableToRegionMap().get(
+ List<RegionInfo> regionInfoList = snapshot.getTableToRegionMap().get(
tableName);
// Get the total region num for the current table
this.totalRegions = regionInfoList.size();
@@ -333,7 +333,7 @@ public class AssignmentVerificationReport {
// Check the favored nodes and its locality information
// Also keep tracker of the most loaded and least loaded region servers
- for (HRegionInfo region : regionInfoList) {
+ for (RegionInfo region : regionInfoList) {
try {
// Get the favored nodes from the assignment plan and verify it.
List<ServerName> favoredNodes = plan.getFavoredNodes(region);
@@ -466,7 +466,7 @@ public class AssignmentVerificationReport {
System.out.println("\tTotal unassigned regions: " +
unAssignedRegionsList.size());
if (isDetailMode) {
- for (HRegionInfo region : unAssignedRegionsList) {
+ for (RegionInfo region : unAssignedRegionsList) {
System.out.println("\t\t" + region.getRegionNameAsString());
}
}
@@ -474,7 +474,7 @@ public class AssignmentVerificationReport {
System.out.println("\tTotal regions NOT on favored nodes: " +
nonFavoredAssignedRegionList.size());
if (isDetailMode) {
- for (HRegionInfo region : nonFavoredAssignedRegionList) {
+ for (RegionInfo region : nonFavoredAssignedRegionList) {
System.out.println("\t\t" + region.getRegionNameAsString());
}
}
@@ -482,7 +482,7 @@ public class AssignmentVerificationReport {
System.out.println("\tTotal regions without favored nodes: " +
regionsWithoutValidFavoredNodes.size());
if (isDetailMode) {
- for (HRegionInfo region : regionsWithoutValidFavoredNodes) {
+ for (RegionInfo region : regionsWithoutValidFavoredNodes) {
System.out.println("\t\t" + region.getRegionNameAsString());
}
}
@@ -572,7 +572,7 @@ public class AssignmentVerificationReport {
* Return the unassigned regions
* @return unassigned regions
*/
- List<HRegionInfo> getUnassignedRegions() {
+ List<RegionInfo> getUnassignedRegions() {
return unAssignedRegionsList;
}
@@ -580,7 +580,7 @@ public class AssignmentVerificationReport {
* Return the regions without favored nodes
* @return regions without favored nodes
*/
- List<HRegionInfo> getRegionsWithoutValidFavoredNodes() {
+ List<RegionInfo> getRegionsWithoutValidFavoredNodes() {
return regionsWithoutValidFavoredNodes;
}
@@ -588,10 +588,10 @@ public class AssignmentVerificationReport {
* Return the regions not assigned to its favored nodes
* @return regions not assigned to its favored nodes
*/
- List<HRegionInfo> getNonFavoredAssignedRegions() {
+ List<RegionInfo> getNonFavoredAssignedRegions() {
return nonFavoredAssignedRegionList;
}
-
+
/**
* Return the number of regions assigned to their favored nodes
* @return number of regions assigned to their favored nodes
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java
index 0b60cef..d3ba231 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java
@@ -32,13 +32,12 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.ScheduledChore;
import org.apache.hadoop.hbase.TableName;
-import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
@@ -53,8 +52,7 @@ import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.PairOfSameType;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.util.Triple;
-
-import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
+import org.apache.yetus.audience.InterfaceAudience;
/**
* A janitor for the catalog tables. Scans the <code>hbase:meta</code> catalog
@@ -137,7 +135,7 @@ public class CatalogJanitor extends ScheduledChore {
* parent regioninfos
* @throws IOException
*/
- Triple<Integer, Map<HRegionInfo, Result>, Map<HRegionInfo, Result>>
+ Triple<Integer, Map<RegionInfo, Result>, Map<RegionInfo, Result>>
getMergedRegionsAndSplitParents() throws IOException {
return getMergedRegionsAndSplitParents(null);
}
@@ -152,15 +150,15 @@ public class CatalogJanitor extends ScheduledChore {
* parent regioninfos
* @throws IOException
*/
- Triple<Integer, Map<HRegionInfo, Result>, Map<HRegionInfo, Result>>
+ Triple<Integer, Map<RegionInfo, Result>, Map<RegionInfo, Result>>
getMergedRegionsAndSplitParents(final TableName tableName) throws IOException {
final boolean isTableSpecified = (tableName != null);
// TODO: Only works with single hbase:meta region currently. Fix.
final AtomicInteger count = new AtomicInteger(0);
// Keep Map of found split parents. There are candidates for cleanup.
// Use a comparator that has split parents come before its daughters.
- final Map<HRegionInfo, Result> splitParents = new TreeMap<>(new SplitParentFirstComparator());
- final Map<HRegionInfo, Result> mergedRegions = new TreeMap<>();
+ final Map<RegionInfo, Result> splitParents = new TreeMap<>(new SplitParentFirstComparator());
+ final Map<RegionInfo, Result> mergedRegions = new TreeMap<>(RegionInfo.COMPARATOR);
// This visitor collects split parents and counts rows in the hbase:meta table
MetaTableAccessor.Visitor visitor = new MetaTableAccessor.Visitor() {
@@ -168,7 +166,7 @@ public class CatalogJanitor extends ScheduledChore {
public boolean visit(Result r) throws IOException {
if (r == null || r.isEmpty()) return true;
count.incrementAndGet();
- HRegionInfo info = MetaTableAccessor.getHRegionInfo(r);
+ RegionInfo info = MetaTableAccessor.getRegionInfo(r);
if (info == null) return true; // Keep scanning
if (isTableSpecified
&& info.getTable().compareTo(tableName) > 0) {
@@ -200,8 +198,8 @@ public class CatalogJanitor extends ScheduledChore {
* the files on the file system
* @throws IOException
*/
- boolean cleanMergeRegion(final HRegionInfo mergedRegion,
- final HRegionInfo regionA, final HRegionInfo regionB) throws IOException {
+ boolean cleanMergeRegion(final RegionInfo mergedRegion,
+ final RegionInfo regionA, final RegionInfo regionB) throws IOException {
FileSystem fs = this.services.getMasterFileSystem().getFileSystem();
Path rootdir = this.services.getMasterFileSystem().getRootDir();
Path tabledir = FSUtils.getTableDir(rootdir, mergedRegion.getTable());
@@ -244,21 +242,21 @@ public class CatalogJanitor extends ScheduledChore {
LOG.debug("CatalogJanitor already running");
return result;
}
- Triple<Integer, Map<HRegionInfo, Result>, Map<HRegionInfo, Result>> scanTriple =
+ Triple<Integer, Map<RegionInfo, Result>, Map<RegionInfo, Result>> scanTriple =
getMergedRegionsAndSplitParents();
/**
* clean merge regions first
*/
- Map<HRegionInfo, Result> mergedRegions = scanTriple.getSecond();
- for (Map.Entry<HRegionInfo, Result> e : mergedRegions.entrySet()) {
+ Map<RegionInfo, Result> mergedRegions = scanTriple.getSecond();
+ for (Map.Entry<RegionInfo, Result> e : mergedRegions.entrySet()) {
if (this.services.isInMaintenanceMode()) {
// Stop cleaning if the master is in maintenance mode
break;
}
- PairOfSameType<HRegionInfo> p = MetaTableAccessor.getMergeRegions(e.getValue());
- HRegionInfo regionA = p.getFirst();
- HRegionInfo regionB = p.getSecond();
+ PairOfSameType<RegionInfo> p = MetaTableAccessor.getMergeRegions(e.getValue());
+ RegionInfo regionA = p.getFirst();
+ RegionInfo regionB = p.getSecond();
if (regionA == null || regionB == null) {
LOG.warn("Unexpected references regionA="
+ (regionA == null ? "null" : regionA.getShortNameToLog())
@@ -274,12 +272,12 @@ public class CatalogJanitor extends ScheduledChore {
/**
* clean split parents
*/
- Map<HRegionInfo, Result> splitParents = scanTriple.getThird();
+ Map<RegionInfo, Result> splitParents = scanTriple.getThird();
// Now work on our list of found parents. See if any we can clean up.
// regions whose parents are still around
HashSet<String> parentNotCleaned = new HashSet<>();
- for (Map.Entry<HRegionInfo, Result> e : splitParents.entrySet()) {
+ for (Map.Entry<RegionInfo, Result> e : splitParents.entrySet()) {
if (this.services.isInMaintenanceMode()) {
// Stop cleaning if the master is in maintenance mode
break;
@@ -291,7 +289,7 @@ public class CatalogJanitor extends ScheduledChore {
} else {
// We could not clean the parent, so it's daughters should not be
// cleaned either (HBASE-6160)
- PairOfSameType<HRegionInfo> daughters =
+ PairOfSameType<RegionInfo> daughters =
MetaTableAccessor.getDaughterRegions(e.getValue());
parentNotCleaned.add(daughters.getFirst().getEncodedName());
parentNotCleaned.add(daughters.getSecond().getEncodedName());
@@ -307,11 +305,11 @@ public class CatalogJanitor extends ScheduledChore {
* Compare HRegionInfos in a way that has split parents sort BEFORE their
* daughters.
*/
- static class SplitParentFirstComparator implements Comparator<HRegionInfo> {
+ static class SplitParentFirstComparator implements Comparator<RegionInfo> {
Comparator<byte[]> rowEndKeyComparator = new Bytes.RowEndKeyComparator();
@Override
- public int compare(HRegionInfo left, HRegionInfo right) {
- // This comparator differs from the one HRegionInfo in that it sorts
+ public int compare(RegionInfo left, RegionInfo right) {
+ // This comparator differs from the one RegionInfo in that it sorts
// parent before daughters.
if (left == null) return -1;
if (right == null) return 1;
@@ -330,14 +328,14 @@ public class CatalogJanitor extends ScheduledChore {
/**
* If daughters no longer hold reference to the parents, delete the parent.
- * @param parent HRegionInfo of split offlined parent
+ * @param parent RegionInfo of split offlined parent
* @param rowContent Content of <code>parent</code> row in
* <code>metaRegionName</code>
* @return True if we removed <code>parent</code> from meta table and from
* the filesystem.
* @throws IOException
*/
- boolean cleanParent(final HRegionInfo parent, Result rowContent)
+ boolean cleanParent(final RegionInfo parent, Result rowContent)
throws IOException {
// Check whether it is a merged region and not clean reference
// No necessary to check MERGEB_QUALIFIER because these two qualifiers will
@@ -347,7 +345,7 @@ public class CatalogJanitor extends ScheduledChore {
return false;
}
// Run checks on each daughter split.
- PairOfSameType<HRegionInfo> daughters = MetaTableAccessor.getDaughterRegions(rowContent);
+ PairOfSameType<RegionInfo> daughters = MetaTableAccessor.getDaughterRegions(rowContent);
Pair<Boolean, Boolean> a = checkDaughterInFs(parent, daughters.getFirst());
Pair<Boolean, Boolean> b = checkDaughterInFs(parent, daughters.getSecond());
if (hasNoReferences(a) && hasNoReferences(b)) {
@@ -388,7 +386,7 @@ public class CatalogJanitor extends ScheduledChore {
* whether the daughter has references to the parent.
* @throws IOException
*/
- Pair<Boolean, Boolean> checkDaughterInFs(final HRegionInfo parent, final HRegionInfo daughter)
+ Pair<Boolean, Boolean> checkDaughterInFs(final RegionInfo parent, final RegionInfo daughter)
throws IOException {
if (daughter == null) {
return new Pair<>(Boolean.FALSE, Boolean.FALSE);
@@ -443,11 +441,11 @@ public class CatalogJanitor extends ScheduledChore {
* @return true if the specified region doesn't have merge qualifier now
* @throws IOException
*/
- public boolean cleanMergeQualifier(final HRegionInfo region)
+ public boolean cleanMergeQualifier(final RegionInfo region)
throws IOException {
// Get merge regions if it is a merged region and already has merge
// qualifier
- Pair<HRegionInfo, HRegionInfo> mergeRegions = MetaTableAccessor
+ Pair<RegionInfo, RegionInfo> mergeRegions = MetaTableAccessor
.getRegionsFromMergeQualifier(this.services.getConnection(),
region.getRegionName());
if (mergeRegions == null
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
index e675d7e..9111f94 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
@@ -18,6 +18,10 @@
*/
package org.apache.hadoop.hbase.master;
+import javax.servlet.ServletException;
+import javax.servlet.http.HttpServlet;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
import java.io.IOException;
import java.io.InterruptedIOException;
import java.lang.reflect.Constructor;
@@ -46,11 +50,6 @@ import java.util.concurrent.atomic.AtomicReference;
import java.util.function.Function;
import java.util.regex.Pattern;
-import javax.servlet.ServletException;
-import javax.servlet.http.HttpServlet;
-import javax.servlet.http.HttpServletRequest;
-import javax.servlet.http.HttpServletResponse;
-
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
@@ -64,7 +63,6 @@ import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HBaseIOException;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.MasterNotRunningException;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.NamespaceDescriptor;
@@ -76,10 +74,10 @@ import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotDisabledException;
import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.UnknownRegionException;
-import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
import org.apache.hadoop.hbase.client.MasterSwitchType;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
@@ -167,16 +165,6 @@ import org.apache.hadoop.hbase.replication.master.TableCFsUpdater;
import org.apache.hadoop.hbase.replication.regionserver.Replication;
import org.apache.hadoop.hbase.security.AccessDeniedException;
import org.apache.hadoop.hbase.security.UserProvider;
-import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.Maps;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse.CompactionState;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionServerInfo;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceViolationPolicy;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos;
import org.apache.hadoop.hbase.util.Addressing;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.CompressionTest;
@@ -200,12 +188,24 @@ import org.apache.hadoop.hbase.zookeeper.SplitOrMergeTracker;
import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.yetus.audience.InterfaceAudience;
import org.apache.zookeeper.KeeperException;
import org.eclipse.jetty.server.Server;
import org.eclipse.jetty.server.ServerConnector;
import org.eclipse.jetty.servlet.ServletHolder;
import org.eclipse.jetty.webapp.WebAppContext;
+import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.Maps;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse.CompactionState;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionServerInfo;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceViolationPolicy;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos;
+
import com.google.protobuf.Descriptors;
import com.google.protobuf.Service;
@@ -1416,7 +1416,7 @@ public class HMaster extends HRegionServer implements MasterServices {
}
}
- Map<TableName, Map<ServerName, List<HRegionInfo>>> assignmentsByTable =
+ Map<TableName, Map<ServerName, List<RegionInfo>>> assignmentsByTable =
this.assignmentManager.getRegionStates().getAssignmentsByTable();
List<RegionPlan> plans = new ArrayList<>();
@@ -1426,7 +1426,7 @@ public class HMaster extends HRegionServer implements MasterServices {
this.balancer.setClusterLoad(
this.assignmentManager.getRegionStates().getAssignmentsByTable());
- for (Entry<TableName, Map<ServerName, List<HRegionInfo>>> e : assignmentsByTable.entrySet()) {
+ for (Entry<TableName, Map<ServerName, List<RegionInfo>>> e : assignmentsByTable.entrySet()) {
List<RegionPlan> partialPlans = this.balancer.balanceCluster(e.getKey(), e.getValue());
if (partialPlans != null) plans.addAll(partialPlans);
}
@@ -1560,7 +1560,7 @@ public class HMaster extends HRegionServer implements MasterServices {
@Override
public long mergeRegions(
- final HRegionInfo[] regionsToMerge,
+ final RegionInfo[] regionsToMerge,
final boolean forcible,
final long nonceGroup,
final long nonce) throws IOException {
@@ -1579,7 +1579,7 @@ public class HMaster extends HRegionServer implements MasterServices {
+ " and " + regionsToMerge[1].getTable());
}
- if (regionsToMerge[0].compareTo(regionsToMerge[1]) == 0) {
+ if (RegionInfo.COMPARATOR.compare(regionsToMerge[0], regionsToMerge[1]) == 0) {
throw new MergeRegionException(
"Cannot merge a region to itself " + regionsToMerge[0] + ", " + regionsToMerge[1]);
}
@@ -1607,7 +1607,7 @@ public class HMaster extends HRegionServer implements MasterServices {
}
@Override
- public long splitRegion(final HRegionInfo regionInfo, final byte[] splitRow,
+ public long splitRegion(final RegionInfo regionInfo, final byte[] splitRow,
final long nonceGroup, final long nonce)
throws IOException {
checkInitialized();
@@ -1637,7 +1637,7 @@ public class HMaster extends HRegionServer implements MasterServices {
RegionState regionState = assignmentManager.getRegionStates().
getRegionState(Bytes.toString(encodedRegionName));
- HRegionInfo hri;
+ RegionInfo hri;
if (regionState != null) {
hri = regionState.getRegion();
} else {
@@ -1736,7 +1736,7 @@ public class HMaster extends HRegionServer implements MasterServices {
String namespace = tableDescriptor.getTableName().getNamespaceAsString();
this.clusterSchemaService.getNamespace(namespace);
- HRegionInfo[] newRegions = ModifyRegionUtils.createHRegionInfos(tableDescriptor, splitKeys);
+ RegionInfo[] newRegions = ModifyRegionUtils.createRegionInfos(tableDescriptor, splitKeys);
sanityCheckTableDescriptor(tableDescriptor);
return MasterProcedureUtil.submitProcedure(
@@ -1776,7 +1776,7 @@ public class HMaster extends HRegionServer implements MasterServices {
"Only system table creation can use this createSystemTable API");
}
- HRegionInfo[] newRegions = ModifyRegionUtils.createHRegionInfos(tableDescriptor, null);
+ RegionInfo[] newRegions = ModifyRegionUtils.createRegionInfos(tableDescriptor, null);
LOG.info(getClientIdAuditPrefix() + " create " + tableDescriptor);
@@ -2357,10 +2357,10 @@ public class HMaster extends HRegionServer implements MasterServices {
* may be null.
*/
@VisibleForTesting // Used by TestMaster.
- Pair<HRegionInfo, ServerName> getTableRegionForRow(
+ Pair<RegionInfo, ServerName> getTableRegionForRow(
final TableName tableName, final byte [] rowKey)
throws IOException {
- final AtomicReference<Pair<HRegionInfo, ServerName>> result = new AtomicReference<>(null);
+ final AtomicReference<Pair<RegionInfo, ServerName>> result = new AtomicReference<>(null);
MetaTableAccessor.Visitor visitor = new MetaTableAccessor.Visitor() {
@Override
@@ -2368,8 +2368,8 @@ public class HMaster extends HRegionServer implements MasterServices {
if (data == null || data.size() <= 0) {
return true;
}
- Pair<HRegionInfo, ServerName> pair =
- new Pair(MetaTableAccessor.getHRegionInfo(data),
+ Pair<RegionInfo, ServerName> pair =
+ new Pair(MetaTableAccessor.getRegionInfo(data),
MetaTableAccessor.getServerName(data,0));
if (pair == null) {
return false;
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/LoadBalancer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/LoadBalancer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/LoadBalancer.java
index 885258d..1dad70d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/LoadBalancer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/LoadBalancer.java
@@ -18,25 +18,21 @@
*/
package org.apache.hadoop.hbase.master;
-import java.util.*;
+import java.util.List;
+import java.util.Map;
-import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.UnmodifiableIterator;
import org.apache.hadoop.conf.Configurable;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.ClusterStatus;
import org.apache.hadoop.hbase.HBaseIOException;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.Stoppable;
import org.apache.hadoop.hbase.TableName;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.conf.ConfigurationObserver;
+import org.apache.yetus.audience.InterfaceAudience;
import edu.umd.cs.findbugs.annotations.Nullable;
-import org.apache.hadoop.hbase.security.access.AccessControlLists;
-import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
-import org.apache.hadoop.util.StringUtils;
/**
* Makes decisions about the placement and movement of Regions across
@@ -81,7 +77,7 @@ public interface LoadBalancer extends Configurable, Stoppable, ConfigurationObse
* Pass RegionStates and allow balancer to set the current cluster load.
* @param ClusterLoad
*/
- void setClusterLoad(Map<TableName, Map<ServerName, List<HRegionInfo>>> ClusterLoad);
+ void setClusterLoad(Map<TableName, Map<ServerName, List<RegionInfo>>> ClusterLoad);
/**
* Set the master service.
@@ -96,7 +92,7 @@ public interface LoadBalancer extends Configurable, Stoppable, ConfigurationObse
* @return List of plans
*/
List<RegionPlan> balanceCluster(TableName tableName, Map<ServerName,
- List<HRegionInfo>> clusterState) throws HBaseIOException;
+ List<RegionInfo>> clusterState) throws HBaseIOException;
/**
* Perform the major balance operation
@@ -104,7 +100,7 @@ public interface LoadBalancer extends Configurable, Stoppable, ConfigurationObse
* @return List of plans
*/
List<RegionPlan> balanceCluster(Map<ServerName,
- List<HRegionInfo>> clusterState) throws HBaseIOException;
+ List<RegionInfo>> clusterState) throws HBaseIOException;
/**
* Perform a Round Robin assignment of regions.
@@ -112,8 +108,8 @@ public interface LoadBalancer extends Configurable, Stoppable, ConfigurationObse
* @param servers
* @return Map of servername to regioninfos
*/
- Map<ServerName, List<HRegionInfo>> roundRobinAssignment(
- List<HRegionInfo> regions,
+ Map<ServerName, List<RegionInfo>> roundRobinAssignment(
+ List<RegionInfo> regions,
List<ServerName> servers
) throws HBaseIOException;
@@ -124,8 +120,8 @@ public interface LoadBalancer extends Configurable, Stoppable, ConfigurationObse
* @return List of plans
*/
@Nullable
- Map<ServerName, List<HRegionInfo>> retainAssignment(
- Map<HRegionInfo, ServerName> regions,
+ Map<ServerName, List<RegionInfo>> retainAssignment(
+ Map<RegionInfo, ServerName> regions,
List<ServerName> servers
) throws HBaseIOException;
@@ -136,7 +132,7 @@ public interface LoadBalancer extends Configurable, Stoppable, ConfigurationObse
* @return Servername
*/
ServerName randomAssignment(
- HRegionInfo regionInfo, List<ServerName> servers
+ RegionInfo regionInfo, List<ServerName> servers
) throws HBaseIOException;
/**
@@ -150,13 +146,13 @@ public interface LoadBalancer extends Configurable, Stoppable, ConfigurationObse
* @param regionInfo
* @param sn
*/
- void regionOnline(HRegionInfo regionInfo, ServerName sn);
+ void regionOnline(RegionInfo regionInfo, ServerName sn);
/**
* Marks the region as offline at balancer.
* @param regionInfo
*/
- void regionOffline(HRegionInfo regionInfo);
+ void regionOffline(RegionInfo regionInfo);
/*
* Notification that config has changed
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java
index 2e7231b..3b268cb 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java
@@ -29,12 +29,12 @@ import org.apache.hadoop.fs.permission.FsAction;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hbase.ClusterId;
import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.backup.HFileArchiver;
-import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
@@ -46,6 +46,7 @@ import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSTableDescriptors;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.ipc.RemoteException;
+import org.apache.yetus.audience.InterfaceAudience;
/**
* This class abstracts a bunch of operations the HMaster needs to interact with
@@ -388,9 +389,9 @@ public class MasterFileSystem {
// created here in bootstrap and it'll need to be cleaned up. Better to
// not make it in first place. Turn off block caching for bootstrap.
// Enable after.
- HRegionInfo metaHRI = new HRegionInfo(HRegionInfo.FIRST_META_REGIONINFO);
TableDescriptor metaDescriptor = new FSTableDescriptors(c).get(TableName.META_TABLE_NAME);
- HRegion meta = HRegion.createHRegion(metaHRI, rd, c, setInfoFamilyCachingForMeta(metaDescriptor, false), null);
+ HRegion meta = HRegion.createHRegion(RegionInfoBuilder.FIRST_META_REGIONINFO, rd,
+ c, setInfoFamilyCachingForMeta(metaDescriptor, false), null);
meta.close();
} catch (IOException e) {
e = e instanceof RemoteException ?
@@ -416,12 +417,12 @@ public class MasterFileSystem {
return builder.build();
}
- public void deleteFamilyFromFS(HRegionInfo region, byte[] familyName)
+ public void deleteFamilyFromFS(RegionInfo region, byte[] familyName)
throws IOException {
deleteFamilyFromFS(rootdir, region, familyName);
}
- public void deleteFamilyFromFS(Path rootDir, HRegionInfo region, byte[] familyName)
+ public void deleteFamilyFromFS(Path rootDir, RegionInfo region, byte[] familyName)
throws IOException {
// archive family store files
Path tableDir = FSUtils.getTableDir(rootDir, region.getTable());
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterMetaBootstrap.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterMetaBootstrap.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterMetaBootstrap.java
index d8f74f0..677a2a0 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterMetaBootstrap.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterMetaBootstrap.java
@@ -25,10 +25,10 @@ import java.util.Set;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
@@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.monitoring.MonitoredTask;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.yetus.audience.InterfaceAudience;
import org.apache.zookeeper.KeeperException;
/**
@@ -118,7 +119,7 @@ public class MasterMetaBootstrap {
// TODO: Unimplemented
// long timeout =
// master.getConfiguration().getLong("hbase.catalog.verification.timeout", 1000);
- if (replicaId == HRegionInfo.DEFAULT_REPLICA_ID) {
+ if (replicaId == RegionInfo.DEFAULT_REPLICA_ID) {
status.setStatus("Assigning hbase:meta region");
} else {
status.setStatus("Assigning hbase:meta region, replicaId " + replicaId);
@@ -127,11 +128,11 @@ public class MasterMetaBootstrap {
// Get current meta state from zk.
RegionState metaState = MetaTableLocator.getMetaRegionState(master.getZooKeeper(), replicaId);
LOG.debug("meta state from zookeeper: " + metaState);
- HRegionInfo hri = RegionReplicaUtil.getRegionInfoForReplica(
- HRegionInfo.FIRST_META_REGIONINFO, replicaId);
+ RegionInfo hri = RegionReplicaUtil.getRegionInfoForReplica(
+ RegionInfoBuilder.FIRST_META_REGIONINFO, replicaId);
assignmentManager.assignMeta(hri, metaState.getServerName());
- if (replicaId == HRegionInfo.DEFAULT_REPLICA_ID) {
+ if (replicaId == RegionInfo.DEFAULT_REPLICA_ID) {
// TODO: should we prevent from using state manager before meta was initialized?
// tableStateManager.start();
master.getTableStateManager()
@@ -144,7 +145,7 @@ public class MasterMetaBootstrap {
// if the meta region server is died at this time, we need it to be re-assigned
// by SSH so that system tables can be assigned.
// No need to wait for meta is assigned = 0 when meta is just verified.
- if (replicaId == HRegionInfo.DEFAULT_REPLICA_ID) enableCrashedServerProcessing(false);
+ if (replicaId == RegionInfo.DEFAULT_REPLICA_ID) enableCrashedServerProcessing(false);
LOG.info("hbase:meta with replicaId " + replicaId + ", location="
+ master.getMetaTableLocator().getMetaRegionLocation(master.getZooKeeper(), replicaId));
status.setStatus("META assigned.");
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
index 4ee020a..740edec 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
@@ -32,16 +32,16 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.ServerLoad;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.UnknownRegionException;
-import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
import org.apache.hadoop.hbase.client.MasterSwitchType;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.client.VersionInfoUtil;
@@ -74,6 +74,15 @@ import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.security.access.AccessController;
import org.apache.hadoop.hbase.security.visibility.VisibilityController;
+import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
+import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.ForeignExceptionUtil;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.zookeeper.KeeperException;
+
import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
@@ -262,13 +271,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.Remov
import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
-import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
-import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.hadoop.hbase.util.ForeignExceptionUtil;
-import org.apache.hadoop.hbase.util.Pair;
-import org.apache.zookeeper.KeeperException;
/**
* Implements the master RPC services.
@@ -501,7 +503,7 @@ public class MasterRpcServices extends RSRpcServices
}
final byte[] regionName = req.getRegion().getValue().toByteArray();
- final HRegionInfo regionInfo = master.getAssignmentManager().getRegionInfo(regionName);
+ final RegionInfo regionInfo = master.getAssignmentManager().getRegionInfo(regionName);
if (regionInfo == null) throw new UnknownRegionException(Bytes.toStringBinary(regionName));
final AssignRegionResponse arr = AssignRegionResponse.newBuilder().build();
@@ -712,7 +714,7 @@ public class MasterRpcServices extends RSRpcServices
RegionStates regionStates = master.getAssignmentManager().getRegionStates();
assert(request.getRegionCount() == 2);
- HRegionInfo[] regionsToMerge = new HRegionInfo[request.getRegionCount()];
+ RegionInfo[] regionsToMerge = new RegionInfo[request.getRegionCount()];
for (int i = 0; i < request.getRegionCount(); i++) {
final byte[] encodedNameOfRegion = request.getRegion(i).getValue().toByteArray();
if (request.getRegion(i).getType() != RegionSpecifierType.ENCODED_REGION_NAME) {
@@ -745,7 +747,7 @@ public class MasterRpcServices extends RSRpcServices
final SplitTableRegionRequest request) throws ServiceException {
try {
long procId = master.splitRegion(
- HRegionInfo.convert(request.getRegionInfo()),
+ ProtobufUtil.toRegionInfo(request.getRegionInfo()),
request.hasSplitRow() ? request.getSplitRow().toByteArray() : null,
request.getNonceGroup(),
request.getNonce());
@@ -1344,7 +1346,7 @@ public class MasterRpcServices extends RSRpcServices
}
final byte[] regionName = request.getRegion().getValue().toByteArray();
- final HRegionInfo hri = master.getAssignmentManager().getRegionInfo(regionName);
+ final RegionInfo hri = master.getAssignmentManager().getRegionInfo(regionName);
if (hri == null) throw new UnknownRegionException(Bytes.toStringBinary(regionName));
if (master.cpHost != null) {
@@ -1502,17 +1504,17 @@ public class MasterRpcServices extends RSRpcServices
LOG.warn("unassignRegion specifier type: expected: " + RegionSpecifierType.REGION_NAME
+ " actual: " + type);
}
- Pair<HRegionInfo, ServerName> pair =
+ Pair<RegionInfo, ServerName> pair =
MetaTableAccessor.getRegion(master.getConnection(), regionName);
- if (Bytes.equals(HRegionInfo.FIRST_META_REGIONINFO.getRegionName(),regionName)) {
- pair = new Pair<>(HRegionInfo.FIRST_META_REGIONINFO,
+ if (Bytes.equals(RegionInfoBuilder.FIRST_META_REGIONINFO.getRegionName(),regionName)) {
+ pair = new Pair<>(RegionInfoBuilder.FIRST_META_REGIONINFO,
master.getMetaTableLocator().getMetaRegionLocation(master.getZooKeeper()));
}
if (pair == null) {
throw new UnknownRegionException(Bytes.toString(regionName));
}
- HRegionInfo hri = pair.getFirst();
+ RegionInfo hri = pair.getFirst();
if (master.cpHost != null) {
if (master.cpHost.preUnassign(hri, force)) {
return urr;
@@ -1598,7 +1600,7 @@ public class MasterRpcServices extends RSRpcServices
try {
master.checkInitialized();
byte[] regionName = request.getRegion().getValue().toByteArray();
- TableName tableName = HRegionInfo.getTable(regionName);
+ TableName tableName = RegionInfo.getTable(regionName);
// if the region is a mob region, do the mob file compaction.
if (MobUtils.isMobRegionName(tableName, regionName)) {
return compactMob(request, tableName);
@@ -1615,12 +1617,12 @@ public class MasterRpcServices extends RSRpcServices
public GetRegionInfoResponse getRegionInfo(final RpcController controller,
final GetRegionInfoRequest request) throws ServiceException {
byte[] regionName = request.getRegion().getValue().toByteArray();
- TableName tableName = HRegionInfo.getTable(regionName);
+ TableName tableName = RegionInfo.getTable(regionName);
if (MobUtils.isMobRegionName(tableName, regionName)) {
// a dummy region info contains the compaction state.
- HRegionInfo mobRegionInfo = MobUtils.getMobRegionInfo(tableName);
+ RegionInfo mobRegionInfo = MobUtils.getMobRegionInfo(tableName);
GetRegionInfoResponse.Builder builder = GetRegionInfoResponse.newBuilder();
- builder.setRegionInfo(HRegionInfo.convert(mobRegionInfo));
+ builder.setRegionInfo(ProtobufUtil.toRegionInfo(mobRegionInfo));
if (request.hasCompactionState() && request.getCompactionState()) {
builder.setCompactionState(master.getMobCompactionState(tableName));
}
@@ -1960,9 +1962,9 @@ public class MasterRpcServices extends RSRpcServices
NonceProcedureRunnable npr;
LockType type = LockType.valueOf(request.getLockType().name());
if (request.getRegionInfoCount() > 0) {
- final HRegionInfo[] regionInfos = new HRegionInfo[request.getRegionInfoCount()];
+ final RegionInfo[] regionInfos = new RegionInfo[request.getRegionInfoCount()];
for (int i = 0; i < request.getRegionInfoCount(); ++i) {
- regionInfos[i] = HRegionInfo.convert(request.getRegionInfo(i));
+ regionInfos[i] = ProtobufUtil.toRegionInfo(request.getRegionInfo(i));
}
npr = new NonceProcedureRunnable(master, request.getNonceGroup(), request.getNonce()) {
@Override
@@ -2051,7 +2053,7 @@ public class MasterRpcServices extends RSRpcServices
MasterQuotaManager quotaManager = this.master.getMasterQuotaManager();
final long now = EnvironmentEdgeManager.currentTime();
for (RegionSpaceUse report : request.getSpaceUseList()) {
- quotaManager.addRegionSize(HRegionInfo.convert(
+ quotaManager.addRegionSize(ProtobufUtil.toRegionInfo(
report.getRegionInfo()), report.getRegionSize(), now);
}
return RegionSpaceUseReportResponse.newBuilder().build();
@@ -2069,10 +2071,10 @@ public class MasterRpcServices extends RSRpcServices
GetSpaceQuotaRegionSizesResponse.Builder builder =
GetSpaceQuotaRegionSizesResponse.newBuilder();
if (quotaManager != null) {
- Map<HRegionInfo,Long> regionSizes = quotaManager.snapshotRegionSizes();
+ Map<RegionInfo,Long> regionSizes = quotaManager.snapshotRegionSizes();
Map<TableName,Long> regionSizesByTable = new HashMap<>();
// Translate hregioninfo+long -> tablename+long
- for (Entry<HRegionInfo,Long> entry : regionSizes.entrySet()) {
+ for (Entry<RegionInfo,Long> entry : regionSizes.entrySet()) {
final TableName tableName = entry.getKey().getTable();
Long prevSize = regionSizesByTable.get(tableName);
if (prevSize == null) {
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
index 80756be..e815950 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
@@ -21,21 +21,20 @@ package org.apache.hadoop.hbase.master;
import java.io.IOException;
import java.util.List;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableDescriptors;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotDisabledException;
import org.apache.hadoop.hbase.TableNotFoundException;
-import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
import org.apache.hadoop.hbase.client.MasterSwitchType;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.executor.ExecutorService;
+import org.apache.hadoop.hbase.favored.FavoredNodesManager;
import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
import org.apache.hadoop.hbase.master.locking.LockManager;
-import org.apache.hadoop.hbase.favored.FavoredNodesManager;
import org.apache.hadoop.hbase.master.normalizer.RegionNormalizer;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
@@ -48,8 +47,10 @@ import org.apache.hadoop.hbase.quotas.MasterQuotaManager;
import org.apache.hadoop.hbase.replication.ReplicationException;
import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
+import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
+
import com.google.protobuf.Service;
/**
@@ -291,7 +292,7 @@ public interface MasterServices extends Server {
* @throws IOException
*/
long mergeRegions(
- final HRegionInfo[] regionsToMerge,
+ final RegionInfo[] regionsToMerge,
final boolean forcible,
final long nonceGroup,
final long nonce) throws IOException;
@@ -306,7 +307,7 @@ public interface MasterServices extends Server {
* @throws IOException
*/
long splitRegion(
- final HRegionInfo regionInfo,
+ final RegionInfo regionInfo,
final byte [] splitRow,
final long nonceGroup,
final long nonce) throws IOException;
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterWalManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterWalManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterWalManager.java
index 91433dd..f6074d9 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterWalManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterWalManager.java
@@ -36,13 +36,13 @@ import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.PathFilter;
import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
import org.apache.hadoop.hbase.wal.WALSplitter;
+import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
@@ -284,7 +284,7 @@ public class MasterWalManager {
* @param serverName Failed region server whose wals to be replayed
* @param regions Set of regions to be recovered
*/
- public void prepareLogReplay(ServerName serverName, Set<HRegionInfo> regions) throws IOException {
+ public void prepareLogReplay(ServerName serverName, Set<RegionInfo> regions) throws IOException {
if (!this.distributedLogReplay) {
return;
}
[07/19] hbase git commit: HBASE-18839 Apply RegionInfo to code base
Posted by ch...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/resources/hbase-webapps/master/table.jsp
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/resources/hbase-webapps/master/table.jsp b/hbase-server/src/main/resources/hbase-webapps/master/table.jsp
index 6c401a9..c7a1aff 100644
--- a/hbase-server/src/main/resources/hbase-webapps/master/table.jsp
+++ b/hbase-server/src/main/resources/hbase-webapps/master/table.jsp
@@ -17,54 +17,57 @@
* limitations under the License.
*/
--%>
-<%@page import="org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType"%>
+<%@page import="java.net.URLEncoder"%>
<%@ page contentType="text/html;charset=UTF-8"
import="static org.apache.commons.lang3.StringEscapeUtils.escapeXml"
- import="org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString"
- import="java.net.URLEncoder"
import="java.util.ArrayList"
- import="java.util.TreeMap"
- import="java.util.List"
- import="java.util.LinkedHashMap"
- import="java.util.Map"
- import="java.util.Set"
import="java.util.Collection"
import="java.util.Collections"
import="java.util.Comparator"
+ import="java.util.LinkedHashMap"
+ import="java.util.List"
+ import="java.util.Map"
+ import="java.util.TreeMap"
import="org.apache.commons.lang3.StringEscapeUtils"
import="org.apache.hadoop.conf.Configuration"
- import="org.apache.hadoop.util.StringUtils"
- import="org.apache.hadoop.hbase.HRegionInfo"
+ import="org.apache.hadoop.hbase.HBaseConfiguration"
+ import="org.apache.hadoop.hbase.HColumnDescriptor"
+ import="org.apache.hadoop.hbase.HConstants"
import="org.apache.hadoop.hbase.HRegionLocation"
- import="org.apache.hadoop.hbase.ServerName"
- import="org.apache.hadoop.hbase.ServerLoad"
import="org.apache.hadoop.hbase.RegionLoad"
- import="org.apache.hadoop.hbase.HConstants"
+ import="org.apache.hadoop.hbase.ServerLoad"
+ import="org.apache.hadoop.hbase.ServerName"
+ import="org.apache.hadoop.hbase.TableName"
+ import="org.apache.hadoop.hbase.TableNotFoundException"
+ import="org.apache.hadoop.hbase.client.Admin"
+ import="org.apache.hadoop.hbase.client.CompactionState"
+ import="org.apache.hadoop.hbase.client.RegionInfo"
+ import="org.apache.hadoop.hbase.client.RegionInfoBuilder"
+ import="org.apache.hadoop.hbase.client.RegionLocator"
+ import="org.apache.hadoop.hbase.client.RegionReplicaUtil"
+ import="org.apache.hadoop.hbase.client.Table"
import="org.apache.hadoop.hbase.master.HMaster"
- import="org.apache.hadoop.hbase.zookeeper.MetaTableLocator"
import="org.apache.hadoop.hbase.quotas.QuotaTableUtil"
import="org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot"
import="org.apache.hadoop.hbase.util.Bytes"
import="org.apache.hadoop.hbase.util.FSUtils"
- import="org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos"
- import="org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos"
- import="org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas"
- import="org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota"
- import="org.apache.hadoop.hbase.TableName"
- import="org.apache.hadoop.hbase.HColumnDescriptor"
- import="org.apache.hadoop.hbase.HBaseConfiguration"
- import="org.apache.hadoop.hbase.TableNotFoundException"%>
-<%@ page import="org.apache.hadoop.hbase.client.*" %>
+ import="org.apache.hadoop.hbase.zookeeper.MetaTableLocator"
+ import="org.apache.hadoop.util.StringUtils"
+ import="org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString"%>
+<%@ page import="org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos" %>
+<%@ page import="org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos" %>
+<%@ page import="org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas" %>
+<%@ page import="org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota" %>
<%!
/**
- * @return An empty region load stamped with the passed in <code>hri</code>
+ * @return An empty region load stamped with the passed in <code>regionInfo</code>
* region name.
*/
- private RegionLoad getEmptyRegionLoad(final HRegionInfo hri) {
+ private RegionLoad getEmptyRegionLoad(final RegionInfo regionInfo) {
return new RegionLoad(ClusterStatusProtos.RegionLoad.newBuilder().
setRegionSpecifier(HBaseProtos.RegionSpecifier.newBuilder().
setType(HBaseProtos.RegionSpecifier.RegionSpecifierType.REGION_NAME).
- setValue(ByteString.copyFrom(hri.getRegionName())).build()).build());
+ setValue(ByteString.copyFrom(regionInfo.getRegionName())).build()).build());
}
%>
<%
@@ -205,10 +208,10 @@ if ( fqtn != null ) {
%> Split request accepted. <%
} else if (action.equals("compact")) {
if (key != null && key.length() > 0) {
- List<HRegionInfo> regions = admin.getTableRegions(TableName.valueOf(fqtn));
+ List<RegionInfo> regions = admin.getRegions(TableName.valueOf(fqtn));
byte[] row = Bytes.toBytes(key);
- for (HRegionInfo region : regions) {
+ for (RegionInfo region : regions) {
if (region.containsRow(row)) {
admin.compactRegion(region.getRegionName());
}
@@ -244,8 +247,8 @@ if ( fqtn != null ) {
<%
// NOTE: Presumes meta with one or more replicas
for (int j = 0; j < numMetaReplicas; j++) {
- HRegionInfo meta = RegionReplicaUtil.getRegionInfoForReplica(
- HRegionInfo.FIRST_META_REGIONINFO, j);
+ RegionInfo meta = RegionReplicaUtil.getRegionInfoForReplica(
+ RegionInfoBuilder.FIRST_META_REGIONINFO, j);
ServerName metaLocation = metaTableLocator.waitMetaRegionLocation(master.getZooKeeper(), j, 1);
for (int i = 0; i < 1; i++) {
String url = "";
@@ -433,10 +436,10 @@ if ( fqtn != null ) {
Map<ServerName, Integer> regDistribution = new TreeMap<>();
Map<ServerName, Integer> primaryRegDistribution = new TreeMap<>();
List<HRegionLocation> regions = r.getAllRegionLocations();
- Map<HRegionInfo, RegionLoad> regionsToLoad = new LinkedHashMap<>();
- Map<HRegionInfo, ServerName> regionsToServer = new LinkedHashMap<>();
+ Map<RegionInfo, RegionLoad> regionsToLoad = new LinkedHashMap<>();
+ Map<RegionInfo, ServerName> regionsToServer = new LinkedHashMap<>();
for (HRegionLocation hriEntry : regions) {
- HRegionInfo regionInfo = hriEntry.getRegionInfo();
+ RegionInfo regionInfo = hriEntry.getRegionInfo();
ServerName addr = hriEntry.getServerName();
regionsToServer.put(regionInfo, addr);
@@ -506,14 +509,14 @@ ShowDetailName&Start/End Key<input type="checkbox" id="showWhole" style="margin-
</tr>
<%
- List<Map.Entry<HRegionInfo, RegionLoad>> entryList = new ArrayList<>(regionsToLoad.entrySet());
+ List<Map.Entry<RegionInfo, RegionLoad>> entryList = new ArrayList<>(regionsToLoad.entrySet());
if(sortKey != null) {
if (sortKey.equals("readrequest")) {
Collections.sort(entryList,
- new Comparator<Map.Entry<HRegionInfo, RegionLoad>>() {
+ new Comparator<Map.Entry<RegionInfo, RegionLoad>>() {
public int compare(
- Map.Entry<HRegionInfo, RegionLoad> entry1,
- Map.Entry<HRegionInfo, RegionLoad> entry2) {
+ Map.Entry<RegionInfo, RegionLoad> entry1,
+ Map.Entry<RegionInfo, RegionLoad> entry2) {
if (entry1 == null || entry1.getValue() == null) {
return -1;
} else if (entry2 == null || entry2.getValue() == null) {
@@ -533,10 +536,10 @@ ShowDetailName&Start/End Key<input type="checkbox" id="showWhole" style="margin-
});
} else if (sortKey.equals("writerequest")) {
Collections.sort(entryList,
- new Comparator<Map.Entry<HRegionInfo, RegionLoad>>() {
+ new Comparator<Map.Entry<RegionInfo, RegionLoad>>() {
public int compare(
- Map.Entry<HRegionInfo, RegionLoad> entry1,
- Map.Entry<HRegionInfo, RegionLoad> entry2) {
+ Map.Entry<RegionInfo, RegionLoad> entry1,
+ Map.Entry<RegionInfo, RegionLoad> entry2) {
if (entry1 == null || entry1.getValue() == null) {
return -1;
} else if (entry2 == null || entry2.getValue() == null) {
@@ -558,10 +561,10 @@ ShowDetailName&Start/End Key<input type="checkbox" id="showWhole" style="margin-
});
} else if (sortKey.equals("size")) {
Collections.sort(entryList,
- new Comparator<Map.Entry<HRegionInfo, RegionLoad>>() {
+ new Comparator<Map.Entry<RegionInfo, RegionLoad>>() {
public int compare(
- Map.Entry<HRegionInfo, RegionLoad> entry1,
- Map.Entry<HRegionInfo, RegionLoad> entry2) {
+ Map.Entry<RegionInfo, RegionLoad> entry1,
+ Map.Entry<RegionInfo, RegionLoad> entry2) {
if (entry1 == null || entry1.getValue() == null) {
return -1;
} else if (entry2 == null || entry2.getValue() == null) {
@@ -583,10 +586,10 @@ ShowDetailName&Start/End Key<input type="checkbox" id="showWhole" style="margin-
});
} else if (sortKey.equals("filecount")) {
Collections.sort(entryList,
- new Comparator<Map.Entry<HRegionInfo, RegionLoad>>() {
+ new Comparator<Map.Entry<RegionInfo, RegionLoad>>() {
public int compare(
- Map.Entry<HRegionInfo, RegionLoad> entry1,
- Map.Entry<HRegionInfo, RegionLoad> entry2) {
+ Map.Entry<RegionInfo, RegionLoad> entry1,
+ Map.Entry<RegionInfo, RegionLoad> entry2) {
if (entry1 == null || entry1.getValue() == null) {
return -1;
} else if (entry2 == null || entry2.getValue() == null) {
@@ -608,10 +611,10 @@ ShowDetailName&Start/End Key<input type="checkbox" id="showWhole" style="margin-
});
} else if (sortKey.equals("memstore")) {
Collections.sort(entryList,
- new Comparator<Map.Entry<HRegionInfo, RegionLoad>>() {
+ new Comparator<Map.Entry<RegionInfo, RegionLoad>>() {
public int compare(
- Map.Entry<HRegionInfo, RegionLoad> entry1,
- Map.Entry<HRegionInfo, RegionLoad> entry2) {
+ Map.Entry<RegionInfo, RegionLoad> entry1,
+ Map.Entry<RegionInfo, RegionLoad> entry2) {
if (entry1 == null || entry1.getValue()==null) {
return -1;
} else if (entry2 == null || entry2.getValue()==null) {
@@ -633,10 +636,10 @@ ShowDetailName&Start/End Key<input type="checkbox" id="showWhole" style="margin-
});
} else if (sortKey.equals("locality")) {
Collections.sort(entryList,
- new Comparator<Map.Entry<HRegionInfo, RegionLoad>>() {
+ new Comparator<Map.Entry<RegionInfo, RegionLoad>>() {
public int compare(
- Map.Entry<HRegionInfo, RegionLoad> entry1,
- Map.Entry<HRegionInfo, RegionLoad> entry2) {
+ Map.Entry<RegionInfo, RegionLoad> entry1,
+ Map.Entry<RegionInfo, RegionLoad> entry2) {
if (entry1 == null || entry1.getValue()==null) {
return -1;
} else if (entry2 == null || entry2.getValue()==null) {
@@ -664,8 +667,8 @@ ShowDetailName&Start/End Key<input type="checkbox" id="showWhole" style="margin-
if (numRegionsToRender < 0) {
numRegionsToRender = numRegions;
}
- for (Map.Entry<HRegionInfo, RegionLoad> hriEntry : entryList) {
- HRegionInfo regionInfo = hriEntry.getKey();
+ for (Map.Entry<RegionInfo, RegionLoad> hriEntry : entryList) {
+ RegionInfo regionInfo = hriEntry.getKey();
ServerName addr = regionsToServer.get(regionInfo);
RegionLoad load = hriEntry.getValue();
String readReq = "N/A";
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/resources/hbase-webapps/regionserver/region.jsp
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/resources/hbase-webapps/regionserver/region.jsp b/hbase-server/src/main/resources/hbase-webapps/regionserver/region.jsp
index 097b8a9..62c589b 100644
--- a/hbase-server/src/main/resources/hbase-webapps/regionserver/region.jsp
+++ b/hbase-server/src/main/resources/hbase-webapps/regionserver/region.jsp
@@ -23,21 +23,20 @@
import="java.util.List"
import="static org.apache.commons.lang3.StringEscapeUtils.escapeXml"
import="org.apache.hadoop.conf.Configuration"
- import="org.apache.hadoop.hbase.HTableDescriptor"
- import="org.apache.hadoop.hbase.HColumnDescriptor"
import="org.apache.hadoop.hbase.HBaseConfiguration"
- import="org.apache.hadoop.hbase.HRegionInfo"
+ import="org.apache.hadoop.hbase.client.RegionInfoDisplay"
import="org.apache.hadoop.hbase.regionserver.HRegionServer"
import="org.apache.hadoop.hbase.regionserver.Region"
import="org.apache.hadoop.hbase.regionserver.Store"
- import="org.apache.hadoop.hbase.regionserver.StoreFile"%>
+ import="org.apache.hadoop.hbase.regionserver.StoreFile"
+%>
<%
String regionName = request.getParameter("name");
HRegionServer rs = (HRegionServer) getServletContext().getAttribute(HRegionServer.REGIONSERVER);
Configuration conf = rs.getConfiguration();
Region region = rs.getRegion(regionName);
- String displayName = HRegionInfo.getRegionNameAsStringForDisplay(region.getRegionInfo(),
+ String displayName = RegionInfoDisplay.getRegionNameAsStringForDisplay(region.getRegionInfo(),
rs.getConfiguration());
%>
<!DOCTYPE html>
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
index 6e30246..59ad6de 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
@@ -49,6 +49,7 @@ import java.util.TreeSet;
import java.util.UUID;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Collectors;
import edu.umd.cs.findbugs.annotations.Nullable;
import org.apache.commons.io.FileUtils;
@@ -63,6 +64,9 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.ClusterStatus.Option;
import org.apache.hadoop.hbase.Waiter.ExplainingPredicate;
import org.apache.hadoop.hbase.Waiter.Predicate;
+import org.apache.hadoop.hbase.client.ImmutableHRegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.BufferedMutator;
@@ -470,7 +474,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
/**
* @return META table descriptor
* @deprecated since 2.0 version and will be removed in 3.0 version.
- * use {@link #getMetaDescriptor()}
+ * use {@link #getMetaTableDescriptorBuilder()}
*/
@Deprecated
public HTableDescriptor getMetaTableDescriptor() {
@@ -1468,22 +1472,6 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
* @param c Configuration to use
* @return A Table instance for the created table.
* @throws IOException
- * @deprecated since 2.0 version and will be removed in 3.0 version.
- * use {@link #createTable(TableDescriptor, byte[][], Configuration)}
- */
- @Deprecated
- public Table createTable(HTableDescriptor htd, byte[][] families, Configuration c)
- throws IOException {
- return createTable((TableDescriptor) htd, families, c);
- }
-
- /**
- * Create a table.
- * @param htd
- * @param families
- * @param c Configuration to use
- * @return A Table instance for the created table.
- * @throws IOException
*/
public Table createTable(TableDescriptor htd, byte[][] families, Configuration c)
throws IOException {
@@ -1498,23 +1486,6 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
* @param c Configuration to use
* @return A Table instance for the created table.
* @throws IOException
- * @deprecated since 2.0 version and will be removed in 3.0 version.
- * use {@link #createTable(TableDescriptor, byte[][], byte[][], Configuration)}
- */
- @Deprecated
- public Table createTable(HTableDescriptor htd, byte[][] families, byte[][] splitKeys,
- Configuration c) throws IOException {
- return createTable((TableDescriptor) htd, families, splitKeys, c);
- }
-
- /**
- * Create a table.
- * @param htd
- * @param families
- * @param splitKeys
- * @param c Configuration to use
- * @return A Table instance for the created table.
- * @throws IOException
*/
public Table createTable(TableDescriptor htd, byte[][] families, byte[][] splitKeys,
Configuration c) throws IOException {
@@ -1541,21 +1512,6 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
* @param splitRows
* @return A Table instance for the created table.
* @throws IOException
- * @deprecated since 2.0 version and will be removed in 3.0 version.
- * use {@link #createTable(TableDescriptor, byte[][])}
- */
- @Deprecated
- public Table createTable(HTableDescriptor htd, byte[][] splitRows)
- throws IOException {
- return createTable((TableDescriptor) htd, splitRows);
- }
-
- /**
- * Create a table.
- * @param htd
- * @param splitRows
- * @return A Table instance for the created table.
- * @throws IOException
*/
public Table createTable(TableDescriptor htd, byte[][] splitRows)
throws IOException {
@@ -1745,17 +1701,6 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
/**
* Modify a table, synchronous. Waiting logic similar to that of {@code admin.rb#alter_status}.
- * @deprecated since 2.0 version and will be removed in 3.0 version.
- * use {@link #modifyTableSync(Admin, TableDescriptor)}
- */
- @Deprecated
- @SuppressWarnings("serial")
- public static void modifyTableSync(Admin admin, HTableDescriptor desc)
- throws IOException, InterruptedException {
- modifyTableSync(admin, (TableDescriptor) desc);
- }
- /**
- * Modify a table, synchronous. Waiting logic similar to that of {@code admin.rb#alter_status}.
*/
@SuppressWarnings("serial")
public static void modifyTableSync(Admin admin, TableDescriptor desc)
@@ -1900,22 +1845,6 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
* @param endKey
* @return
* @throws IOException
- * @deprecated since 2.0 version and will be removed in 3.0 version.
- * use {@link #createLocalHRegion(TableDescriptor, byte[], byte[])}
- */
- @Deprecated
- public HRegion createLocalHRegion(HTableDescriptor desc, byte [] startKey,
- byte [] endKey) throws IOException {
- return createLocalHRegion((TableDescriptor) desc, startKey, endKey);
- }
-
- /**
- * Create an HRegion that writes to the local tmp dirs
- * @param desc
- * @param startKey
- * @param endKey
- * @return
- * @throws IOException
*/
public HRegion createLocalHRegion(TableDescriptor desc, byte [] startKey,
byte [] endKey)
@@ -1927,19 +1856,8 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
/**
* Create an HRegion that writes to the local tmp dirs. Creates the WAL for you. Be sure to call
* {@link HBaseTestingUtility#closeRegionAndWAL(HRegion)} when you're finished with it.
- * @deprecated since 2.0 version and will be removed in 3.0 version.
- * use {@link #createLocalHRegion(HRegionInfo, TableDescriptor)}
*/
- @Deprecated
- public HRegion createLocalHRegion(HRegionInfo info, HTableDescriptor desc) throws IOException {
- return createLocalHRegion(info, (TableDescriptor) desc);
- }
-
- /**
- * Create an HRegion that writes to the local tmp dirs. Creates the WAL for you. Be sure to call
- * {@link HBaseTestingUtility#closeRegionAndWAL(HRegion)} when you're finished with it.
- */
- public HRegion createLocalHRegion(HRegionInfo info, TableDescriptor desc) throws IOException {
+ public HRegion createLocalHRegion(RegionInfo info, TableDescriptor desc) throws IOException {
return createRegionAndWAL(info, getDataTestDir(), getConfiguration(), desc);
}
@@ -1950,24 +1868,8 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
* @param wal wal for this region.
* @return created hregion
* @throws IOException
- * @deprecated since 2.0 version and will be removed in 3.0 version.
- * use {@link #createLocalHRegion(HRegionInfo, TableDescriptor, WAL)}
- */
- @Deprecated
- public HRegion createLocalHRegion(HRegionInfo info, HTableDescriptor desc, WAL wal)
- throws IOException {
- return createLocalHRegion(info, (TableDescriptor) desc, wal);
- }
-
- /**
- * Create an HRegion that writes to the local tmp dirs with specified wal
- * @param info regioninfo
- * @param desc table descriptor
- * @param wal wal for this region.
- * @return created hregion
- * @throws IOException
*/
- public HRegion createLocalHRegion(HRegionInfo info, TableDescriptor desc, WAL wal)
+ public HRegion createLocalHRegion(RegionInfo info, TableDescriptor desc, WAL wal)
throws IOException {
return HRegion.createHRegion(info, getDataTestDir(), getConfiguration(), desc, wal);
}
@@ -2482,7 +2384,8 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
@Deprecated
public List<HRegionInfo> createMultiRegionsInMeta(final Configuration conf,
final HTableDescriptor htd, byte [][] startKeys) throws IOException {
- return createMultiRegionsInMeta(conf, (TableDescriptor) htd, startKeys);
+ return createMultiRegionsInMeta(conf, (TableDescriptor) htd, startKeys)
+ .stream().map(ImmutableHRegionInfo::new).collect(Collectors.toList());
}
/**
* Create rows in hbase:meta for regions of the specified table with the specified
@@ -2494,19 +2397,21 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
* @return list of region info for regions added to meta
* @throws IOException
*/
- public List<HRegionInfo> createMultiRegionsInMeta(final Configuration conf,
+ public List<RegionInfo> createMultiRegionsInMeta(final Configuration conf,
final TableDescriptor htd, byte [][] startKeys)
throws IOException {
Table meta = getConnection().getTable(TableName.META_TABLE_NAME);
Arrays.sort(startKeys, Bytes.BYTES_COMPARATOR);
- List<HRegionInfo> newRegions = new ArrayList<>(startKeys.length);
+ List<RegionInfo> newRegions = new ArrayList<>(startKeys.length);
MetaTableAccessor
.updateTableState(getConnection(), htd.getTableName(), TableState.State.ENABLED);
// add custom ones
for (int i = 0; i < startKeys.length; i++) {
int j = (i + 1) % startKeys.length;
- HRegionInfo hri = new HRegionInfo(htd.getTableName(), startKeys[i],
- startKeys[j]);
+ RegionInfo hri = RegionInfoBuilder.newBuilder(htd.getTableName())
+ .setStartKey(startKeys[i])
+ .setEndKey(startKeys[j])
+ .build();
MetaTableAccessor.addRegionToMeta(meta, hri);
newRegions.add(hri);
}
@@ -2518,7 +2423,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
/**
* Create an unmanaged WAL. Be sure to close it when you're through.
*/
- public static WAL createWal(final Configuration conf, final Path rootDir, final HRegionInfo hri)
+ public static WAL createWal(final Configuration conf, final Path rootDir, final RegionInfo hri)
throws IOException {
// The WAL subsystem will use the default rootDir rather than the passed in rootDir
// unless I pass along via the conf.
@@ -2533,20 +2438,8 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
/**
* Create a region with it's own WAL. Be sure to call
* {@link HBaseTestingUtility#closeRegionAndWAL(HRegion)} to clean up all resources.
- * @deprecated since 2.0 version and will be removed in 3.0 version.
- * use {@link #createRegionAndWAL(HRegionInfo, Path, Configuration, TableDescriptor)}
- */
- @Deprecated
- public static HRegion createRegionAndWAL(final HRegionInfo info, final Path rootDir,
- final Configuration conf, final HTableDescriptor htd) throws IOException {
- return createRegionAndWAL(info, rootDir, conf, (TableDescriptor) htd);
- }
-
- /**
- * Create a region with it's own WAL. Be sure to call
- * {@link HBaseTestingUtility#closeRegionAndWAL(HRegion)} to clean up all resources.
*/
- public static HRegion createRegionAndWAL(final HRegionInfo info, final Path rootDir,
+ public static HRegion createRegionAndWAL(final RegionInfo info, final Path rootDir,
final Configuration conf, final TableDescriptor htd) throws IOException {
return createRegionAndWAL(info, rootDir, conf, htd, true);
}
@@ -2554,21 +2447,8 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
/**
* Create a region with it's own WAL. Be sure to call
* {@link HBaseTestingUtility#closeRegionAndWAL(HRegion)} to clean up all resources.
- * @deprecated since 2.0 version and will be removed in 3.0 version.
- * use {@link #createRegionAndWAL(HRegionInfo, Path, Configuration, TableDescriptor, boolean)}
*/
- @Deprecated
- public static HRegion createRegionAndWAL(final HRegionInfo info, final Path rootDir,
- final Configuration conf, final HTableDescriptor htd, boolean initialize)
- throws IOException {
- return createRegionAndWAL(info, rootDir, conf, (TableDescriptor) htd, initialize);
- }
-
- /**
- * Create a region with it's own WAL. Be sure to call
- * {@link HBaseTestingUtility#closeRegionAndWAL(HRegion)} to clean up all resources.
- */
- public static HRegion createRegionAndWAL(final HRegionInfo info, final Path rootDir,
+ public static HRegion createRegionAndWAL(final RegionInfo info, final Path rootDir,
final Configuration conf, final TableDescriptor htd, boolean initialize)
throws IOException {
ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
@@ -2607,7 +2487,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
List<byte[]> rows = new ArrayList<>();
ResultScanner s = t.getScanner(new Scan());
for (Result result : s) {
- HRegionInfo info = MetaTableAccessor.getHRegionInfo(result);
+ RegionInfo info = MetaTableAccessor.getRegionInfo(result);
if (info == null) {
LOG.error("No region info for row " + Bytes.toString(result.getRow()));
// TODO figure out what to do for this new hosed case.
@@ -3175,13 +3055,13 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
public String explainTableAvailability(TableName tableName) throws IOException {
String msg = explainTableState(tableName, TableState.State.ENABLED) + ", ";
if (getHBaseCluster().getMaster().isAlive()) {
- Map<HRegionInfo, ServerName> assignments =
+ Map<RegionInfo, ServerName> assignments =
getHBaseCluster().getMaster().getAssignmentManager().getRegionStates()
.getRegionAssignments();
- final List<Pair<HRegionInfo, ServerName>> metaLocations =
+ final List<Pair<RegionInfo, ServerName>> metaLocations =
MetaTableAccessor.getTableRegionsAndLocations(connection, tableName);
- for (Pair<HRegionInfo, ServerName> metaLocation : metaLocations) {
- HRegionInfo hri = metaLocation.getFirst();
+ for (Pair<RegionInfo, ServerName> metaLocation : metaLocations) {
+ RegionInfo hri = metaLocation.getFirst();
ServerName sn = metaLocation.getSecond();
if (!assignments.containsKey(hri)) {
msg += ", region " + hri
@@ -3377,7 +3257,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
NavigableSet<String> online = new TreeSet<>();
for (RegionServerThread rst : cluster.getLiveRegionServerThreads()) {
try {
- for (HRegionInfo region :
+ for (RegionInfo region :
ProtobufUtil.getOnlineRegions(rst.getRegionServer().getRSRpcServices())) {
online.add(region.getRegionNameAsString());
}
@@ -3387,7 +3267,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
}
for (MasterThread mt : cluster.getLiveMasterThreads()) {
try {
- for (HRegionInfo region :
+ for (RegionInfo region :
ProtobufUtil.getOnlineRegions(mt.getMaster().getRSRpcServices())) {
online.add(region.getRegionNameAsString());
}
@@ -3444,7 +3324,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
* @throw InterruptedException
* @return true if the region is assigned false otherwise.
*/
- public boolean assignRegion(final HRegionInfo regionInfo)
+ public boolean assignRegion(final RegionInfo regionInfo)
throws IOException, InterruptedException {
final AssignmentManager am = getHBaseCluster().getMaster().getAssignmentManager();
am.assign(regionInfo);
@@ -3459,7 +3339,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
* @throws InterruptedException
* @throws IOException
*/
- public void moveRegionAndWait(HRegionInfo destRegion, ServerName destServer)
+ public void moveRegionAndWait(RegionInfo destRegion, ServerName destServer)
throws InterruptedException, IOException {
HMaster master = getMiniHBaseCluster().getMaster();
// TODO: Here we start the move. The move can take a while.
@@ -3559,7 +3439,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
@Override
public boolean evaluate() throws IOException {
- List<HRegionInfo> hris = states.getRegionsOfTable(tableName);
+ List<RegionInfo> hris = states.getRegionsOfTable(tableName);
return hris != null && !hris.isEmpty();
}
});
@@ -3941,21 +3821,6 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
* Creates a pre-split table for load testing. If the table already exists,
* logs a warning and continues.
* @return the number of regions the table was split into
- * @deprecated since 2.0 version and will be removed in 3.0 version.
- * use {@link #createPreSplitLoadTestTable(Configuration,
- * TableDescriptor, ColumnFamilyDescriptor)}
- */
- @Deprecated
- public static int createPreSplitLoadTestTable(Configuration conf,
- HTableDescriptor desc, HColumnDescriptor hcd) throws IOException {
- return createPreSplitLoadTestTable(conf, (TableDescriptor) desc,
- (ColumnFamilyDescriptor) hcd);
- }
-
- /**
- * Creates a pre-split table for load testing. If the table already exists,
- * logs a warning and continues.
- * @return the number of regions the table was split into
*/
public static int createPreSplitLoadTestTable(Configuration conf,
TableDescriptor desc, ColumnFamilyDescriptor hcd) throws IOException {
@@ -3966,21 +3831,6 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
* Creates a pre-split table for load testing. If the table already exists,
* logs a warning and continues.
* @return the number of regions the table was split into
- * @deprecated since 2.0 version and will be removed in 3.0 version.
- * use {@link #createPreSplitLoadTestTable(Configuration,
- * TableDescriptor, ColumnFamilyDescriptor, int)}
- */
- @Deprecated
- public static int createPreSplitLoadTestTable(Configuration conf,
- HTableDescriptor desc, HColumnDescriptor hcd, int numRegionsPerServer) throws IOException {
- return createPreSplitLoadTestTable(conf, (TableDescriptor) desc,
- (ColumnFamilyDescriptor) hcd, numRegionsPerServer);
- }
-
- /**
- * Creates a pre-split table for load testing. If the table already exists,
- * logs a warning and continues.
- * @return the number of regions the table was split into
*/
public static int createPreSplitLoadTestTable(Configuration conf,
TableDescriptor desc, ColumnFamilyDescriptor hcd, int numRegionsPerServer) throws IOException {
@@ -3992,22 +3842,6 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
* Creates a pre-split table for load testing. If the table already exists,
* logs a warning and continues.
* @return the number of regions the table was split into
- * @deprecated since 2.0 version and will be removed in 3.0 version.
- * use {@link #createPreSplitLoadTestTable(Configuration,
- * TableDescriptor, ColumnFamilyDescriptor[], int)}
- */
- @Deprecated
- public static int createPreSplitLoadTestTable(Configuration conf,
- HTableDescriptor desc, HColumnDescriptor[] hcds,
- int numRegionsPerServer) throws IOException {
- return createPreSplitLoadTestTable(conf, (TableDescriptor) desc,
- (ColumnFamilyDescriptor[]) hcds, numRegionsPerServer);
- }
-
- /**
- * Creates a pre-split table for load testing. If the table already exists,
- * logs a warning and continues.
- * @return the number of regions the table was split into
*/
public static int createPreSplitLoadTestTable(Configuration conf,
TableDescriptor desc, ColumnFamilyDescriptor[] hcds,
@@ -4081,12 +3915,12 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
* is recorded in master.
*/
public void assertRegionOnServer(
- final HRegionInfo hri, final ServerName server,
+ final RegionInfo hri, final ServerName server,
final long timeout) throws IOException, InterruptedException {
long timeoutTime = System.currentTimeMillis() + timeout;
while (true) {
- List<HRegionInfo> regions = getAdmin().getOnlineRegions(server);
- if (regions.contains(hri)) return;
+ List<RegionInfo> regions = getAdmin().getRegions(server);
+ if (regions.stream().anyMatch(r -> RegionInfo.COMPARATOR.compare(r, hri) == 0)) return;
long now = System.currentTimeMillis();
if (now > timeoutTime) break;
Thread.sleep(10);
@@ -4100,12 +3934,12 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
* region server, but not on any other one.
*/
public void assertRegionOnlyOnServer(
- final HRegionInfo hri, final ServerName server,
+ final RegionInfo hri, final ServerName server,
final long timeout) throws IOException, InterruptedException {
long timeoutTime = System.currentTimeMillis() + timeout;
while (true) {
- List<HRegionInfo> regions = getAdmin().getOnlineRegions(server);
- if (regions.contains(hri)) {
+ List<RegionInfo> regions = getAdmin().getRegions(server);
+ if (regions.stream().anyMatch(r -> RegionInfo.COMPARATOR.compare(r, hri) == 0)) {
List<JVMClusterUtil.RegionServerThread> rsThreads =
getHBaseCluster().getLiveRegionServerThreads();
for (JVMClusterUtil.RegionServerThread rsThread: rsThreads) {
@@ -4129,16 +3963,6 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
+ " on server " + server);
}
- /**
- * @deprecated since 2.0 version and will be removed in 3.0 version.
- * use {@link #createTestRegion(String, ColumnFamilyDescriptor)}
- */
- @Deprecated
- public HRegion createTestRegion(String tableName, HColumnDescriptor cd)
- throws IOException {
- return createTestRegion(tableName, (ColumnFamilyDescriptor) cd);
- }
-
public HRegion createTestRegion(String tableName, ColumnFamilyDescriptor cd)
throws IOException {
TableDescriptor td
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java
index ef398ff..e453be2 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java
@@ -17,8 +17,6 @@
*/
package org.apache.hadoop.hbase;
-import com.google.protobuf.Service;
-
import java.io.IOException;
import java.net.InetSocketAddress;
import java.util.Collections;
@@ -34,6 +32,7 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.hbase.client.ClusterConnection;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.locking.EntityLock;
import org.apache.hadoop.hbase.executor.ExecutorService;
import org.apache.hadoop.hbase.fs.HFileSystem;
@@ -50,14 +49,17 @@ import org.apache.hadoop.hbase.regionserver.RegionServerServices;
import org.apache.hadoop.hbase.regionserver.SecureBulkLoadManager;
import org.apache.hadoop.hbase.regionserver.ServerNonceManager;
import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.wal.WAL;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.zookeeper.KeeperException;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
+
+import com.google.protobuf.Service;
+
/**
* Basic mock region server services. Should only be instantiated by HBaseTestingUtility.b
*/
@@ -242,7 +244,7 @@ public class MockRegionServerServices implements RegionServerServices {
}
@Override
- public WAL getWAL(HRegionInfo regionInfo) throws IOException {
+ public WAL getWAL(RegionInfo regionInfo) throws IOException {
return null;
}
@@ -280,13 +282,13 @@ public class MockRegionServerServices implements RegionServerServices {
@Override
public boolean reportRegionStateTransition(TransitionCode code, long openSeqNum,
- HRegionInfo... hris) {
+ RegionInfo... hris) {
return false;
}
@Override
public boolean reportRegionStateTransition(TransitionCode code,
- HRegionInfo... hris) {
+ RegionInfo... hris) {
return false;
}
@@ -332,7 +334,7 @@ public class MockRegionServerServices implements RegionServerServices {
}
@Override
- public EntityLock regionLock(List<HRegionInfo> regionInfos, String description, Abortable abort)
+ public EntityLock regionLock(List<RegionInfo> regionInfos, String description, Abortable abort)
throws IOException {
return null;
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/TestGlobalMemStoreSize.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestGlobalMemStoreSize.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestGlobalMemStoreSize.java
index e8ae7d1..21b36b7 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestGlobalMemStoreSize.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestGlobalMemStoreSize.java
@@ -28,11 +28,11 @@ import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.RegionLocator;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.regionserver.Region;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.testclassification.MiscTests;
import org.apache.hadoop.hbase.util.Bytes;
@@ -43,6 +43,8 @@ import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.junit.rules.TestName;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+
/**
* Test HBASE-3694 whether the GlobalMemStoreSize is the same as the summary
* of all the online region's MemStoreSize
@@ -91,7 +93,7 @@ public class TestGlobalMemStoreSize {
for (HRegionServer server : getOnlineRegionServers()) {
long globalMemStoreSize = 0;
- for (HRegionInfo regionInfo :
+ for (RegionInfo regionInfo :
ProtobufUtil.getOnlineRegions(null, server.getRSRpcServices())) {
globalMemStoreSize += server.getRegion(regionInfo.getEncodedName()).getMemstoreSize();
}
@@ -105,7 +107,7 @@ public class TestGlobalMemStoreSize {
LOG.info("Starting flushes on " + server.getServerName() +
", size=" + server.getRegionServerAccounting().getGlobalMemstoreDataSize());
- for (HRegionInfo regionInfo :
+ for (RegionInfo regionInfo :
ProtobufUtil.getOnlineRegions(null, server.getRSRpcServices())) {
Region r = server.getRegion(regionInfo.getEncodedName());
flush(r, server);
@@ -121,7 +123,7 @@ public class TestGlobalMemStoreSize {
if (size > 0) {
// If size > 0, see if its because the meta region got edits while
// our test was running....
- for (HRegionInfo regionInfo :
+ for (RegionInfo regionInfo :
ProtobufUtil.getOnlineRegions(null, server.getRSRpcServices())) {
Region r = server.getRegion(regionInfo.getEncodedName());
long l = r.getMemstoreSize();
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/TestIOFencing.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestIOFencing.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestIOFencing.java
index c9881c6..b759261 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestIOFencing.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestIOFencing.java
@@ -32,6 +32,7 @@ import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.regionserver.CompactingMemStore;
@@ -99,7 +100,7 @@ public class TestIOFencing {
@SuppressWarnings("deprecation")
public CompactionBlockerRegion(Path tableDir, WAL log,
- FileSystem fs, Configuration confParam, HRegionInfo info,
+ FileSystem fs, Configuration confParam, RegionInfo info,
TableDescriptor htd, RegionServerServices rsServices) {
super(tableDir, log, fs, confParam, info, htd, rsServices);
}
@@ -159,7 +160,7 @@ public class TestIOFencing {
public static class BlockCompactionsInPrepRegion extends CompactionBlockerRegion {
public BlockCompactionsInPrepRegion(Path tableDir, WAL log,
- FileSystem fs, Configuration confParam, HRegionInfo info,
+ FileSystem fs, Configuration confParam, RegionInfo info,
TableDescriptor htd, RegionServerServices rsServices) {
super(tableDir, log, fs, confParam, info, htd, rsServices);
}
@@ -182,7 +183,7 @@ public class TestIOFencing {
*/
public static class BlockCompactionsInCompletionRegion extends CompactionBlockerRegion {
public BlockCompactionsInCompletionRegion(Path tableDir, WAL log,
- FileSystem fs, Configuration confParam, HRegionInfo info,
+ FileSystem fs, Configuration confParam, RegionInfo info,
TableDescriptor htd, RegionServerServices rsServices) {
super(tableDir, log, fs, confParam, info, htd, rsServices);
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/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 a0cd236..15c0b0c 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
@@ -41,6 +41,8 @@ import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
import org.apache.hadoop.hbase.client.RegionLocator;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Table;
@@ -63,9 +65,9 @@ import org.junit.BeforeClass;
import org.junit.Rule;
import org.junit.Test;
import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
-import org.junit.rules.TestName;
/**
* Test {@link org.apache.hadoop.hbase.MetaTableAccessor}.
@@ -114,7 +116,7 @@ public class TestMetaTableAccessor {
regionCount = r.getStartKeys().length;
}
// Test it works getting a region from just made user table.
- final List<HRegionInfo> regions =
+ final List<RegionInfo> regions =
testGettingTableRegions(connection, tableName, regionCount);
MetaTask reader = new MetaTask(connection, "reader") {
@Override
@@ -227,7 +229,7 @@ public class TestMetaTableAccessor {
@Test public void testGetRegionsFromMetaTable()
throws IOException, InterruptedException {
- List<HRegionInfo> regions =
+ List<RegionInfo> regions =
new MetaTableLocator().getMetaRegions(UTIL.getZooKeeperWatcher());
assertTrue(regions.size() >= 1);
assertTrue(new MetaTableLocator().getMetaRegionsAndLocations(
@@ -256,7 +258,7 @@ public class TestMetaTableAccessor {
final String name = this.name.getMethodName();
LOG.info("Started " + name);
// Test get on non-existent region.
- Pair<HRegionInfo, ServerName> pair =
+ Pair<RegionInfo, ServerName> pair =
MetaTableAccessor.getRegion(connection, Bytes.toBytes("nonexistent-region"));
assertNull(pair);
LOG.info("Finished " + name);
@@ -285,12 +287,12 @@ public class TestMetaTableAccessor {
assertEquals(1, MetaTableAccessor.getTableRegions(connection, greaterName).size());
}
- private static List<HRegionInfo> testGettingTableRegions(final Connection connection,
+ private static List<RegionInfo> testGettingTableRegions(final Connection connection,
final TableName name, final int regionCount)
throws IOException, InterruptedException {
- List<HRegionInfo> regions = MetaTableAccessor.getTableRegions(connection, name);
+ List<RegionInfo> regions = MetaTableAccessor.getTableRegions(connection, name);
assertEquals(regionCount, regions.size());
- Pair<HRegionInfo, ServerName> pair =
+ Pair<RegionInfo, ServerName> pair =
MetaTableAccessor.getRegion(connection, regions.get(0).getRegionName());
assertEquals(regions.get(0).getEncodedName(),
pair.getFirst().getEncodedName());
@@ -298,9 +300,9 @@ public class TestMetaTableAccessor {
}
private static void testGetRegion(final Connection connection,
- final HRegionInfo region)
+ final RegionInfo region)
throws IOException, InterruptedException {
- Pair<HRegionInfo, ServerName> pair =
+ Pair<RegionInfo, ServerName> pair =
MetaTableAccessor.getRegion(connection, region.getRegionName());
assertEquals(region.getEncodedName(),
pair.getFirst().getEncodedName());
@@ -349,12 +351,27 @@ public class TestMetaTableAccessor {
ServerName serverName100 = ServerName.valueOf("baz", 60010, random.nextLong());
long regionId = System.currentTimeMillis();
- HRegionInfo primary = new HRegionInfo(TableName.valueOf(name.getMethodName()),
- HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW, false, regionId, 0);
- HRegionInfo replica1 = new HRegionInfo(TableName.valueOf(name.getMethodName()),
- HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW, false, regionId, 1);
- HRegionInfo replica100 = new HRegionInfo(TableName.valueOf(name.getMethodName()),
- HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW, false, regionId, 100);
+ RegionInfo primary = RegionInfoBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
+ .setStartKey(HConstants.EMPTY_START_ROW)
+ .setEndKey(HConstants.EMPTY_END_ROW)
+ .setSplit(false)
+ .setRegionId(regionId)
+ .setReplicaId(0)
+ .build();
+ RegionInfo replica1 = RegionInfoBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
+ .setStartKey(HConstants.EMPTY_START_ROW)
+ .setEndKey(HConstants.EMPTY_END_ROW)
+ .setSplit(false)
+ .setRegionId(regionId)
+ .setReplicaId(1)
+ .build();
+ RegionInfo replica100 = RegionInfoBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
+ .setStartKey(HConstants.EMPTY_START_ROW)
+ .setEndKey(HConstants.EMPTY_END_ROW)
+ .setSplit(false)
+ .setRegionId(regionId)
+ .setReplicaId(100)
+ .build();
long seqNum0 = random.nextLong();
long seqNum1 = random.nextLong();
@@ -422,12 +439,17 @@ public class TestMetaTableAccessor {
@Test
public void testMetaLocationForRegionReplicasIsAddedAtTableCreation() throws IOException {
long regionId = System.currentTimeMillis();
- HRegionInfo primary = new HRegionInfo(TableName.valueOf(name.getMethodName()),
- HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW, false, regionId, 0);
+ RegionInfo primary = RegionInfoBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
+ .setStartKey(HConstants.EMPTY_START_ROW)
+ .setEndKey(HConstants.EMPTY_END_ROW)
+ .setSplit(false)
+ .setRegionId(regionId)
+ .setReplicaId(0)
+ .build();
Table meta = MetaTableAccessor.getMetaHTable(connection);
try {
- List<HRegionInfo> regionInfos = Lists.newArrayList(primary);
+ List<RegionInfo> regionInfos = Lists.newArrayList(primary);
MetaTableAccessor.addRegionsToMeta(connection, regionInfos, 3);
assertEmptyMetaLocation(meta, primary.getRegionName(), 1);
@@ -441,17 +463,33 @@ public class TestMetaTableAccessor {
public void testMetaLocationForRegionReplicasIsAddedAtRegionSplit() throws IOException {
long regionId = System.currentTimeMillis();
ServerName serverName0 = ServerName.valueOf("foo", 60010, random.nextLong());
- HRegionInfo parent = new HRegionInfo(TableName.valueOf(name.getMethodName()),
- HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW, false, regionId, 0);
- HRegionInfo splitA = new HRegionInfo(TableName.valueOf(name.getMethodName()),
- HConstants.EMPTY_START_ROW, Bytes.toBytes("a"), false, regionId+1, 0);
- HRegionInfo splitB = new HRegionInfo(TableName.valueOf(name.getMethodName()),
- Bytes.toBytes("a"), HConstants.EMPTY_END_ROW, false, regionId+1, 0);
+ RegionInfo parent = RegionInfoBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
+ .setStartKey(HConstants.EMPTY_START_ROW)
+ .setEndKey(HConstants.EMPTY_END_ROW)
+ .setSplit(false)
+ .setRegionId(regionId)
+ .setReplicaId(0)
+ .build();
+
+ RegionInfo splitA = RegionInfoBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
+ .setStartKey(HConstants.EMPTY_START_ROW)
+ .setEndKey(Bytes.toBytes("a"))
+ .setSplit(false)
+ .setRegionId(regionId + 1)
+ .setReplicaId(0)
+ .build();
+ RegionInfo splitB = RegionInfoBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
+ .setStartKey(Bytes.toBytes("a"))
+ .setEndKey(HConstants.EMPTY_END_ROW)
+ .setSplit(false)
+ .setRegionId(regionId + 1)
+ .setReplicaId(0)
+ .build();
Table meta = MetaTableAccessor.getMetaHTable(connection);
try {
- List<HRegionInfo> regionInfos = Lists.newArrayList(parent);
+ List<RegionInfo> regionInfos = Lists.newArrayList(parent);
MetaTableAccessor.addRegionsToMeta(connection, regionInfos, 3);
MetaTableAccessor.splitRegion(connection, parent, splitA, splitB, serverName0, 3, false);
@@ -470,16 +508,32 @@ public class TestMetaTableAccessor {
long regionId = System.currentTimeMillis();
ServerName serverName0 = ServerName.valueOf("foo", 60010, random.nextLong());
- HRegionInfo parentA = new HRegionInfo(TableName.valueOf(name.getMethodName()),
- Bytes.toBytes("a"), HConstants.EMPTY_END_ROW, false, regionId, 0);
- HRegionInfo parentB = new HRegionInfo(TableName.valueOf(name.getMethodName()),
- HConstants.EMPTY_START_ROW, Bytes.toBytes("a"), false, regionId, 0);
- HRegionInfo merged = new HRegionInfo(TableName.valueOf(name.getMethodName()),
- HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW, false, regionId+1, 0);
+ RegionInfo parentA = RegionInfoBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
+ .setStartKey(Bytes.toBytes("a"))
+ .setEndKey(HConstants.EMPTY_END_ROW)
+ .setSplit(false)
+ .setRegionId(regionId)
+ .setReplicaId(0)
+ .build();
+
+ RegionInfo parentB = RegionInfoBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
+ .setStartKey(HConstants.EMPTY_START_ROW)
+ .setEndKey(Bytes.toBytes("a"))
+ .setSplit(false)
+ .setRegionId(regionId)
+ .setReplicaId(0)
+ .build();
+ RegionInfo merged = RegionInfoBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
+ .setStartKey(HConstants.EMPTY_START_ROW)
+ .setEndKey(HConstants.EMPTY_END_ROW)
+ .setSplit(false)
+ .setRegionId(regionId + 1)
+ .setReplicaId(0)
+ .build();
Table meta = MetaTableAccessor.getMetaHTable(connection);
try {
- List<HRegionInfo> regionInfos = Lists.newArrayList(parentA, parentB);
+ List<RegionInfo> regionInfos = Lists.newArrayList(parentA, parentB);
MetaTableAccessor.addRegionsToMeta(connection, regionInfos, 3);
MetaTableAccessor.mergeRegions(connection, merged, parentA, parentB, serverName0, 3,
@@ -542,13 +596,18 @@ public class TestMetaTableAccessor {
@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);
+ RegionInfo regionInfo = RegionInfoBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
+ .setStartKey(HConstants.EMPTY_START_ROW)
+ .setEndKey(HConstants.EMPTY_END_ROW)
+ .setSplit(false)
+ .setRegionId(regionId)
+ .setReplicaId(0)
+ .build();
ServerName sn = ServerName.valueOf("bar", 0, 0);
Table meta = MetaTableAccessor.getMetaHTable(connection);
try {
- List<HRegionInfo> regionInfos = Lists.newArrayList(regionInfo);
+ List<RegionInfo> regionInfos = Lists.newArrayList(regionInfo);
MetaTableAccessor.addRegionsToMeta(connection, regionInfos, 1);
long masterSystemTime = EnvironmentEdgeManager.currentTime() + 123456789;
@@ -579,17 +638,34 @@ public class TestMetaTableAccessor {
@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);
+
+ RegionInfo regionInfoA = RegionInfoBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
+ .setStartKey(HConstants.EMPTY_START_ROW)
+ .setEndKey(new byte[] {'a'})
+ .setSplit(false)
+ .setRegionId(regionId)
+ .setReplicaId(0)
+ .build();
+
+ RegionInfo regionInfoB = RegionInfoBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
+ .setStartKey(new byte[] {'a'})
+ .setEndKey(HConstants.EMPTY_END_ROW)
+ .setSplit(false)
+ .setRegionId(regionId)
+ .setReplicaId(0)
+ .build();
+ RegionInfo mergedRegionInfo = RegionInfoBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
+ .setStartKey(HConstants.EMPTY_START_ROW)
+ .setEndKey(HConstants.EMPTY_END_ROW)
+ .setSplit(false)
+ .setRegionId(regionId)
+ .setReplicaId(0)
+ .build();
ServerName sn = ServerName.valueOf("bar", 0, 0);
Table meta = MetaTableAccessor.getMetaHTable(connection);
try {
- List<HRegionInfo> regionInfos = Lists.newArrayList(regionInfoA, regionInfoB);
+ List<RegionInfo> 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
@@ -674,13 +750,21 @@ public class TestMetaTableAccessor {
UTIL.createTable(tableName, "cf1");
HRegionLocation loc = rl.getAllRegionLocations().get(0);
- HRegionInfo parent = loc.getRegionInfo();
+ RegionInfo parent = loc.getRegionInfo();
long rid = 1000;
byte[] splitKey = Bytes.toBytes("a");
- HRegionInfo splitA = new HRegionInfo(parent.getTable(), parent.getStartKey(),
- splitKey, false, rid);
- HRegionInfo splitB = new HRegionInfo(parent.getTable(), splitKey,
- parent.getEndKey(), false, rid);
+ RegionInfo splitA = RegionInfoBuilder.newBuilder(parent.getTable())
+ .setStartKey(parent.getStartKey())
+ .setEndKey(splitKey)
+ .setSplit(false)
+ .setRegionId(rid)
+ .build();
+ RegionInfo splitB = RegionInfoBuilder.newBuilder(parent.getTable())
+ .setStartKey(splitKey)
+ .setEndKey(parent.getEndKey())
+ .setSplit(false)
+ .setRegionId(rid)
+ .build();
// find the meta server
MiniHBaseCluster cluster = UTIL.getMiniHBaseCluster();
@@ -704,16 +788,31 @@ public class TestMetaTableAccessor {
public void testEmptyMetaDaughterLocationDuringSplit() throws IOException {
long regionId = System.currentTimeMillis();
ServerName serverName0 = ServerName.valueOf("foo", 60010, random.nextLong());
- HRegionInfo parent = new HRegionInfo(TableName.valueOf("table_foo"), HConstants.EMPTY_START_ROW,
- HConstants.EMPTY_END_ROW, false, regionId, 0);
- HRegionInfo splitA = new HRegionInfo(TableName.valueOf("table_foo"), HConstants.EMPTY_START_ROW,
- Bytes.toBytes("a"), false, regionId + 1, 0);
- HRegionInfo splitB = new HRegionInfo(TableName.valueOf("table_foo"), Bytes.toBytes("a"),
- HConstants.EMPTY_END_ROW, false, regionId + 1, 0);
+ RegionInfo parent = RegionInfoBuilder.newBuilder(TableName.valueOf("table_foo"))
+ .setStartKey(HConstants.EMPTY_START_ROW)
+ .setEndKey(HConstants.EMPTY_END_ROW)
+ .setSplit(false)
+ .setRegionId(regionId)
+ .setReplicaId(0)
+ .build();
+ RegionInfo splitA = RegionInfoBuilder.newBuilder(TableName.valueOf("table_foo"))
+ .setStartKey(HConstants.EMPTY_START_ROW)
+ .setEndKey(Bytes.toBytes("a"))
+ .setSplit(false)
+ .setRegionId(regionId + 1)
+ .setReplicaId(0)
+ .build();
+ RegionInfo splitB = RegionInfoBuilder.newBuilder(TableName.valueOf("table_foo"))
+ .setStartKey(Bytes.toBytes("a"))
+ .setEndKey(HConstants.EMPTY_END_ROW)
+ .setSplit(false)
+ .setRegionId(regionId + 1)
+ .setReplicaId(0)
+ .build();
Table meta = MetaTableAccessor.getMetaHTable(connection);
try {
- List<HRegionInfo> regionInfos = Lists.newArrayList(parent);
+ List<RegionInfo> regionInfos = Lists.newArrayList(parent);
MetaTableAccessor.addRegionsToMeta(connection, regionInfos, 3);
MetaTableAccessor.splitRegion(connection, parent, splitA, splitB, serverName0, 3, false);
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessorNoCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessorNoCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessorNoCluster.java
index 9915f99..cec2c20 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessorNoCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessorNoCluster.java
@@ -31,6 +31,8 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.client.ClusterConnection;
import org.apache.hadoop.hbase.client.HConnectionTestingUtility;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.ipc.HBaseRpcController;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
@@ -85,31 +87,30 @@ public class TestMetaTableAccessorNoCluster {
@Test
public void testGetHRegionInfo() throws IOException {
- assertNull(MetaTableAccessor.getHRegionInfo(new Result()));
+ assertNull(MetaTableAccessor.getRegionInfo(new Result()));
List<Cell> kvs = new ArrayList<>();
Result r = Result.create(kvs);
- assertNull(MetaTableAccessor.getHRegionInfo(r));
+ assertNull(MetaTableAccessor.getRegionInfo(r));
byte [] f = HConstants.CATALOG_FAMILY;
// Make a key value that doesn't have the expected qualifier.
kvs.add(new KeyValue(HConstants.EMPTY_BYTE_ARRAY, f,
HConstants.SERVER_QUALIFIER, f));
r = Result.create(kvs);
- assertNull(MetaTableAccessor.getHRegionInfo(r));
+ assertNull(MetaTableAccessor.getRegionInfo(r));
// Make a key that does not have a regioninfo value.
kvs.add(new KeyValue(HConstants.EMPTY_BYTE_ARRAY, f,
HConstants.REGIONINFO_QUALIFIER, f));
- HRegionInfo hri = MetaTableAccessor.getHRegionInfo(Result.create(kvs));
+ RegionInfo hri = MetaTableAccessor.getRegionInfo(Result.create(kvs));
assertTrue(hri == null);
// OK, give it what it expects
kvs.clear();
kvs.add(new KeyValue(HConstants.EMPTY_BYTE_ARRAY, f,
- HConstants.REGIONINFO_QUALIFIER,
- HRegionInfo.FIRST_META_REGIONINFO.toByteArray()));
- hri = MetaTableAccessor.getHRegionInfo(Result.create(kvs));
+ HConstants.REGIONINFO_QUALIFIER, RegionInfo.toByteArray(RegionInfoBuilder.FIRST_META_REGIONINFO)));
+ hri = MetaTableAccessor.getRegionInfo(Result.create(kvs));
assertNotNull(hri);
- assertTrue(hri.equals(HRegionInfo.FIRST_META_REGIONINFO));
+ assertTrue(RegionInfo.COMPARATOR.compare(hri, RegionInfoBuilder.FIRST_META_REGIONINFO) == 0);
}
/**
@@ -145,7 +146,7 @@ public class TestMetaTableAccessorNoCluster {
final byte [] rowToVerify = Bytes.toBytes("rowToVerify");
kvs.add(new KeyValue(rowToVerify,
HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER,
- HRegionInfo.FIRST_META_REGIONINFO.toByteArray()));
+ RegionInfo.toByteArray(RegionInfoBuilder.FIRST_META_REGIONINFO)));
kvs.add(new KeyValue(rowToVerify,
HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER,
Bytes.toBytes(sn.getHostAndPort())));
@@ -173,11 +174,11 @@ public class TestMetaTableAccessorNoCluster {
// to shove this in here first so it gets picked up all over; e.g. by
// HTable.
connection = HConnectionTestingUtility.getSpiedConnection(UTIL.getConfiguration());
-
+
// Fix the location lookup so it 'works' though no network. First
// make an 'any location' object.
final HRegionLocation anyLocation =
- new HRegionLocation(HRegionInfo.FIRST_META_REGIONINFO, sn);
+ new HRegionLocation(RegionInfoBuilder.FIRST_META_REGIONINFO, sn);
final RegionLocations rl = new RegionLocations(anyLocation);
// Return the RegionLocations object when locateRegion
// The ugly format below comes of 'Important gotcha on spying real objects!' from
@@ -191,10 +192,10 @@ public class TestMetaTableAccessorNoCluster {
when(connection).getClient(Mockito.any(ServerName.class));
// Scan meta for user tables and verify we got back expected answer.
- NavigableMap<HRegionInfo, Result> hris =
+ NavigableMap<RegionInfo, Result> hris =
MetaTableAccessor.getServerUserRegions(connection, sn);
assertEquals(1, hris.size());
- assertTrue(hris.firstEntry().getKey().equals(HRegionInfo.FIRST_META_REGIONINFO));
+ assertTrue(RegionInfo.COMPARATOR.compare(hris.firstEntry().getKey(), RegionInfoBuilder.FIRST_META_REGIONINFO) == 0);
assertTrue(Bytes.equals(rowToVerify, hris.firstEntry().getValue().getRow()));
// Finally verify that scan was called four times -- three times
// with exception and then on 4th attempt we succeed
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/TestPartialResultsFromClientSide.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestPartialResultsFromClientSide.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestPartialResultsFromClientSide.java
index 20ad023..d23b4e4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestPartialResultsFromClientSide.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestPartialResultsFromClientSide.java
@@ -35,6 +35,7 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.client.ClientScanner;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
@@ -817,11 +818,11 @@ public class TestPartialResultsFromClientSide {
}
private void moveRegion(Table table, int index) throws IOException{
- List<Pair<HRegionInfo, ServerName>> regions = MetaTableAccessor
+ List<Pair<RegionInfo, ServerName>> regions = MetaTableAccessor
.getTableRegionsAndLocations(TEST_UTIL.getConnection(),
table.getName());
assertEquals(1, regions.size());
- HRegionInfo regionInfo = regions.get(0).getFirst();
+ RegionInfo regionInfo = regions.get(0).getFirst();
ServerName name = TEST_UTIL.getHBaseCluster().getRegionServer(index).getServerName();
TEST_UTIL.getAdmin().move(regionInfo.getEncodedNameAsBytes(),
Bytes.toBytes(name.getServerName()));
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/TestRegionRebalancing.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestRegionRebalancing.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestRegionRebalancing.java
index cff1a8d..3f7ea3b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestRegionRebalancing.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestRegionRebalancing.java
@@ -32,9 +32,9 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.RegionLocator;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.testclassification.FlakeyTests;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.util.Bytes;
@@ -49,6 +49,8 @@ import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import org.junit.runners.Parameterized.Parameters;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+
/**
* Test whether region re-balancing works. (HBASE-71)
*/
@@ -199,7 +201,7 @@ public class TestRegionRebalancing {
LOG.debug(server.getServerName() + " Avg: " + avg + " actual: " + serverLoad);
if (!(avg > 2.0 && serverLoad <= avgLoadPlusSlop
&& serverLoad >= avgLoadMinusSlop)) {
- for (HRegionInfo hri :
+ for (RegionInfo hri :
ProtobufUtil.getOnlineRegions(server.getRSRpcServices())) {
if (hri.isMetaRegion()) serverLoad--;
// LOG.debug(hri.getRegionNameAsString());
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/TestStochasticBalancerJmxMetrics.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestStochasticBalancerJmxMetrics.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestStochasticBalancerJmxMetrics.java
index 7ad99c5..f987ea7 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestStochasticBalancerJmxMetrics.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestStochasticBalancerJmxMetrics.java
@@ -20,6 +20,13 @@ package org.apache.hadoop.hbase;
import static org.junit.Assert.assertTrue;
+import javax.management.MBeanAttributeInfo;
+import javax.management.MBeanInfo;
+import javax.management.MBeanServerConnection;
+import javax.management.ObjectInstance;
+import javax.management.ObjectName;
+import javax.management.remote.JMXConnector;
+import javax.management.remote.JMXConnectorFactory;
import java.io.IOException;
import java.util.HashSet;
import java.util.Hashtable;
@@ -29,17 +36,10 @@ import java.util.Map;
import java.util.Random;
import java.util.Set;
-import javax.management.MBeanAttributeInfo;
-import javax.management.MBeanInfo;
-import javax.management.MBeanServerConnection;
-import javax.management.ObjectInstance;
-import javax.management.ObjectName;
-import javax.management.remote.JMXConnector;
-import javax.management.remote.JMXConnectorFactory;
-
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
import org.apache.hadoop.hbase.master.balancer.BalancerTestBase;
import org.apache.hadoop.hbase.master.balancer.StochasticLoadBalancer;
@@ -128,7 +128,7 @@ public class TestStochasticBalancerJmxMetrics extends BalancerTestBase {
loadBalancer.setConf(conf);
TableName tableName = HConstants.ENSEMBLE_TABLE_NAME;
- Map<ServerName, List<HRegionInfo>> clusterState = mockClusterServers(mockCluster_ensemble);
+ Map<ServerName, List<RegionInfo>> clusterState = mockClusterServers(mockCluster_ensemble);
loadBalancer.balanceCluster(tableName, clusterState);
String[] tableNames = new String[] { tableName.getNameAsString() };
@@ -164,7 +164,7 @@ public class TestStochasticBalancerJmxMetrics extends BalancerTestBase {
// table 1
TableName tableName = TableName.valueOf(TABLE_NAME_1);
- Map<ServerName, List<HRegionInfo>> clusterState = mockClusterServers(mockCluster_pertable_1);
+ Map<ServerName, List<RegionInfo>> clusterState = mockClusterServers(mockCluster_pertable_1);
loadBalancer.balanceCluster(tableName, clusterState);
// table 2
@@ -204,7 +204,7 @@ public class TestStochasticBalancerJmxMetrics extends BalancerTestBase {
/**
* Read the attributes from Hadoop->HBase->Master->Balancer in JMX
- * @throws IOException
+ * @throws IOException
*/
private Set<String> readJmxMetrics() throws IOException {
JMXConnector connector = null;
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java
index 461ff8c..4ed8d91 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java
@@ -39,6 +39,7 @@ import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
@@ -576,8 +577,8 @@ public class TestZooKeeper {
static boolean retainAssignCalled = false;
@Override
- public Map<ServerName, List<HRegionInfo>> retainAssignment(
- Map<HRegionInfo, ServerName> regions, List<ServerName> servers) throws HBaseIOException {
+ public Map<ServerName, List<RegionInfo>> retainAssignment(
+ Map<RegionInfo, ServerName> regions, List<ServerName> servers) throws HBaseIOException {
retainAssignCalled = true;
return super.retainAssignment(regions, servers);
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java
index cfe4d1f..d7d1b3a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java
@@ -83,7 +83,7 @@ public class HConnectionTestingUtility {
* itself a mock.
* @param sn ServerName to include in the region location returned by this
* <code>connection</code>
- * @param hri HRegionInfo to include in the location returned when
+ * @param hri RegionInfo to include in the location returned when
* getRegionLocator is called on the mocked connection
* @return Mock up a connection that returns a {@link Configuration} when
* {@link ClusterConnection#getConfiguration()} is called, a 'location' when
@@ -99,7 +99,7 @@ public class HConnectionTestingUtility {
public static ClusterConnection getMockedConnectionAndDecorate(final Configuration conf,
final AdminProtos.AdminService.BlockingInterface admin,
final ClientProtos.ClientService.BlockingInterface client,
- final ServerName sn, final HRegionInfo hri)
+ final ServerName sn, final RegionInfo hri)
throws IOException {
ConnectionImplementation c = Mockito.mock(ConnectionImplementation.class);
Mockito.when(c.getConfiguration()).thenReturn(conf);
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java
index 9c100a2..0b7da28 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java
@@ -38,7 +38,6 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.InvalidFamilyOperationException;
@@ -588,10 +587,10 @@ public class TestAdmin1 {
expectedRegions) throws IOException {
int numRS = c.getCurrentNrHRS();
List<HRegionLocation> regions = regionLocator.getAllRegionLocations();
- Map<ServerName, List<HRegionInfo>> server2Regions = new HashMap<>();
+ Map<ServerName, List<RegionInfo>> server2Regions = new HashMap<>();
for (HRegionLocation loc : regions) {
ServerName server = loc.getServerName();
- List<HRegionInfo> regs = server2Regions.get(server);
+ List<RegionInfo> regs = server2Regions.get(server);
if (regs == null) {
regs = new ArrayList<>();
server2Regions.put(server, regs);
@@ -607,7 +606,7 @@ public class TestAdmin1 {
float average = (float) expectedRegions/numRS;
int min = (int)Math.floor(average);
int max = (int)Math.ceil(average);
- for (List<HRegionInfo> regionList : server2Regions.values()) {
+ for (List<RegionInfo> regionList : server2Regions.values()) {
assertTrue("numRS=" + numRS + ", min=" + min + ", max=" + max +
", size=" + regionList.size() + ", tablesOnMaster=" + tablesOnMaster,
regionList.size() == min || regionList.size() == max);
@@ -690,7 +689,7 @@ public class TestAdmin1 {
List<HRegionLocation> regions;
Iterator<HRegionLocation> hris;
- HRegionInfo hri;
+ RegionInfo hri;
ClusterConnection conn = (ClusterConnection) TEST_UTIL.getConnection();
try (RegionLocator l = TEST_UTIL.getConnection().getRegionLocator(tableName)) {
regions = l.getAllRegionLocations();
@@ -1160,7 +1159,7 @@ public class TestAdmin1 {
puts.add(put);
ht.put(puts);
ht.close();
- List<Pair<HRegionInfo, ServerName>> regions =
+ List<Pair<RegionInfo, ServerName>> regions =
MetaTableAccessor.getTableRegionsAndLocations(TEST_UTIL.getConnection(), tableName);
boolean gotException = false;
// the element at index 1 would be a replica (since the metareader gives us ordered
@@ -1214,7 +1213,7 @@ public class TestAdmin1 {
nameofRegionsToMerge,
true,
HConstants.NO_NONCE,
- HConstants.NO_NONCE);
+ HConstants.NO_NONCE);
((ClusterConnection) TEST_UTIL.getAdmin().getConnection()).getMaster()
.mergeTableRegions(null, request);
} catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException m) {
@@ -1357,12 +1356,12 @@ public class TestAdmin1 {
TEST_UTIL.createTable(td, splitRows);
TEST_UTIL.waitTableAvailable(tableName);
- List<HRegionInfo> tableRegions;
- HRegionInfo regionA;
- HRegionInfo regionB;
+ List<RegionInfo> tableRegions;
+ RegionInfo regionA;
+ RegionInfo regionB;
// merge with full name
- tableRegions = admin.getTableRegions(tableName);
+ tableRegions = admin.getRegions(tableName);
assertEquals(3, admin.getTableRegions(tableName).size());
regionA = tableRegions.get(0);
regionB = tableRegions.get(1);
@@ -1373,7 +1372,7 @@ public class TestAdmin1 {
assertEquals(2, admin.getTableRegions(tableName).size());
// merge with encoded name
- tableRegions = admin.getTableRegions(tableName);
+ tableRegions = admin.getRegions(tableName);
regionA = tableRegions.get(0);
regionB = tableRegions.get(1);
// TODO convert this to version that is synchronous (See HBASE-16668)
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java
index e95f382..30cd8bf 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java
@@ -42,7 +42,6 @@ import org.apache.hadoop.hbase.ClusterStatus.Option;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MiniHBaseCluster;
@@ -55,11 +54,10 @@ import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.UnknownRegionException;
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
import org.apache.hadoop.hbase.constraint.ConstraintException;
-import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
import org.apache.hadoop.hbase.master.HMaster;
+import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.regionserver.Region;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.testclassification.ClientTests;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.util.Bytes;
@@ -69,12 +67,14 @@ import org.junit.After;
import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
+import org.junit.Ignore;
import org.junit.Rule;
import org.junit.Test;
-import org.junit.Ignore;
import org.junit.experimental.categories.Category;
import org.junit.rules.TestName;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+
/**
* Class to test HBaseAdmin.
@@ -316,10 +316,10 @@ public class TestAdmin2 {
final TableName tableName = TableName.valueOf(name.getMethodName());
createTableWithDefaultConf(tableName);
- HRegionInfo info = null;
+ RegionInfo info = null;
HRegionServer rs = TEST_UTIL.getRSForFirstRegionInTable(tableName);
- List<HRegionInfo> onlineRegions = ProtobufUtil.getOnlineRegions(rs.getRSRpcServices());
- for (HRegionInfo regionInfo : onlineRegions) {
+ List<RegionInfo> onlineRegions = ProtobufUtil.getOnlineRegions(rs.getRSRpcServices());
+ for (RegionInfo regionInfo : onlineRegions) {
if (!regionInfo.getTable().isSystemTable()) {
info = regionInfo;
admin.unassign(regionInfo.getRegionName(), true);
@@ -344,10 +344,10 @@ public class TestAdmin2 {
byte[] tableName = Bytes.toBytes(name);
createTableWithDefaultConf(tableName);
- HRegionInfo info = null;
+ RegionInfo info = null;
HRegionServer rs = TEST_UTIL.getRSForFirstRegionInTable(TableName.valueOf(tableName));
- List<HRegionInfo> onlineRegions = ProtobufUtil.getOnlineRegions(rs.getRSRpcServices());
- for (HRegionInfo regionInfo : onlineRegions) {
+ List<RegionInfo> onlineRegions = ProtobufUtil.getOnlineRegions(rs.getRSRpcServices());
+ for (RegionInfo regionInfo : onlineRegions) {
if (!regionInfo.isMetaTable()) {
if (regionInfo.getRegionNameAsString().contains(name)) {
info = regionInfo;
@@ -369,10 +369,10 @@ public class TestAdmin2 {
final TableName tableName = TableName.valueOf(name.getMethodName());
createTableWithDefaultConf(tableName);
- HRegionInfo info = null;
+ RegionInfo info = null;
HRegionServer rs = TEST_UTIL.getRSForFirstRegionInTable(tableName);
- List<HRegionInfo> onlineRegions = ProtobufUtil.getOnlineRegions(rs.getRSRpcServices());
- for (HRegionInfo regionInfo : onlineRegions) {
+ List<RegionInfo> onlineRegions = ProtobufUtil.getOnlineRegions(rs.getRSRpcServices());
+ for (RegionInfo regionInfo : onlineRegions) {
if (!regionInfo.isMetaTable()) {
if (regionInfo.getRegionNameAsString().contains("TestHBACloseRegion2")) {
info = regionInfo;
@@ -436,7 +436,7 @@ public class TestAdmin2 {
desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
admin.createTable(desc, startKey, endKey, expectedRegions);
- List<HRegionInfo> RegionInfos = admin.getTableRegions(tableName);
+ List<RegionInfo> RegionInfos = admin.getRegions(tableName);
assertEquals("Tried to create " + expectedRegions + " regions " +
"but only found " + RegionInfos.size(),
@@ -449,8 +449,8 @@ public class TestAdmin2 {
HMaster master = cluster.getMaster();
final TableName tableName = TableName.valueOf(name.getMethodName());
Admin localAdmin = createTable(tableName);
- List<HRegionInfo> tableRegions = localAdmin.getTableRegions(tableName);
- HRegionInfo hri = tableRegions.get(0);
+ List<RegionInfo> tableRegions = localAdmin.getRegions(tableName);
+ RegionInfo hri = tableRegions.get(0);
AssignmentManager am = master.getAssignmentManager();
ServerName server = am.getRegionStates().getRegionServerOfRegion(hri);
localAdmin.move(hri.getEncodedNameAsBytes(), Bytes.toBytes(server.getServerName()));
@@ -620,9 +620,9 @@ public class TestAdmin2 {
try (RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName)) {
HRegionLocation regionLocation = locator.getRegionLocation(Bytes.toBytes("mmm"));
- HRegionInfo region = regionLocation.getRegionInfo();
+ RegionInfo region = regionLocation.getRegionInfo();
byte[] regionName = region.getRegionName();
- Pair<HRegionInfo, ServerName> pair = rawAdmin.getRegion(regionName);
+ Pair<RegionInfo, ServerName> pair = rawAdmin.getRegion(regionName);
assertTrue(Bytes.equals(regionName, pair.getFirst().getRegionName()));
pair = rawAdmin.getRegion(region.getEncodedNameAsBytes());
assertTrue(Bytes.equals(regionName, pair.getFirst().getRegionName()));
[13/19] hbase git commit: HBASE-18839 Apply RegionInfo to code base
Posted by ch...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java
index 9e37292..da6afc9 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java
@@ -30,15 +30,15 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.MetaMutationAnnotation;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.UnknownRegionException;
-import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
import org.apache.hadoop.hbase.client.MasterSwitchType;
import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.exceptions.MergeRegionException;
@@ -59,13 +59,16 @@ import org.apache.hadoop.hbase.quotas.QuotaExceededException;
import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
import org.apache.hadoop.hbase.regionserver.HStoreFile;
import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.yetus.audience.InterfaceAudience;
+
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsState;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.hadoop.hbase.util.FSUtils;
/**
* The procedure to Merge a region in a table.
@@ -79,8 +82,8 @@ public class MergeTableRegionsProcedure
private Boolean traceEnabled;
private volatile boolean lock = false;
private ServerName regionLocation;
- private HRegionInfo[] regionsToMerge;
- private HRegionInfo mergedRegion;
+ private RegionInfo[] regionsToMerge;
+ private RegionInfo mergedRegion;
private boolean forcible;
public MergeTableRegionsProcedure() {
@@ -88,18 +91,18 @@ public class MergeTableRegionsProcedure
}
public MergeTableRegionsProcedure(final MasterProcedureEnv env,
- final HRegionInfo regionToMergeA, final HRegionInfo regionToMergeB) throws IOException {
+ final RegionInfo regionToMergeA, final RegionInfo regionToMergeB) throws IOException {
this(env, regionToMergeA, regionToMergeB, false);
}
public MergeTableRegionsProcedure(final MasterProcedureEnv env,
- final HRegionInfo regionToMergeA, final HRegionInfo regionToMergeB,
+ final RegionInfo regionToMergeA, final RegionInfo regionToMergeB,
final boolean forcible) throws MergeRegionException {
- this(env, new HRegionInfo[] {regionToMergeA, regionToMergeB}, forcible);
+ this(env, new RegionInfo[] {regionToMergeA, regionToMergeB}, forcible);
}
public MergeTableRegionsProcedure(final MasterProcedureEnv env,
- final HRegionInfo[] regionsToMerge, final boolean forcible)
+ final RegionInfo[] regionsToMerge, final boolean forcible)
throws MergeRegionException {
super(env);
@@ -117,7 +120,7 @@ public class MergeTableRegionsProcedure
this.forcible = forcible;
}
- private static void checkRegionsToMerge(final HRegionInfo[] regionsToMerge,
+ private static void checkRegionsToMerge(final RegionInfo[] regionsToMerge,
final boolean forcible) throws MergeRegionException {
// For now, we only merge 2 regions.
// It could be extended to more than 2 regions in the future.
@@ -129,19 +132,19 @@ public class MergeTableRegionsProcedure
checkRegionsToMerge(regionsToMerge[0], regionsToMerge[1], forcible);
}
- private static void checkRegionsToMerge(final HRegionInfo regionToMergeA,
- final HRegionInfo regionToMergeB, final boolean forcible) throws MergeRegionException {
+ private static void checkRegionsToMerge(final RegionInfo regionToMergeA,
+ final RegionInfo regionToMergeB, final boolean forcible) throws MergeRegionException {
if (!regionToMergeA.getTable().equals(regionToMergeB.getTable())) {
throw new MergeRegionException("Can't merge regions from two different tables: " +
regionToMergeA + ", " + regionToMergeB);
}
- if (regionToMergeA.getReplicaId() != HRegionInfo.DEFAULT_REPLICA_ID ||
- regionToMergeB.getReplicaId() != HRegionInfo.DEFAULT_REPLICA_ID) {
+ if (regionToMergeA.getReplicaId() != RegionInfo.DEFAULT_REPLICA_ID ||
+ regionToMergeB.getReplicaId() != RegionInfo.DEFAULT_REPLICA_ID) {
throw new MergeRegionException("Can't merge non-default replicas");
}
- if (!HRegionInfo.areAdjacent(regionToMergeA, regionToMergeB)) {
+ if (!RegionInfo.areAdjacent(regionToMergeA, regionToMergeB)) {
String msg = "Unable to merge not adjacent regions " + regionToMergeA.getShortNameToLog() +
", " + regionToMergeB.getShortNameToLog() + " where forcible = " + forcible;
LOG.warn(msg);
@@ -151,18 +154,18 @@ public class MergeTableRegionsProcedure
}
}
- private static HRegionInfo createMergedRegionInfo(final HRegionInfo[] regionsToMerge) {
+ private static RegionInfo createMergedRegionInfo(final RegionInfo[] regionsToMerge) {
return createMergedRegionInfo(regionsToMerge[0], regionsToMerge[1]);
}
/**
* Create merged region info through the specified two regions
*/
- private static HRegionInfo createMergedRegionInfo(final HRegionInfo regionToMergeA,
- final HRegionInfo regionToMergeB) {
+ private static RegionInfo createMergedRegionInfo(final RegionInfo regionToMergeA,
+ final RegionInfo regionToMergeB) {
// Choose the smaller as start key
final byte[] startKey;
- if (regionToMergeA.compareTo(regionToMergeB) <= 0) {
+ if (RegionInfo.COMPARATOR.compare(regionToMergeA, regionToMergeB) <= 0) {
startKey = regionToMergeA.getStartKey();
} else {
startKey = regionToMergeB.getStartKey();
@@ -179,12 +182,16 @@ public class MergeTableRegionsProcedure
}
// Merged region is sorted between two merging regions in META
- final long rid = getMergedRegionIdTimestamp(regionToMergeA, regionToMergeB);
- return new HRegionInfo(regionToMergeA.getTable(), startKey, endKey, false, rid);
+ return RegionInfoBuilder.newBuilder(regionToMergeA.getTable())
+ .setStartKey(startKey)
+ .setEndKey(endKey)
+ .setSplit(false)
+ .setRegionId(getMergedRegionIdTimestamp(regionToMergeA, regionToMergeB))
+ .build();
}
- private static long getMergedRegionIdTimestamp(final HRegionInfo regionToMergeA,
- final HRegionInfo regionToMergeB) {
+ private static long getMergedRegionIdTimestamp(final RegionInfo regionToMergeA,
+ final RegionInfo regionToMergeB) {
long rid = EnvironmentEdgeManager.currentTime();
// Regionid is timestamp. Merged region's id can't be less than that of
// merging regions else will insert at wrong location in hbase:meta (See HBASE-710).
@@ -252,7 +259,7 @@ public class MergeTableRegionsProcedure
throw new UnsupportedOperationException(this + " unhandled state=" + state);
}
} catch (IOException e) {
- LOG.warn("Error trying to merge regions " + HRegionInfo.getShortNameToLog(regionsToMerge) +
+ LOG.warn("Error trying to merge regions " + RegionInfo.getShortNameToLog(regionsToMerge) +
" in the table " + getTableName() + " (in state=" + state + ")", e);
setFailure("master-merge-regions", e);
@@ -305,7 +312,7 @@ public class MergeTableRegionsProcedure
// This will be retried. Unless there is a bug in the code,
// this should be just a "temporary error" (e.g. network down)
LOG.warn("Failed rollback attempt step " + state + " for merging the regions "
- + HRegionInfo.getShortNameToLog(regionsToMerge) + " in table " + getTableName(), e);
+ + RegionInfo.getShortNameToLog(regionsToMerge) + " in table " + getTableName(), e);
throw e;
}
}
@@ -351,10 +358,10 @@ public class MergeTableRegionsProcedure
final MasterProcedureProtos.MergeTableRegionsStateData.Builder mergeTableRegionsMsg =
MasterProcedureProtos.MergeTableRegionsStateData.newBuilder()
.setUserInfo(MasterProcedureUtil.toProtoUserInfo(getUser()))
- .setMergedRegionInfo(HRegionInfo.convert(mergedRegion))
+ .setMergedRegionInfo(ProtobufUtil.toRegionInfo(mergedRegion))
.setForcible(forcible);
for (int i = 0; i < regionsToMerge.length; ++i) {
- mergeTableRegionsMsg.addRegionInfo(HRegionInfo.convert(regionsToMerge[i]));
+ mergeTableRegionsMsg.addRegionInfo(ProtobufUtil.toRegionInfo(regionsToMerge[i]));
}
serializer.serialize(mergeTableRegionsMsg.build());
}
@@ -369,12 +376,12 @@ public class MergeTableRegionsProcedure
setUser(MasterProcedureUtil.toUserInfo(mergeTableRegionsMsg.getUserInfo()));
assert(mergeTableRegionsMsg.getRegionInfoCount() == 2);
- regionsToMerge = new HRegionInfo[mergeTableRegionsMsg.getRegionInfoCount()];
+ regionsToMerge = new RegionInfo[mergeTableRegionsMsg.getRegionInfoCount()];
for (int i = 0; i < regionsToMerge.length; i++) {
- regionsToMerge[i] = HRegionInfo.convert(mergeTableRegionsMsg.getRegionInfo(i));
+ regionsToMerge[i] = ProtobufUtil.toRegionInfo(mergeTableRegionsMsg.getRegionInfo(i));
}
- mergedRegion = HRegionInfo.convert(mergeTableRegionsMsg.getMergedRegionInfo());
+ mergedRegion = ProtobufUtil.toRegionInfo(mergeTableRegionsMsg.getMergedRegionInfo());
}
@Override
@@ -383,7 +390,7 @@ public class MergeTableRegionsProcedure
sb.append(" table=");
sb.append(getTableName());
sb.append(", regions=");
- sb.append(HRegionInfo.getShortNameToLog(regionsToMerge));
+ sb.append(RegionInfo.getShortNameToLog(regionsToMerge));
sb.append(", forcibly=");
sb.append(forcible);
}
@@ -450,7 +457,7 @@ public class MergeTableRegionsProcedure
boolean regionAHasMergeQualifier = !catalogJanitor.cleanMergeQualifier(regionsToMerge[0]);
if (regionAHasMergeQualifier
|| !catalogJanitor.cleanMergeQualifier(regionsToMerge[1])) {
- String msg = "Skip merging regions " + HRegionInfo.getShortNameToLog(regionsToMerge) +
+ String msg = "Skip merging regions " + RegionInfo.getShortNameToLog(regionsToMerge) +
", because region "
+ (regionAHasMergeQualifier ? regionsToMerge[0].getEncodedName() : regionsToMerge[1]
.getEncodedName()) + " has merge qualifier";
@@ -526,7 +533,7 @@ public class MergeTableRegionsProcedure
boolean ret = cpHost.preMergeRegionsAction(regionsToMerge, getUser());
if (ret) {
throw new IOException(
- "Coprocessor bypassing regions " + HRegionInfo.getShortNameToLog(regionsToMerge) +
+ "Coprocessor bypassing regions " + RegionInfo.getShortNameToLog(regionsToMerge) +
" merge.");
}
}
@@ -649,7 +656,7 @@ public class MergeTableRegionsProcedure
int procsIdx = 0;
for (int i = 0; i < regionsToMerge.length; ++i) {
for (int j = 0; j < regionReplication; ++j) {
- final HRegionInfo hri = RegionReplicaUtil.getRegionInfoForReplica(regionsToMerge[i], j);
+ final RegionInfo hri = RegionReplicaUtil.getRegionInfoForReplica(regionsToMerge[i], j);
procs[procsIdx++] = env.getAssignmentManager().createAssignProcedure(hri, serverName);
}
}
@@ -663,7 +670,7 @@ public class MergeTableRegionsProcedure
int procsIdx = 0;
for (int i = 0; i < regionsToMerge.length; ++i) {
for (int j = 0; j < regionReplication; ++j) {
- final HRegionInfo hri = RegionReplicaUtil.getRegionInfoForReplica(regionsToMerge[i], j);
+ final RegionInfo hri = RegionReplicaUtil.getRegionInfoForReplica(regionsToMerge[i], j);
procs[procsIdx++] = env.getAssignmentManager().createUnassignProcedure(hri,null,true);
}
}
@@ -675,7 +682,7 @@ public class MergeTableRegionsProcedure
final ServerName targetServer = getServerName(env);
final AssignProcedure[] procs = new AssignProcedure[regionReplication];
for (int i = 0; i < procs.length; ++i) {
- final HRegionInfo hri = RegionReplicaUtil.getRegionInfoForReplica(mergedRegion, i);
+ final RegionInfo hri = RegionReplicaUtil.getRegionInfoForReplica(mergedRegion, i);
procs[i] = env.getAssignmentManager().createAssignProcedure(hri, targetServer);
}
return procs;
@@ -699,12 +706,12 @@ public class MergeTableRegionsProcedure
if (ret) {
throw new IOException(
- "Coprocessor bypassing regions " + HRegionInfo.getShortNameToLog(regionsToMerge) +
+ "Coprocessor bypassing regions " + RegionInfo.getShortNameToLog(regionsToMerge) +
" merge.");
}
try {
for (Mutation p : metaEntries) {
- HRegionInfo.parseRegionName(p.getRow());
+ RegionInfo.parseRegionName(p.getRow());
}
} catch (IOException e) {
LOG.error("Row key of mutation from coprocessor is not parsable as region name."
@@ -780,7 +787,7 @@ public class MergeTableRegionsProcedure
* @return The merged region. Maybe be null if called to early or we failed.
*/
@VisibleForTesting
- public HRegionInfo getMergedRegion() {
+ public RegionInfo getMergedRegion() {
return this.mergedRegion;
}
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MoveRegionProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MoveRegionProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MoveRegionProcedure.java
index b07298e..624806a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MoveRegionProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MoveRegionProcedure.java
@@ -23,14 +23,15 @@ import java.io.IOException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.master.RegionPlan;
import org.apache.hadoop.hbase.master.procedure.AbstractStateMachineRegionProcedure;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
+import org.apache.yetus.audience.InterfaceAudience;
+
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MoveRegionState;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MoveRegionStateData;
@@ -128,7 +129,7 @@ public class MoveRegionProcedure extends AbstractStateMachineRegionProcedure<Mov
super.serializeStateData(serializer);
final MoveRegionStateData.Builder state = MoveRegionStateData.newBuilder()
- // No need to serialize the HRegionInfo. The super class has the region.
+ // No need to serialize the RegionInfo. The super class has the region.
.setSourceServer(ProtobufUtil.toServerName(plan.getSource()));
if (plan.getDestination() != null) {
state.setDestinationServer(ProtobufUtil.toServerName(plan.getDestination()));
@@ -143,7 +144,7 @@ public class MoveRegionProcedure extends AbstractStateMachineRegionProcedure<Mov
super.deserializeStateData(serializer);
final MoveRegionStateData state = serializer.deserialize(MoveRegionStateData.class);
- final HRegionInfo regionInfo = getRegion(); // Get it from super class deserialization.
+ final RegionInfo regionInfo = getRegion(); // Get it from super class deserialization.
final ServerName sourceServer = ProtobufUtil.toServerName(state.getSourceServer());
final ServerName destinationServer = state.hasDestinationServer() ?
ProtobufUtil.toServerName(state.getDestinationServer()) : null;
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/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 1f43ad3..f9a1b43 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
@@ -28,24 +28,24 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.RegionLocations;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
-import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.master.RegionState;
import org.apache.hadoop.hbase.master.RegionState.State;
import org.apache.hadoop.hbase.procedure2.util.StringUtils;
-import org.apache.hadoop.hbase.util.MultiHConnection;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.MultiHConnection;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
+import org.apache.yetus.audience.InterfaceAudience;
import org.apache.zookeeper.KeeperException;
import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
@@ -79,7 +79,7 @@ public class RegionStateStore {
}
public interface RegionStateVisitor {
- void visitRegionState(HRegionInfo regionInfo, State state,
+ void visitRegionState(RegionInfo regionInfo, State state,
ServerName regionLocation, ServerName lastHost, long openSeqNum);
}
@@ -114,7 +114,7 @@ public class RegionStateStore {
final HRegionLocation hrl = locations[i];
if (hrl == null) continue;
- final HRegionInfo regionInfo = hrl.getRegionInfo();
+ final RegionInfo regionInfo = hrl.getRegionInfo();
if (regionInfo == null) continue;
final int replicaId = regionInfo.getReplicaId();
@@ -132,7 +132,7 @@ public class RegionStateStore {
}
}
- public void updateRegionLocation(final HRegionInfo regionInfo, final State state,
+ public void updateRegionLocation(final RegionInfo regionInfo, final State state,
final ServerName regionLocation, final ServerName lastHost, final long openSeqNum,
final long pid)
throws IOException {
@@ -149,7 +149,7 @@ public class RegionStateStore {
oldState != null ? oldState.getServerName() : null, openSeqNum, pid);
}
- protected void updateMetaLocation(final HRegionInfo regionInfo, final ServerName serverName)
+ protected void updateMetaLocation(final RegionInfo regionInfo, final ServerName serverName)
throws IOException {
try {
MetaTableLocator.setMetaLocation(master.getZooKeeper(), serverName,
@@ -159,7 +159,7 @@ public class RegionStateStore {
}
}
- protected void updateUserRegionLocation(final HRegionInfo regionInfo, final State state,
+ protected void updateUserRegionLocation(final RegionInfo regionInfo, final State state,
final ServerName regionLocation, final ServerName lastHost, final long openSeqNum,
final long pid)
throws IOException {
@@ -195,7 +195,7 @@ public class RegionStateStore {
}
}
- protected void updateRegionLocation(final HRegionInfo regionInfo, final State state,
+ protected void updateRegionLocation(final RegionInfo regionInfo, final State state,
final Put... put) throws IOException {
synchronized (this) {
if (multiHConnection == null) {
@@ -219,8 +219,8 @@ public class RegionStateStore {
// ============================================================================================
// Update Region Splitting State helpers
// ============================================================================================
- public void splitRegion(final HRegionInfo parent, final HRegionInfo hriA,
- final HRegionInfo hriB, final ServerName serverName) throws IOException {
+ public void splitRegion(final RegionInfo parent, final RegionInfo hriA,
+ final RegionInfo hriB, final ServerName serverName) throws IOException {
final TableDescriptor htd = getTableDescriptor(parent.getTable());
MetaTableAccessor.splitRegion(master.getConnection(), parent, hriA, hriB, serverName,
getRegionReplication(htd), hasSerialReplicationScope(htd));
@@ -229,8 +229,8 @@ public class RegionStateStore {
// ============================================================================================
// Update Region Merging State helpers
// ============================================================================================
- public void mergeRegions(final HRegionInfo parent, final HRegionInfo hriA,
- final HRegionInfo hriB, final ServerName serverName) throws IOException {
+ public void mergeRegions(final RegionInfo parent, final RegionInfo hriA,
+ final RegionInfo hriB, final ServerName serverName) throws IOException {
final TableDescriptor htd = getTableDescriptor(parent.getTable());
MetaTableAccessor.mergeRegions(master.getConnection(), parent, hriA, hriB, serverName,
getRegionReplication(htd), EnvironmentEdgeManager.currentTime(),
@@ -240,11 +240,11 @@ public class RegionStateStore {
// ============================================================================================
// Delete Region State helpers
// ============================================================================================
- public void deleteRegion(final HRegionInfo regionInfo) throws IOException {
+ public void deleteRegion(final RegionInfo regionInfo) throws IOException {
deleteRegions(Collections.singletonList(regionInfo));
}
- public void deleteRegions(final List<HRegionInfo> regions) throws IOException {
+ public void deleteRegions(final List<RegionInfo> regions) throws IOException {
MetaTableAccessor.deleteRegions(master.getConnection(), regions);
}
@@ -300,7 +300,7 @@ public class RegionStateStore {
return replicaId == 0
? HConstants.SERVERNAME_QUALIFIER
: Bytes.toBytes(HConstants.SERVERNAME_QUALIFIER_STR + META_REPLICA_ID_DELIMITER
- + String.format(HRegionInfo.REPLICA_ID_FORMAT, replicaId));
+ + String.format(RegionInfo.REPLICA_ID_FORMAT, replicaId));
}
// ==========================================================================
@@ -322,6 +322,6 @@ public class RegionStateStore {
return replicaId == 0
? HConstants.STATE_QUALIFIER
: Bytes.toBytes(HConstants.STATE_QUALIFIER_STR + META_REPLICA_ID_DELIMITER
- + String.format(HRegionInfo.REPLICA_ID_FORMAT, replicaId));
+ + String.format(RegionInfo.REPLICA_ID_FORMAT, replicaId));
}
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStates.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStates.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStates.java
index 3a9c34a..c13a49d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStates.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStates.java
@@ -19,21 +19,6 @@
package org.apache.hadoop.hbase.master.assignment;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.exceptions.UnexpectedStateException;
-import org.apache.hadoop.hbase.master.RegionState;
-import org.apache.hadoop.hbase.master.RegionState.State;
-import org.apache.hadoop.hbase.procedure2.ProcedureEvent;
-import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.yetus.audience.InterfaceAudience;
-
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
@@ -50,6 +35,22 @@ import java.util.TreeSet;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentSkipListMap;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.exceptions.UnexpectedStateException;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.master.RegionState.State;
+import org.apache.hadoop.hbase.procedure2.ProcedureEvent;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
+
/**
* RegionStates contains a set of Maps that describes the in-memory state of the AM, with
* the regions available in the system, the region in transition, the offline regions and
@@ -71,8 +72,8 @@ public class RegionStates {
State.CLOSING // already in-progress (retrying)
};
- private static class AssignmentProcedureEvent extends ProcedureEvent<HRegionInfo> {
- public AssignmentProcedureEvent(final HRegionInfo regionInfo) {
+ private static class AssignmentProcedureEvent extends ProcedureEvent<RegionInfo> {
+ public AssignmentProcedureEvent(final RegionInfo regionInfo) {
super(regionInfo);
}
}
@@ -96,7 +97,7 @@ public class RegionStates {
// so for now. Odd is that elsewhere in this RegionStates, we synchronize on
// the RegionStateNode instance. TODO.
public static class RegionStateNode implements Comparable<RegionStateNode> {
- private final HRegionInfo regionInfo;
+ private final RegionInfo regionInfo;
private final ProcedureEvent<?> event;
private volatile RegionTransitionProcedure procedure = null;
@@ -117,7 +118,7 @@ public class RegionStates {
private volatile long openSeqNum = HConstants.NO_SEQNUM;
- public RegionStateNode(final HRegionInfo regionInfo) {
+ public RegionStateNode(final RegionInfo regionInfo) {
this.regionInfo = regionInfo;
this.event = new AssignmentProcedureEvent(regionInfo);
}
@@ -184,7 +185,6 @@ public class RegionStates {
this.openSeqNum = seqId;
}
-
public ServerName setRegionLocation(final ServerName serverName) {
ServerName lastRegionLocation = this.regionLocation;
if (LOG.isTraceEnabled() && serverName == null) {
@@ -219,7 +219,7 @@ public class RegionStates {
return event;
}
- public HRegionInfo getRegionInfo() {
+ public RegionInfo getRegionInfo() {
return regionInfo;
}
@@ -255,9 +255,9 @@ public class RegionStates {
@Override
public int compareTo(final RegionStateNode other) {
- // NOTE: HRegionInfo sort by table first, so we are relying on that.
+ // NOTE: RegionInfo sort by table first, so we are relying on that.
// we have a TestRegionState#testOrderedByTable() that check for that.
- return getRegionInfo().compareTo(other.getRegionInfo());
+ return RegionInfo.COMPARATOR.compare(getRegionInfo(), other.getRegionInfo());
}
@Override
@@ -276,7 +276,7 @@ public class RegionStates {
public String toString() {
return toDescriptiveString();
}
-
+
public String toShortString() {
// rit= is the current Region-In-Transition State -- see State enum.
return String.format("rit=%s, location=%s", getState(), getRegionLocation());
@@ -295,7 +295,7 @@ public class RegionStates {
@Override
public int compare(final RegionState l, final RegionState r) {
int stampCmp = Long.compare(l.getStamp(), r.getStamp());
- return stampCmp != 0 ? stampCmp : l.getRegion().compareTo(r.getRegion());
+ return stampCmp != 0 ? stampCmp : RegionInfo.COMPARATOR.compare(l.getRegion(), r.getRegion());
}
}
@@ -357,8 +357,8 @@ public class RegionStates {
return regions.size();
}
- public ArrayList<HRegionInfo> getRegionInfoList() {
- ArrayList<HRegionInfo> hris = new ArrayList<HRegionInfo>(regions.size());
+ public ArrayList<RegionInfo> getRegionInfoList() {
+ ArrayList<RegionInfo> hris = new ArrayList<RegionInfo>(regions.size());
for (RegionStateNode region: regions) {
hris.add(region.getRegionInfo());
}
@@ -401,20 +401,20 @@ public class RegionStates {
// TODO: Replace the ConcurrentSkipListMaps
/**
- * RegionName -- i.e. HRegionInfo.getRegionName() -- as bytes to {@link RegionStateNode}
+ * RegionName -- i.e. RegionInfo.getRegionName() -- as bytes to {@link RegionStateNode}
*/
private final ConcurrentSkipListMap<byte[], RegionStateNode> regionsMap =
new ConcurrentSkipListMap<byte[], RegionStateNode>(Bytes.BYTES_COMPARATOR);
- private final ConcurrentSkipListMap<HRegionInfo, RegionStateNode> regionInTransition =
- new ConcurrentSkipListMap<HRegionInfo, RegionStateNode>();
+ private final ConcurrentSkipListMap<RegionInfo, RegionStateNode> regionInTransition =
+ new ConcurrentSkipListMap<RegionInfo, RegionStateNode>(RegionInfo.COMPARATOR);
/**
* Regions marked as offline on a read of hbase:meta. Unused or at least, once
* offlined, regions have no means of coming on line again. TODO.
*/
- private final ConcurrentSkipListMap<HRegionInfo, RegionStateNode> regionOffline =
- new ConcurrentSkipListMap<HRegionInfo, RegionStateNode>();
+ private final ConcurrentSkipListMap<RegionInfo, RegionStateNode> regionOffline =
+ new ConcurrentSkipListMap<RegionInfo, RegionStateNode>();
private final ConcurrentSkipListMap<byte[], RegionFailedOpen> regionFailedOpen =
new ConcurrentSkipListMap<byte[], RegionFailedOpen>(Bytes.BYTES_COMPARATOR);
@@ -432,7 +432,7 @@ public class RegionStates {
}
@VisibleForTesting
- public boolean isRegionInRegionStates(final HRegionInfo hri) {
+ public boolean isRegionInRegionStates(final RegionInfo hri) {
return (regionsMap.containsKey(hri.getRegionName()) || regionInTransition.containsKey(hri)
|| regionOffline.containsKey(hri));
}
@@ -440,13 +440,13 @@ public class RegionStates {
// ==========================================================================
// RegionStateNode helpers
// ==========================================================================
- protected RegionStateNode createRegionNode(final HRegionInfo regionInfo) {
+ protected RegionStateNode createRegionNode(final RegionInfo regionInfo) {
RegionStateNode newNode = new RegionStateNode(regionInfo);
RegionStateNode oldNode = regionsMap.putIfAbsent(regionInfo.getRegionName(), newNode);
return oldNode != null ? oldNode : newNode;
}
- protected RegionStateNode getOrCreateRegionNode(final HRegionInfo regionInfo) {
+ protected RegionStateNode getOrCreateRegionNode(final RegionInfo regionInfo) {
RegionStateNode node = regionsMap.get(regionInfo.getRegionName());
return node != null ? node : createRegionNode(regionInfo);
}
@@ -455,7 +455,7 @@ public class RegionStates {
return regionsMap.get(regionName);
}
- protected RegionStateNode getRegionNode(final HRegionInfo regionInfo) {
+ protected RegionStateNode getRegionNode(final RegionInfo regionInfo) {
return getRegionNodeFromName(regionInfo.getRegionName());
}
@@ -469,7 +469,7 @@ public class RegionStates {
return null;
}
- public void deleteRegion(final HRegionInfo regionInfo) {
+ public void deleteRegion(final RegionInfo regionInfo) {
regionsMap.remove(regionInfo.getRegionName());
// Remove from the offline regions map too if there.
if (this.regionOffline.containsKey(regionInfo)) {
@@ -496,8 +496,8 @@ public class RegionStates {
return regions;
}
- ArrayList<HRegionInfo> getTableRegionsInfo(final TableName tableName) {
- final ArrayList<HRegionInfo> regions = new ArrayList<HRegionInfo>();
+ ArrayList<RegionInfo> getTableRegionsInfo(final TableName tableName) {
+ final ArrayList<RegionInfo> regions = new ArrayList<RegionInfo>();
for (RegionStateNode node: regionsMap.tailMap(tableName.getName()).values()) {
if (!node.getTable().equals(tableName)) break;
regions.add(node.getRegionInfo());
@@ -520,7 +520,7 @@ public class RegionStates {
// ==========================================================================
// RegionState helpers
// ==========================================================================
- public RegionState getRegionState(final HRegionInfo regionInfo) {
+ public RegionState getRegionState(final RegionInfo regionInfo) {
return createRegionState(getRegionNode(regionInfo));
}
@@ -542,13 +542,13 @@ public class RegionStates {
return !getTableRegionStates(tableName).isEmpty();
}
- public List<HRegionInfo> getRegionsOfTable(final TableName table) {
+ public List<RegionInfo> getRegionsOfTable(final TableName table) {
return getRegionsOfTable(table, false);
}
- List<HRegionInfo> getRegionsOfTable(final TableName table, final boolean offline) {
+ List<RegionInfo> getRegionsOfTable(final TableName table, final boolean offline) {
final ArrayList<RegionStateNode> nodes = getTableRegionStateNodes(table);
- final ArrayList<HRegionInfo> hris = new ArrayList<HRegionInfo>(nodes.size());
+ final ArrayList<RegionInfo> hris = new ArrayList<RegionInfo>(nodes.size());
for (RegionStateNode node: nodes) {
if (include(node, offline)) hris.add(node.getRegionInfo());
}
@@ -567,7 +567,7 @@ public class RegionStates {
}
if (node.isInState(State.SPLIT)) return false;
if (node.isInState(State.OFFLINE) && !offline) return false;
- final HRegionInfo hri = node.getRegionInfo();
+ final RegionInfo hri = node.getRegionInfo();
return (!hri.isOffline() && !hri.isSplit()) ||
((hri.isOffline() || hri.isSplit()) && offline);
}
@@ -575,9 +575,9 @@ public class RegionStates {
/**
* Returns the set of regions hosted by the specified server
* @param serverName the server we are interested in
- * @return set of HRegionInfo hosted by the specified server
+ * @return set of RegionInfo hosted by the specified server
*/
- public List<HRegionInfo> getServerRegionInfoSet(final ServerName serverName) {
+ public List<RegionInfo> getServerRegionInfoSet(final ServerName serverName) {
final ServerStateNode serverInfo = getServerNode(serverName);
if (serverInfo == null) return Collections.emptyList();
@@ -603,7 +603,7 @@ public class RegionStates {
}
}
- public void logSplit(final HRegionInfo regionInfo) {
+ public void logSplit(final RegionInfo regionInfo) {
final RegionStateNode regionNode = getRegionNode(regionInfo);
synchronized (regionNode) {
regionNode.setState(State.SPLIT);
@@ -611,7 +611,7 @@ public class RegionStates {
}
@VisibleForTesting
- public void updateRegionState(final HRegionInfo regionInfo, final State state) {
+ public void updateRegionState(final RegionInfo regionInfo, final State state) {
final RegionStateNode regionNode = getOrCreateRegionNode(regionInfo);
synchronized (regionNode) {
regionNode.setState(state);
@@ -621,8 +621,8 @@ public class RegionStates {
// ============================================================================================
// TODO:
// ============================================================================================
- public List<HRegionInfo> getAssignedRegions() {
- final List<HRegionInfo> result = new ArrayList<HRegionInfo>();
+ public List<RegionInfo> getAssignedRegions() {
+ final List<RegionInfo> result = new ArrayList<RegionInfo>();
for (RegionStateNode node: regionsMap.values()) {
if (!node.isInTransition()) {
result.add(node.getRegionInfo());
@@ -631,7 +631,7 @@ public class RegionStates {
return result;
}
- public boolean isRegionInState(final HRegionInfo regionInfo, final State... state) {
+ public boolean isRegionInState(final RegionInfo regionInfo, final State... state) {
final RegionStateNode region = getRegionNode(regionInfo);
if (region != null) {
synchronized (region) {
@@ -641,21 +641,21 @@ public class RegionStates {
return false;
}
- public boolean isRegionOnline(final HRegionInfo regionInfo) {
+ public boolean isRegionOnline(final RegionInfo regionInfo) {
return isRegionInState(regionInfo, State.OPEN);
}
/**
* @return True if region is offline (In OFFLINE or CLOSED state).
*/
- public boolean isRegionOffline(final HRegionInfo regionInfo) {
+ public boolean isRegionOffline(final RegionInfo regionInfo) {
return isRegionInState(regionInfo, State.OFFLINE, State.CLOSED);
}
- public Map<ServerName, List<HRegionInfo>> getSnapShotOfAssignment(
- final Collection<HRegionInfo> regions) {
- final Map<ServerName, List<HRegionInfo>> result = new HashMap<ServerName, List<HRegionInfo>>();
- for (HRegionInfo hri: regions) {
+ public Map<ServerName, List<RegionInfo>> getSnapShotOfAssignment(
+ final Collection<RegionInfo> regions) {
+ final Map<ServerName, List<RegionInfo>> result = new HashMap<ServerName, List<RegionInfo>>();
+ for (RegionInfo hri: regions) {
final RegionStateNode node = getRegionNode(hri);
if (node == null) continue;
@@ -663,9 +663,9 @@ public class RegionStates {
final ServerName serverName = node.getRegionLocation();
if (serverName == null) continue;
- List<HRegionInfo> serverRegions = result.get(serverName);
+ List<RegionInfo> serverRegions = result.get(serverName);
if (serverRegions == null) {
- serverRegions = new ArrayList<HRegionInfo>();
+ serverRegions = new ArrayList<RegionInfo>();
result.put(serverName, serverRegions);
}
@@ -674,20 +674,20 @@ public class RegionStates {
return result;
}
- public Map<HRegionInfo, ServerName> getRegionAssignments() {
- final HashMap<HRegionInfo, ServerName> assignments = new HashMap<HRegionInfo, ServerName>();
+ public Map<RegionInfo, ServerName> getRegionAssignments() {
+ final HashMap<RegionInfo, ServerName> assignments = new HashMap<RegionInfo, ServerName>();
for (RegionStateNode node: regionsMap.values()) {
assignments.put(node.getRegionInfo(), node.getRegionLocation());
}
return assignments;
}
- public Map<RegionState.State, List<HRegionInfo>> getRegionByStateOfTable(TableName tableName) {
+ public Map<RegionState.State, List<RegionInfo>> getRegionByStateOfTable(TableName tableName) {
final State[] states = State.values();
- final Map<RegionState.State, List<HRegionInfo>> tableRegions =
- new HashMap<State, List<HRegionInfo>>(states.length);
+ final Map<RegionState.State, List<RegionInfo>> tableRegions =
+ new HashMap<State, List<RegionInfo>>(states.length);
for (int i = 0; i < states.length; ++i) {
- tableRegions.put(states[i], new ArrayList<HRegionInfo>());
+ tableRegions.put(states[i], new ArrayList<RegionInfo>());
}
for (RegionStateNode node: regionsMap.values()) {
@@ -698,7 +698,7 @@ public class RegionStates {
return tableRegions;
}
- public ServerName getRegionServerOfRegion(final HRegionInfo regionInfo) {
+ public ServerName getRegionServerOfRegion(final RegionInfo regionInfo) {
final RegionStateNode region = getRegionNode(regionInfo);
if (region != null) {
synchronized (region) {
@@ -717,29 +717,29 @@ public class RegionStates {
* @param forceByCluster a flag to force to aggregate the server-load to the cluster level
* @return A clone of current assignments by table.
*/
- public Map<TableName, Map<ServerName, List<HRegionInfo>>> getAssignmentsByTable(
+ public Map<TableName, Map<ServerName, List<RegionInfo>>> getAssignmentsByTable(
final boolean forceByCluster) {
if (!forceByCluster) return getAssignmentsByTable();
- final HashMap<ServerName, List<HRegionInfo>> ensemble =
- new HashMap<ServerName, List<HRegionInfo>>(serverMap.size());
+ final HashMap<ServerName, List<RegionInfo>> ensemble =
+ new HashMap<ServerName, List<RegionInfo>>(serverMap.size());
for (ServerStateNode serverNode: serverMap.values()) {
ensemble.put(serverNode.getServerName(), serverNode.getRegionInfoList());
}
// TODO: can we use Collections.singletonMap(HConstants.ENSEMBLE_TABLE_NAME, ensemble)?
- final Map<TableName, Map<ServerName, List<HRegionInfo>>> result =
- new HashMap<TableName, Map<ServerName, List<HRegionInfo>>>(1);
+ final Map<TableName, Map<ServerName, List<RegionInfo>>> result =
+ new HashMap<TableName, Map<ServerName, List<RegionInfo>>>(1);
result.put(HConstants.ENSEMBLE_TABLE_NAME, ensemble);
return result;
}
- public Map<TableName, Map<ServerName, List<HRegionInfo>>> getAssignmentsByTable() {
- final Map<TableName, Map<ServerName, List<HRegionInfo>>> result = new HashMap<>();
+ public Map<TableName, Map<ServerName, List<RegionInfo>>> getAssignmentsByTable() {
+ final Map<TableName, Map<ServerName, List<RegionInfo>>> result = new HashMap<>();
for (RegionStateNode node: regionsMap.values()) {
- Map<ServerName, List<HRegionInfo>> tableResult = result.get(node.getTable());
+ Map<ServerName, List<RegionInfo>> tableResult = result.get(node.getTable());
if (tableResult == null) {
- tableResult = new HashMap<ServerName, List<HRegionInfo>>();
+ tableResult = new HashMap<ServerName, List<RegionInfo>>();
result.put(node.getTable(), tableResult);
}
@@ -748,9 +748,9 @@ public class RegionStates {
LOG.info("Skipping, no server for " + node);
continue;
}
- List<HRegionInfo> serverResult = tableResult.get(serverName);
+ List<RegionInfo> serverResult = tableResult.get(serverName);
if (serverResult == null) {
- serverResult = new ArrayList<HRegionInfo>();
+ serverResult = new ArrayList<RegionInfo>();
tableResult.put(serverName, serverResult);
}
@@ -780,7 +780,7 @@ public class RegionStates {
return !regionInTransition.isEmpty();
}
- public boolean isRegionInTransition(final HRegionInfo regionInfo) {
+ public boolean isRegionInTransition(final RegionInfo regionInfo) {
final RegionStateNode node = regionInTransition.get(regionInfo);
return node != null ? node.isInTransition() : false;
}
@@ -788,13 +788,13 @@ public class RegionStates {
/**
* @return If a procedure-in-transition for <code>hri</code>, return it else null.
*/
- public RegionTransitionProcedure getRegionTransitionProcedure(final HRegionInfo hri) {
+ public RegionTransitionProcedure getRegionTransitionProcedure(final RegionInfo hri) {
RegionStateNode node = regionInTransition.get(hri);
if (node == null) return null;
return node.getProcedure();
}
- public RegionState getRegionTransitionState(final HRegionInfo hri) {
+ public RegionState getRegionTransitionState(final RegionInfo hri) {
RegionStateNode node = regionInTransition.get(hri);
if (node == null) return null;
@@ -840,7 +840,7 @@ public class RegionStates {
}
// TODO: Unused.
- public void removeFromOfflineRegions(final HRegionInfo regionInfo) {
+ public void removeFromOfflineRegions(final RegionInfo regionInfo) {
regionOffline.remove(regionInfo);
}
@@ -861,7 +861,7 @@ public class RegionStates {
return regionNode;
}
- public HRegionInfo getRegionInfo() {
+ public RegionInfo getRegionInfo() {
return regionNode.getRegionInfo();
}
@@ -893,11 +893,11 @@ public class RegionStates {
return node;
}
- public RegionFailedOpen getFailedOpen(final HRegionInfo regionInfo) {
+ public RegionFailedOpen getFailedOpen(final RegionInfo regionInfo) {
return regionFailedOpen.get(regionInfo.getRegionName());
}
- public void removeFromFailedOpen(final HRegionInfo regionInfo) {
+ public void removeFromFailedOpen(final RegionInfo regionInfo) {
regionFailedOpen.remove(regionInfo.getRegionName());
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionTransitionProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionTransitionProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionTransitionProcedure.java
index 9a10e2b..6f54dcf 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionTransitionProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionTransitionProcedure.java
@@ -24,10 +24,9 @@ import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.exceptions.UnexpectedStateException;
import org.apache.hadoop.hbase.master.assignment.RegionStates.RegionStateNode;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
@@ -36,6 +35,8 @@ import org.apache.hadoop.hbase.procedure2.Procedure;
import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
import org.apache.hadoop.hbase.procedure2.RemoteProcedureDispatcher.RemoteOperation;
import org.apache.hadoop.hbase.procedure2.RemoteProcedureDispatcher.RemoteProcedure;
+import org.apache.yetus.audience.InterfaceAudience;
+
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RegionTransitionState;
import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
@@ -89,7 +90,7 @@ public abstract class RegionTransitionProcedure
private RegionTransitionState transitionState =
RegionTransitionState.REGION_TRANSITION_QUEUE;
- private HRegionInfo regionInfo;
+ private RegionInfo regionInfo;
private volatile boolean lock = false;
public RegionTransitionProcedure() {
@@ -97,22 +98,22 @@ public abstract class RegionTransitionProcedure
super();
}
- public RegionTransitionProcedure(final HRegionInfo regionInfo) {
+ public RegionTransitionProcedure(final RegionInfo regionInfo) {
this.regionInfo = regionInfo;
}
- public HRegionInfo getRegionInfo() {
+ public RegionInfo getRegionInfo() {
return regionInfo;
}
- protected void setRegionInfo(final HRegionInfo regionInfo) {
+ protected void setRegionInfo(final RegionInfo regionInfo) {
// Setter is for deserialization.
this.regionInfo = regionInfo;
}
@Override
public TableName getTableName() {
- HRegionInfo hri = getRegionInfo();
+ RegionInfo hri = getRegionInfo();
return hri != null? hri.getTable(): null;
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java
index f67aa5b..cbd334e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java
@@ -40,12 +40,13 @@ import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
import org.apache.hadoop.hbase.client.MasterSwitchType;
import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
@@ -72,6 +73,7 @@ import org.apache.hadoop.hbase.util.Threads;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.SplitTableRegionState;
@@ -86,8 +88,8 @@ public class SplitTableRegionProcedure
extends AbstractStateMachineRegionProcedure<SplitTableRegionState> {
private static final Log LOG = LogFactory.getLog(SplitTableRegionProcedure.class);
private Boolean traceEnabled = null;
- private HRegionInfo daughter_1_HRI;
- private HRegionInfo daughter_2_HRI;
+ private RegionInfo daughter_1_RI;
+ private RegionInfo daughter_2_RI;
private byte[] bestSplitRow;
public SplitTableRegionProcedure() {
@@ -95,14 +97,24 @@ public class SplitTableRegionProcedure
}
public SplitTableRegionProcedure(final MasterProcedureEnv env,
- final HRegionInfo regionToSplit, final byte[] splitRow) throws IOException {
+ final RegionInfo regionToSplit, final byte[] splitRow) throws IOException {
super(env, regionToSplit);
this.bestSplitRow = splitRow;
checkSplittable(env, regionToSplit, bestSplitRow);
final TableName table = regionToSplit.getTable();
final long rid = getDaughterRegionIdTimestamp(regionToSplit);
- this.daughter_1_HRI = new HRegionInfo(table, regionToSplit.getStartKey(), bestSplitRow, false, rid);
- this.daughter_2_HRI = new HRegionInfo(table, bestSplitRow, regionToSplit.getEndKey(), false, rid);
+ this.daughter_1_RI = RegionInfoBuilder.newBuilder(table)
+ .setStartKey(regionToSplit.getStartKey())
+ .setEndKey(bestSplitRow)
+ .setSplit(false)
+ .setRegionId(rid)
+ .build();
+ this.daughter_2_RI = RegionInfoBuilder.newBuilder(table)
+ .setStartKey(bestSplitRow)
+ .setEndKey(regionToSplit.getEndKey())
+ .setSplit(false)
+ .setRegionId(rid)
+ .build();
}
/**
@@ -113,10 +125,10 @@ public class SplitTableRegionProcedure
* @throws IOException
*/
private void checkSplittable(final MasterProcedureEnv env,
- final HRegionInfo regionToSplit, final byte[] splitRow) throws IOException {
+ final RegionInfo regionToSplit, final byte[] splitRow) throws IOException {
// Ask the remote RS if this region is splittable.
// If we get an IOE, report it along w/ the failure so can see why we are not splittable at this time.
- if(regionToSplit.getReplicaId() != HRegionInfo.DEFAULT_REPLICA_ID) {
+ if(regionToSplit.getReplicaId() != RegionInfo.DEFAULT_REPLICA_ID) {
throw new IllegalArgumentException ("Can't invoke split on non-default regions directly");
}
RegionStateNode node =
@@ -169,10 +181,10 @@ public class SplitTableRegionProcedure
/**
* Calculate daughter regionid to use.
- * @param hri Parent {@link HRegionInfo}
+ * @param hri Parent {@link RegionInfo}
* @return Daughter region id (timestamp) to use.
*/
- private static long getDaughterRegionIdTimestamp(final HRegionInfo hri) {
+ private static long getDaughterRegionIdTimestamp(final RegionInfo hri) {
long rid = EnvironmentEdgeManager.currentTime();
// Regionid is timestamp. Can't be less than that of parent else will insert
// at wrong location in hbase:meta (See HBASE-710).
@@ -332,9 +344,9 @@ public class SplitTableRegionProcedure
final MasterProcedureProtos.SplitTableRegionStateData.Builder splitTableRegionMsg =
MasterProcedureProtos.SplitTableRegionStateData.newBuilder()
.setUserInfo(MasterProcedureUtil.toProtoUserInfo(getUser()))
- .setParentRegionInfo(HRegionInfo.convert(getRegion()))
- .addChildRegionInfo(HRegionInfo.convert(daughter_1_HRI))
- .addChildRegionInfo(HRegionInfo.convert(daughter_2_HRI));
+ .setParentRegionInfo(ProtobufUtil.toRegionInfo(getRegion()))
+ .addChildRegionInfo(ProtobufUtil.toRegionInfo(daughter_1_RI))
+ .addChildRegionInfo(ProtobufUtil.toRegionInfo(daughter_2_RI));
serializer.serialize(splitTableRegionMsg.build());
}
@@ -346,10 +358,10 @@ public class SplitTableRegionProcedure
final MasterProcedureProtos.SplitTableRegionStateData splitTableRegionsMsg =
serializer.deserialize(MasterProcedureProtos.SplitTableRegionStateData.class);
setUser(MasterProcedureUtil.toUserInfo(splitTableRegionsMsg.getUserInfo()));
- setRegion(HRegionInfo.convert(splitTableRegionsMsg.getParentRegionInfo()));
+ setRegion(ProtobufUtil.toRegionInfo(splitTableRegionsMsg.getParentRegionInfo()));
assert(splitTableRegionsMsg.getChildRegionInfoCount() == 2);
- daughter_1_HRI = HRegionInfo.convert(splitTableRegionsMsg.getChildRegionInfo(0));
- daughter_2_HRI = HRegionInfo.convert(splitTableRegionsMsg.getChildRegionInfo(1));
+ daughter_1_RI = ProtobufUtil.toRegionInfo(splitTableRegionsMsg.getChildRegionInfo(0));
+ daughter_2_RI = ProtobufUtil.toRegionInfo(splitTableRegionsMsg.getChildRegionInfo(1));
}
@Override
@@ -360,12 +372,12 @@ public class SplitTableRegionProcedure
sb.append(", parent=");
sb.append(getParentRegion().getShortNameToLog());
sb.append(", daughterA=");
- sb.append(daughter_1_HRI.getShortNameToLog());
+ sb.append(daughter_1_RI.getShortNameToLog());
sb.append(", daughterB=");
- sb.append(daughter_2_HRI.getShortNameToLog());
+ sb.append(daughter_2_RI.getShortNameToLog());
}
- private HRegionInfo getParentRegion() {
+ private RegionInfo getParentRegion() {
return getRegion();
}
@@ -380,7 +392,7 @@ public class SplitTableRegionProcedure
}
private byte[] getSplitRow() {
- return daughter_2_HRI.getStartKey();
+ return daughter_2_RI.getStartKey();
}
private static State [] EXPECTED_SPLIT_STATES = new State [] {State.OPEN, State.CLOSED};
@@ -394,7 +406,7 @@ public class SplitTableRegionProcedure
// Check whether the region is splittable
RegionStateNode node =
env.getAssignmentManager().getRegionStates().getRegionNode(getParentRegion());
- HRegionInfo parentHRI = null;
+ RegionInfo parentHRI = null;
if (node != null) {
parentHRI = node.getRegionInfo();
@@ -479,7 +491,7 @@ public class SplitTableRegionProcedure
final AssignProcedure[] procs = new AssignProcedure[regionReplication];
for (int i = 0; i < regionReplication; ++i) {
- final HRegionInfo hri = RegionReplicaUtil.getRegionInfoForReplica(getParentRegion(), i);
+ final RegionInfo hri = RegionReplicaUtil.getRegionInfoForReplica(getParentRegion(), i);
procs[i] = env.getAssignmentManager().createAssignProcedure(hri, serverName);
}
env.getMasterServices().getMasterProcedureExecutor().submitProcedures(procs);
@@ -502,17 +514,17 @@ public class SplitTableRegionProcedure
Pair<Integer, Integer> expectedReferences = splitStoreFiles(env, regionFs);
assertReferenceFileCount(fs, expectedReferences.getFirst(),
- regionFs.getSplitsDir(daughter_1_HRI));
+ regionFs.getSplitsDir(daughter_1_RI));
//Move the files from the temporary .splits to the final /table/region directory
- regionFs.commitDaughterRegion(daughter_1_HRI);
+ regionFs.commitDaughterRegion(daughter_1_RI);
assertReferenceFileCount(fs, expectedReferences.getFirst(),
- new Path(tabledir, daughter_1_HRI.getEncodedName()));
+ new Path(tabledir, daughter_1_RI.getEncodedName()));
assertReferenceFileCount(fs, expectedReferences.getSecond(),
- regionFs.getSplitsDir(daughter_2_HRI));
- regionFs.commitDaughterRegion(daughter_2_HRI);
+ regionFs.getSplitsDir(daughter_2_RI));
+ regionFs.commitDaughterRegion(daughter_2_RI);
assertReferenceFileCount(fs, expectedReferences.getSecond(),
- new Path(tabledir, daughter_2_HRI.getEncodedName()));
+ new Path(tabledir, daughter_2_RI.getEncodedName()));
}
/**
@@ -650,9 +662,9 @@ public class SplitTableRegionProcedure
final byte[] splitRow = getSplitRow();
final String familyName = Bytes.toString(family);
final Path path_first =
- regionFs.splitStoreFile(this.daughter_1_HRI, familyName, sf, splitRow, false, null);
+ regionFs.splitStoreFile(this.daughter_1_RI, familyName, sf, splitRow, false, null);
final Path path_second =
- regionFs.splitStoreFile(this.daughter_2_HRI, familyName, sf, splitRow, true, null);
+ regionFs.splitStoreFile(this.daughter_2_RI, familyName, sf, splitRow, true, null);
if (LOG.isDebugEnabled()) {
LOG.debug("pid=" + getProcId() + " splitting complete for store file: " +
sf.getPath() + " for region: " + getParentRegion().getShortNameToLog());
@@ -702,7 +714,7 @@ public class SplitTableRegionProcedure
}
try {
for (Mutation p : metaEntries) {
- HRegionInfo.parseRegionName(p.getRow());
+ RegionInfo.parseRegionName(p.getRow());
}
} catch (IOException e) {
LOG.error("pid=" + getProcId() + " row key of mutation from coprocessor not parsable as "
@@ -720,7 +732,7 @@ public class SplitTableRegionProcedure
*/
private void updateMetaForDaughterRegions(final MasterProcedureEnv env) throws IOException {
env.getAssignmentManager().markRegionAsSplit(getParentRegion(), getParentRegionServerName(env),
- daughter_1_HRI, daughter_2_HRI);
+ daughter_1_RI, daughter_2_RI);
}
/**
@@ -742,7 +754,7 @@ public class SplitTableRegionProcedure
private void postSplitRegion(final MasterProcedureEnv env) throws IOException {
final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
if (cpHost != null) {
- cpHost.postCompletedSplitRegionAction(daughter_1_HRI, daughter_2_HRI, getUser());
+ cpHost.postCompletedSplitRegionAction(daughter_1_RI, daughter_2_RI, getUser());
}
}
@@ -755,7 +767,7 @@ public class SplitTableRegionProcedure
final int regionReplication) {
final UnassignProcedure[] procs = new UnassignProcedure[regionReplication];
for (int i = 0; i < procs.length; ++i) {
- final HRegionInfo hri = RegionReplicaUtil.getRegionInfoForReplica(getParentRegion(), i);
+ final RegionInfo hri = RegionReplicaUtil.getRegionInfoForReplica(getParentRegion(), i);
procs[i] = env.getAssignmentManager().createUnassignProcedure(hri, null, true);
}
return procs;
@@ -767,11 +779,11 @@ public class SplitTableRegionProcedure
final AssignProcedure[] procs = new AssignProcedure[regionReplication * 2];
int procsIdx = 0;
for (int i = 0; i < regionReplication; ++i) {
- final HRegionInfo hri = RegionReplicaUtil.getRegionInfoForReplica(daughter_1_HRI, i);
+ final RegionInfo hri = RegionReplicaUtil.getRegionInfoForReplica(daughter_1_RI, i);
procs[procsIdx++] = env.getAssignmentManager().createAssignProcedure(hri, targetServer);
}
for (int i = 0; i < regionReplication; ++i) {
- final HRegionInfo hri = RegionReplicaUtil.getRegionInfoForReplica(daughter_2_HRI, i);
+ final RegionInfo hri = RegionReplicaUtil.getRegionInfoForReplica(daughter_2_RI, i);
procs[procsIdx++] = env.getAssignmentManager().createAssignProcedure(hri, targetServer);
}
return procs;
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/UnassignProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/UnassignProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/UnassignProcedure.java
index 4cb6368..66277be 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/UnassignProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/UnassignProcedure.java
@@ -20,29 +20,31 @@
package org.apache.hadoop.hbase.master.assignment;
import java.io.IOException;
+
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.NotServingRegionException;
import org.apache.hadoop.hbase.ServerName;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.exceptions.UnexpectedStateException;
import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
+import org.apache.hadoop.hbase.master.RegionState.State;
import org.apache.hadoop.hbase.master.assignment.RegionStates.RegionStateNode;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
-import org.apache.hadoop.hbase.master.procedure.ServerCrashException;
import org.apache.hadoop.hbase.master.procedure.RSProcedureDispatcher.RegionCloseOperation;
-import org.apache.hadoop.hbase.master.RegionState.State;
+import org.apache.hadoop.hbase.master.procedure.ServerCrashException;
import org.apache.hadoop.hbase.procedure2.ProcedureMetrics;
import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
import org.apache.hadoop.hbase.procedure2.RemoteProcedureDispatcher.RemoteOperation;
+import org.apache.hadoop.hbase.regionserver.RegionServerAbortedException;
+import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;
+import org.apache.yetus.audience.InterfaceAudience;
+
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RegionTransitionState;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.UnassignRegionStateData;
import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
-import org.apache.hadoop.hbase.regionserver.RegionServerAbortedException;
-import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;
/**
* Procedure that describes the unassignment of a single region.
@@ -90,12 +92,12 @@ public class UnassignProcedure extends RegionTransitionProcedure {
super();
}
- public UnassignProcedure(final HRegionInfo regionInfo, final ServerName hostingServer,
+ public UnassignProcedure(final RegionInfo regionInfo, final ServerName hostingServer,
final boolean force) {
this(regionInfo, hostingServer, null, force);
}
- public UnassignProcedure(final HRegionInfo regionInfo,
+ public UnassignProcedure(final RegionInfo regionInfo,
final ServerName hostingServer, final ServerName destinationServer, final boolean force) {
super(regionInfo);
this.hostingServer = hostingServer;
@@ -128,7 +130,7 @@ public class UnassignProcedure extends RegionTransitionProcedure {
UnassignRegionStateData.Builder state = UnassignRegionStateData.newBuilder()
.setTransitionState(getTransitionState())
.setHostingServer(ProtobufUtil.toServerName(this.hostingServer))
- .setRegionInfo(HRegionInfo.convert(getRegionInfo()));
+ .setRegionInfo(ProtobufUtil.toRegionInfo(getRegionInfo()));
if (this.destinationServer != null) {
state.setDestinationServer(ProtobufUtil.toServerName(destinationServer));
}
@@ -144,7 +146,7 @@ public class UnassignProcedure extends RegionTransitionProcedure {
final UnassignRegionStateData state =
serializer.deserialize(UnassignRegionStateData.class);
setTransitionState(state.getTransitionState());
- setRegionInfo(HRegionInfo.convert(state.getRegionInfo()));
+ setRegionInfo(ProtobufUtil.toRegionInfo(state.getRegionInfo()));
this.hostingServer = ProtobufUtil.toServerName(state.getHostingServer());
force = state.getForce();
if (state.hasDestinationServer()) {
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/Util.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/Util.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/Util.java
index 156fe7a..e6b1495 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/Util.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/Util.java
@@ -19,11 +19,12 @@ package org.apache.hadoop.hbase.master.assignment;
import java.io.IOException;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.ipc.HBaseRpcController;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.yetus.audience.InterfaceAudience;
+
import org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
@@ -43,13 +44,13 @@ class Util {
* @throws IOException Let it out so can report this IOE as reason for failure
*/
static GetRegionInfoResponse getRegionInfoResponse(final MasterProcedureEnv env,
- final ServerName regionLocation, final HRegionInfo hri)
+ final ServerName regionLocation, final RegionInfo hri)
throws IOException {
return getRegionInfoResponse(env, regionLocation, hri, false);
}
static GetRegionInfoResponse getRegionInfoResponse(final MasterProcedureEnv env,
- final ServerName regionLocation, final HRegionInfo hri, boolean includeBestSplitRow)
+ final ServerName regionLocation, final RegionInfo hri, boolean includeBestSplitRow)
throws IOException {
// TODO: There is no timeout on this controller. Set one!
HBaseRpcController controller = env.getMasterServices().getClusterConnection().
[19/19] hbase git commit: HBASE-18839 Apply RegionInfo to code base
Posted by ch...@apache.org.
HBASE-18839 Apply RegionInfo to code base
Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/a11a35a1
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/a11a35a1
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/a11a35a1
Branch: refs/heads/master
Commit: a11a35a1135c431ee12534451c925727165eded5
Parents: 7f4c3b3
Author: Chia-Ping Tsai <ch...@gmail.com>
Authored: Thu Sep 28 16:16:21 2017 +0800
Committer: Chia-Ping Tsai <ch...@gmail.com>
Committed: Thu Sep 28 20:19:29 2017 +0800
----------------------------------------------------------------------
.../hadoop/hbase/backup/util/BackupUtils.java | 14 +-
.../hadoop/hbase/AsyncMetaTableAccessor.java | 47 +--
.../org/apache/hadoop/hbase/HRegionInfo.java | 19 +-
.../apache/hadoop/hbase/HRegionLocation.java | 26 +-
.../apache/hadoop/hbase/MetaTableAccessor.java | 245 ++++++-------
.../apache/hadoop/hbase/RegionLocations.java | 5 +-
.../apache/hadoop/hbase/client/AsyncAdmin.java | 15 +-
.../hadoop/hbase/client/AsyncHBaseAdmin.java | 9 +-
.../hbase/client/ConnectionImplementation.java | 38 +-
.../hbase/client/FlushRegionCallable.java | 10 +-
.../apache/hadoop/hbase/client/HBaseAdmin.java | 134 ++++---
.../hadoop/hbase/client/HRegionLocator.java | 9 +-
.../hadoop/hbase/client/HTableMultiplexer.java | 15 +-
.../hbase/client/ImmutableHRegionInfo.java | 2 +-
.../hadoop/hbase/client/RawAsyncHBaseAdmin.java | 64 ++--
.../hadoop/hbase/client/RawAsyncTable.java | 25 +-
.../hadoop/hbase/client/RawAsyncTableImpl.java | 18 +-
.../client/RegionCoprocessorRpcChannelImpl.java | 22 +-
.../apache/hadoop/hbase/client/RegionInfo.java | 27 +-
.../hadoop/hbase/client/RegionInfoBuilder.java | 360 ++++++-------------
.../hadoop/hbase/client/RegionReplicaUtil.java | 65 ++--
.../hadoop/hbase/client/ZooKeeperRegistry.java | 7 +-
.../apache/hadoop/hbase/master/RegionState.java | 23 +-
.../hbase/shaded/protobuf/ProtobufUtil.java | 38 +-
.../hbase/shaded/protobuf/RequestConverter.java | 48 +--
.../shaded/protobuf/ResponseConverter.java | 20 +-
.../hbase/zookeeper/MetaTableLocator.java | 71 ++--
.../hadoop/hbase/client/TestAsyncProcess.java | 28 +-
.../coprocessor/AsyncAggregationClient.java | 28 +-
.../apache/hadoop/hbase/coprocessor/Export.java | 22 +-
.../example/TestRefreshHFilesEndpoint.java | 20 +-
.../mapreduce/TableSnapshotInputFormat.java | 26 +-
.../hbase/regionserver/CompactionTool.java | 10 +-
.../hadoop/hbase/snapshot/ExportSnapshot.java | 23 +-
.../hbase/mapreduce/TestImportExport.java | 18 +-
.../replication/TestReplicationSmallTests.java | 25 +-
.../hbase/snapshot/TestExportSnapshot.java | 13 +-
.../hbase/snapshot/TestMobExportSnapshot.java | 4 +-
.../hadoop/hbase/rest/RegionsResource.java | 20 +-
.../hbase/rsgroup/RSGroupAdminServer.java | 43 +--
.../hbase/rsgroup/RSGroupBasedLoadBalancer.java | 104 +++---
.../hbase/rsgroup/RSGroupInfoManagerImpl.java | 15 +-
.../balancer/TestRSGroupBasedLoadBalancer.java | 133 +++----
.../hadoop/hbase/rsgroup/TestRSGroupsBase.java | 22 +-
.../hbase/tmpl/master/MasterStatusTmpl.jamon | 6 +-
.../hbase/tmpl/regionserver/RSStatusTmpl.jamon | 4 +-
.../tmpl/regionserver/RegionListTmpl.jamon | 41 +--
.../hadoop/hbase/RegionStateListener.java | 11 +-
.../hadoop/hbase/backup/HFileArchiver.java | 18 +-
.../hbase/client/ClientSideRegionScanner.java | 5 +-
.../hbase/client/TableSnapshotScanner.java | 13 +-
.../hbase/client/locking/LockServiceClient.java | 18 +-
.../SplitLogManagerCoordination.java | 8 +-
.../ZKSplitLogManagerCoordination.java | 16 +-
.../coprocessor/MultiRowMutationEndpoint.java | 16 +-
.../favored/FavoredNodeAssignmentHelper.java | 104 +++---
.../hbase/favored/FavoredNodeLoadBalancer.java | 85 ++---
.../hbase/favored/FavoredNodesManager.java | 44 +--
.../hadoop/hbase/favored/FavoredNodesPlan.java | 10 +-
.../hbase/favored/FavoredNodesPromoter.java | 8 +-
.../org/apache/hadoop/hbase/io/HFileLink.java | 19 +-
.../hadoop/hbase/master/AssignmentListener.java | 9 +-
.../master/AssignmentVerificationReport.java | 38 +-
.../hadoop/hbase/master/CatalogJanitor.java | 56 ++-
.../org/apache/hadoop/hbase/master/HMaster.java | 58 +--
.../hadoop/hbase/master/LoadBalancer.java | 32 +-
.../hadoop/hbase/master/MasterFileSystem.java | 13 +-
.../hbase/master/MasterMetaBootstrap.java | 15 +-
.../hadoop/hbase/master/MasterRpcServices.java | 54 +--
.../hadoop/hbase/master/MasterServices.java | 11 +-
.../hadoop/hbase/master/MasterWalManager.java | 6 +-
.../hbase/master/RegionPlacementMaintainer.java | 67 ++--
.../apache/hadoop/hbase/master/RegionPlan.java | 10 +-
.../hadoop/hbase/master/ServerManager.java | 32 +-
.../SnapshotOfRegionAssignmentFromMeta.java | 64 ++--
.../hadoop/hbase/master/SplitLogManager.java | 12 +-
.../master/assignment/AssignProcedure.java | 16 +-
.../master/assignment/AssignmentManager.java | 217 +++++------
.../assignment/GCMergedRegionsProcedure.java | 33 +-
.../master/assignment/GCRegionProcedure.java | 16 +-
.../assignment/MergeTableRegionsProcedure.java | 89 ++---
.../master/assignment/MoveRegionProcedure.java | 9 +-
.../master/assignment/RegionStateStore.java | 34 +-
.../hbase/master/assignment/RegionStates.java | 168 ++++-----
.../assignment/RegionTransitionProcedure.java | 15 +-
.../assignment/SplitTableRegionProcedure.java | 84 +++--
.../master/assignment/UnassignProcedure.java | 22 +-
.../hadoop/hbase/master/assignment/Util.java | 9 +-
.../hbase/master/balancer/BaseLoadBalancer.java | 118 +++---
.../hbase/master/balancer/ClusterLoadState.java | 16 +-
.../balancer/FavoredStochasticBalancer.java | 102 +++---
.../master/balancer/RegionInfoComparator.java | 6 +-
.../master/balancer/RegionLocationFinder.java | 38 +-
.../master/balancer/SimpleLoadBalancer.java | 42 +--
.../master/balancer/StochasticLoadBalancer.java | 14 +-
.../hbase/master/locking/LockManager.java | 15 +-
.../hbase/master/locking/LockProcedure.java | 27 +-
.../normalizer/MergeNormalizationPlan.java | 19 +-
.../master/normalizer/RegionNormalizer.java | 8 +-
.../normalizer/SimpleRegionNormalizer.java | 17 +-
.../normalizer/SplitNormalizationPlan.java | 19 +-
.../AbstractStateMachineRegionProcedure.java | 21 +-
.../procedure/AddColumnFamilyProcedure.java | 9 +-
.../procedure/CloneSnapshotProcedure.java | 38 +-
.../master/procedure/CreateTableProcedure.java | 66 ++--
.../procedure/DeleteColumnFamilyProcedure.java | 11 +-
.../master/procedure/DeleteTableProcedure.java | 23 +-
.../procedure/MasterDDLOperationHelper.java | 10 +-
.../procedure/MasterProcedureScheduler.java | 19 +-
.../master/procedure/ModifyTableProcedure.java | 11 +-
.../master/procedure/ProcedureSyncWait.java | 10 +-
.../master/procedure/RSProcedureDispatcher.java | 26 +-
.../master/procedure/RecoverMetaProcedure.java | 24 +-
.../procedure/RestoreSnapshotProcedure.java | 39 +-
.../master/procedure/ServerCrashProcedure.java | 31 +-
.../procedure/TruncateTableProcedure.java | 29 +-
.../snapshot/DisabledTableSnapshotHandler.java | 21 +-
.../snapshot/EnabledTableSnapshotHandler.java | 20 +-
.../master/snapshot/MasterSnapshotVerifier.java | 31 +-
.../master/snapshot/TakeSnapshotHandler.java | 17 +-
.../org/apache/hadoop/hbase/mob/MobUtils.java | 26 +-
.../hbase/namespace/NamespaceAuditor.java | 12 +-
.../hbase/namespace/NamespaceStateManager.java | 16 +-
.../flush/MasterFlushTableProcedureManager.java | 15 +-
.../quotas/FileSystemUtilizationChore.java | 12 +-
.../hadoop/hbase/quotas/MasterQuotaManager.java | 32 +-
.../quotas/NamespaceQuotaSnapshotStore.java | 24 +-
.../hadoop/hbase/quotas/QuotaObserverChore.java | 15 +-
.../hadoop/hbase/quotas/QuotaSnapshotStore.java | 11 +-
.../hbase/quotas/TableQuotaSnapshotStore.java | 24 +-
.../hadoop/hbase/regionserver/HRegion.java | 64 ++--
.../hbase/regionserver/HRegionFileSystem.java | 74 ++--
.../hbase/regionserver/HRegionServer.java | 64 ++--
.../hadoop/hbase/regionserver/HStore.java | 12 +-
.../regionserver/MetricsRegionWrapperImpl.java | 4 +-
.../hbase/regionserver/RSRpcServices.java | 101 +++---
.../hadoop/hbase/regionserver/Region.java | 8 +-
.../hbase/regionserver/RegionMergeRequest.java | 15 +-
.../hbase/regionserver/RegionScanner.java | 8 +-
.../regionserver/RegionServerServices.java | 20 +-
.../regionserver/RegionServicesForStores.java | 6 +-
.../hbase/regionserver/RegionSplitPolicy.java | 2 +-
.../hadoop/hbase/regionserver/SplitRequest.java | 19 +-
.../regionserver/handler/CloseMetaHandler.java | 6 +-
.../handler/CloseRegionHandler.java | 15 +-
.../regionserver/handler/OpenMetaHandler.java | 6 +-
.../handler/OpenPriorityRegionHandler.java | 6 +-
.../regionserver/handler/OpenRegionHandler.java | 17 +-
.../hbase/regionserver/wal/AbstractFSWAL.java | 19 +-
.../hbase/regionserver/wal/AsyncFSWAL.java | 24 +-
.../hadoop/hbase/regionserver/wal/FSHLog.java | 20 +-
.../hbase/regionserver/wal/FSWALEntry.java | 20 +-
.../regionserver/wal/WALCoprocessorHost.java | 6 +-
.../hadoop/hbase/regionserver/wal/WALUtil.java | 24 +-
.../RegionReplicaReplicationEndpoint.java | 12 +-
.../hbase/snapshot/RestoreSnapshotHelper.java | 129 +++----
.../hadoop/hbase/snapshot/SnapshotInfo.java | 30 +-
.../hadoop/hbase/snapshot/SnapshotManifest.java | 16 +-
.../hbase/snapshot/SnapshotManifestV1.java | 18 +-
.../hbase/snapshot/SnapshotManifestV2.java | 23 +-
.../hbase/snapshot/SnapshotReferenceUtil.java | 23 +-
.../org/apache/hadoop/hbase/tool/Canary.java | 68 ++--
.../org/apache/hadoop/hbase/util/HBaseFsck.java | 184 +++++-----
.../hadoop/hbase/util/HBaseFsckRepair.java | 40 +--
.../hadoop/hbase/util/HFileArchiveUtil.java | 14 +-
.../hadoop/hbase/util/ModifyRegionUtils.java | 68 ++--
.../apache/hadoop/hbase/util/RegionMover.java | 60 ++--
.../hbase/util/ServerRegionReplicaUtil.java | 13 +-
.../hadoop/hbase/wal/DisabledWALProvider.java | 12 +-
.../java/org/apache/hadoop/hbase/wal/WAL.java | 16 +-
.../org/apache/hadoop/hbase/wal/WALEdit.java | 24 +-
.../resources/hbase-webapps/master/table.jsp | 111 +++---
.../hbase-webapps/regionserver/region.jsp | 9 +-
.../hadoop/hbase/HBaseTestingUtility.java | 244 ++-----------
.../hadoop/hbase/MockRegionServerServices.java | 18 +-
.../hadoop/hbase/TestGlobalMemStoreSize.java | 10 +-
.../org/apache/hadoop/hbase/TestIOFencing.java | 7 +-
.../hadoop/hbase/TestMetaTableAccessor.java | 207 ++++++++---
.../hbase/TestMetaTableAccessorNoCluster.java | 27 +-
.../hbase/TestPartialResultsFromClientSide.java | 5 +-
.../hadoop/hbase/TestRegionRebalancing.java | 6 +-
.../hbase/TestStochasticBalancerJmxMetrics.java | 22 +-
.../org/apache/hadoop/hbase/TestZooKeeper.java | 5 +-
.../hbase/client/HConnectionTestingUtility.java | 4 +-
.../apache/hadoop/hbase/client/TestAdmin1.java | 23 +-
.../apache/hadoop/hbase/client/TestAdmin2.java | 36 +-
.../hbase/client/TestAsyncClusterAdminApi.java | 11 +-
.../hbase/client/TestAsyncRegionAdminApi.java | 48 +--
.../client/TestBlockEvictionFromClient.java | 7 +-
.../hbase/client/TestMetaWithReplicas.java | 11 +-
.../hbase/client/TestMvccConsistentScanner.java | 3 +-
.../coprocessor/TestCoprocessorInterface.java | 13 +-
.../hbase/coprocessor/TestMasterObserver.java | 22 +-
.../TestRegionObserverInterface.java | 4 +-
.../TestRegionObserverScannerOpenHook.java | 3 +-
.../TestFavoredNodeAssignmentHelper.java | 108 +++---
.../hbase/master/MockNoopMasterServices.java | 10 +-
.../hadoop/hbase/master/MockRegionServer.java | 13 +-
.../hbase/master/TestAssignmentListener.java | 8 +-
.../TestCatalogJanitorInMemoryStates.java | 51 +--
.../master/TestDistributedLogSplitting.java | 138 +++----
.../apache/hadoop/hbase/master/TestMaster.java | 23 +-
.../TestMasterOperationsForRegionReplicas.java | 55 ++-
.../hbase/master/TestMasterTransitions.java | 16 +-
.../hbase/master/TestRegionPlacement.java | 14 +-
.../hbase/master/TestRegionPlacement2.java | 21 +-
.../hadoop/hbase/master/TestRestartCluster.java | 13 +-
.../hadoop/hbase/master/TestRollingRestart.java | 7 +-
.../hadoop/hbase/master/TestWarmupRegion.java | 6 +-
.../assignment/AssignmentTestingUtil.java | 22 +-
.../master/assignment/MockMasterServices.java | 16 +-
.../assignment/TestAssignmentManager.java | 82 +++--
.../assignment/TestAssignmentOnRSCrash.java | 25 +-
.../TestMergeTableRegionsProcedure.java | 28 +-
.../master/assignment/TestRegionStates.java | 50 +--
.../TestSplitTableRegionProcedure.java | 20 +-
.../hbase/master/balancer/BalancerTestBase.java | 100 +++---
.../LoadBalancerPerformanceEvaluation.java | 37 +-
.../master/balancer/TestBaseLoadBalancer.java | 149 ++++----
.../balancer/TestDefaultLoadBalancer.java | 32 +-
.../TestFavoredStochasticBalancerPickers.java | 20 +-
.../TestFavoredStochasticLoadBalancer.java | 62 ++--
.../balancer/TestRegionLocationFinder.java | 8 +-
.../balancer/TestStochasticLoadBalancer.java | 38 +-
.../hbase/master/locking/TestLockProcedure.java | 27 +-
.../normalizer/TestSimpleRegionNormalizer.java | 228 ++++++++----
.../TestSimpleRegionNormalizerOnCluster.java | 30 +-
.../MasterProcedureTestingUtility.java | 14 +-
.../procedure/TestCreateTableProcedure.java | 18 +-
.../TestDeleteColumnFamilyProcedure.java | 4 +-
.../procedure/TestDeleteTableProcedure.java | 8 +-
.../TestMasterFailoverWithProcedures.java | 21 +-
.../procedure/TestMasterProcedureWalLease.java | 6 +-
.../procedure/TestModifyTableProcedure.java | 10 +-
.../master/procedure/TestProcedureAdmin.java | 6 +-
.../procedure/TestSafemodeBringsDownMaster.java | 5 +-
.../procedure/TestTruncateTableProcedure.java | 10 +-
.../master/snapshot/TestSnapshotFileCache.java | 22 +-
.../quotas/TestFileSystemUtilizationChore.java | 29 +-
.../TestNamespaceQuotaViolationStore.java | 59 ++-
.../hbase/quotas/TestQuotaObserverChore.java | 20 +-
.../TestQuotaObserverChoreRegionReports.java | 10 +-
.../hbase/quotas/TestQuotaStatusRPCs.java | 14 +-
.../hadoop/hbase/quotas/TestRegionSizeUse.java | 10 +-
.../hadoop/hbase/quotas/TestSpaceQuotas.java | 10 +-
.../quotas/TestTableQuotaViolationStore.java | 63 +++-
.../hbase/quotas/TestTablesWithQuotas.java | 4 +-
.../hbase/regionserver/TestAtomicOperation.java | 25 +-
.../hbase/regionserver/TestDefaultMemStore.java | 4 +-
.../TestEndToEndSplitTransaction.java | 42 +--
.../hadoop/hbase/regionserver/TestHRegion.java | 3 +-
.../regionserver/TestHRegionFileSystem.java | 14 +-
.../hbase/regionserver/TestHRegionInfo.java | 8 +-
.../hbase/regionserver/TestMobStoreScanner.java | 4 +-
.../TestRSKilledWhenInitializing.java | 12 +-
.../hbase/regionserver/TestRSStatusServlet.java | 45 +--
.../regionserver/TestRegionInfoBuilder.java | 4 +-
.../TestRegionMergeTransactionOnCluster.java | 64 ++--
.../regionserver/TestRegionServerAbort.java | 15 +-
.../TestRegionServerRegionSpaceUseReport.java | 44 ++-
.../TestScannerHeartbeatMessages.java | 16 +-
.../TestSplitTransactionOnCluster.java | 32 +-
.../regionserver/wal/AbstractTestFSWAL.java | 23 +-
.../wal/AbstractTestLogRolling.java | 5 +-
.../regionserver/wal/TestAsyncLogRolling.java | 6 +-
.../hbase/regionserver/wal/TestLogRolling.java | 6 +-
.../replication/TestReplicationEndpoint.java | 20 +-
.../replication/TestSerialReplication.java | 20 +-
.../hbase/snapshot/SnapshotTestingUtils.java | 53 +--
.../TestLoadIncrementalHFilesSplitRecovery.java | 26 +-
.../hadoop/hbase/util/BaseTestHBaseFsck.java | 32 +-
.../hadoop/hbase/util/TestHBaseFsckOneRS.java | 70 ++--
.../hbase/util/TestHBaseFsckReplicas.java | 71 ++--
.../hadoop/hbase/util/TestHBaseFsckTwoRS.java | 46 ++-
.../util/hbck/OfflineMetaRebuildTestCore.java | 18 +-
.../apache/hadoop/hbase/wal/FaultyFSLog.java | 12 +-
.../hbase/wal/WALPerformanceEvaluation.java | 29 +-
.../hadoop/hbase/thrift/ThriftServerRunner.java | 64 ++--
278 files changed, 4817 insertions(+), 4611 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/BackupUtils.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/BackupUtils.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/BackupUtils.java
index 631183d..9d87612 100644
--- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/BackupUtils.java
+++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/BackupUtils.java
@@ -43,7 +43,6 @@ import org.apache.hadoop.fs.PathFilter;
import org.apache.hadoop.fs.RemoteIterator;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
@@ -53,15 +52,16 @@ import org.apache.hadoop.hbase.backup.HBackupFileSystem;
import org.apache.hadoop.hbase.backup.RestoreRequest;
import org.apache.hadoop.hbase.backup.impl.BackupManifest;
import org.apache.hadoop.hbase.backup.impl.BackupManifest.BackupImage;
-import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.TableDescriptor;
-import org.apache.hadoop.hbase.tool.LoadIncrementalHFiles;
import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.tool.LoadIncrementalHFiles;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.FSTableDescriptors;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
+import org.apache.yetus.audience.InterfaceAudience;
/**
* A collection for methods used by multiple classes to backup HBase tables.
@@ -150,11 +150,11 @@ public final class BackupUtils {
LOG.debug("Attempting to copy table info for:" + table + " target: " + target
+ " descriptor: " + orig);
LOG.debug("Finished copying tableinfo.");
- List<HRegionInfo> regions = null;
+ List<RegionInfo> regions = null;
regions = MetaTableAccessor.getTableRegions(conn, table);
// For each region, write the region info to disk
LOG.debug("Starting to write region info for table " + table);
- for (HRegionInfo regionInfo : regions) {
+ for (RegionInfo regionInfo : regions) {
Path regionDir =
HRegion.getRegionDir(new Path(backupInfo.getTableBackupDir(table)), regionInfo);
regionDir = new Path(backupInfo.getTableBackupDir(table), regionDir.getName());
@@ -168,8 +168,8 @@ public final class BackupUtils {
* Write the .regioninfo file on-disk.
*/
public static void writeRegioninfoOnFilesystem(final Configuration conf, final FileSystem fs,
- final Path regionInfoDir, HRegionInfo regionInfo) throws IOException {
- final byte[] content = regionInfo.toDelimitedByteArray();
+ final Path regionInfoDir, RegionInfo regionInfo) throws IOException {
+ final byte[] content = RegionInfo.toDelimitedByteArray(regionInfo);
Path regionInfoFile = new Path(regionInfoDir, "." + HConstants.REGIONINFO_QUALIFIER_STR);
// First check to get the permissions
FsPermission perms = FSUtils.getFilePermissions(fs, conf, HConstants.DATA_FILE_UMASK_KEY);
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-client/src/main/java/org/apache/hadoop/hbase/AsyncMetaTableAccessor.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/AsyncMetaTableAccessor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/AsyncMetaTableAccessor.java
index 678dd91..6f41bd0 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/AsyncMetaTableAccessor.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/AsyncMetaTableAccessor.java
@@ -38,20 +38,21 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.MetaTableAccessor.CollectingVisitor;
import org.apache.hadoop.hbase.MetaTableAccessor.QueryType;
import org.apache.hadoop.hbase.MetaTableAccessor.Visitor;
-import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.Consistency;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.RawAsyncTable;
import org.apache.hadoop.hbase.client.RawScanResultConsumer;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.client.Scan.ReadType;
+import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.Pair;
+import org.apache.yetus.audience.InterfaceAudience;
/**
* The asynchronous meta table accessor. Used to read/write region and assignment information store
@@ -112,7 +113,7 @@ public class AsyncMetaTableAccessor {
RawAsyncTable metaTable, byte[] regionName) {
CompletableFuture<Optional<HRegionLocation>> future = new CompletableFuture<>();
try {
- HRegionInfo parsedRegionInfo = MetaTableAccessor.parseRegionInfoFromRegionName(regionName);
+ RegionInfo parsedRegionInfo = MetaTableAccessor.parseRegionInfoFromRegionName(regionName);
metaTable.get(
new Get(MetaTableAccessor.getMetaKeyForRegion(parsedRegionInfo))
.addFamily(HConstants.CATALOG_FAMILY)).whenComplete(
@@ -151,14 +152,14 @@ public class AsyncMetaTableAccessor {
results
.stream()
.filter(result -> !result.isEmpty())
- .filter(result -> MetaTableAccessor.getHRegionInfo(result) != null)
+ .filter(result -> MetaTableAccessor.getRegionInfo(result) != null)
.forEach(
result -> {
getRegionLocations(result).ifPresent(
locations -> {
for (HRegionLocation location : locations.getRegionLocations()) {
if (location != null
- && encodedRegionNameStr.equals(location.getRegionInfo()
+ && encodedRegionNameStr.equals(location.getRegion()
.getEncodedName())) {
future.complete(Optional.of(location));
return;
@@ -218,27 +219,27 @@ public class AsyncMetaTableAccessor {
* @return the list of regioninfos and server. The return value will be wrapped by a
* {@link CompletableFuture}.
*/
- private static CompletableFuture<List<Pair<HRegionInfo, ServerName>>> getTableRegionsAndLocations(
+ private static CompletableFuture<List<Pair<RegionInfo, ServerName>>> getTableRegionsAndLocations(
RawAsyncTable metaTable, final Optional<TableName> tableName,
final boolean excludeOfflinedSplitParents) {
- CompletableFuture<List<Pair<HRegionInfo, ServerName>>> future = new CompletableFuture<>();
+ CompletableFuture<List<Pair<RegionInfo, ServerName>>> future = new CompletableFuture<>();
if (tableName.filter((t) -> t.equals(TableName.META_TABLE_NAME)).isPresent()) {
future.completeExceptionally(new IOException(
"This method can't be used to locate meta regions;" + " use MetaTableLocator instead"));
}
- // Make a version of CollectingVisitor that collects HRegionInfo and ServerAddress
- CollectingVisitor<Pair<HRegionInfo, ServerName>> visitor = new CollectingVisitor<Pair<HRegionInfo, ServerName>>() {
+ // Make a version of CollectingVisitor that collects RegionInfo and ServerAddress
+ CollectingVisitor<Pair<RegionInfo, ServerName>> visitor = new CollectingVisitor<Pair<RegionInfo, ServerName>>() {
private Optional<RegionLocations> current = null;
@Override
public boolean visit(Result r) throws IOException {
current = getRegionLocations(r);
- if (!current.isPresent() || current.get().getRegionLocation().getRegionInfo() == null) {
- LOG.warn("No serialized HRegionInfo in " + r);
+ if (!current.isPresent() || current.get().getRegionLocation().getRegion() == null) {
+ LOG.warn("No serialized RegionInfo in " + r);
return true;
}
- HRegionInfo hri = current.get().getRegionLocation().getRegionInfo();
+ RegionInfo hri = current.get().getRegionLocation().getRegion();
if (excludeOfflinedSplitParents && hri.isSplitParent()) return true;
// Else call super and add this Result to the collection.
return super.visit(r);
@@ -251,7 +252,7 @@ public class AsyncMetaTableAccessor {
}
for (HRegionLocation loc : current.get().getRegionLocations()) {
if (loc != null) {
- this.results.add(new Pair<HRegionInfo, ServerName>(loc.getRegionInfo(), loc
+ this.results.add(new Pair<RegionInfo, ServerName>(loc.getRegionInfo(), loc
.getServerName()));
}
}
@@ -381,7 +382,7 @@ public class AsyncMetaTableAccessor {
*/
private static Optional<RegionLocations> getRegionLocations(final Result r) {
if (r == null) return Optional.empty();
- Optional<HRegionInfo> regionInfo = getHRegionInfo(r, getRegionInfoColumn());
+ Optional<RegionInfo> regionInfo = getHRegionInfo(r, getRegionInfoColumn());
if (!regionInfo.isPresent()) return Optional.empty();
List<HRegionLocation> locations = new ArrayList<HRegionLocation>(1);
@@ -427,11 +428,11 @@ public class AsyncMetaTableAccessor {
* @param replicaId the replicaId for the HRegionLocation
* @return HRegionLocation parsed from the given meta row Result for the given replicaId
*/
- private static HRegionLocation getRegionLocation(final Result r, final HRegionInfo regionInfo,
+ private static HRegionLocation getRegionLocation(final Result r, final RegionInfo regionInfo,
final int replicaId) {
Optional<ServerName> serverName = getServerName(r, replicaId);
long seqNum = getSeqNumDuringOpen(r, replicaId);
- HRegionInfo replicaInfo = RegionReplicaUtil.getRegionInfoForReplica(regionInfo, replicaId);
+ RegionInfo replicaInfo = RegionReplicaUtil.getRegionInfoForReplica(regionInfo, replicaId);
return new HRegionLocation(replicaInfo, serverName.orElse(null), seqNum);
}
@@ -521,16 +522,16 @@ public class AsyncMetaTableAccessor {
}
/**
- * Returns the HRegionInfo object from the column {@link HConstants#CATALOG_FAMILY} and
+ * Returns the RegionInfo object from the column {@link HConstants#CATALOG_FAMILY} and
* <code>qualifier</code> of the catalog table result.
* @param r a Result object from the catalog table scan
* @param qualifier Column family qualifier
- * @return An HRegionInfo instance.
+ * @return An RegionInfo instance.
*/
- private static Optional<HRegionInfo> getHRegionInfo(final Result r, byte[] qualifier) {
+ private static Optional<RegionInfo> getHRegionInfo(final Result r, byte[] qualifier) {
Cell cell = r.getColumnLatestCell(getCatalogFamily(), qualifier);
if (cell == null) return Optional.empty();
- return Optional.ofNullable(HRegionInfo.parseFromOrNull(cell.getValueArray(),
+ return Optional.ofNullable(RegionInfo.parseFromOrNull(cell.getValueArray(),
cell.getValueOffset(), cell.getValueLength()));
}
@@ -575,7 +576,7 @@ public class AsyncMetaTableAccessor {
return replicaId == 0
? HConstants.SERVER_QUALIFIER
: Bytes.toBytes(HConstants.SERVER_QUALIFIER_STR + META_REPLICA_ID_DELIMITER
- + String.format(HRegionInfo.REPLICA_ID_FORMAT, replicaId));
+ + String.format(RegionInfo.REPLICA_ID_FORMAT, replicaId));
}
/**
@@ -587,7 +588,7 @@ public class AsyncMetaTableAccessor {
return replicaId == 0
? HConstants.STARTCODE_QUALIFIER
: Bytes.toBytes(HConstants.STARTCODE_QUALIFIER_STR + META_REPLICA_ID_DELIMITER
- + String.format(HRegionInfo.REPLICA_ID_FORMAT, replicaId));
+ + String.format(RegionInfo.REPLICA_ID_FORMAT, replicaId));
}
/**
@@ -599,7 +600,7 @@ public class AsyncMetaTableAccessor {
return replicaId == 0
? HConstants.SEQNUM_QUALIFIER
: Bytes.toBytes(HConstants.SEQNUM_QUALIFIER_STR + META_REPLICA_ID_DELIMITER
- + String.format(HRegionInfo.REPLICA_ID_FORMAT, replicaId));
+ + String.format(RegionInfo.REPLICA_ID_FORMAT, replicaId));
}
/**
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java
index cc88733..f2cb821 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java
@@ -28,17 +28,18 @@ import java.util.stream.Collectors;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.client.RegionInfoBuilder;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.KeyValue.KVComparator;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.client.RegionInfoDisplay;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.master.RegionState;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.client.RegionInfoDisplay;
import org.apache.hadoop.io.DataInputBuffer;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
/**
* Information about a region. A region is a range of keys in the whole keyspace of a table, an
@@ -308,7 +309,7 @@ public class HRegionInfo implements RegionInfo, Comparable<HRegionInfo> {
*
* @param other
*/
- public HRegionInfo(HRegionInfo other) {
+ public HRegionInfo(RegionInfo other) {
super();
this.endKey = other.getEndKey();
this.offLine = other.isOffline();
@@ -710,13 +711,13 @@ public class HRegionInfo implements RegionInfo, Comparable<HRegionInfo> {
* @param info the HRegionInfo to convert
* @return the converted RegionInfo
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
- * Use toProtoRegionInfo(org.apache.hadoop.hbase.client.RegionInfo)
+ * Use toRegionInfo(org.apache.hadoop.hbase.client.RegionInfo)
* in org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil.
*/
@Deprecated
@InterfaceAudience.Private
public static HBaseProtos.RegionInfo convert(final HRegionInfo info) {
- return ProtobufUtil.toProtoRegionInfo(info);
+ return ProtobufUtil.toRegionInfo(info);
}
/**
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionLocation.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionLocation.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionLocation.java
index c6c8af5..fd679bd 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionLocation.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionLocation.java
@@ -18,11 +18,13 @@
*/
package org.apache.hadoop.hbase;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.client.ImmutableHRegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.util.Addressing;
+import org.apache.yetus.audience.InterfaceAudience;
/**
- * Data structure to hold HRegionInfo and the address for the hosting
+ * Data structure to hold RegionInfo and the address for the hosting
* HRegionServer. Immutable. Comparable, but we compare the 'location' only:
* i.e. the hostname and port, and *not* the regioninfo. This means two
* instances are the same if they refer to the same 'location' (the same
@@ -36,15 +38,15 @@ import org.apache.hadoop.hbase.util.Addressing;
*/
@InterfaceAudience.Public
public class HRegionLocation implements Comparable<HRegionLocation> {
- private final HRegionInfo regionInfo;
+ private final RegionInfo regionInfo;
private final ServerName serverName;
private final long seqNum;
- public HRegionLocation(HRegionInfo regionInfo, ServerName serverName) {
+ public HRegionLocation(RegionInfo regionInfo, ServerName serverName) {
this(regionInfo, serverName, HConstants.NO_SEQNUM);
}
- public HRegionLocation(HRegionInfo regionInfo, ServerName serverName, long seqNum) {
+ public HRegionLocation(RegionInfo regionInfo, ServerName serverName, long seqNum) {
this.regionInfo = regionInfo;
this.serverName = serverName;
this.seqNum = seqNum;
@@ -84,8 +86,20 @@ public class HRegionLocation implements Comparable<HRegionLocation> {
return this.serverName.hashCode();
}
- /** @return HRegionInfo */
+ /**
+ *
+ * @return Immutable HRegionInfo
+ * @deprecated Since 2.0.0. Will remove in 3.0.0. Use {@link #getRegion()}} instead.
+ */
+ @Deprecated
public HRegionInfo getRegionInfo(){
+ return regionInfo == null ? null : new ImmutableHRegionInfo(regionInfo);
+ }
+
+ /**
+ * @return regionInfo
+ */
+ public RegionInfo getRegion(){
return regionInfo;
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/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 47fffa2..8b6cf66 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
@@ -22,6 +22,7 @@ import java.io.IOException;
import java.io.InterruptedIOException;
import java.util.ArrayList;
import java.util.Arrays;
+import java.util.Collections;
import java.util.HashMap;
import java.util.LinkedHashMap;
import java.util.List;
@@ -36,7 +37,6 @@ import java.util.regex.Pattern;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
-import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.Consistency;
@@ -44,6 +44,8 @@ import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
import org.apache.hadoop.hbase.client.RegionLocator;
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
import org.apache.hadoop.hbase.client.RegionServerCallable;
@@ -66,6 +68,7 @@ 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;
+import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
@@ -152,9 +155,9 @@ public class MetaTableAccessor {
static {
// Copy the prefix from FIRST_META_REGIONINFO into META_REGION_PREFIX.
// FIRST_META_REGIONINFO == 'hbase:meta,,1'. META_REGION_PREFIX == 'hbase:meta,'
- int len = HRegionInfo.FIRST_META_REGIONINFO.getRegionName().length - 2;
+ int len = RegionInfoBuilder.FIRST_META_REGIONINFO.getRegionName().length - 2;
META_REGION_PREFIX = new byte [len];
- System.arraycopy(HRegionInfo.FIRST_META_REGIONINFO.getRegionName(), 0,
+ System.arraycopy(RegionInfoBuilder.FIRST_META_REGIONINFO.getRegionName(), 0,
META_REGION_PREFIX, 0, len);
}
@@ -168,9 +171,9 @@ public class MetaTableAccessor {
* @deprecated use {@link #getTableRegionsAndLocations}, region can have multiple locations
*/
@Deprecated
- public static NavigableMap<HRegionInfo, ServerName> allTableRegions(
+ public static NavigableMap<RegionInfo, ServerName> allTableRegions(
Connection connection, final TableName tableName) throws IOException {
- final NavigableMap<HRegionInfo, ServerName> regions = new TreeMap<>();
+ final NavigableMap<RegionInfo, ServerName> regions = new TreeMap<>();
Visitor visitor = new TableVisitorBase(tableName) {
@Override
public boolean visitInternal(Result result) throws IOException {
@@ -178,7 +181,7 @@ public class MetaTableAccessor {
if (locations == null) return true;
for (HRegionLocation loc : locations.getRegionLocations()) {
if (loc != null) {
- HRegionInfo regionInfo = loc.getRegionInfo();
+ RegionInfo regionInfo = loc.getRegionInfo();
regions.put(regionInfo, loc.getServerName());
}
}
@@ -300,12 +303,12 @@ public class MetaTableAccessor {
* Gets the region info and assignment for the specified region.
* @param connection connection we're using
* @param regionName Region to lookup.
- * @return Location and HRegionInfo for <code>regionName</code>
+ * @return Location and RegionInfo for <code>regionName</code>
* @throws IOException
* @deprecated use {@link #getRegionLocation(Connection, byte[])} instead
*/
@Deprecated
- public static Pair<HRegionInfo, ServerName> getRegion(Connection connection, byte [] regionName)
+ public static Pair<RegionInfo, ServerName> getRegion(Connection connection, byte [] regionName)
throws IOException {
HRegionLocation location = getRegionLocation(connection, regionName);
return location == null
@@ -323,7 +326,7 @@ public class MetaTableAccessor {
public static HRegionLocation getRegionLocation(Connection connection,
byte[] regionName) throws IOException {
byte[] row = regionName;
- HRegionInfo parsedInfo = null;
+ RegionInfo parsedInfo = null;
try {
parsedInfo = parseRegionInfoFromRegionName(regionName);
row = getMetaKeyForRegion(parsedInfo);
@@ -347,7 +350,7 @@ public class MetaTableAccessor {
* @throws IOException
*/
public static HRegionLocation getRegionLocation(Connection connection,
- HRegionInfo regionInfo) throws IOException {
+ RegionInfo regionInfo) throws IOException {
byte[] row = getMetaKeyForRegion(regionInfo);
Get get = new Get(row);
get.addFamily(HConstants.CATALOG_FAMILY);
@@ -356,7 +359,7 @@ public class MetaTableAccessor {
}
/** Returns the row key to use for this regionInfo */
- public static byte[] getMetaKeyForRegion(HRegionInfo regionInfo) {
+ public static byte[] getMetaKeyForRegion(RegionInfo regionInfo) {
return RegionReplicaUtil.getRegionInfoForDefaultReplica(regionInfo).getRegionName();
}
@@ -364,12 +367,17 @@ public class MetaTableAccessor {
* is stored in the name, so the returned object should only be used for the fields
* in the regionName.
*/
- public static HRegionInfo parseRegionInfoFromRegionName(byte[] regionName) throws IOException {
- byte[][] fields = HRegionInfo.parseRegionName(regionName);
+ public static RegionInfo parseRegionInfoFromRegionName(byte[] regionName) throws IOException {
+ byte[][] fields = RegionInfo.parseRegionName(regionName);
long regionId = Long.parseLong(Bytes.toString(fields[2]));
int replicaId = fields.length > 3 ? Integer.parseInt(Bytes.toString(fields[3]), 16) : 0;
- return new HRegionInfo(TableName.valueOf(fields[0]), fields[1], fields[1], false, regionId,
- replicaId);
+ return RegionInfoBuilder.newBuilder(TableName.valueOf(fields[0]))
+ .setStartKey(fields[1])
+ .setEndKey(fields[2])
+ .setSplit(false)
+ .setRegionId(regionId)
+ .setReplicaId(replicaId)
+ .build();
}
/**
@@ -392,11 +400,11 @@ public class MetaTableAccessor {
* @throws IOException
*/
@Nullable
- public static Pair<HRegionInfo, HRegionInfo> getRegionsFromMergeQualifier(
+ public static Pair<RegionInfo, RegionInfo> getRegionsFromMergeQualifier(
Connection connection, byte[] regionName) throws IOException {
Result result = getRegionResult(connection, regionName);
- HRegionInfo mergeA = getHRegionInfo(result, HConstants.MERGEA_QUALIFIER);
- HRegionInfo mergeB = getHRegionInfo(result, HConstants.MERGEB_QUALIFIER);
+ RegionInfo mergeA = getRegionInfo(result, HConstants.MERGEA_QUALIFIER);
+ RegionInfo mergeB = getRegionInfo(result, HConstants.MERGEB_QUALIFIER);
if (mergeA == null && mergeB == null) {
return null;
}
@@ -429,15 +437,15 @@ public class MetaTableAccessor {
* @throws IOException
*/
@VisibleForTesting
- public static List<HRegionInfo> getAllRegions(Connection connection,
+ public static List<RegionInfo> getAllRegions(Connection connection,
boolean excludeOfflinedSplitParents)
throws IOException {
- List<Pair<HRegionInfo, ServerName>> result;
+ List<Pair<RegionInfo, ServerName>> result;
result = getTableRegionsAndLocations(connection, null,
excludeOfflinedSplitParents);
- return getListOfHRegionInfos(result);
+ return getListOfRegionInfos(result);
}
@@ -446,10 +454,10 @@ public class MetaTableAccessor {
* to get meta table regions, use methods in MetaTableLocator instead.
* @param connection connection we're using
* @param tableName table we're looking for
- * @return Ordered list of {@link HRegionInfo}.
+ * @return Ordered list of {@link RegionInfo}.
* @throws IOException
*/
- public static List<HRegionInfo> getTableRegions(Connection connection, TableName tableName)
+ public static List<RegionInfo> getTableRegions(Connection connection, TableName tableName)
throws IOException {
return getTableRegions(connection, tableName, false);
}
@@ -461,22 +469,21 @@ public class MetaTableAccessor {
* @param tableName table we're looking for
* @param excludeOfflinedSplitParents If true, do not include offlined split
* parents in the return.
- * @return Ordered list of {@link HRegionInfo}.
+ * @return Ordered list of {@link RegionInfo}.
* @throws IOException
*/
- public static List<HRegionInfo> getTableRegions(Connection connection,
+ public static List<RegionInfo> getTableRegions(Connection connection,
TableName tableName, final boolean excludeOfflinedSplitParents)
throws IOException {
- List<Pair<HRegionInfo, ServerName>> result =
+ List<Pair<RegionInfo, ServerName>> result =
getTableRegionsAndLocations(connection, tableName, excludeOfflinedSplitParents);
- return getListOfHRegionInfos(result);
+ return getListOfRegionInfos(result);
}
- @Nullable
- static List<HRegionInfo> getListOfHRegionInfos(final List<Pair<HRegionInfo, ServerName>> pairs) {
- if (pairs == null || pairs.isEmpty()) return null;
- List<HRegionInfo> result = new ArrayList<>(pairs.size());
- for (Pair<HRegionInfo, ServerName> pair: pairs) {
+ static List<RegionInfo> getListOfRegionInfos(final List<Pair<RegionInfo, ServerName>> pairs) {
+ if (pairs == null || pairs.isEmpty()) return Collections.EMPTY_LIST;
+ List<RegionInfo> result = new ArrayList<>(pairs.size());
+ for (Pair<RegionInfo, ServerName> pair: pairs) {
result.add(pair.getFirst());
}
return result;
@@ -488,7 +495,7 @@ public class MetaTableAccessor {
* @return True if <code>current</code> tablename is equal to
* <code>tableName</code>
*/
- static boolean isInsideTable(final HRegionInfo current, final TableName tableName) {
+ static boolean isInsideTable(final RegionInfo current, final TableName tableName) {
return tableName.equals(current.getTable());
}
@@ -587,7 +594,7 @@ public class MetaTableAccessor {
* @return Return list of regioninfos and server.
* @throws IOException
*/
- public static List<Pair<HRegionInfo, ServerName>>
+ public static List<Pair<RegionInfo, ServerName>>
getTableRegionsAndLocations(Connection connection, TableName tableName)
throws IOException {
return getTableRegionsAndLocations(connection, tableName, true);
@@ -601,26 +608,26 @@ public class MetaTableAccessor {
* @return Return list of regioninfos and server addresses.
* @throws IOException
*/
- public static List<Pair<HRegionInfo, ServerName>> getTableRegionsAndLocations(
+ public static List<Pair<RegionInfo, ServerName>> getTableRegionsAndLocations(
Connection connection, @Nullable final TableName tableName,
final boolean excludeOfflinedSplitParents) throws IOException {
if (tableName != null && tableName.equals(TableName.META_TABLE_NAME)) {
throw new IOException("This method can't be used to locate meta regions;"
+ " use MetaTableLocator instead");
}
- // Make a version of CollectingVisitor that collects HRegionInfo and ServerAddress
- CollectingVisitor<Pair<HRegionInfo, ServerName>> visitor =
- new CollectingVisitor<Pair<HRegionInfo, ServerName>>() {
+ // Make a version of CollectingVisitor that collects RegionInfo and ServerAddress
+ CollectingVisitor<Pair<RegionInfo, ServerName>> visitor =
+ new CollectingVisitor<Pair<RegionInfo, ServerName>>() {
private RegionLocations current = null;
@Override
public boolean visit(Result r) throws IOException {
current = getRegionLocations(r);
if (current == null || current.getRegionLocation().getRegionInfo() == null) {
- LOG.warn("No serialized HRegionInfo in " + r);
+ LOG.warn("No serialized RegionInfo in " + r);
return true;
}
- HRegionInfo hri = current.getRegionLocation().getRegionInfo();
+ RegionInfo hri = current.getRegionLocation().getRegionInfo();
if (excludeOfflinedSplitParents && hri.isSplitParent()) return true;
// Else call super and add this Result to the collection.
return super.visit(r);
@@ -652,10 +659,10 @@ public class MetaTableAccessor {
* catalog regions).
* @throws IOException
*/
- public static NavigableMap<HRegionInfo, Result>
+ public static NavigableMap<RegionInfo, Result>
getServerUserRegions(Connection connection, final ServerName serverName)
throws IOException {
- final NavigableMap<HRegionInfo, Result> hris = new TreeMap<>();
+ final NavigableMap<RegionInfo, Result> hris = new TreeMap<>();
// Fill the above hris map with entries from hbase:meta that have the passed
// servername.
CollectingVisitor<Result> v = new CollectingVisitor<Result>() {
@@ -741,9 +748,9 @@ public class MetaTableAccessor {
startRow =
getTableStartRowForMeta(tableName, QueryType.REGION);
if (row != null) {
- HRegionInfo closestRi =
+ RegionInfo closestRi =
getClosestRegionInfo(connection, tableName, row);
- startRow = HRegionInfo
+ startRow = RegionInfo
.createRegionName(tableName, closestRi.getStartKey(), HConstants.ZEROES, false);
}
stopRow =
@@ -813,11 +820,11 @@ public class MetaTableAccessor {
* @throws java.io.IOException
*/
@NonNull
- public static HRegionInfo getClosestRegionInfo(Connection connection,
+ public static RegionInfo getClosestRegionInfo(Connection connection,
@NonNull final TableName tableName,
@NonNull final byte[] row)
throws IOException {
- byte[] searchRow = HRegionInfo.createRegionName(tableName, row, HConstants.NINES, false);
+ byte[] searchRow = RegionInfo.createRegionName(tableName, row, HConstants.NINES, false);
Scan scan = getMetaScan(connection, 1);
scan.setReversed(true);
scan.setStartRow(searchRow);
@@ -827,9 +834,9 @@ public class MetaTableAccessor {
throw new TableNotFoundException("Cannot find row in META " +
" for table: " + tableName + ", row=" + Bytes.toStringBinary(row));
}
- HRegionInfo regionInfo = getHRegionInfo(result);
+ RegionInfo regionInfo = getRegionInfo(result);
if (regionInfo == null) {
- throw new IOException("HRegionInfo was null or empty in Meta for " +
+ throw new IOException("RegionInfo was null or empty in Meta for " +
tableName + ", row=" + Bytes.toStringBinary(row));
}
return regionInfo;
@@ -879,7 +886,7 @@ public class MetaTableAccessor {
return replicaId == 0
? HConstants.SERVER_QUALIFIER
: Bytes.toBytes(HConstants.SERVER_QUALIFIER_STR + META_REPLICA_ID_DELIMITER
- + String.format(HRegionInfo.REPLICA_ID_FORMAT, replicaId));
+ + String.format(RegionInfo.REPLICA_ID_FORMAT, replicaId));
}
/**
@@ -892,7 +899,7 @@ public class MetaTableAccessor {
return replicaId == 0
? HConstants.STARTCODE_QUALIFIER
: Bytes.toBytes(HConstants.STARTCODE_QUALIFIER_STR + META_REPLICA_ID_DELIMITER
- + String.format(HRegionInfo.REPLICA_ID_FORMAT, replicaId));
+ + String.format(RegionInfo.REPLICA_ID_FORMAT, replicaId));
}
/**
@@ -905,7 +912,7 @@ public class MetaTableAccessor {
return replicaId == 0
? HConstants.SEQNUM_QUALIFIER
: Bytes.toBytes(HConstants.SEQNUM_QUALIFIER_STR + META_REPLICA_ID_DELIMITER
- + String.format(HRegionInfo.REPLICA_ID_FORMAT, replicaId));
+ + String.format(RegionInfo.REPLICA_ID_FORMAT, replicaId));
}
/**
@@ -975,7 +982,7 @@ public class MetaTableAccessor {
@Nullable
public static RegionLocations getRegionLocations(final Result r) {
if (r == null) return null;
- HRegionInfo regionInfo = getHRegionInfo(r, getRegionInfoColumn());
+ RegionInfo regionInfo = getRegionInfo(r, getRegionInfoColumn());
if (regionInfo == null) return null;
List<HRegionLocation> locations = new ArrayList<>(1);
@@ -1021,37 +1028,37 @@ public class MetaTableAccessor {
* @param replicaId the replicaId for the HRegionLocation
* @return HRegionLocation parsed from the given meta row Result for the given replicaId
*/
- private static HRegionLocation getRegionLocation(final Result r, final HRegionInfo regionInfo,
+ private static HRegionLocation getRegionLocation(final Result r, final RegionInfo regionInfo,
final int replicaId) {
ServerName serverName = getServerName(r, replicaId);
long seqNum = getSeqNumDuringOpen(r, replicaId);
- HRegionInfo replicaInfo = RegionReplicaUtil.getRegionInfoForReplica(regionInfo, replicaId);
+ RegionInfo replicaInfo = RegionReplicaUtil.getRegionInfoForReplica(regionInfo, replicaId);
return new HRegionLocation(replicaInfo, serverName, seqNum);
}
/**
- * Returns HRegionInfo object from the column
+ * Returns RegionInfo object from the column
* HConstants.CATALOG_FAMILY:HConstants.REGIONINFO_QUALIFIER of the catalog
* table Result.
* @param data a Result object from the catalog table scan
- * @return HRegionInfo or null
+ * @return RegionInfo or null
*/
- public static HRegionInfo getHRegionInfo(Result data) {
- return getHRegionInfo(data, HConstants.REGIONINFO_QUALIFIER);
+ public static RegionInfo getRegionInfo(Result data) {
+ return getRegionInfo(data, HConstants.REGIONINFO_QUALIFIER);
}
/**
- * Returns the HRegionInfo object from the column {@link HConstants#CATALOG_FAMILY} and
+ * Returns the RegionInfo object from the column {@link HConstants#CATALOG_FAMILY} and
* <code>qualifier</code> of the catalog table result.
* @param r a Result object from the catalog table scan
* @param qualifier Column family qualifier
- * @return An HRegionInfo instance or null.
+ * @return An RegionInfo instance or null.
*/
@Nullable
- private static HRegionInfo getHRegionInfo(final Result r, byte [] qualifier) {
+ private static RegionInfo getRegionInfo(final Result r, byte [] qualifier) {
Cell cell = r.getColumnLatestCell(getCatalogFamily(), qualifier);
if (cell == null) return null;
- return HRegionInfo.parseFromOrNull(cell.getValueArray(),
+ return RegionInfo.parseFromOrNull(cell.getValueArray(),
cell.getValueOffset(), cell.getValueLength());
}
@@ -1059,12 +1066,12 @@ public class MetaTableAccessor {
* Returns the daughter regions by reading the corresponding columns of the catalog table
* Result.
* @param data a Result object from the catalog table scan
- * @return a pair of HRegionInfo or PairOfSameType(null, null) if the region is not a split
+ * @return a pair of RegionInfo or PairOfSameType(null, null) if the region is not a split
* parent
*/
- public static PairOfSameType<HRegionInfo> getDaughterRegions(Result data) {
- HRegionInfo splitA = getHRegionInfo(data, HConstants.SPLITA_QUALIFIER);
- HRegionInfo splitB = getHRegionInfo(data, HConstants.SPLITB_QUALIFIER);
+ public static PairOfSameType<RegionInfo> getDaughterRegions(Result data) {
+ RegionInfo splitA = getRegionInfo(data, HConstants.SPLITA_QUALIFIER);
+ RegionInfo splitB = getRegionInfo(data, HConstants.SPLITB_QUALIFIER);
return new PairOfSameType<>(splitA, splitB);
}
@@ -1073,12 +1080,12 @@ public class MetaTableAccessor {
* Returns the merge regions by reading the corresponding columns of the catalog table
* Result.
* @param data a Result object from the catalog table scan
- * @return a pair of HRegionInfo or PairOfSameType(null, null) if the region is not a split
+ * @return a pair of RegionInfo or PairOfSameType(null, null) if the region is not a split
* parent
*/
- public static PairOfSameType<HRegionInfo> getMergeRegions(Result data) {
- HRegionInfo mergeA = getHRegionInfo(data, HConstants.MERGEA_QUALIFIER);
- HRegionInfo mergeB = getHRegionInfo(data, HConstants.MERGEB_QUALIFIER);
+ public static PairOfSameType<RegionInfo> getMergeRegions(Result data) {
+ RegionInfo mergeA = getRegionInfo(data, HConstants.MERGEA_QUALIFIER);
+ RegionInfo mergeB = getRegionInfo(data, HConstants.MERGEB_QUALIFIER);
return new PairOfSameType<>(mergeA, mergeB);
}
@@ -1222,7 +1229,7 @@ public class MetaTableAccessor {
@Override
public boolean visit(Result rowResult) throws IOException {
- HRegionInfo info = getHRegionInfo(rowResult);
+ RegionInfo info = getRegionInfo(rowResult);
if (info == null) {
return true;
}
@@ -1251,7 +1258,7 @@ public class MetaTableAccessor {
@Override
public final boolean visit(Result rowResult) throws IOException {
- HRegionInfo info = getHRegionInfo(rowResult);
+ RegionInfo info = getRegionInfo(rowResult);
if (info == null) {
return true;
}
@@ -1298,7 +1305,7 @@ public class MetaTableAccessor {
/**
* Generates and returns a Put containing the region into for the catalog table
*/
- public static Put makePutFromRegionInfo(HRegionInfo regionInfo)
+ public static Put makePutFromRegionInfo(RegionInfo regionInfo)
throws IOException {
return makePutFromRegionInfo(regionInfo, EnvironmentEdgeManager.currentTime());
}
@@ -1306,7 +1313,7 @@ public class MetaTableAccessor {
/**
* Generates and returns a Put containing the region into for the catalog table
*/
- public static Put makePutFromRegionInfo(HRegionInfo regionInfo, long ts)
+ public static Put makePutFromRegionInfo(RegionInfo regionInfo, long ts)
throws IOException {
Put put = new Put(regionInfo.getRegionName(), ts);
addRegionInfo(put, regionInfo);
@@ -1317,7 +1324,7 @@ public class MetaTableAccessor {
* Generates and returns a Delete containing the region info for the catalog
* table
*/
- public static Delete makeDeleteFromRegionInfo(HRegionInfo regionInfo) {
+ public static Delete makeDeleteFromRegionInfo(RegionInfo regionInfo) {
long now = EnvironmentEdgeManager.currentTime();
return makeDeleteFromRegionInfo(regionInfo, now);
}
@@ -1326,7 +1333,7 @@ public class MetaTableAccessor {
* Generates and returns a Delete containing the region info for the catalog
* table
*/
- public static Delete makeDeleteFromRegionInfo(HRegionInfo regionInfo, long ts) {
+ public static Delete makeDeleteFromRegionInfo(RegionInfo regionInfo, long ts) {
if (regionInfo == null) {
throw new IllegalArgumentException("Can't make a delete for null region");
}
@@ -1356,14 +1363,14 @@ public class MetaTableAccessor {
/**
* Adds split daughters to the Put
*/
- public static Put addDaughtersToPut(Put put, HRegionInfo splitA, HRegionInfo splitB) {
+ public static Put addDaughtersToPut(Put put, RegionInfo splitA, RegionInfo splitB) {
if (splitA != null) {
put.addImmutable(
- HConstants.CATALOG_FAMILY, HConstants.SPLITA_QUALIFIER, splitA.toByteArray());
+ HConstants.CATALOG_FAMILY, HConstants.SPLITA_QUALIFIER, RegionInfo.toByteArray(splitA));
}
if (splitB != null) {
put.addImmutable(
- HConstants.CATALOG_FAMILY, HConstants.SPLITB_QUALIFIER, splitB.toByteArray());
+ HConstants.CATALOG_FAMILY, HConstants.SPLITB_QUALIFIER, RegionInfo.toByteArray(splitB));
}
return put;
}
@@ -1505,7 +1512,7 @@ public class MetaTableAccessor {
* @throws IOException if problem connecting or updating meta
*/
public static void addRegionToMeta(Connection connection,
- HRegionInfo regionInfo)
+ RegionInfo regionInfo)
throws IOException {
putToMetaTable(connection, makePutFromRegionInfo(regionInfo));
LOG.info("Added " + regionInfo.getRegionNameAsString());
@@ -1518,7 +1525,7 @@ public class MetaTableAccessor {
* @param regionInfo region information
* @throws IOException if problem connecting or updating meta
*/
- public static void addRegionToMeta(Table meta, HRegionInfo regionInfo) throws IOException {
+ public static void addRegionToMeta(Table meta, RegionInfo regionInfo) throws IOException {
addRegionToMeta(meta, regionInfo, null, null);
}
@@ -1526,7 +1533,7 @@ public class MetaTableAccessor {
* Adds a (single) hbase:meta row for the specified new region and its daughters. Note that this
* does not add its daughter's as different rows, but adds information about the daughters
* in the same row as the parent. Use
- * {@link #splitRegion(Connection, HRegionInfo, HRegionInfo, HRegionInfo, ServerName,int,boolean)}
+ * {@link #splitRegion(Connection, RegionInfo, RegionInfo, RegionInfo, ServerName,int,boolean)}
* if you want to do that.
* @param meta the Table for META
* @param regionInfo region information
@@ -1534,8 +1541,8 @@ public class MetaTableAccessor {
* @param splitB second split daughter of the parent regionInfo
* @throws IOException if problem connecting or updating meta
*/
- public static void addRegionToMeta(Table meta, HRegionInfo regionInfo,
- HRegionInfo splitA, HRegionInfo splitB) throws IOException {
+ public static void addRegionToMeta(Table meta, RegionInfo regionInfo,
+ RegionInfo splitA, RegionInfo splitB) throws IOException {
Put put = makePutFromRegionInfo(regionInfo);
addDaughtersToPut(put, splitA, splitB);
meta.put(put);
@@ -1551,7 +1558,7 @@ public class MetaTableAccessor {
* Adds a (single) hbase:meta row for the specified new region and its daughters. Note that this
* does not add its daughter's as different rows, but adds information about the daughters
* in the same row as the parent. Use
- * {@link #splitRegion(Connection, HRegionInfo, HRegionInfo, HRegionInfo, ServerName,int,boolean)}
+ * {@link #splitRegion(Connection, RegionInfo, RegionInfo, RegionInfo, ServerName,int,boolean)}
* if you want to do that.
* @param connection connection we're using
* @param regionInfo region information
@@ -1559,8 +1566,8 @@ public class MetaTableAccessor {
* @param splitB second split daughter of the parent regionInfo
* @throws IOException if problem connecting or updating meta
*/
- public static void addRegionToMeta(Connection connection, HRegionInfo regionInfo,
- HRegionInfo splitA, HRegionInfo splitB) throws IOException {
+ public static void addRegionToMeta(Connection connection, RegionInfo regionInfo,
+ RegionInfo splitA, RegionInfo splitB) throws IOException {
Table meta = getMetaHTable(connection);
try {
addRegionToMeta(meta, regionInfo, splitA, splitB);
@@ -1576,7 +1583,7 @@ public class MetaTableAccessor {
* @throws IOException if problem connecting or updating meta
*/
public static void addRegionsToMeta(Connection connection,
- List<HRegionInfo> regionInfos, int regionReplication)
+ List<RegionInfo> regionInfos, int regionReplication)
throws IOException {
addRegionsToMeta(connection, regionInfos, regionReplication, HConstants.LATEST_TIMESTAMP);
}
@@ -1589,10 +1596,10 @@ public class MetaTableAccessor {
* @throws IOException if problem connecting or updating meta
*/
public static void addRegionsToMeta(Connection connection,
- List<HRegionInfo> regionInfos, int regionReplication, long ts)
+ List<RegionInfo> regionInfos, int regionReplication, long ts)
throws IOException {
List<Put> puts = new ArrayList<>();
- for (HRegionInfo regionInfo : regionInfos) {
+ for (RegionInfo regionInfo : regionInfos) {
if (RegionReplicaUtil.isDefaultReplica(regionInfo)) {
Put put = makePutFromRegionInfo(regionInfo, ts);
// Add empty locations for region replicas so that number of replicas can be cached
@@ -1614,7 +1621,7 @@ public class MetaTableAccessor {
* @param openSeqNum the latest sequence number obtained when the region was open
*/
public static void addDaughter(final Connection connection,
- final HRegionInfo regionInfo, final ServerName sn, final long openSeqNum)
+ final RegionInfo regionInfo, final ServerName sn, final long openSeqNum)
throws NotAllMetaRegionsOnlineException, IOException {
long now = EnvironmentEdgeManager.currentTime();
Put put = new Put(regionInfo.getRegionName(), now);
@@ -1640,23 +1647,21 @@ public class MetaTableAccessor {
* @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,
+ public static void mergeRegions(final Connection connection, RegionInfo mergedRegion,
+ RegionInfo regionA, RegionInfo regionB, ServerName sn, int regionReplication,
long masterSystemTime, 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(mergedRegion, time);
putOfMerged.addImmutable(HConstants.CATALOG_FAMILY, HConstants.MERGEA_QUALIFIER,
- regionA.toByteArray());
+ RegionInfo.toByteArray(regionA));
putOfMerged.addImmutable(HConstants.CATALOG_FAMILY, HConstants.MERGEB_QUALIFIER,
- regionB.toByteArray());
+ RegionInfo.toByteArray(regionB));
// Deletes for merging regions
Delete deleteA = makeDeleteFromRegionInfo(regionA, time);
@@ -1707,17 +1712,15 @@ public class MetaTableAccessor {
* @param sn the location of the region
* @param saveBarrier true if need save replication barrier in meta, used for serial replication
*/
- public static void splitRegion(final Connection connection, HRegionInfo parent,
- HRegionInfo splitA, HRegionInfo splitB, ServerName sn, int regionReplication,
+ public static void splitRegion(final Connection connection, RegionInfo parent,
+ RegionInfo splitA, RegionInfo splitB, ServerName sn, int regionReplication,
boolean saveBarrier) throws IOException {
Table meta = getMetaHTable(connection);
try {
- HRegionInfo copyOfParent = new HRegionInfo(parent);
- copyOfParent.setOffline(true);
- copyOfParent.setSplit(true);
-
//Put for parent
- Put putParent = makePutFromRegionInfo(copyOfParent);
+ Put putParent = makePutFromRegionInfo(RegionInfoBuilder.newBuilder(parent)
+ .setOffline(true)
+ .setSplit(true).build());
addDaughtersToPut(putParent, splitA, splitB);
//Puts for daughters
@@ -1878,7 +1881,7 @@ public class MetaTableAccessor {
* @throws IOException
*/
public static void updateRegionLocation(Connection connection,
- HRegionInfo regionInfo, ServerName sn, long openSeqNum,
+ RegionInfo regionInfo, ServerName sn, long openSeqNum,
long masterSystemTime)
throws IOException {
updateLocation(connection, regionInfo, sn, openSeqNum, masterSystemTime);
@@ -1920,7 +1923,7 @@ public class MetaTableAccessor {
* if the server is down on other end.
*/
private static void updateLocation(final Connection connection,
- HRegionInfo regionInfo, ServerName sn, long openSeqNum,
+ RegionInfo regionInfo, ServerName sn, long openSeqNum,
long masterSystemTime)
throws IOException {
@@ -1943,7 +1946,7 @@ public class MetaTableAccessor {
* @throws IOException
*/
public static void deleteRegion(Connection connection,
- HRegionInfo regionInfo)
+ RegionInfo regionInfo)
throws IOException {
long time = EnvironmentEdgeManager.currentTime();
Delete delete = new Delete(regionInfo.getRegionName());
@@ -1959,7 +1962,7 @@ public class MetaTableAccessor {
* @throws IOException
*/
public static void deleteRegions(Connection connection,
- List<HRegionInfo> regionsInfo) throws IOException {
+ List<RegionInfo> regionsInfo) throws IOException {
deleteRegions(connection, regionsInfo, EnvironmentEdgeManager.currentTime());
}
/**
@@ -1968,10 +1971,10 @@ public class MetaTableAccessor {
* @param regionsInfo list of regions to be deleted from META
* @throws IOException
*/
- public static void deleteRegions(Connection connection, List<HRegionInfo> regionsInfo, long ts)
+ public static void deleteRegions(Connection connection, List<RegionInfo> regionsInfo, long ts)
throws IOException {
List<Delete> deletes = new ArrayList<>(regionsInfo.size());
- for (HRegionInfo hri: regionsInfo) {
+ for (RegionInfo hri: regionsInfo) {
Delete e = new Delete(hri.getRegionName());
e.addFamily(getCatalogFamily(), ts);
deletes.add(e);
@@ -1991,26 +1994,26 @@ public class MetaTableAccessor {
* @throws IOException
*/
public static void mutateRegions(Connection connection,
- final List<HRegionInfo> regionsToRemove,
- final List<HRegionInfo> regionsToAdd)
+ final List<RegionInfo> regionsToRemove,
+ final List<RegionInfo> regionsToAdd)
throws IOException {
List<Mutation> mutation = new ArrayList<>();
if (regionsToRemove != null) {
- for (HRegionInfo hri: regionsToRemove) {
+ for (RegionInfo hri: regionsToRemove) {
mutation.add(makeDeleteFromRegionInfo(hri));
}
}
if (regionsToAdd != null) {
- for (HRegionInfo hri: regionsToAdd) {
+ for (RegionInfo hri: regionsToAdd) {
mutation.add(makePutFromRegionInfo(hri));
}
}
mutateMetaTable(connection, mutation);
if (regionsToRemove != null && regionsToRemove.size() > 0) {
- LOG.debug("Deleted " + HRegionInfo.getShortNameToLog(regionsToRemove));
+ LOG.debug("Deleted " + RegionInfo.getShortNameToLog(regionsToRemove));
}
if (regionsToAdd != null && regionsToAdd.size() > 0) {
- LOG.debug("Added " + HRegionInfo.getShortNameToLog(regionsToAdd));
+ LOG.debug("Added " + RegionInfo.getShortNameToLog(regionsToAdd));
}
}
@@ -2021,7 +2024,7 @@ public class MetaTableAccessor {
* @throws IOException
*/
public static void overwriteRegions(Connection connection,
- List<HRegionInfo> regionInfos, int regionReplication) throws IOException {
+ List<RegionInfo> regionInfos, int regionReplication) throws IOException {
// use master time for delete marker and the Put
long now = EnvironmentEdgeManager.currentTime();
deleteRegions(connection, regionInfos, now);
@@ -2045,7 +2048,7 @@ public class MetaTableAccessor {
* @throws IOException
*/
public static void deleteMergeQualifiers(Connection connection,
- final HRegionInfo mergedRegion) throws IOException {
+ final RegionInfo mergedRegion) throws IOException {
long time = EnvironmentEdgeManager.currentTime();
Delete delete = new Delete(mergedRegion.getRegionName());
delete.addColumns(getCatalogFamily(), HConstants.MERGEA_QUALIFIER, time);
@@ -2057,10 +2060,10 @@ public class MetaTableAccessor {
+ Bytes.toStringBinary(HConstants.MERGEB_QUALIFIER));
}
- public static Put addRegionInfo(final Put p, final HRegionInfo hri)
+ public static Put addRegionInfo(final Put p, final RegionInfo hri)
throws IOException {
p.addImmutable(getCatalogFamily(), HConstants.REGIONINFO_QUALIFIER,
- hri.toByteArray());
+ RegionInfo.toByteArray(hri));
return p;
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionLocations.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionLocations.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionLocations.java
index bbff8ce..8b3fbb4 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionLocations.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionLocations.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hbase;
import java.util.Collection;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
import org.apache.hadoop.hbase.util.Bytes;
@@ -143,14 +144,14 @@ public class RegionLocations {
*/
public RegionLocations remove(HRegionLocation location) {
if (location == null) return this;
- if (location.getRegionInfo() == null) return this;
+ if (location.getRegion() == null) return this;
int replicaId = location.getRegionInfo().getReplicaId();
if (replicaId >= locations.length) return this;
// check whether something to remove. HRL.compareTo() compares ONLY the
// serverName. We want to compare the HRI's as well.
if (locations[replicaId] == null
- || !location.getRegionInfo().equals(locations[replicaId].getRegionInfo())
+ || RegionInfo.COMPARATOR.compare(location.getRegion(), locations[replicaId].getRegion()) != 0
|| !location.equals(locations[replicaId])) {
return this;
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
index b2c6969..e539c6a 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
@@ -17,9 +17,9 @@
*/
package org.apache.hadoop.hbase.client;
-import java.util.List;
import java.util.Collection;
import java.util.EnumSet;
+import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.Set;
@@ -29,20 +29,19 @@ import java.util.regex.Pattern;
import org.apache.hadoop.hbase.ClusterStatus;
import org.apache.hadoop.hbase.ClusterStatus.Option;
-import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.RegionLoad;
import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.TableName;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.hbase.quotas.QuotaFilter;
-import org.apache.hadoop.hbase.quotas.QuotaSettings;
import org.apache.hadoop.hbase.client.RawAsyncTable.CoprocessorCallable;
import org.apache.hadoop.hbase.client.replication.TableCFs;
import org.apache.hadoop.hbase.client.security.SecurityCapability;
+import org.apache.hadoop.hbase.quotas.QuotaFilter;
+import org.apache.hadoop.hbase.quotas.QuotaSettings;
import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
import org.apache.hadoop.hbase.util.Pair;
+import org.apache.yetus.audience.InterfaceAudience;
import com.google.protobuf.RpcChannel;
@@ -306,12 +305,12 @@ public interface AsyncAdmin {
/**
* Get all the online regions on a region server.
*/
- CompletableFuture<List<HRegionInfo>> getOnlineRegions(ServerName serverName);
+ CompletableFuture<List<RegionInfo>> getOnlineRegions(ServerName serverName);
/**
* Get the regions of a given table.
*/
- CompletableFuture<List<HRegionInfo>> getTableRegions(TableName tableName);
+ CompletableFuture<List<RegionInfo>> getTableRegions(TableName tableName);
/**
* Flush a table.
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java
index a3cc89f..3cb8360 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java
@@ -32,12 +32,10 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.ClusterStatus;
import org.apache.hadoop.hbase.ClusterStatus.Option;
-import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.RegionLoad;
import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.TableName;
-import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.RawAsyncTable.CoprocessorCallable;
import org.apache.hadoop.hbase.client.replication.TableCFs;
import org.apache.hadoop.hbase.client.security.SecurityCapability;
@@ -46,6 +44,7 @@ import org.apache.hadoop.hbase.quotas.QuotaSettings;
import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
import org.apache.hadoop.hbase.util.Pair;
+import org.apache.yetus.audience.InterfaceAudience;
import com.google.protobuf.RpcChannel;
@@ -215,12 +214,12 @@ public class AsyncHBaseAdmin implements AsyncAdmin {
}
@Override
- public CompletableFuture<List<HRegionInfo>> getOnlineRegions(ServerName serverName) {
+ public CompletableFuture<List<RegionInfo>> getOnlineRegions(ServerName serverName) {
return wrap(rawAdmin.getOnlineRegions(serverName));
}
@Override
- public CompletableFuture<List<HRegionInfo>> getTableRegions(TableName tableName) {
+ public CompletableFuture<List<RegionInfo>> getTableRegions(TableName tableName) {
return wrap(rawAdmin.getTableRegions(tableName));
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
index 860eef3..9d888e6 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
@@ -57,7 +57,6 @@ import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotEnabledException;
import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
-import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.backoff.ClientBackoffPolicy;
import org.apache.hadoop.hbase.client.backoff.ClientBackoffPolicyFactory;
import org.apache.hadoop.hbase.exceptions.ClientExceptionsUtil;
@@ -67,6 +66,20 @@ import org.apache.hadoop.hbase.ipc.RpcClientFactory;
import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;
import org.apache.hadoop.hbase.security.User;
+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.ReflectionUtils;
+import org.apache.hadoop.hbase.util.Threads;
+import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
+import org.apache.hadoop.hbase.zookeeper.ZKUtil;
+import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.zookeeper.KeeperException;
+
+import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.BlockingRpcChannel;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
@@ -110,19 +123,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.Remov
import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.RemoveReplicationPeerResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigResponse;
-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.ReflectionUtils;
-import org.apache.hadoop.hbase.util.Threads;
-import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
-import org.apache.hadoop.hbase.zookeeper.ZKUtil;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
-import org.apache.hadoop.ipc.RemoteException;
-import org.apache.zookeeper.KeeperException;
-
-import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
import edu.umd.cs.findbugs.annotations.Nullable;
@@ -600,13 +600,13 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
LOG.debug("Table " + tableName + " not enabled");
return false;
}
- List<Pair<HRegionInfo, ServerName>> locations =
+ List<Pair<RegionInfo, ServerName>> locations =
MetaTableAccessor.getTableRegionsAndLocations(this, tableName, true);
int notDeployed = 0;
int regionCount = 0;
- for (Pair<HRegionInfo, ServerName> pair : locations) {
- HRegionInfo info = pair.getFirst();
+ for (Pair<RegionInfo, ServerName> pair : locations) {
+ RegionInfo info = pair.getFirst();
if (pair.getSecond() == null) {
if (LOG.isDebugEnabled()) {
LOG.debug("Table " + tableName + " has not deployed region " + pair.getFirst()
@@ -675,10 +675,10 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
@Override
public List<HRegionLocation> locateRegions(final TableName tableName,
final boolean useCache, final boolean offlined) throws IOException {
- List<HRegionInfo> regions = MetaTableAccessor
+ List<RegionInfo> regions = MetaTableAccessor
.getTableRegions(this, tableName, !offlined);
final List<HRegionLocation> locations = new ArrayList<>();
- for (HRegionInfo regionInfo : regions) {
+ for (RegionInfo regionInfo : regions) {
RegionLocations list = locateRegion(tableName, regionInfo.getStartKey(), useCache, true);
if (list != null) {
for (HRegionLocation loc : list.getRegionLocations()) {
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-client/src/main/java/org/apache/hadoop/hbase/client/FlushRegionCallable.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/FlushRegionCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/FlushRegionCallable.java
index af5c4d6..442bf1d 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/FlushRegionCallable.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/FlushRegionCallable.java
@@ -22,16 +22,16 @@ import java.io.IOException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.TableName;
-import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.ipc.HBaseRpcController;
import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.yetus.audience.InterfaceAudience;
+
import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
/**
* A Callable for flushRegion() RPC.
@@ -52,7 +52,7 @@ public class FlushRegionCallable extends RegionAdminServiceCallable<FlushRegionR
}
public FlushRegionCallable(ClusterConnection connection,
- RpcControllerFactory rpcControllerFactory, HRegionInfo regionInfo,
+ RpcControllerFactory rpcControllerFactory, RegionInfo regionInfo,
boolean writeFlushWalMarker) {
this(connection, rpcControllerFactory, regionInfo.getTable(), regionInfo.getRegionName(),
regionInfo.getStartKey(), writeFlushWalMarker);
[11/19] hbase git commit: HBASE-18839 Apply RegionInfo to code base
Posted by ch...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CloneSnapshotProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CloneSnapshotProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CloneSnapshotProcedure.java
index 9f2baf4..6155f16 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CloneSnapshotProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CloneSnapshotProcedure.java
@@ -30,11 +30,10 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.TableExistsException;
import org.apache.hadoop.hbase.TableName;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
import org.apache.hadoop.hbase.errorhandling.ForeignException;
@@ -46,11 +45,6 @@ import org.apache.hadoop.hbase.master.procedure.CreateTableProcedure.CreateHdfsR
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
import org.apache.hadoop.hbase.monitoring.TaskMonitor;
import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.CloneSnapshotState;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException;
import org.apache.hadoop.hbase.snapshot.RestoreSnapshotHelper;
@@ -59,8 +53,14 @@ import org.apache.hadoop.hbase.snapshot.SnapshotManifest;
import org.apache.hadoop.hbase.util.FSTableDescriptors;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.Pair;
+import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.CloneSnapshotState;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
@InterfaceAudience.Private
public class CloneSnapshotProcedure
@@ -70,7 +70,7 @@ public class CloneSnapshotProcedure
private TableDescriptor tableDescriptor;
private SnapshotDescription snapshot;
private boolean restoreAcl;
- private List<HRegionInfo> newRegions = null;
+ private List<RegionInfo> newRegions = null;
private Map<String, Pair<String, String> > parentsToChildrenPairMap = new HashMap<>();
// Monitor
@@ -253,8 +253,8 @@ public class CloneSnapshotProcedure
.setSnapshot(this.snapshot)
.setTableSchema(ProtobufUtil.toTableSchema(tableDescriptor));
if (newRegions != null) {
- for (HRegionInfo hri: newRegions) {
- cloneSnapshotMsg.addRegionInfo(HRegionInfo.convert(hri));
+ for (RegionInfo hri: newRegions) {
+ cloneSnapshotMsg.addRegionInfo(ProtobufUtil.toRegionInfo(hri));
}
}
if (!parentsToChildrenPairMap.isEmpty()) {
@@ -289,7 +289,7 @@ public class CloneSnapshotProcedure
} else {
newRegions = new ArrayList<>(cloneSnapshotMsg.getRegionInfoCount());
for (HBaseProtos.RegionInfo hri: cloneSnapshotMsg.getRegionInfoList()) {
- newRegions.add(HRegionInfo.convert(hri));
+ newRegions.add(ProtobufUtil.toRegionInfo(hri));
}
}
if (cloneSnapshotMsg.getParentToChildRegionsPairListCount() > 0) {
@@ -357,8 +357,8 @@ public class CloneSnapshotProcedure
throws IOException, InterruptedException {
final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
if (cpHost != null) {
- final HRegionInfo[] regions = (newRegions == null) ? null :
- newRegions.toArray(new HRegionInfo[newRegions.size()]);
+ final RegionInfo[] regions = (newRegions == null) ? null :
+ newRegions.toArray(new RegionInfo[newRegions.size()]);
cpHost.postCompletedCreateTableAction(tableDescriptor, regions, getUser());
}
}
@@ -368,16 +368,16 @@ public class CloneSnapshotProcedure
* @param env MasterProcedureEnv
* @throws IOException
*/
- private List<HRegionInfo> createFilesystemLayout(
+ private List<RegionInfo> createFilesystemLayout(
final MasterProcedureEnv env,
final TableDescriptor tableDescriptor,
- final List<HRegionInfo> newRegions) throws IOException {
+ final List<RegionInfo> newRegions) throws IOException {
return createFsLayout(env, tableDescriptor, newRegions, new CreateHdfsRegions() {
@Override
- public List<HRegionInfo> createHdfsRegions(
+ public List<RegionInfo> createHdfsRegions(
final MasterProcedureEnv env,
final Path tableRootDir, final TableName tableName,
- final List<HRegionInfo> newRegions) throws IOException {
+ final List<RegionInfo> newRegions) throws IOException {
final MasterFileSystem mfs = env.getMasterServices().getMasterFileSystem();
final FileSystem fs = mfs.getFileSystem();
@@ -429,10 +429,10 @@ public class CloneSnapshotProcedure
* @param env MasterProcedureEnv
* @throws IOException
*/
- private List<HRegionInfo> createFsLayout(
+ private List<RegionInfo> createFsLayout(
final MasterProcedureEnv env,
final TableDescriptor tableDescriptor,
- List<HRegionInfo> newRegions,
+ List<RegionInfo> newRegions,
final CreateHdfsRegions hdfsRegionHandler) throws IOException {
final MasterFileSystem mfs = env.getMasterServices().getMasterFileSystem();
final Path tempdir = mfs.getTempDir();
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateTableProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateTableProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateTableProcedure.java
index ba41f8b..a1c82c7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateTableProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateTableProcedure.java
@@ -27,27 +27,27 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.DoNotRetryIOException;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.TableExistsException;
import org.apache.hadoop.hbase.TableName;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
import org.apache.hadoop.hbase.master.MasterFileSystem;
import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.CreateTableState;
import org.apache.hadoop.hbase.util.FSTableDescriptors;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.ModifyRegionUtils;
import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil;
+import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.CreateTableState;
@InterfaceAudience.Private
public class CreateTableProcedure
@@ -55,7 +55,7 @@ public class CreateTableProcedure
private static final Log LOG = LogFactory.getLog(CreateTableProcedure.class);
private TableDescriptor tableDescriptor;
- private List<HRegionInfo> newRegions;
+ private List<RegionInfo> newRegions;
public CreateTableProcedure() {
// Required by the Procedure framework to create the procedure on replay
@@ -63,12 +63,12 @@ public class CreateTableProcedure
}
public CreateTableProcedure(final MasterProcedureEnv env,
- final TableDescriptor tableDescriptor, final HRegionInfo[] newRegions) {
+ final TableDescriptor tableDescriptor, final RegionInfo[] newRegions) {
this(env, tableDescriptor, newRegions, null);
}
public CreateTableProcedure(final MasterProcedureEnv env,
- final TableDescriptor tableDescriptor, final HRegionInfo[] newRegions,
+ final TableDescriptor tableDescriptor, final RegionInfo[] newRegions,
final ProcedurePrepareLatch syncLatch) {
super(env, syncLatch);
this.tableDescriptor = tableDescriptor;
@@ -191,8 +191,8 @@ public class CreateTableProcedure
.setUserInfo(MasterProcedureUtil.toProtoUserInfo(getUser()))
.setTableSchema(ProtobufUtil.toTableSchema(tableDescriptor));
if (newRegions != null) {
- for (HRegionInfo hri: newRegions) {
- state.addRegionInfo(HRegionInfo.convert(hri));
+ for (RegionInfo hri: newRegions) {
+ state.addRegionInfo(ProtobufUtil.toRegionInfo(hri));
}
}
serializer.serialize(state.build());
@@ -212,7 +212,7 @@ public class CreateTableProcedure
} else {
newRegions = new ArrayList<>(state.getRegionInfoCount());
for (HBaseProtos.RegionInfo hri: state.getRegionInfoList()) {
- newRegions.add(HRegionInfo.convert(hri));
+ newRegions.add(ProtobufUtil.toRegionInfo(hri));
}
}
}
@@ -255,8 +255,8 @@ public class CreateTableProcedure
final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
if (cpHost != null) {
- final HRegionInfo[] regions = newRegions == null ? null :
- newRegions.toArray(new HRegionInfo[newRegions.size()]);
+ final RegionInfo[] regions = newRegions == null ? null :
+ newRegions.toArray(new RegionInfo[newRegions.size()]);
cpHost.preCreateTableAction(tableDescriptor, regions, getUser());
}
}
@@ -265,36 +265,36 @@ public class CreateTableProcedure
throws IOException, InterruptedException {
final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
if (cpHost != null) {
- final HRegionInfo[] regions = (newRegions == null) ? null :
- newRegions.toArray(new HRegionInfo[newRegions.size()]);
+ final RegionInfo[] regions = (newRegions == null) ? null :
+ newRegions.toArray(new RegionInfo[newRegions.size()]);
cpHost.postCompletedCreateTableAction(tableDescriptor, regions, getUser());
}
}
protected interface CreateHdfsRegions {
- List<HRegionInfo> createHdfsRegions(final MasterProcedureEnv env,
+ List<RegionInfo> createHdfsRegions(final MasterProcedureEnv env,
final Path tableRootDir, final TableName tableName,
- final List<HRegionInfo> newRegions) throws IOException;
+ final List<RegionInfo> newRegions) throws IOException;
}
- protected static List<HRegionInfo> createFsLayout(final MasterProcedureEnv env,
- final TableDescriptor tableDescriptor, final List<HRegionInfo> newRegions)
+ protected static List<RegionInfo> createFsLayout(final MasterProcedureEnv env,
+ final TableDescriptor tableDescriptor, final List<RegionInfo> newRegions)
throws IOException {
return createFsLayout(env, tableDescriptor, newRegions, new CreateHdfsRegions() {
@Override
- public List<HRegionInfo> createHdfsRegions(final MasterProcedureEnv env,
+ public List<RegionInfo> createHdfsRegions(final MasterProcedureEnv env,
final Path tableRootDir, final TableName tableName,
- final List<HRegionInfo> newRegions) throws IOException {
- HRegionInfo[] regions = newRegions != null ?
- newRegions.toArray(new HRegionInfo[newRegions.size()]) : null;
+ final List<RegionInfo> newRegions) throws IOException {
+ RegionInfo[] regions = newRegions != null ?
+ newRegions.toArray(new RegionInfo[newRegions.size()]) : null;
return ModifyRegionUtils.createRegions(env.getMasterConfiguration(),
tableRootDir, tableDescriptor, regions, null);
}
});
}
- protected static List<HRegionInfo> createFsLayout(final MasterProcedureEnv env,
- final TableDescriptor tableDescriptor, List<HRegionInfo> newRegions,
+ protected static List<RegionInfo> createFsLayout(final MasterProcedureEnv env,
+ final TableDescriptor tableDescriptor, List<RegionInfo> newRegions,
final CreateHdfsRegions hdfsRegionHandler) throws IOException {
final MasterFileSystem mfs = env.getMasterServices().getMasterFileSystem();
final Path tempdir = mfs.getTempDir();
@@ -332,15 +332,15 @@ public class CreateTableProcedure
}
}
- protected static List<HRegionInfo> addTableToMeta(final MasterProcedureEnv env,
+ protected static List<RegionInfo> addTableToMeta(final MasterProcedureEnv env,
final TableDescriptor tableDescriptor,
- final List<HRegionInfo> regions) throws IOException {
+ final List<RegionInfo> regions) throws IOException {
assert (regions != null && regions.size() > 0) : "expected at least 1 region, got " + regions;
ProcedureSyncWait.waitMetaRegions(env);
// Add replicas if needed
- List<HRegionInfo> newRegions = addReplicas(env, tableDescriptor, regions);
+ List<RegionInfo> newRegions = addReplicas(env, tableDescriptor, regions);
// Add regions to META
addRegionsToMeta(env, tableDescriptor, newRegions);
@@ -359,14 +359,14 @@ public class CreateTableProcedure
* @param regions default replicas
* @return the combined list of default and non-default replicas
*/
- private static List<HRegionInfo> addReplicas(final MasterProcedureEnv env,
+ private static List<RegionInfo> addReplicas(final MasterProcedureEnv env,
final TableDescriptor tableDescriptor,
- final List<HRegionInfo> regions) {
+ final List<RegionInfo> regions) {
int numRegionReplicas = tableDescriptor.getRegionReplication() - 1;
if (numRegionReplicas <= 0) {
return regions;
}
- List<HRegionInfo> hRegionInfos = new ArrayList<>((numRegionReplicas+1)*regions.size());
+ List<RegionInfo> hRegionInfos = new ArrayList<>((numRegionReplicas+1)*regions.size());
for (int i = 0; i < regions.size(); i++) {
for (int j = 1; j <= numRegionReplicas; j++) {
hRegionInfos.add(RegionReplicaUtil.getRegionInfoForReplica(regions.get(i), j));
@@ -396,7 +396,7 @@ public class CreateTableProcedure
*/
private static void addRegionsToMeta(final MasterProcedureEnv env,
final TableDescriptor tableDescriptor,
- final List<HRegionInfo> regionInfos) throws IOException {
+ final List<RegionInfo> regionInfos) throws IOException {
MetaTableAccessor.addRegionsToMeta(env.getMasterServices().getConnection(),
regionInfos, tableDescriptor.getRegionReplication());
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteColumnFamilyProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteColumnFamilyProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteColumnFamilyProcedure.java
index 92d0c5d..fd99378 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteColumnFamilyProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteColumnFamilyProcedure.java
@@ -23,19 +23,20 @@ import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.InvalidFamilyOperationException;
import org.apache.hadoop.hbase.TableName;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
+
import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DeleteColumnFamilyState;
-import org.apache.hadoop.hbase.util.Bytes;
/**
* The procedure to delete a column family from an existing table.
@@ -50,7 +51,7 @@ public class DeleteColumnFamilyProcedure
private byte [] familyName;
private boolean hasMob;
- private List<HRegionInfo> regionInfoList;
+ private List<RegionInfo> regionInfoList;
private Boolean traceEnabled;
public DeleteColumnFamilyProcedure() {
@@ -360,7 +361,7 @@ public class DeleteColumnFamilyProcedure
}
}
- private List<HRegionInfo> getRegionInfoList(final MasterProcedureEnv env) throws IOException {
+ private List<RegionInfo> getRegionInfoList(final MasterProcedureEnv env) throws IOException {
if (regionInfoList == null) {
regionInfoList = env.getAssignmentManager().getRegionStates()
.getRegionsOfTable(getTableName());
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java
index 04e1327..4cc1875 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java
@@ -27,15 +27,14 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotDisabledException;
import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.backup.HFileArchiver;
-import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
@@ -47,19 +46,21 @@ import org.apache.hadoop.hbase.master.MasterFileSystem;
import org.apache.hadoop.hbase.mob.MobConstants;
import org.apache.hadoop.hbase.mob.MobUtils;
import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
+import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.yetus.audience.InterfaceAudience;
+
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DeleteTableState;
-import org.apache.hadoop.hbase.regionserver.HRegion;
-import org.apache.hadoop.hbase.util.FSUtils;
@InterfaceAudience.Private
public class DeleteTableProcedure
extends AbstractStateMachineTableProcedure<DeleteTableState> {
private static final Log LOG = LogFactory.getLog(DeleteTableProcedure.class);
- private List<HRegionInfo> regions;
+ private List<RegionInfo> regions;
private TableName tableName;
public DeleteTableProcedure() {
@@ -211,8 +212,8 @@ public class DeleteTableProcedure
.setUserInfo(MasterProcedureUtil.toProtoUserInfo(getUser()))
.setTableName(ProtobufUtil.toProtoTableName(tableName));
if (regions != null) {
- for (HRegionInfo hri: regions) {
- state.addRegionInfo(HRegionInfo.convert(hri));
+ for (RegionInfo hri: regions) {
+ state.addRegionInfo(ProtobufUtil.toRegionInfo(hri));
}
}
serializer.serialize(state.build());
@@ -232,7 +233,7 @@ public class DeleteTableProcedure
} else {
regions = new ArrayList<>(state.getRegionInfoCount());
for (HBaseProtos.RegionInfo hri: state.getRegionInfoList()) {
- regions.add(HRegionInfo.convert(hri));
+ regions.add(ProtobufUtil.toRegionInfo(hri));
}
}
}
@@ -269,7 +270,7 @@ public class DeleteTableProcedure
}
protected static void deleteFromFs(final MasterProcedureEnv env,
- final TableName tableName, final List<HRegionInfo> regions,
+ final TableName tableName, final List<RegionInfo> regions,
final boolean archive) throws IOException {
final MasterFileSystem mfs = env.getMasterServices().getMasterFileSystem();
final FileSystem fs = mfs.getFileSystem();
@@ -310,7 +311,7 @@ public class DeleteTableProcedure
// Archive regions from FS (temp directory)
if (archive) {
- for (HRegionInfo hri : regions) {
+ for (RegionInfo hri : regions) {
LOG.debug("Archiving region " + hri.getRegionNameAsString() + " from FS");
HFileArchiver.archiveRegion(fs, mfs.getRootDir(),
tempTableDir, HRegion.getRegionDir(tempTableDir, hri.getEncodedName()));
@@ -366,7 +367,7 @@ public class DeleteTableProcedure
}
protected static void deleteFromMeta(final MasterProcedureEnv env,
- final TableName tableName, List<HRegionInfo> regions) throws IOException {
+ final TableName tableName, List<RegionInfo> regions) throws IOException {
MetaTableAccessor.deleteRegions(env.getMasterServices().getConnection(), regions);
// Clean any remaining rows for this table.
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterDDLOperationHelper.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterDDLOperationHelper.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterDDLOperationHelper.java
index df9cefb..02ecdc6 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterDDLOperationHelper.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterDDLOperationHelper.java
@@ -24,13 +24,13 @@ import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.TableName;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.master.MasterFileSystem;
import org.apache.hadoop.hbase.mob.MobConstants;
import org.apache.hadoop.hbase.mob.MobUtils;
import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
/**
* Helper class for schema change procedures
@@ -47,21 +47,21 @@ public final class MasterDDLOperationHelper {
public static void deleteColumnFamilyFromFileSystem(
final MasterProcedureEnv env,
final TableName tableName,
- final List<HRegionInfo> regionInfoList,
+ final List<RegionInfo> regionInfoList,
final byte[] familyName,
final boolean hasMob) throws IOException {
final MasterFileSystem mfs = env.getMasterServices().getMasterFileSystem();
if (LOG.isDebugEnabled()) {
LOG.debug("Removing family=" + Bytes.toString(familyName) + " from table=" + tableName);
}
- for (HRegionInfo hri : regionInfoList) {
+ for (RegionInfo hri : regionInfoList) {
// Delete the family directory in FS for all the regions one by one
mfs.deleteFamilyFromFS(hri, familyName);
}
if (hasMob) {
// Delete the mob region
Path mobRootDir = new Path(mfs.getRootDir(), MobConstants.MOB_DIR_NAME);
- HRegionInfo mobRegionInfo = MobUtils.getMobRegionInfo(tableName);
+ RegionInfo mobRegionInfo = MobUtils.getMobRegionInfo(tableName);
mfs.deleteFamilyFromFS(mobRootDir, mobRegionInfo, familyName);
}
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java
index e49966c..9402845 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java
@@ -29,12 +29,11 @@ import java.util.Map.Entry;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableExistsException;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotFoundException;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.master.locking.LockProcedure;
import org.apache.hadoop.hbase.master.procedure.TableProcedureInterface.TableOperationType;
import org.apache.hadoop.hbase.procedure2.AbstractProcedureScheduler;
@@ -45,12 +44,14 @@ import org.apache.hadoop.hbase.procedure2.LockedResource;
import org.apache.hadoop.hbase.procedure2.LockedResourceType;
import org.apache.hadoop.hbase.procedure2.Procedure;
import org.apache.hadoop.hbase.procedure2.ProcedureDeque;
-import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.hbase.util.AvlUtil.AvlIterableList;
import org.apache.hadoop.hbase.util.AvlUtil.AvlKeyComparator;
import org.apache.hadoop.hbase.util.AvlUtil.AvlLinkedNode;
import org.apache.hadoop.hbase.util.AvlUtil.AvlTree;
import org.apache.hadoop.hbase.util.AvlUtil.AvlTreeIterator;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
/**
* ProcedureScheduler for the Master Procedures.
@@ -783,7 +784,7 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
* @param regionInfo the region we are trying to lock
* @return true if the procedure has to wait for the regions to be available
*/
- public boolean waitRegion(final Procedure procedure, final HRegionInfo regionInfo) {
+ public boolean waitRegion(final Procedure procedure, final RegionInfo regionInfo) {
return waitRegions(procedure, regionInfo.getTable(), regionInfo);
}
@@ -795,8 +796,8 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
* @return true if the procedure has to wait for the regions to be available
*/
public boolean waitRegions(final Procedure procedure, final TableName table,
- final HRegionInfo... regionInfo) {
- Arrays.sort(regionInfo);
+ final RegionInfo... regionInfo) {
+ Arrays.sort(regionInfo, RegionInfo.COMPARATOR);
schedLock();
try {
// If there is parent procedure, it would have already taken xlock, so no need to take
@@ -842,7 +843,7 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
* @param procedure the procedure that was holding the region
* @param regionInfo the region the procedure was holding
*/
- public void wakeRegion(final Procedure procedure, final HRegionInfo regionInfo) {
+ public void wakeRegion(final Procedure procedure, final RegionInfo regionInfo) {
wakeRegions(procedure, regionInfo.getTable(), regionInfo);
}
@@ -852,8 +853,8 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
* @param regionInfo the list of regions the procedure was holding
*/
public void wakeRegions(final Procedure procedure,final TableName table,
- final HRegionInfo... regionInfo) {
- Arrays.sort(regionInfo);
+ final RegionInfo... regionInfo) {
+ Arrays.sort(regionInfo, RegionInfo.COMPARATOR);
schedLock();
try {
int numProcs = 0;
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableProcedure.java
index 61c57f0..21487d4 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableProcedure.java
@@ -27,12 +27,11 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotFoundException;
-import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
@@ -41,10 +40,12 @@ import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
+import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil;
+import org.apache.yetus.audience.InterfaceAudience;
+
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ModifyTableState;
-import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil;
@InterfaceAudience.Private
public class ModifyTableProcedure
@@ -55,7 +56,7 @@ public class ModifyTableProcedure
private TableDescriptor modifiedTableDescriptor;
private boolean deleteColumnFamilyInModify;
- private List<HRegionInfo> regionInfoList;
+ private List<RegionInfo> regionInfoList;
private Boolean traceEnabled = null;
public ModifyTableProcedure() {
@@ -413,7 +414,7 @@ public class ModifyTableProcedure
}
}
- private List<HRegionInfo> getRegionInfoList(final MasterProcedureEnv env) throws IOException {
+ private List<RegionInfo> getRegionInfoList(final MasterProcedureEnv env) throws IOException {
if (regionInfoList == null) {
regionInfoList = env.getAssignmentManager().getRegionStates()
.getRegionsOfTable(getTableName());
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ProcedureSyncWait.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ProcedureSyncWait.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ProcedureSyncWait.java
index fcbcbea..18fa91d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ProcedureSyncWait.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ProcedureSyncWait.java
@@ -30,16 +30,16 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.CoordinatedStateException;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.NotAllMetaRegionsOnlineException;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.yetus.audience.InterfaceStability;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.exceptions.TimeoutIOException;
import org.apache.hadoop.hbase.master.assignment.RegionStates;
import org.apache.hadoop.hbase.procedure2.Procedure;
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
import org.apache.hadoop.hbase.quotas.MasterQuotaManager;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.yetus.audience.InterfaceStability;
/**
* Helper to synchronously wait on conditions.
@@ -209,9 +209,9 @@ public final class ProcedureSyncWait {
}
protected static void waitRegionInTransition(final MasterProcedureEnv env,
- final List<HRegionInfo> regions) throws IOException, CoordinatedStateException {
+ final List<RegionInfo> regions) throws IOException, CoordinatedStateException {
final RegionStates states = env.getAssignmentManager().getRegionStates();
- for (final HRegionInfo region : regions) {
+ for (final RegionInfo region : regions) {
ProcedureSyncWait.waitFor(env, "regions " + region.getRegionNameAsString() + " in transition",
new ProcedureSyncWait.Predicate<Boolean>() {
@Override
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java
index c4cca2b..a0c06ce 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java
@@ -18,25 +18,26 @@
package org.apache.hadoop.hbase.master.procedure;
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.ArrayListMultimap;
-
import java.io.IOException;
import java.net.SocketTimeoutException;
-import java.util.concurrent.Callable;
-import java.util.concurrent.TimeUnit;
import java.util.List;
import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.concurrent.TimeUnit;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.ipc.RemoteException;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.DoNotRetryIOException;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.master.ServerListener;
import org.apache.hadoop.hbase.procedure2.RemoteProcedureDispatcher;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.ipc.RemoteException;
+
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.ArrayListMultimap;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
@@ -47,7 +48,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ExecuteProc
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ExecuteProceduresResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionResponse;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
/**
* A remote procecdure dispatcher for regionservers.
@@ -473,15 +473,15 @@ public class RSProcedureDispatcher
}
public static abstract class RegionOperation extends RemoteOperation {
- private final HRegionInfo regionInfo;
+ private final RegionInfo regionInfo;
protected RegionOperation(final RemoteProcedure remoteProcedure,
- final HRegionInfo regionInfo) {
+ final RegionInfo regionInfo) {
super(remoteProcedure);
this.regionInfo = regionInfo;
}
- public HRegionInfo getRegionInfo() {
+ public RegionInfo getRegionInfo() {
return this.regionInfo;
}
}
@@ -492,7 +492,7 @@ public class RSProcedureDispatcher
private boolean failedOpen;
public RegionOpenOperation(final RemoteProcedure remoteProcedure,
- final HRegionInfo regionInfo, final List<ServerName> favoredNodes,
+ final RegionInfo regionInfo, final List<ServerName> favoredNodes,
final boolean openForReplay) {
super(remoteProcedure, regionInfo);
this.favoredNodes = favoredNodes;
@@ -519,7 +519,7 @@ public class RSProcedureDispatcher
private boolean closed = false;
public RegionCloseOperation(final RemoteProcedure remoteProcedure,
- final HRegionInfo regionInfo, final ServerName destinationServer) {
+ final RegionInfo regionInfo, final ServerName destinationServer) {
super(remoteProcedure, regionInfo);
this.destinationServer = destinationServer;
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RecoverMetaProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RecoverMetaProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RecoverMetaProcedure.java
index 7ae81ba..fe3a445 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RecoverMetaProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RecoverMetaProcedure.java
@@ -18,12 +18,15 @@
package org.apache.hadoop.hbase.master.procedure;
-import com.google.common.base.Preconditions;
+import java.io.IOException;
+import java.util.Set;
+
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.master.assignment.AssignProcedure;
@@ -31,15 +34,14 @@ import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
import org.apache.hadoop.hbase.procedure2.ProcedureYieldException;
import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
+import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
+import org.apache.zookeeper.KeeperException;
+
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RecoverMetaState;
-import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
-import org.apache.zookeeper.KeeperException;
-import java.io.IOException;
-import java.io.InputStream;
-import java.util.Set;
+import com.google.common.base.Preconditions;
/**
* This procedure recovers meta from prior shutdown/ crash of a server, and brings meta online by
@@ -75,7 +77,7 @@ public class RecoverMetaProcedure
final ProcedurePrepareLatch latch) {
this.failedMetaServer = failedMetaServer;
this.shouldSplitWal = shouldSplitLog;
- this.replicaId = HRegionInfo.DEFAULT_REPLICA_ID;
+ this.replicaId = RegionInfo.DEFAULT_REPLICA_ID;
this.syncLatch = latch;
}
@@ -120,8 +122,8 @@ public class RecoverMetaProcedure
break;
case RECOVER_META_ASSIGN_REGIONS:
- HRegionInfo hri = RegionReplicaUtil.getRegionInfoForReplica(
- HRegionInfo.FIRST_META_REGIONINFO, this.replicaId);
+ RegionInfo hri = RegionReplicaUtil.getRegionInfoForReplica(
+ RegionInfoBuilder.FIRST_META_REGIONINFO, this.replicaId);
AssignProcedure metaAssignProcedure;
if (failedMetaServer != null) {
@@ -204,7 +206,7 @@ public class RecoverMetaProcedure
this.shouldSplitWal = state.hasShouldSplitWal() && state.getShouldSplitWal();
this.failedMetaServer = state.hasFailedMetaServer() ?
ProtobufUtil.toServerName(state.getFailedMetaServer()) : null;
- this.replicaId = state.hasReplicaId() ? state.getReplicaId() : HRegionInfo.DEFAULT_REPLICA_ID;
+ this.replicaId = state.hasReplicaId() ? state.getReplicaId() : RegionInfo.DEFAULT_REPLICA_ID;
}
@Override
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RestoreSnapshotProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RestoreSnapshotProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RestoreSnapshotProcedure.java
index 9d775ac..2cf5584 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RestoreSnapshotProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RestoreSnapshotProcedure.java
@@ -30,12 +30,11 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.DoNotRetryIOException;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotFoundException;
-import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.errorhandling.ForeignException;
import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
@@ -44,16 +43,18 @@ import org.apache.hadoop.hbase.master.MetricsSnapshot;
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
import org.apache.hadoop.hbase.monitoring.TaskMonitor;
import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RestoreSnapshotState;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
import org.apache.hadoop.hbase.snapshot.RestoreSnapshotHelper;
import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
import org.apache.hadoop.hbase.snapshot.SnapshotManifest;
import org.apache.hadoop.hbase.util.Pair;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RestoreSnapshotState;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
@InterfaceAudience.Private
public class RestoreSnapshotProcedure
@@ -61,9 +62,9 @@ public class RestoreSnapshotProcedure
private static final Log LOG = LogFactory.getLog(RestoreSnapshotProcedure.class);
private TableDescriptor modifiedTableDescriptor;
- private List<HRegionInfo> regionsToRestore = null;
- private List<HRegionInfo> regionsToRemove = null;
- private List<HRegionInfo> regionsToAdd = null;
+ private List<RegionInfo> regionsToRestore = null;
+ private List<RegionInfo> regionsToRemove = null;
+ private List<RegionInfo> regionsToAdd = null;
private Map<String, Pair<String, String>> parentsToChildrenPairMap = new HashMap<>();
private SnapshotDescription snapshot;
@@ -239,18 +240,18 @@ public class RestoreSnapshotProcedure
.setModifiedTableSchema(ProtobufUtil.toTableSchema(modifiedTableDescriptor));
if (regionsToRestore != null) {
- for (HRegionInfo hri: regionsToRestore) {
- restoreSnapshotMsg.addRegionInfoForRestore(HRegionInfo.convert(hri));
+ for (RegionInfo hri: regionsToRestore) {
+ restoreSnapshotMsg.addRegionInfoForRestore(ProtobufUtil.toRegionInfo(hri));
}
}
if (regionsToRemove != null) {
- for (HRegionInfo hri: regionsToRemove) {
- restoreSnapshotMsg.addRegionInfoForRemove(HRegionInfo.convert(hri));
+ for (RegionInfo hri: regionsToRemove) {
+ restoreSnapshotMsg.addRegionInfoForRemove(ProtobufUtil.toRegionInfo(hri));
}
}
if (regionsToAdd != null) {
- for (HRegionInfo hri: regionsToAdd) {
- restoreSnapshotMsg.addRegionInfoForAdd(HRegionInfo.convert(hri));
+ for (RegionInfo hri: regionsToAdd) {
+ restoreSnapshotMsg.addRegionInfoForAdd(ProtobufUtil.toRegionInfo(hri));
}
}
if (!parentsToChildrenPairMap.isEmpty()) {
@@ -287,7 +288,7 @@ public class RestoreSnapshotProcedure
} else {
regionsToRestore = new ArrayList<>(restoreSnapshotMsg.getRegionInfoForRestoreCount());
for (HBaseProtos.RegionInfo hri: restoreSnapshotMsg.getRegionInfoForRestoreList()) {
- regionsToRestore.add(HRegionInfo.convert(hri));
+ regionsToRestore.add(ProtobufUtil.toRegionInfo(hri));
}
}
if (restoreSnapshotMsg.getRegionInfoForRemoveCount() == 0) {
@@ -295,7 +296,7 @@ public class RestoreSnapshotProcedure
} else {
regionsToRemove = new ArrayList<>(restoreSnapshotMsg.getRegionInfoForRemoveCount());
for (HBaseProtos.RegionInfo hri: restoreSnapshotMsg.getRegionInfoForRemoveList()) {
- regionsToRemove.add(HRegionInfo.convert(hri));
+ regionsToRemove.add(ProtobufUtil.toRegionInfo(hri));
}
}
if (restoreSnapshotMsg.getRegionInfoForAddCount() == 0) {
@@ -303,7 +304,7 @@ public class RestoreSnapshotProcedure
} else {
regionsToAdd = new ArrayList<>(restoreSnapshotMsg.getRegionInfoForAddCount());
for (HBaseProtos.RegionInfo hri: restoreSnapshotMsg.getRegionInfoForAddList()) {
- regionsToAdd.add(HRegionInfo.convert(hri));
+ regionsToAdd.add(ProtobufUtil.toRegionInfo(hri));
}
}
if (restoreSnapshotMsg.getParentToChildRegionsPairListCount() > 0) {
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashProcedure.java
index 9307c45..a0ee628 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashProcedure.java
@@ -18,16 +18,15 @@
package org.apache.hadoop.hbase.master.procedure;
import java.io.IOException;
-import java.io.InputStream;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.master.MasterWalManager;
@@ -39,8 +38,8 @@ import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
import org.apache.hadoop.hbase.procedure2.ProcedureYieldException;
import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
+
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ServerCrashState;
@@ -72,7 +71,7 @@ implements ServerProcedureInterface {
/**
* Regions that were on the crashed server.
*/
- private List<HRegionInfo> regionsOnCrashedServer;
+ private List<RegionInfo> regionsOnCrashedServer;
private boolean carryingMeta = false;
private boolean shouldSplitWal;
@@ -159,7 +158,7 @@ implements ServerProcedureInterface {
if (filterDefaultMetaRegions(regionsOnCrashedServer)) {
if (LOG.isTraceEnabled()) {
LOG.trace("Assigning regions " +
- HRegionInfo.getShortNameToLog(regionsOnCrashedServer) + ", " + this +
+ RegionInfo.getShortNameToLog(regionsOnCrashedServer) + ", " + this +
"; cycles=" + getCycles());
}
handleRIT(env, regionsOnCrashedServer);
@@ -195,7 +194,7 @@ implements ServerProcedureInterface {
// Assign meta if still carrying it. Check again: region may be assigned because of RIT timeout
final AssignmentManager am = env.getMasterServices().getAssignmentManager();
- for (HRegionInfo hri: am.getRegionStates().getServerRegionInfoSet(serverName)) {
+ for (RegionInfo hri: am.getRegionStates().getServerRegionInfoSet(serverName)) {
if (!isDefaultMetaRegion(hri)) continue;
am.offlineRegion(hri);
@@ -203,13 +202,13 @@ implements ServerProcedureInterface {
}
}
- private boolean filterDefaultMetaRegions(final List<HRegionInfo> regions) {
+ private boolean filterDefaultMetaRegions(final List<RegionInfo> regions) {
if (regions == null) return false;
regions.removeIf(this::isDefaultMetaRegion);
return !regions.isEmpty();
}
- private boolean isDefaultMetaRegion(final HRegionInfo hri) {
+ private boolean isDefaultMetaRegion(final RegionInfo hri) {
return hri.getTable().equals(TableName.META_TABLE_NAME) &&
RegionReplicaUtil.isDefaultReplica(hri);
}
@@ -295,8 +294,8 @@ implements ServerProcedureInterface {
setCarryingMeta(this.carryingMeta).
setShouldSplitWal(this.shouldSplitWal);
if (this.regionsOnCrashedServer != null && !this.regionsOnCrashedServer.isEmpty()) {
- for (HRegionInfo hri: this.regionsOnCrashedServer) {
- state.addRegionsOnCrashedServer(HRegionInfo.convert(hri));
+ for (RegionInfo hri: this.regionsOnCrashedServer) {
+ state.addRegionsOnCrashedServer(ProtobufUtil.toRegionInfo(hri));
}
}
serializer.serialize(state.build());
@@ -315,9 +314,9 @@ implements ServerProcedureInterface {
this.shouldSplitWal = state.getShouldSplitWal();
int size = state.getRegionsOnCrashedServerCount();
if (size > 0) {
- this.regionsOnCrashedServer = new ArrayList<HRegionInfo>(size);
- for (RegionInfo ri: state.getRegionsOnCrashedServerList()) {
- this.regionsOnCrashedServer.add(HRegionInfo.convert(ri));
+ this.regionsOnCrashedServer = new ArrayList<>(size);
+ for (org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo ri: state.getRegionsOnCrashedServerList()) {
+ this.regionsOnCrashedServer.add(ProtobufUtil.toRegionInfo(ri));
}
}
}
@@ -365,13 +364,13 @@ implements ServerProcedureInterface {
* @param env
* @param regions Regions that were on crashed server
*/
- private void handleRIT(final MasterProcedureEnv env, final List<HRegionInfo> regions) {
+ private void handleRIT(final MasterProcedureEnv env, final List<RegionInfo> regions) {
if (regions == null) return;
AssignmentManager am = env.getMasterServices().getAssignmentManager();
- final Iterator<HRegionInfo> it = regions.iterator();
+ final Iterator<RegionInfo> it = regions.iterator();
ServerCrashException sce = null;
while (it.hasNext()) {
- final HRegionInfo hri = it.next();
+ final RegionInfo hri = it.next();
RegionTransitionProcedure rtp = am.getRegionStates().getRegionTransitionProcedure(hri);
if (rtp == null) continue;
// Make sure the RIT is against this crashed server. In the case where there are many
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TruncateTableProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TruncateTableProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TruncateTableProcedure.java
index f54df3d..c82f8d1 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TruncateTableProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TruncateTableProcedure.java
@@ -25,20 +25,22 @@ import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotDisabledException;
import org.apache.hadoop.hbase.TableNotFoundException;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.exceptions.HBaseException;
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
+import org.apache.hadoop.hbase.util.ModifyRegionUtils;
+import org.apache.yetus.audience.InterfaceAudience;
+
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.TruncateTableState;
-import org.apache.hadoop.hbase.util.ModifyRegionUtils;
@InterfaceAudience.Private
public class TruncateTableProcedure
@@ -46,7 +48,7 @@ public class TruncateTableProcedure
private static final Log LOG = LogFactory.getLog(TruncateTableProcedure.class);
private boolean preserveSplits;
- private List<HRegionInfo> regions;
+ private List<RegionInfo> regions;
private TableDescriptor tableDescriptor;
private TableName tableName;
@@ -104,7 +106,7 @@ public class TruncateTableProcedure
DeleteTableProcedure.deleteFromFs(env, getTableName(), regions, true);
if (!preserveSplits) {
// if we are not preserving splits, generate a new single region
- regions = Arrays.asList(ModifyRegionUtils.createHRegionInfos(tableDescriptor, null));
+ regions = Arrays.asList(ModifyRegionUtils.createRegionInfos(tableDescriptor, null));
} else {
regions = recreateRegionInfo(regions);
}
@@ -222,8 +224,8 @@ public class TruncateTableProcedure
state.setTableName(ProtobufUtil.toProtoTableName(tableName));
}
if (regions != null) {
- for (HRegionInfo hri: regions) {
- state.addRegionInfo(HRegionInfo.convert(hri));
+ for (RegionInfo hri: regions) {
+ state.addRegionInfo(ProtobufUtil.toRegionInfo(hri));
}
}
serializer.serialize(state.build());
@@ -249,15 +251,18 @@ public class TruncateTableProcedure
} else {
regions = new ArrayList<>(state.getRegionInfoCount());
for (HBaseProtos.RegionInfo hri: state.getRegionInfoList()) {
- regions.add(HRegionInfo.convert(hri));
+ regions.add(ProtobufUtil.toRegionInfo(hri));
}
}
}
- private static List<HRegionInfo> recreateRegionInfo(final List<HRegionInfo> regions) {
- ArrayList<HRegionInfo> newRegions = new ArrayList<>(regions.size());
- for (HRegionInfo hri: regions) {
- newRegions.add(new HRegionInfo(hri.getTable(), hri.getStartKey(), hri.getEndKey()));
+ private static List<RegionInfo> recreateRegionInfo(final List<RegionInfo> regions) {
+ ArrayList<RegionInfo> newRegions = new ArrayList<>(regions.size());
+ for (RegionInfo hri: regions) {
+ newRegions.add(RegionInfoBuilder.newBuilder(hri.getTable())
+ .setStartKey(hri.getStartKey())
+ .setEndKey(hri.getEndKey())
+ .build());
}
return newRegions;
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/DisabledTableSnapshotHandler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/DisabledTableSnapshotHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/DisabledTableSnapshotHandler.java
index 970a6ca..84c154f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/DisabledTableSnapshotHandler.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/DisabledTableSnapshotHandler.java
@@ -25,22 +25,23 @@ import java.util.concurrent.ThreadPoolExecutor;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.yetus.audience.InterfaceStability;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
import org.apache.hadoop.hbase.errorhandling.ForeignException;
import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.mob.MobUtils;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
import org.apache.hadoop.hbase.snapshot.SnapshotManifest;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.ModifyRegionUtils;
import org.apache.hadoop.hbase.util.Pair;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.yetus.audience.InterfaceStability;
import org.apache.zookeeper.KeeperException;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
+
/**
* Take a snapshot of a disabled table.
* <p>
@@ -68,16 +69,16 @@ public class DisabledTableSnapshotHandler extends TakeSnapshotHandler {
// TODO consider parallelizing these operations since they are independent. Right now its just
// easier to keep them serial though
@Override
- public void snapshotRegions(List<Pair<HRegionInfo, ServerName>> regionsAndLocations)
+ public void snapshotRegions(List<Pair<RegionInfo, ServerName>> regionsAndLocations)
throws IOException, KeeperException {
try {
// 1. get all the regions hosting this table.
// extract each pair to separate lists
- Set<HRegionInfo> regions = new HashSet<>();
- for (Pair<HRegionInfo, ServerName> p : regionsAndLocations) {
+ Set<RegionInfo> regions = new HashSet<>();
+ for (Pair<RegionInfo, ServerName> p : regionsAndLocations) {
// Don't include non-default regions
- HRegionInfo hri = p.getFirst();
+ RegionInfo hri = p.getFirst();
if (RegionReplicaUtil.isDefaultReplica(hri)) {
regions.add(hri);
}
@@ -86,7 +87,7 @@ public class DisabledTableSnapshotHandler extends TakeSnapshotHandler {
boolean mobEnabled = MobUtils.hasMobColumns(htd);
if (mobEnabled) {
// snapshot the mob files as a offline region.
- HRegionInfo mobRegionInfo = MobUtils.getMobRegionInfo(htd.getTableName());
+ RegionInfo mobRegionInfo = MobUtils.getMobRegionInfo(htd.getTableName());
regions.add(mobRegionInfo);
}
@@ -100,7 +101,7 @@ public class DisabledTableSnapshotHandler extends TakeSnapshotHandler {
try {
ModifyRegionUtils.editRegions(exec, regions, new ModifyRegionUtils.RegionEditTask() {
@Override
- public void editRegion(final HRegionInfo regionInfo) throws IOException {
+ public void editRegion(final RegionInfo regionInfo) throws IOException {
snapshotManifest.addRegion(FSUtils.getTableDir(rootDir, snapshotTable), regionInfo);
}
});
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/EnabledTableSnapshotHandler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/EnabledTableSnapshotHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/EnabledTableSnapshotHandler.java
index cb3d890..399a127 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/EnabledTableSnapshotHandler.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/EnabledTableSnapshotHandler.java
@@ -24,19 +24,19 @@ import java.util.Set;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.errorhandling.ForeignException;
import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.mob.MobUtils;
import org.apache.hadoop.hbase.procedure.Procedure;
import org.apache.hadoop.hbase.procedure.ProcedureCoordinator;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
import org.apache.hadoop.hbase.snapshot.HBaseSnapshotException;
import org.apache.hadoop.hbase.util.Pair;
+import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
/**
* Handle the master side of taking a snapshot of an online table, regardless of snapshot type.
@@ -69,12 +69,12 @@ public class EnabledTableSnapshotHandler extends TakeSnapshotHandler {
* phases to complete.
*/
@Override
- protected void snapshotRegions(List<Pair<HRegionInfo, ServerName>> regions)
+ protected void snapshotRegions(List<Pair<RegionInfo, ServerName>> regions)
throws HBaseSnapshotException, IOException {
Set<String> regionServers = new HashSet<>(regions.size());
- for (Pair<HRegionInfo, ServerName> region : regions) {
+ for (Pair<RegionInfo, ServerName> region : regions) {
if (region != null && region.getFirst() != null && region.getSecond() != null) {
- HRegionInfo hri = region.getFirst();
+ RegionInfo hri = region.getFirst();
if (hri.isOffline() && (hri.isSplit() || hri.isSplitParent())) continue;
regionServers.add(region.getSecond().toString());
}
@@ -97,8 +97,8 @@ public class EnabledTableSnapshotHandler extends TakeSnapshotHandler {
LOG.info("Done waiting - online snapshot for " + this.snapshot.getName());
// Take the offline regions as disabled
- for (Pair<HRegionInfo, ServerName> region : regions) {
- HRegionInfo regionInfo = region.getFirst();
+ for (Pair<RegionInfo, ServerName> region : regions) {
+ RegionInfo regionInfo = region.getFirst();
if (regionInfo.isOffline() && (regionInfo.isSplit() || regionInfo.isSplitParent())) {
LOG.info("Take disabled snapshot of offline region=" + regionInfo);
snapshotDisabledRegion(regionInfo);
@@ -109,7 +109,7 @@ public class EnabledTableSnapshotHandler extends TakeSnapshotHandler {
if (mobEnabled) {
LOG.info("Taking snapshot for mob files in table " + htd.getTableName());
// snapshot the mob files as a offline region.
- HRegionInfo mobRegionInfo = MobUtils.getMobRegionInfo(htd.getTableName());
+ RegionInfo mobRegionInfo = MobUtils.getMobRegionInfo(htd.getTableName());
snapshotMobRegion(mobRegionInfo);
}
} catch (InterruptedException e) {
@@ -125,7 +125,7 @@ public class EnabledTableSnapshotHandler extends TakeSnapshotHandler {
/**
* Takes a snapshot of the mob region
*/
- private void snapshotMobRegion(final HRegionInfo regionInfo)
+ private void snapshotMobRegion(final RegionInfo regionInfo)
throws IOException {
snapshotManifest.addMobRegion(regionInfo);
monitor.rethrowException();
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/MasterSnapshotVerifier.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/MasterSnapshotVerifier.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/MasterSnapshotVerifier.java
index 2f4d98c..b698082 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/MasterSnapshotVerifier.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/MasterSnapshotVerifier.java
@@ -24,26 +24,27 @@ import java.util.Set;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.yetus.audience.InterfaceStability;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
-import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.mob.MobUtils;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest;
import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
import org.apache.hadoop.hbase.snapshot.CorruptedSnapshotException;
import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
import org.apache.hadoop.hbase.snapshot.SnapshotManifest;
import org.apache.hadoop.hbase.snapshot.SnapshotReferenceUtil;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.yetus.audience.InterfaceStability;
+
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest;
/**
* General snapshot verification on the master.
@@ -67,7 +68,7 @@ import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
* </ol>
* <ul>
* <li>Matching regions in the snapshot as currently in the table</li>
- * <li>{@link HRegionInfo} matches the current and stored regions</li>
+ * <li>{@link RegionInfo} matches the current and stored regions</li>
* <li>All referenced hfiles have valid names</li>
* <li>All the hfiles are present (either in .archive directory in the region)</li>
* <li>All recovered.edits files are present (by name) and have the correct file size</li>
@@ -100,7 +101,7 @@ public final class MasterSnapshotVerifier {
/**
* Verify that the snapshot in the directory is a valid snapshot
* @param snapshotDir snapshot directory to check
- * @param snapshotServers {@link org.apache.hadoop.hbase.ServerName} of the servers
+ * @param snapshotServers {@link org.apache.hadoop.hbase.ServerName} of the servers
* that are involved in the snapshot
* @throws CorruptedSnapshotException if the snapshot is invalid
* @throws IOException if there is an unexpected connection issue to the filesystem
@@ -156,7 +157,7 @@ public final class MasterSnapshotVerifier {
* @throws IOException if we can't reach hbase:meta or read the files from the FS
*/
private void verifyRegions(final SnapshotManifest manifest) throws IOException {
- List<HRegionInfo> regions;
+ List<RegionInfo> regions;
if (TableName.META_TABLE_NAME.equals(tableName)) {
regions = new MetaTableLocator().getMetaRegions(services.getZooKeeper());
} else {
@@ -187,8 +188,8 @@ public final class MasterSnapshotVerifier {
LOG.error(errorMsg);
}
- // Verify HRegionInfo
- for (HRegionInfo region : regions) {
+ // Verify RegionInfo
+ for (RegionInfo region : regions) {
SnapshotRegionManifest regionManifest = regionManifests.get(region.getEncodedName());
if (regionManifest == null) {
// could happen due to a move or split race.
@@ -214,10 +215,10 @@ public final class MasterSnapshotVerifier {
* @param region the region to check
* @param manifest snapshot manifest to inspect
*/
- private void verifyRegionInfo(final HRegionInfo region,
+ private void verifyRegionInfo(final RegionInfo region,
final SnapshotRegionManifest manifest) throws IOException {
- HRegionInfo manifestRegionInfo = HRegionInfo.convert(manifest.getRegionInfo());
- if (!region.equals(manifestRegionInfo)) {
+ RegionInfo manifestRegionInfo = ProtobufUtil.toRegionInfo(manifest.getRegionInfo());
+ if (RegionInfo.COMPARATOR.compare(region, manifestRegionInfo) != 0) {
String msg = "Manifest region info " + manifestRegionInfo +
"doesn't match expected region:" + region;
throw new CorruptedSnapshotException(msg, ProtobufUtil.createSnapshotDesc(snapshot));
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/TakeSnapshotHandler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/TakeSnapshotHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/TakeSnapshotHandler.java
index 2ca0489..808cab5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/TakeSnapshotHandler.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/TakeSnapshotHandler.java
@@ -30,11 +30,10 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.errorhandling.ForeignException;
import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
@@ -48,7 +47,6 @@ import org.apache.hadoop.hbase.master.locking.LockManager;
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
import org.apache.hadoop.hbase.monitoring.TaskMonitor;
import org.apache.hadoop.hbase.procedure2.LockType;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
import org.apache.hadoop.hbase.snapshot.SnapshotCreationException;
import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
@@ -56,8 +54,11 @@ import org.apache.hadoop.hbase.snapshot.SnapshotManifest;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
+import org.apache.yetus.audience.InterfaceAudience;
import org.apache.zookeeper.KeeperException;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
+
/**
* A handler for taking snapshots from the master.
*
@@ -170,7 +171,7 @@ public abstract class TakeSnapshotHandler extends EventHandler implements Snapsh
snapshotManifest.addTableDescriptor(this.htd);
monitor.rethrowException();
- List<Pair<HRegionInfo, ServerName>> regionsAndLocations;
+ List<Pair<RegionInfo, ServerName>> regionsAndLocations;
if (TableName.META_TABLE_NAME.equals(snapshotTable)) {
regionsAndLocations = new MetaTableLocator().getMetaRegionsAndLocations(
server.getZooKeeper());
@@ -185,9 +186,9 @@ public abstract class TakeSnapshotHandler extends EventHandler implements Snapsh
// extract each pair to separate lists
Set<String> serverNames = new HashSet<>();
- for (Pair<HRegionInfo, ServerName> p : regionsAndLocations) {
+ for (Pair<RegionInfo, ServerName> p : regionsAndLocations) {
if (p != null && p.getFirst() != null && p.getSecond() != null) {
- HRegionInfo hri = p.getFirst();
+ RegionInfo hri = p.getFirst();
if (hri.isOffline() && (hri.isSplit() || hri.isSplitParent())) continue;
serverNames.add(p.getSecond().toString());
}
@@ -256,13 +257,13 @@ public abstract class TakeSnapshotHandler extends EventHandler implements Snapsh
/**
* Snapshot the specified regions
*/
- protected abstract void snapshotRegions(List<Pair<HRegionInfo, ServerName>> regions)
+ protected abstract void snapshotRegions(List<Pair<RegionInfo, ServerName>> regions)
throws IOException, KeeperException;
/**
* Take a snapshot of the specified disabled region
*/
- protected void snapshotDisabledRegion(final HRegionInfo regionInfo)
+ protected void snapshotDisabledRegion(final RegionInfo regionInfo)
throws IOException {
snapshotManifest.addRegion(FSUtils.getTableDir(rootDir, snapshotTable), regionInfo);
monitor.rethrowException();
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobUtils.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobUtils.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobUtils.java
index 54f1373..e87cb3d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobUtils.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobUtils.java
@@ -45,7 +45,6 @@ import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellComparator;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.Tag;
import org.apache.hadoop.hbase.TagType;
@@ -53,6 +52,8 @@ import org.apache.hadoop.hbase.TagUtil;
import org.apache.hadoop.hbase.backup.HFileArchiver;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
import org.apache.hadoop.hbase.client.MobCompactPartitionPolicy;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.io.HFileLink;
@@ -385,7 +386,7 @@ public final class MobUtils {
*/
public static Path getMobRegionPath(Configuration conf, TableName tableName) {
Path tablePath = FSUtils.getTableDir(getMobHome(conf), tableName);
- HRegionInfo regionInfo = getMobRegionInfo(tableName);
+ RegionInfo regionInfo = getMobRegionInfo(tableName);
return new Path(tablePath, regionInfo.getEncodedName());
}
@@ -413,24 +414,27 @@ public final class MobUtils {
}
/**
- * Gets the HRegionInfo of the mob files.
+ * Gets the RegionInfo of the mob files.
* This is a dummy region. The mob files are not saved in a region in HBase.
* This is only used in mob snapshot. It's internally used only.
* @param tableName
* @return A dummy mob region info.
*/
- public static HRegionInfo getMobRegionInfo(TableName tableName) {
- HRegionInfo info = new HRegionInfo(tableName, MobConstants.MOB_REGION_NAME_BYTES,
- HConstants.EMPTY_END_ROW, false, 0);
- return info;
+ public static RegionInfo getMobRegionInfo(TableName tableName) {
+ return RegionInfoBuilder.newBuilder(tableName)
+ .setStartKey(MobConstants.MOB_REGION_NAME_BYTES)
+ .setEndKey(HConstants.EMPTY_END_ROW)
+ .setSplit(false)
+ .setRegionId(0)
+ .build();
}
/**
- * Gets whether the current HRegionInfo is a mob one.
- * @param regionInfo The current HRegionInfo.
- * @return If true, the current HRegionInfo is a mob one.
+ * Gets whether the current RegionInfo is a mob one.
+ * @param regionInfo The current RegionInfo.
+ * @return If true, the current RegionInfo is a mob one.
*/
- public static boolean isMobRegionInfo(HRegionInfo regionInfo) {
+ public static boolean isMobRegionInfo(RegionInfo regionInfo) {
return regionInfo == null ? false : getMobRegionInfo(regionInfo.getTable()).getEncodedName()
.equals(regionInfo.getEncodedName());
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/namespace/NamespaceAuditor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/namespace/NamespaceAuditor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/namespace/NamespaceAuditor.java
index b12777d..efe2c1e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/namespace/NamespaceAuditor.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/namespace/NamespaceAuditor.java
@@ -22,14 +22,14 @@ import java.io.IOException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HBaseIOException;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.TableExistsException;
import org.apache.hadoop.hbase.TableName;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.quotas.QuotaExceededException;
+import org.apache.yetus.audience.InterfaceAudience;
/**
* The Class NamespaceAuditor performs checks to ensure operations like table creation
@@ -74,7 +74,7 @@ public class NamespaceAuditor {
checkTableTypeAndThrowException(tName);
}
}
-
+
/**
* Check and update region count quota for an existing table.
* @param tName - table name for which region count to be updated.
@@ -113,7 +113,7 @@ public class NamespaceAuditor {
return -1;
}
- public void checkQuotaToSplitRegion(HRegionInfo hri) throws IOException {
+ public void checkQuotaToSplitRegion(RegionInfo hri) throws IOException {
if (!stateManager.isInitialized()) {
throw new IOException(
"Split operation is being performed even before namespace auditor is initialized.");
@@ -124,7 +124,7 @@ public class NamespaceAuditor {
}
}
- public void updateQuotaForRegionMerge(HRegionInfo mergedRegion) throws IOException {
+ public void updateQuotaForRegionMerge(RegionInfo mergedRegion) throws IOException {
if (!stateManager.isInitialized()) {
throw new IOException(
"Merge operation is being performed even before namespace auditor is initialized.");
@@ -148,7 +148,7 @@ public class NamespaceAuditor {
stateManager.removeTable(tableName);
}
- public void removeRegionFromNamespaceUsage(HRegionInfo hri) throws IOException {
+ public void removeRegionFromNamespaceUsage(RegionInfo hri) throws IOException {
stateManager.removeRegionFromTable(hri);
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/namespace/NamespaceStateManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/namespace/NamespaceStateManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/namespace/NamespaceStateManager.java
index f3e9ffb..c62594a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/namespace/NamespaceStateManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/namespace/NamespaceStateManager.java
@@ -24,15 +24,15 @@ import java.util.concurrent.ConcurrentMap;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.TableName;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.master.TableNamespaceManager;
import org.apache.hadoop.hbase.quotas.QuotaExceededException;
import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
/**
* NamespaceStateManager manages state (in terms of quota) of all the namespaces. It contains
@@ -75,7 +75,7 @@ class NamespaceStateManager {
/**
* Check if adding a region violates namespace quota, if not update namespace cache.
*
- * @param TableName
+ * @param name
* @param regionName
* @param incr
* @return true, if region can be added to table.
@@ -106,16 +106,16 @@ class NamespaceStateManager {
}
return true;
}
-
+
/**
* Check and update region count for an existing table. To handle scenarios like restore snapshot
- * @param TableName name of the table for region count needs to be checked and updated
+ * @param name name of the table for region count needs to be checked and updated
* @param incr count of regions
* @throws QuotaExceededException if quota exceeds for the number of regions allowed in a
* namespace
* @throws IOException Signals that an I/O exception has occurred.
*/
- synchronized void checkAndUpdateNamespaceRegionCount(TableName name, int incr)
+ synchronized void checkAndUpdateNamespaceRegionCount(TableName name, int incr)
throws IOException {
String namespace = name.getNamespaceAsString();
NamespaceDescriptor nspdesc = getNamespaceDescriptor(namespace);
@@ -221,7 +221,7 @@ class NamespaceStateManager {
if (table.isSystemTable()) {
continue;
}
- List<HRegionInfo> regions =
+ List<RegionInfo> regions =
MetaTableAccessor.getTableRegions(this.master.getConnection(), table, true);
addTable(table, regions.size());
}
@@ -234,7 +234,7 @@ class NamespaceStateManager {
return initialized;
}
- public synchronized void removeRegionFromTable(HRegionInfo hri) throws IOException {
+ public synchronized void removeRegionFromTable(RegionInfo hri) throws IOException {
String namespace = hri.getTable().getNamespaceAsString();
NamespaceTableAndRegionInfo nsInfo = nsStateCache.get(namespace);
if (nsInfo != null) {
[04/19] hbase git commit: HBASE-18839 Apply RegionInfo to code base
Posted by ch...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestBaseLoadBalancer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestBaseLoadBalancer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestBaseLoadBalancer.java
index b360145..c33cd56 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestBaseLoadBalancer.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestBaseLoadBalancer.java
@@ -17,7 +17,12 @@
*/
package org.apache.hadoop.hbase.master.balancer;
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
@@ -28,15 +33,17 @@ import java.util.Set;
import java.util.TreeMap;
import java.util.TreeSet;
import java.util.stream.Collectors;
+
import org.apache.commons.lang3.ArrayUtils;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseIOException;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
import org.apache.hadoop.hbase.master.LoadBalancer;
import org.apache.hadoop.hbase.master.MasterServices;
@@ -48,17 +55,14 @@ import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer.Cluster.MoveRegi
import org.apache.hadoop.hbase.testclassification.MasterTests;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.net.DNSToSwitchMapping;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
import org.junit.BeforeClass;
import org.junit.Rule;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.junit.rules.TestName;
import org.mockito.Mockito;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
+
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
@Category({MasterTests.class, MediumTests.class})
public class TestBaseLoadBalancer extends BalancerTestBase {
@@ -108,13 +112,13 @@ public class TestBaseLoadBalancer extends BalancerTestBase {
public static class MockBalancer extends BaseLoadBalancer {
@Override
- public List<RegionPlan> balanceCluster(Map<ServerName, List<HRegionInfo>> clusterState) {
+ public List<RegionPlan> balanceCluster(Map<ServerName, List<RegionInfo>> clusterState) {
return null;
}
@Override
public List<RegionPlan> balanceCluster(TableName tableName,
- Map<ServerName, List<HRegionInfo>> clusterState) throws HBaseIOException {
+ Map<ServerName, List<RegionInfo>> clusterState) throws HBaseIOException {
return null;
}
}
@@ -125,9 +129,9 @@ public class TestBaseLoadBalancer extends BalancerTestBase {
* @param servers
* @param assignments
*/
- private void assertImmediateAssignment(List<HRegionInfo> regions, List<ServerName> servers,
- Map<HRegionInfo, ServerName> assignments) {
- for (HRegionInfo region : regions) {
+ private void assertImmediateAssignment(List<RegionInfo> regions, List<ServerName> servers,
+ Map<RegionInfo, ServerName> assignments) {
+ for (RegionInfo region : regions) {
assertTrue(assignments.containsKey(region));
}
}
@@ -143,31 +147,31 @@ public class TestBaseLoadBalancer extends BalancerTestBase {
@Test (timeout=180000)
public void testBulkAssignment() throws Exception {
List<ServerName> tmp = getListOfServerNames(randomServers(5, 0));
- List<HRegionInfo> hris = randomRegions(20);
- hris.add(HRegionInfo.FIRST_META_REGIONINFO);
+ List<RegionInfo> hris = randomRegions(20);
+ hris.add(RegionInfoBuilder.FIRST_META_REGIONINFO);
tmp.add(master);
- Map<ServerName, List<HRegionInfo>> plans = loadBalancer.roundRobinAssignment(hris, tmp);
+ Map<ServerName, List<RegionInfo>> plans = loadBalancer.roundRobinAssignment(hris, tmp);
if (LoadBalancer.isTablesOnMaster(loadBalancer.getConf())) {
- assertTrue(plans.get(master).contains(HRegionInfo.FIRST_META_REGIONINFO));
+ assertTrue(plans.get(master).contains(RegionInfoBuilder.FIRST_META_REGIONINFO));
assertEquals(1, plans.get(master).size());
}
int totalRegion = 0;
- for (List<HRegionInfo> regions: plans.values()) {
+ for (List<RegionInfo> regions: plans.values()) {
totalRegion += regions.size();
}
assertEquals(hris.size(), totalRegion);
for (int[] mock : regionsAndServersMocks) {
LOG.debug("testBulkAssignment with " + mock[0] + " regions and " + mock[1] + " servers");
- List<HRegionInfo> regions = randomRegions(mock[0]);
+ List<RegionInfo> regions = randomRegions(mock[0]);
List<ServerAndLoad> servers = randomServers(mock[1], 0);
List<ServerName> list = getListOfServerNames(servers);
- Map<ServerName, List<HRegionInfo>> assignments =
+ Map<ServerName, List<RegionInfo>> assignments =
loadBalancer.roundRobinAssignment(regions, list);
float average = (float) regions.size() / servers.size();
int min = (int) Math.floor(average);
int max = (int) Math.ceil(average);
if (assignments != null && !assignments.isEmpty()) {
- for (List<HRegionInfo> regionList : assignments.values()) {
+ for (List<RegionInfo> regionList : assignments.values()) {
assertTrue(regionList.size() == min || regionList.size() == max);
}
}
@@ -185,8 +189,8 @@ public class TestBaseLoadBalancer extends BalancerTestBase {
public void testRetainAssignment() throws Exception {
// Test simple case where all same servers are there
List<ServerAndLoad> servers = randomServers(10, 10);
- List<HRegionInfo> regions = randomRegions(100);
- Map<HRegionInfo, ServerName> existing = new TreeMap<>();
+ List<RegionInfo> regions = randomRegions(100);
+ Map<RegionInfo, ServerName> existing = new TreeMap<>(RegionInfo.COMPARATOR);
for (int i = 0; i < regions.size(); i++) {
ServerName sn = servers.get(i % servers.size()).getServerName();
// The old server would have had same host and port, but different
@@ -196,7 +200,7 @@ public class TestBaseLoadBalancer extends BalancerTestBase {
existing.put(regions.get(i), snWithOldStartCode);
}
List<ServerName> listOfServerNames = getListOfServerNames(servers);
- Map<ServerName, List<HRegionInfo>> assignment =
+ Map<ServerName, List<RegionInfo>> assignment =
loadBalancer.retainAssignment(existing, listOfServerNames);
assertRetainedAssignment(existing, listOfServerNames, assignment);
@@ -236,7 +240,7 @@ public class TestBaseLoadBalancer extends BalancerTestBase {
allServers.addAll(idleServers);
LoadBalancer balancer = new MockBalancer() {
@Override
- public boolean shouldBeOnMaster(HRegionInfo region) {
+ public boolean shouldBeOnMaster(RegionInfo region) {
return false;
}
};
@@ -249,9 +253,12 @@ public class TestBaseLoadBalancer extends BalancerTestBase {
MasterServices services = Mockito.mock(MasterServices.class);
Mockito.when(services.getServerManager()).thenReturn(sm);
balancer.setMasterServices(services);
- HRegionInfo hri1 = new HRegionInfo(
- TableName.valueOf(name.getMethodName()), "key1".getBytes(), "key2".getBytes(),
- false, 100);
+ RegionInfo hri1 = RegionInfoBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
+ .setStartKey("key1".getBytes())
+ .setEndKey("key2".getBytes())
+ .setSplit(false)
+ .setRegionId(100)
+ .build();
assertNull(balancer.randomAssignment(hri1, Collections.EMPTY_LIST));
assertNull(balancer.randomAssignment(hri1, null));
for (int i = 0; i != 3; ++i) {
@@ -267,23 +274,29 @@ public class TestBaseLoadBalancer extends BalancerTestBase {
// replica from one node to a specific other node or rack lowers the
// availability of the region or not
- List<HRegionInfo> list0 = new ArrayList<>();
- List<HRegionInfo> list1 = new ArrayList<>();
- List<HRegionInfo> list2 = new ArrayList<>();
+ List<RegionInfo> list0 = new ArrayList<>();
+ List<RegionInfo> list1 = new ArrayList<>();
+ List<RegionInfo> list2 = new ArrayList<>();
// create a region (region1)
- HRegionInfo hri1 = new HRegionInfo(
- TableName.valueOf(name.getMethodName()), "key1".getBytes(), "key2".getBytes(),
- false, 100);
+ RegionInfo hri1 = RegionInfoBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
+ .setStartKey("key1".getBytes())
+ .setEndKey("key2".getBytes())
+ .setSplit(false)
+ .setRegionId(100)
+ .build();
// create a replica of the region (replica_of_region1)
- HRegionInfo hri2 = RegionReplicaUtil.getRegionInfoForReplica(hri1, 1);
+ RegionInfo hri2 = RegionReplicaUtil.getRegionInfoForReplica(hri1, 1);
// create a second region (region2)
- HRegionInfo hri3 = new HRegionInfo(
- TableName.valueOf(name.getMethodName()), "key2".getBytes(), "key3".getBytes(),
- false, 101);
+ RegionInfo hri3 = RegionInfoBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
+ .setStartKey("key2".getBytes())
+ .setEndKey("key3".getBytes())
+ .setSplit(false)
+ .setRegionId(101)
+ .build();
list0.add(hri1); //only region1
list1.add(hri2); //only replica_of_region1
list2.add(hri3); //only region2
- Map<ServerName, List<HRegionInfo>> clusterState = new LinkedHashMap<>();
+ Map<ServerName, List<RegionInfo>> clusterState = new LinkedHashMap<>();
clusterState.put(servers[0], list0); //servers[0] hosts region1
clusterState.put(servers[1], list1); //servers[1] hosts replica_of_region1
clusterState.put(servers[2], list2); //servers[2] hosts region2
@@ -335,23 +348,29 @@ public class TestBaseLoadBalancer extends BalancerTestBase {
@Test (timeout=180000)
public void testRegionAvailabilityWithRegionMoves() throws Exception {
- List<HRegionInfo> list0 = new ArrayList<>();
- List<HRegionInfo> list1 = new ArrayList<>();
- List<HRegionInfo> list2 = new ArrayList<>();
+ List<RegionInfo> list0 = new ArrayList<>();
+ List<RegionInfo> list1 = new ArrayList<>();
+ List<RegionInfo> list2 = new ArrayList<>();
// create a region (region1)
- HRegionInfo hri1 = new HRegionInfo(
- TableName.valueOf(name.getMethodName()), "key1".getBytes(), "key2".getBytes(),
- false, 100);
+ RegionInfo hri1 = RegionInfoBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
+ .setStartKey("key1".getBytes())
+ .setEndKey("key2".getBytes())
+ .setSplit(false)
+ .setRegionId(100)
+ .build();
// create a replica of the region (replica_of_region1)
- HRegionInfo hri2 = RegionReplicaUtil.getRegionInfoForReplica(hri1, 1);
+ RegionInfo hri2 = RegionReplicaUtil.getRegionInfoForReplica(hri1, 1);
// create a second region (region2)
- HRegionInfo hri3 = new HRegionInfo(
- TableName.valueOf(name.getMethodName()), "key2".getBytes(), "key3".getBytes(),
- false, 101);
+ RegionInfo hri3 = RegionInfoBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
+ .setStartKey("key2".getBytes())
+ .setEndKey("key3".getBytes())
+ .setSplit(false)
+ .setRegionId(101)
+ .build();
list0.add(hri1); //only region1
list1.add(hri2); //only replica_of_region1
list2.add(hri3); //only region2
- Map<ServerName, List<HRegionInfo>> clusterState = new LinkedHashMap<>();
+ Map<ServerName, List<RegionInfo>> clusterState = new LinkedHashMap<>();
clusterState.put(servers[0], list0); //servers[0] hosts region1
clusterState.put(servers[1], list1); //servers[1] hosts replica_of_region1
clusterState.put(servers[2], list2); //servers[2] hosts region2
@@ -373,8 +392,8 @@ public class TestBaseLoadBalancer extends BalancerTestBase {
// start over again
clusterState.clear();
- List<HRegionInfo> list3 = new ArrayList<>();
- HRegionInfo hri4 = RegionReplicaUtil.getRegionInfoForReplica(hri3, 1);
+ List<RegionInfo> list3 = new ArrayList<>();
+ RegionInfo hri4 = RegionReplicaUtil.getRegionInfoForReplica(hri3, 1);
list3.add(hri4);
clusterState.put(servers[0], list0); //servers[0], rack1 hosts region1
clusterState.put(servers[5], list1); //servers[5], rack2 hosts replica_of_region1
@@ -409,15 +428,15 @@ public class TestBaseLoadBalancer extends BalancerTestBase {
* @param servers
* @param assignment
*/
- private void assertRetainedAssignment(Map<HRegionInfo, ServerName> existing,
- List<ServerName> servers, Map<ServerName, List<HRegionInfo>> assignment) {
+ private void assertRetainedAssignment(Map<RegionInfo, ServerName> existing,
+ List<ServerName> servers, Map<ServerName, List<RegionInfo>> assignment) {
// Verify condition 1, every region assigned, and to online server
Set<ServerName> onlineServerSet = new TreeSet<>(servers);
- Set<HRegionInfo> assignedRegions = new TreeSet<>();
- for (Map.Entry<ServerName, List<HRegionInfo>> a : assignment.entrySet()) {
+ Set<RegionInfo> assignedRegions = new TreeSet<>(RegionInfo.COMPARATOR);
+ for (Map.Entry<ServerName, List<RegionInfo>> a : assignment.entrySet()) {
assertTrue("Region assigned to server that was not listed as online",
onlineServerSet.contains(a.getKey()));
- for (HRegionInfo r : a.getValue())
+ for (RegionInfo r : a.getValue())
assignedRegions.add(r);
}
assertEquals(existing.size(), assignedRegions.size());
@@ -428,9 +447,9 @@ public class TestBaseLoadBalancer extends BalancerTestBase {
onlineHostNames.add(s.getHostname());
}
- for (Map.Entry<ServerName, List<HRegionInfo>> a : assignment.entrySet()) {
+ for (Map.Entry<ServerName, List<RegionInfo>> a : assignment.entrySet()) {
ServerName assignedTo = a.getKey();
- for (HRegionInfo r : a.getValue()) {
+ for (RegionInfo r : a.getValue()) {
ServerName address = existing.get(r);
if (address != null && onlineHostNames.contains(address.getHostname())) {
// this region was prevously assigned somewhere, and that
@@ -447,8 +466,8 @@ public class TestBaseLoadBalancer extends BalancerTestBase {
// tests whether the BaseLoadBalancer.Cluster can be constructed with servers
// sharing same host and port
List<ServerName> servers = getListOfServerNames(randomServers(10, 10));
- List<HRegionInfo> regions = randomRegions(101);
- Map<ServerName, List<HRegionInfo>> clusterState = new HashMap<>();
+ List<RegionInfo> regions = randomRegions(101);
+ Map<ServerName, List<RegionInfo>> clusterState = new HashMap<>();
assignRegions(regions, servers, clusterState);
@@ -468,11 +487,11 @@ public class TestBaseLoadBalancer extends BalancerTestBase {
assertEquals(10, cluster.numServers); // only 10 servers because they share the same host + port
}
- private void assignRegions(List<HRegionInfo> regions, List<ServerName> servers,
- Map<ServerName, List<HRegionInfo>> clusterState) {
+ private void assignRegions(List<RegionInfo> regions, List<ServerName> servers,
+ Map<ServerName, List<RegionInfo>> clusterState) {
for (int i = 0; i < regions.size(); i++) {
ServerName sn = servers.get(i % servers.size());
- List<HRegionInfo> regionsOfServer = clusterState.get(sn);
+ List<RegionInfo> regionsOfServer = clusterState.get(sn);
if (regionsOfServer == null) {
regionsOfServer = new ArrayList<>(10);
clusterState.put(sn, regionsOfServer);
@@ -486,8 +505,8 @@ public class TestBaseLoadBalancer extends BalancerTestBase {
public void testClusterRegionLocations() {
// tests whether region locations are handled correctly in Cluster
List<ServerName> servers = getListOfServerNames(randomServers(10, 10));
- List<HRegionInfo> regions = randomRegions(101);
- Map<ServerName, List<HRegionInfo>> clusterState = new HashMap<>();
+ List<RegionInfo> regions = randomRegions(101);
+ Map<ServerName, List<RegionInfo>> clusterState = new HashMap<>();
assignRegions(regions, servers, clusterState);
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestDefaultLoadBalancer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestDefaultLoadBalancer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestDefaultLoadBalancer.java
index 610ecf7..4d09bf8 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestDefaultLoadBalancer.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestDefaultLoadBalancer.java
@@ -17,6 +17,8 @@
*/
package org.apache.hadoop.hbase.master.balancer;
+import static org.junit.Assert.assertTrue;
+
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
@@ -27,15 +29,14 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.master.LoadBalancer;
import org.apache.hadoop.hbase.master.RegionPlan;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.testclassification.MasterTests;
import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.net.DNSToSwitchMapping;
import org.junit.BeforeClass;
import org.junit.Rule;
@@ -43,9 +44,6 @@ import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.junit.rules.TestName;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
/**
* Test the load balancer that is created by default.
*/
@@ -129,16 +127,16 @@ public class TestDefaultLoadBalancer extends BalancerTestBase {
*/
@Test (timeout=60000)
public void testBalanceClusterOverall() throws Exception {
- Map<TableName, Map<ServerName, List<HRegionInfo>>> clusterLoad = new TreeMap<>();
+ Map<TableName, Map<ServerName, List<RegionInfo>>> clusterLoad = new TreeMap<>();
for (int[] mockCluster : clusterStateMocks) {
- Map<ServerName, List<HRegionInfo>> clusterServers = mockClusterServers(mockCluster, 50);
+ Map<ServerName, List<RegionInfo>> clusterServers = mockClusterServers(mockCluster, 50);
List<ServerAndLoad> clusterList = convertToList(clusterServers);
clusterLoad.put(TableName.valueOf(name.getMethodName()), clusterServers);
- HashMap<TableName, TreeMap<ServerName, List<HRegionInfo>>> result = mockClusterServersWithTables(clusterServers);
+ HashMap<TableName, TreeMap<ServerName, List<RegionInfo>>> result = mockClusterServersWithTables(clusterServers);
loadBalancer.setClusterLoad(clusterLoad);
List<RegionPlan> clusterplans = new ArrayList<>();
List<Pair<TableName, Integer>> regionAmountList = new ArrayList<>();
- for(TreeMap<ServerName, List<HRegionInfo>> servers : result.values()){
+ for(TreeMap<ServerName, List<RegionInfo>> servers : result.values()){
List<ServerAndLoad> list = convertToList(servers);
LOG.info("Mock Cluster : " + printMock(list) + " " + printStats(list));
List<RegionPlan> partialplans = loadBalancer.balanceCluster(servers);
@@ -146,7 +144,7 @@ public class TestDefaultLoadBalancer extends BalancerTestBase {
List<ServerAndLoad> balancedClusterPerTable = reconcile(list, partialplans, servers);
LOG.info("Mock Balance : " + printMock(balancedClusterPerTable));
assertClusterAsBalanced(balancedClusterPerTable);
- for (Map.Entry<ServerName, List<HRegionInfo>> entry : servers.entrySet()) {
+ for (Map.Entry<ServerName, List<RegionInfo>> entry : servers.entrySet()) {
returnRegions(entry.getValue());
returnServer(entry.getKey());
}
@@ -167,16 +165,16 @@ public class TestDefaultLoadBalancer extends BalancerTestBase {
*/
@Test (timeout=60000)
public void testImpactOfBalanceClusterOverall() throws Exception {
- Map<TableName, Map<ServerName, List<HRegionInfo>>> clusterLoad = new TreeMap<>();
- Map<ServerName, List<HRegionInfo>> clusterServers = mockUniformClusterServers(mockUniformCluster);
+ Map<TableName, Map<ServerName, List<RegionInfo>>> clusterLoad = new TreeMap<>();
+ Map<ServerName, List<RegionInfo>> clusterServers = mockUniformClusterServers(mockUniformCluster);
List<ServerAndLoad> clusterList = convertToList(clusterServers);
clusterLoad.put(TableName.valueOf(name.getMethodName()), clusterServers);
// use overall can achieve both table and cluster level balance
- HashMap<TableName, TreeMap<ServerName, List<HRegionInfo>>> result1 = mockClusterServersWithTables(clusterServers);
+ HashMap<TableName, TreeMap<ServerName, List<RegionInfo>>> result1 = mockClusterServersWithTables(clusterServers);
loadBalancer.setClusterLoad(clusterLoad);
List<RegionPlan> clusterplans1 = new ArrayList<RegionPlan>();
List<Pair<TableName, Integer>> regionAmountList = new ArrayList<Pair<TableName, Integer>>();
- for(TreeMap<ServerName, List<HRegionInfo>> servers : result1.values()){
+ for(TreeMap<ServerName, List<RegionInfo>> servers : result1.values()){
List<ServerAndLoad> list = convertToList(servers);
LOG.info("Mock Cluster : " + printMock(list) + " " + printStats(list));
List<RegionPlan> partialplans = loadBalancer.balanceCluster(servers);
@@ -184,7 +182,7 @@ public class TestDefaultLoadBalancer extends BalancerTestBase {
List<ServerAndLoad> balancedClusterPerTable = reconcile(list, partialplans, servers);
LOG.info("Mock Balance : " + printMock(balancedClusterPerTable));
assertClusterAsBalanced(balancedClusterPerTable);
- for (Map.Entry<ServerName, List<HRegionInfo>> entry : servers.entrySet()) {
+ for (Map.Entry<ServerName, List<RegionInfo>> entry : servers.entrySet()) {
returnRegions(entry.getValue());
returnServer(entry.getKey());
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticBalancerPickers.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticBalancerPickers.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticBalancerPickers.java
index 8f2e893..e636cb0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticBalancerPickers.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticBalancerPickers.java
@@ -29,26 +29,25 @@ import java.util.EnumSet;
import java.util.List;
import java.util.Map;
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.Maps;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.ClusterStatus;
+import org.apache.hadoop.hbase.ClusterStatus.Option;
import org.apache.hadoop.hbase.HBaseTestingUtility;
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.client.Admin;
-import org.apache.hadoop.hbase.master.RackManager;
-import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.Waiter;
-import org.apache.hadoop.hbase.ClusterStatus.Option;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.favored.FavoredNodesManager;
import org.apache.hadoop.hbase.master.LoadBalancer;
+import org.apache.hadoop.hbase.master.RackManager;
import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer.Cluster;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.After;
import org.junit.Before;
@@ -58,6 +57,7 @@ import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.Maps;
@Category(LargeTests.class)
public class TestFavoredStochasticBalancerPickers extends BalancerTestBase {
@@ -116,7 +116,7 @@ public class TestFavoredStochasticBalancerPickers extends BalancerTestBase {
ServerName source = getRSWithMaxRegions(Lists.newArrayList(masterServerName, mostLoadedServer));
assertNotNull(source);
int regionsToMove = admin.getOnlineRegions(source).size()/2;
- List<HRegionInfo> hris = admin.getOnlineRegions(source);
+ List<RegionInfo> hris = admin.getRegions(source);
for (int i = 0; i < regionsToMove; i++) {
admin.move(hris.get(i).getEncodedNameAsBytes(), Bytes.toBytes(mostLoadedServer.getServerName()));
LOG.info("Moving region: " + hris.get(i).getRegionNameAsString() + " to " + mostLoadedServer);
@@ -132,11 +132,11 @@ public class TestFavoredStochasticBalancerPickers extends BalancerTestBase {
});
TEST_UTIL.getHBaseCluster().startRegionServerAndWait(60000);
- Map<ServerName, List<HRegionInfo>> serverAssignments = Maps.newHashMap();
+ Map<ServerName, List<RegionInfo>> serverAssignments = Maps.newHashMap();
ClusterStatus status = admin.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS));
for (ServerName sn : status.getServers()) {
if (!ServerName.isSameAddress(sn, masterServerName)) {
- serverAssignments.put(sn, admin.getOnlineRegions(sn));
+ serverAssignments.put(sn, admin.getRegions(sn));
}
}
RegionLocationFinder regionFinder = new RegionLocationFinder();
@@ -165,7 +165,7 @@ public class TestFavoredStochasticBalancerPickers extends BalancerTestBase {
Cluster.Action action = loadPicker.generate(cluster);
if (action.type == Cluster.Action.Type.MOVE_REGION) {
Cluster.MoveRegionAction moveRegionAction = (Cluster.MoveRegionAction) action;
- HRegionInfo region = cluster.regions[moveRegionAction.region];
+ RegionInfo region = cluster.regions[moveRegionAction.region];
assertNotEquals(-1, moveRegionAction.toServer);
ServerName destinationServer = cluster.servers[moveRegionAction.toServer];
assertEquals(cluster.servers[moveRegionAction.fromServer], mostLoadedServer);
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticLoadBalancer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticLoadBalancer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticLoadBalancer.java
index 4cf3fb6..2f22eeb 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticLoadBalancer.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticLoadBalancer.java
@@ -34,27 +34,27 @@ import java.util.Set;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.ClusterStatus.Option;
import org.apache.hadoop.hbase.HBaseTestingUtility;
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.MiniHBaseCluster;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.Waiter;
-import org.apache.hadoop.hbase.ClusterStatus.Option;
import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.favored.FavoredNodeAssignmentHelper;
+import org.apache.hadoop.hbase.favored.FavoredNodesManager;
import org.apache.hadoop.hbase.favored.FavoredNodesPlan;
import org.apache.hadoop.hbase.master.HMaster;
+import org.apache.hadoop.hbase.master.LoadBalancer;
+import org.apache.hadoop.hbase.master.ServerManager;
import org.apache.hadoop.hbase.master.assignment.RegionStates;
import org.apache.hadoop.hbase.master.assignment.RegionStates.RegionStateNode;
-import org.apache.hadoop.hbase.master.ServerManager;
import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.testclassification.MediumTests;
-import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.favored.FavoredNodesManager;
-import org.apache.hadoop.hbase.master.LoadBalancer;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.JVMClusterUtil;
import org.junit.After;
@@ -126,13 +126,13 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
assertTrue("Balancer did not run", admin.balancer());
TEST_UTIL.waitUntilNoRegionsInTransition(120000);
- List<HRegionInfo> hris = admin.getOnlineRegions(rs1.getRegionServer().getServerName());
- for (HRegionInfo hri : hris) {
+ List<RegionInfo> hris = admin.getRegions(rs1.getRegionServer().getServerName());
+ for (RegionInfo hri : hris) {
assertFalse("New RS contains regions belonging to table: " + tableName,
hri.getTable().equals(tableName));
}
- hris = admin.getOnlineRegions(rs2.getRegionServer().getServerName());
- for (HRegionInfo hri : hris) {
+ hris = admin.getRegions(rs2.getRegionServer().getServerName());
+ for (RegionInfo hri : hris) {
assertFalse("New RS contains regions belonging to table: " + tableName,
hri.getTable().equals(tableName));
}
@@ -150,13 +150,13 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
admin.flush(tableName);
LoadBalancer balancer = master.getLoadBalancer();
- List<HRegionInfo> regions = admin.getTableRegions(tableName);
+ List<RegionInfo> regions = admin.getRegions(tableName);
regions.addAll(admin.getTableRegions(TableName.META_TABLE_NAME));
regions.addAll(admin.getTableRegions(TableName.NAMESPACE_TABLE_NAME));
List<ServerName> servers = Lists.newArrayList(
admin.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS)).getServers());
- Map<ServerName, List<HRegionInfo>> map = balancer.roundRobinAssignment(regions, servers);
- for (List<HRegionInfo> regionInfos : map.values()) {
+ Map<ServerName, List<RegionInfo>> map = balancer.roundRobinAssignment(regions, servers);
+ for (List<RegionInfo> regionInfos : map.values()) {
regions.removeAll(regionInfos);
}
assertEquals("No region should be missed by balancer", 0, regions.size());
@@ -173,8 +173,8 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
TEST_UTIL.waitTableAvailable(desc.getTableName());
FavoredNodesManager fnm = master.getFavoredNodesManager();
- List<HRegionInfo> regionsOfTable = admin.getTableRegions(TableName.valueOf(tableName));
- for (HRegionInfo rInfo : regionsOfTable) {
+ List<RegionInfo> regionsOfTable = admin.getRegions(TableName.valueOf(tableName));
+ for (RegionInfo rInfo : regionsOfTable) {
Set<ServerName> favNodes = Sets.newHashSet(fnm.getFavoredNodes(rInfo));
assertNotNull(favNodes);
assertEquals(FavoredNodeAssignmentHelper.FAVORED_NODES_NUM, favNodes.size());
@@ -212,7 +212,7 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
admin.createTable(desc);
TEST_UTIL.waitTableAvailable(desc.getTableName());
- HRegionInfo hri = admin.getTableRegions(TableName.valueOf(tableName)).get(0);
+ RegionInfo hri = admin.getTableRegions(TableName.valueOf(tableName)).get(0);
FavoredNodesManager fnm = master.getFavoredNodesManager();
fnm.deleteFavoredNodesForRegions(Lists.newArrayList(hri));
@@ -242,7 +242,7 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
admin.createTable(desc, Bytes.toBytes("aaa"), Bytes.toBytes("zzz"), REGION_NUM);
TEST_UTIL.waitTableAvailable(tableName);
- final HRegionInfo region = admin.getTableRegions(tableName).get(0);
+ final RegionInfo region = admin.getTableRegions(tableName).get(0);
LOG.info("Region thats supposed to be in transition: " + region);
FavoredNodesManager fnm = master.getFavoredNodesManager();
List<ServerName> currentFN = fnm.getFavoredNodes(region);
@@ -280,7 +280,7 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
admin.createTable(desc, Bytes.toBytes("aaa"), Bytes.toBytes("zzz"), REGION_NUM);
TEST_UTIL.waitTableAvailable(tableName);
- final HRegionInfo misplacedRegion = admin.getTableRegions(tableName).get(0);
+ final RegionInfo misplacedRegion = admin.getTableRegions(tableName).get(0);
FavoredNodesManager fnm = master.getFavoredNodesManager();
List<ServerName> currentFN = fnm.getFavoredNodes(misplacedRegion);
assertNotNull(currentFN);
@@ -297,7 +297,7 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
List<ServerName> newFavoredNodes = helper.generateFavoredNodes(misplacedRegion);
assertNotNull(newFavoredNodes);
assertEquals(FavoredNodeAssignmentHelper.FAVORED_NODES_NUM, newFavoredNodes.size());
- Map<HRegionInfo, List<ServerName>> regionFNMap = Maps.newHashMap();
+ Map<RegionInfo, List<ServerName>> regionFNMap = Maps.newHashMap();
regionFNMap.put(misplacedRegion, newFavoredNodes);
fnm.updateFavoredNodes(regionFNMap);
@@ -326,7 +326,7 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
admin.createTable(desc, Bytes.toBytes("aaa"), Bytes.toBytes("zzz"), REGION_NUM);
TEST_UTIL.waitTableAvailable(tableName);
- final HRegionInfo region = admin.getTableRegions(tableName).get(0);
+ final RegionInfo region = admin.getTableRegions(tableName).get(0);
LOG.info("Region that's supposed to be in transition: " + region);
FavoredNodesManager fnm = master.getFavoredNodesManager();
List<ServerName> currentFN = fnm.getFavoredNodes(region);
@@ -364,7 +364,7 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
admin.createTable(desc, Bytes.toBytes("aaa"), Bytes.toBytes("zzz"), REGION_NUM);
TEST_UTIL.waitTableAvailable(tableName);
- final HRegionInfo region = admin.getTableRegions(tableName).get(0);
+ final RegionInfo region = admin.getTableRegions(tableName).get(0);
LOG.info("Region that's supposed to be in transition: " + region);
FavoredNodesManager fnm = master.getFavoredNodesManager();
List<ServerName> currentFN = fnm.getFavoredNodes(region);
@@ -394,13 +394,13 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
helper.initialize();
for (RegionStateNode regionState: regionStates.getRegionsInTransition()) {
- HRegionInfo regionInfo = regionState.getRegionInfo();
+ RegionInfo regionInfo = regionState.getRegionInfo();
List<ServerName> newFavoredNodes = helper.generateFavoredNodes(regionInfo);
assertNotNull(newFavoredNodes);
assertEquals(FavoredNodeAssignmentHelper.FAVORED_NODES_NUM, newFavoredNodes.size());
LOG.info("Region: " + regionInfo.getEncodedName() + " FN: " + newFavoredNodes);
- Map<HRegionInfo, List<ServerName>> regionFNMap = Maps.newHashMap();
+ Map<RegionInfo, List<ServerName>> regionFNMap = Maps.newHashMap();
regionFNMap.put(regionInfo, newFavoredNodes);
fnm.updateFavoredNodes(regionFNMap);
LOG.info("Assigning region: " + regionInfo.getEncodedName());
@@ -425,7 +425,7 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
admin.createTable(desc, Bytes.toBytes("aaa"), Bytes.toBytes("zzz"), REGION_NUM);
TEST_UTIL.waitTableAvailable(tableName);
- final HRegionInfo region = admin.getTableRegions(tableName).get(0);
+ final RegionInfo region = admin.getTableRegions(tableName).get(0);
LOG.info("Region that's supposed to be in transition: " + region);
FavoredNodesManager fnm = master.getFavoredNodesManager();
List<ServerName> currentFN = fnm.getFavoredNodes(region);
@@ -446,9 +446,9 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
assertTrue("Region: " + region + " should be RIT",
regionStatesBeforeMaster.getRegionState(region).isFailedOpen());
- List<HRegionInfo> rit = Lists.newArrayList();
+ List<RegionInfo> rit = Lists.newArrayList();
for (RegionStateNode regionState: regionStatesBeforeMaster.getRegionsInTransition()) {
- HRegionInfo regionInfo = regionState.getRegionInfo();
+ RegionInfo regionInfo = regionState.getRegionInfo();
LOG.debug("Region in transition after stopping FN's: " + regionInfo);
rit.add(regionInfo);
assertTrue("Region: " + regionInfo + " should be RIT",
@@ -473,7 +473,7 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
assertTrue("Region: " + region + " should be RIT",
regionStates.getRegionState(region).isFailedOpen());
- for (HRegionInfo regionInfo : rit) {
+ for (RegionInfo regionInfo : rit) {
assertTrue("Region: " + regionInfo + " should be RIT",
regionStates.getRegionState(regionInfo).isFailedOpen());
}
@@ -487,13 +487,13 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
FavoredNodeAssignmentHelper helper = new FavoredNodeAssignmentHelper(serversForNewFN, conf);
helper.initialize();
- for (HRegionInfo regionInfo : rit) {
+ for (RegionInfo regionInfo : rit) {
List<ServerName> newFavoredNodes = helper.generateFavoredNodes(regionInfo);
assertNotNull(newFavoredNodes);
assertEquals(FavoredNodeAssignmentHelper.FAVORED_NODES_NUM, newFavoredNodes.size());
LOG.info("Region: " + regionInfo.getEncodedName() + " FN: " + newFavoredNodes);
- Map<HRegionInfo, List<ServerName>> regionFNMap = Maps.newHashMap();
+ Map<RegionInfo, List<ServerName>> regionFNMap = Maps.newHashMap();
regionFNMap.put(regionInfo, newFavoredNodes);
fnm.updateFavoredNodes(regionFNMap);
LOG.info("Assigning region: " + regionInfo.getEncodedName());
@@ -511,7 +511,7 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
private void checkFavoredNodeAssignments(TableName tableName, FavoredNodesManager fnm,
RegionStates regionStates) throws IOException {
- for (HRegionInfo hri : admin.getTableRegions(tableName)) {
+ for (RegionInfo hri : admin.getTableRegions(tableName)) {
ServerName host = regionStates.getRegionServerOfRegion(hri);
assertNotNull("Region: " + hri.getEncodedName() + " not on FN, current: " + host
+ " FN list: " + fnm.getFavoredNodes(hri),
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRegionLocationFinder.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRegionLocationFinder.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRegionLocationFinder.java
index b96dcb5..010f57a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRegionLocationFinder.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRegionLocationFinder.java
@@ -26,10 +26,10 @@ import java.util.List;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HDFSBlocksDistribution;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.regionserver.Region;
@@ -84,7 +84,7 @@ public class TestRegionLocationFinder {
for (int i = 0; i < ServerNum; i++) {
HRegionServer server = cluster.getRegionServer(i);
for (Region region : server.getRegions(tableName)) {
- // get region's hdfs block distribution by region and RegionLocationFinder,
+ // get region's hdfs block distribution by region and RegionLocationFinder,
// they should have same result
HDFSBlocksDistribution blocksDistribution1 = region.getHDFSBlocksDistribution();
HDFSBlocksDistribution blocksDistribution2 = finder.getBlockDistribution(region
@@ -151,12 +151,12 @@ public class TestRegionLocationFinder {
if (regions.size() <= 0) {
continue;
}
- List<HRegionInfo> regionInfos = new ArrayList<>(regions.size());
+ List<RegionInfo> regionInfos = new ArrayList<>(regions.size());
for (Region region : regions) {
regionInfos.add(region.getRegionInfo());
}
finder.refreshAndWait(regionInfos);
- for (HRegionInfo regionInfo : regionInfos) {
+ for (RegionInfo regionInfo : regionInfos) {
assertNotNull(finder.getCache().getIfPresent(regionInfo));
}
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancer.java
index b97567d..e547f87 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancer.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancer.java
@@ -39,10 +39,10 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.ClusterStatus;
import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.RegionLoad;
import org.apache.hadoop.hbase.ServerLoad;
import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
import org.apache.hadoop.hbase.master.MockNoopMasterServices;
import org.apache.hadoop.hbase.master.RackManager;
@@ -159,7 +159,7 @@ public class TestStochasticLoadBalancer extends BalancerTestBase {
conf.setFloat("hbase.master.balancer.stochastic.minCostNeedBalance", 1.0f);
loadBalancer.setConf(conf);
for (int[] mockCluster : clusterStateMocks) {
- Map<ServerName, List<HRegionInfo>> servers = mockClusterServers(mockCluster);
+ Map<ServerName, List<RegionInfo>> servers = mockClusterServers(mockCluster);
List<RegionPlan> plans = loadBalancer.balanceCluster(servers);
assertNull(plans);
}
@@ -183,7 +183,7 @@ public class TestStochasticLoadBalancer extends BalancerTestBase {
conf.setFloat("hbase.master.balancer.stochastic.maxMovePercent", 1.0f);
loadBalancer.setConf(conf);
for (int[] mockCluster : clusterStateMocks) {
- Map<ServerName, List<HRegionInfo>> servers = mockClusterServers(mockCluster);
+ Map<ServerName, List<RegionInfo>> servers = mockClusterServers(mockCluster);
List<ServerAndLoad> list = convertToList(servers);
LOG.info("Mock Cluster : " + printMock(list) + " " + printStats(list));
List<RegionPlan> plans = loadBalancer.balanceCluster(servers);
@@ -192,7 +192,7 @@ public class TestStochasticLoadBalancer extends BalancerTestBase {
assertClusterAsBalanced(balancedCluster);
List<RegionPlan> secondPlans = loadBalancer.balanceCluster(servers);
assertNull(secondPlans);
- for (Map.Entry<ServerName, List<HRegionInfo>> entry : servers.entrySet()) {
+ for (Map.Entry<ServerName, List<RegionInfo>> entry : servers.entrySet()) {
returnRegions(entry.getValue());
returnServer(entry.getKey());
}
@@ -375,7 +375,7 @@ public class TestStochasticLoadBalancer extends BalancerTestBase {
int replication = 1;
int numTables = 2;
- Map<ServerName, List<HRegionInfo>> serverMap =
+ Map<ServerName, List<RegionInfo>> serverMap =
createServerMap(numNodes, numRegions, numRegionsPerServer, replication, numTables);
List<ServerAndLoad> list = convertToList(serverMap);
@@ -419,7 +419,7 @@ public class TestStochasticLoadBalancer extends BalancerTestBase {
costFunction = new StochasticLoadBalancer.RegionReplicaHostCostFunction(conf);
int [] servers = new int[] {3,3,3,3,3};
- TreeMap<ServerName, List<HRegionInfo>> clusterState = mockClusterServers(servers);
+ TreeMap<ServerName, List<RegionInfo>> clusterState = mockClusterServers(servers);
BaseLoadBalancer.Cluster cluster;
@@ -429,7 +429,7 @@ public class TestStochasticLoadBalancer extends BalancerTestBase {
assertEquals(0, costWithoutReplicas, 0);
// replicate the region from first server to the last server
- HRegionInfo replica1 = RegionReplicaUtil.getRegionInfoForReplica(
+ RegionInfo replica1 = RegionReplicaUtil.getRegionInfoForReplica(
clusterState.firstEntry().getValue().get(0),1);
clusterState.lastEntry().getValue().add(replica1);
@@ -440,7 +440,7 @@ public class TestStochasticLoadBalancer extends BalancerTestBase {
assertEquals(0, costWith1ReplicaDifferentServer, 0);
// add a third replica to the last server
- HRegionInfo replica2 = RegionReplicaUtil.getRegionInfoForReplica(replica1, 2);
+ RegionInfo replica2 = RegionReplicaUtil.getRegionInfoForReplica(replica1, 2);
clusterState.lastEntry().getValue().add(replica2);
cluster = new BaseLoadBalancer.Cluster(clusterState, null, null, null);
@@ -451,14 +451,14 @@ public class TestStochasticLoadBalancer extends BalancerTestBase {
// test with replication = 4 for following:
- HRegionInfo replica3;
- Iterator<Entry<ServerName, List<HRegionInfo>>> it;
- Entry<ServerName, List<HRegionInfo>> entry;
+ RegionInfo replica3;
+ Iterator<Entry<ServerName, List<RegionInfo>>> it;
+ Entry<ServerName, List<RegionInfo>> entry;
clusterState = mockClusterServers(servers);
it = clusterState.entrySet().iterator();
entry = it.next(); //first server
- HRegionInfo hri = entry.getValue().get(0);
+ RegionInfo hri = entry.getValue().get(0);
replica1 = RegionReplicaUtil.getRegionInfoForReplica(hri, 1);
replica2 = RegionReplicaUtil.getRegionInfoForReplica(hri, 2);
replica3 = RegionReplicaUtil.getRegionInfoForReplica(hri, 3);
@@ -491,10 +491,10 @@ public class TestStochasticLoadBalancer extends BalancerTestBase {
public void testNeedsBalanceForColocatedReplicas() {
// check for the case where there are two hosts and with one rack, and where
// both the replicas are hosted on the same server
- List<HRegionInfo> regions = randomRegions(1);
+ List<RegionInfo> regions = randomRegions(1);
ServerName s1 = ServerName.valueOf("host1", 1000, 11111);
ServerName s2 = ServerName.valueOf("host11", 1000, 11111);
- Map<ServerName, List<HRegionInfo>> map = new HashMap<>();
+ Map<ServerName, List<RegionInfo>> map = new HashMap<>();
map.put(s1, regions);
regions.add(RegionReplicaUtil.getRegionInfoForReplica(regions.get(0), 1));
// until the step above s1 holds two replicas of a region
@@ -505,7 +505,7 @@ public class TestStochasticLoadBalancer extends BalancerTestBase {
// and both the replicas are on the same rack
map.clear();
regions = randomRegions(1);
- List<HRegionInfo> regionsOnS2 = new ArrayList<>(1);
+ List<RegionInfo> regionsOnS2 = new ArrayList<>(1);
regionsOnS2.add(RegionReplicaUtil.getRegionInfoForReplica(regions.get(0), 1));
map.put(s1, regions);
map.put(s2, regionsOnS2);
@@ -615,13 +615,13 @@ public class TestStochasticLoadBalancer extends BalancerTestBase {
int replication = 3; // 3 replicas per region
int numRegionsPerServer = 5;
int numTables = 10;
- Map<ServerName, List<HRegionInfo>> serverMap =
+ Map<ServerName, List<RegionInfo>> serverMap =
createServerMap(numHosts, numRegions, numRegionsPerServer, replication, numTables);
int numNodesPerHost = 4;
// create a new map with 4 RS per host.
- Map<ServerName, List<HRegionInfo>> newServerMap = new TreeMap<>(serverMap);
- for (Map.Entry<ServerName, List<HRegionInfo>> entry : serverMap.entrySet()) {
+ Map<ServerName, List<RegionInfo>> newServerMap = new TreeMap<>(serverMap);
+ for (Map.Entry<ServerName, List<RegionInfo>> entry : serverMap.entrySet()) {
for (int i=1; i < numNodesPerHost; i++) {
ServerName s1 = entry.getKey();
ServerName s2 = ServerName.valueOf(s1.getHostname(), s1.getPort() + i, 1); // create an RS for the same host
@@ -662,7 +662,7 @@ public class TestStochasticLoadBalancer extends BalancerTestBase {
int numRegionsPerServer = 28;
int numTables = 10;
int numRacks = 4; // all replicas should be on a different rack
- Map<ServerName, List<HRegionInfo>> serverMap =
+ Map<ServerName, List<RegionInfo>> serverMap =
createServerMap(numNodes, numRegions, numRegionsPerServer, replication, numTables);
RackManager rm = new ForTestRackManager(numRacks);
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/master/locking/TestLockProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/locking/TestLockProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/locking/TestLockProcedure.java
index e338849..ce02395 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/locking/TestLockProcedure.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/locking/TestLockProcedure.java
@@ -35,9 +35,9 @@ import org.apache.hadoop.hbase.CategoryBasedTimeout;
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.locking.LockServiceClient;
import org.apache.hadoop.hbase.master.MasterRpcServices;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureConstants;
@@ -46,12 +46,6 @@ import org.apache.hadoop.hbase.procedure2.LockType;
import org.apache.hadoop.hbase.procedure2.Procedure;
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
-import org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockHeartbeatRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockHeartbeatResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockResponse;
import org.apache.hadoop.hbase.testclassification.MasterTests;
import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.hamcrest.core.IsInstanceOf;
@@ -67,6 +61,13 @@ import org.junit.rules.ExpectedException;
import org.junit.rules.TestName;
import org.junit.rules.TestRule;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockHeartbeatRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockHeartbeatResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockResponse;
+
@Category({MasterTests.class, SmallTests.class})
public class TestLockProcedure {
@Rule
@@ -87,9 +88,9 @@ public class TestLockProcedure {
private static String namespace = "namespace";
private static TableName tableName1 = TableName.valueOf(namespace, "table1");
- private static List<HRegionInfo> tableRegions1;
+ private static List<RegionInfo> tableRegions1;
private static TableName tableName2 = TableName.valueOf(namespace, "table2");
- private static List<HRegionInfo> tableRegions2;
+ private static List<RegionInfo> tableRegions2;
private String testMethodName;
@@ -109,8 +110,8 @@ public class TestLockProcedure {
UTIL.createTable(tableName2, new byte[][]{"fam".getBytes()}, new byte[][] {"1".getBytes()});
masterRpcService = UTIL.getHBaseCluster().getMaster().getMasterRpcServices();
procExec = UTIL.getMiniHBaseCluster().getMaster().getMasterProcedureExecutor();
- tableRegions1 = UTIL.getAdmin().getTableRegions(tableName1);
- tableRegions2 = UTIL.getAdmin().getTableRegions(tableName2);
+ tableRegions1 = UTIL.getAdmin().getRegions(tableName1);
+ tableRegions2 = UTIL.getAdmin().getRegions(tableName2);
assert tableRegions1.size() > 0;
assert tableRegions2.size() > 0;
}
@@ -151,7 +152,7 @@ public class TestLockProcedure {
null, tableName, null, description, HConstants.NO_NONCE, HConstants.NO_NONCE);
}
- private LockRequest getRegionLock(List<HRegionInfo> regionInfos, String description) {
+ private LockRequest getRegionLock(List<RegionInfo> regionInfos, String description) {
return LockServiceClient.buildLockRequest(LockServiceProtos.LockType.EXCLUSIVE,
null, null, regionInfos, description, HConstants.NO_NONCE, HConstants.NO_NONCE);
}
@@ -178,7 +179,7 @@ public class TestLockProcedure {
@Test
public void testLockRequestValidationRegionsFromDifferentTable() throws Exception {
- List<HRegionInfo> regions = new ArrayList<>();
+ List<RegionInfo> regions = new ArrayList<>();
regions.addAll(tableRegions1);
regions.addAll(tableRegions2);
validateLockRequestException(getRegionLock(regions, "desc"),
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizer.java
index e25a61f..ab6d7d0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizer.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizer.java
@@ -32,16 +32,13 @@ import java.util.Map;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HBaseIOException;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.RegionLoad;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
import org.apache.hadoop.hbase.master.MasterRpcServices;
import org.apache.hadoop.hbase.master.MasterServices;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledResponse;
-import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController;
-import org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
import org.apache.hadoop.hbase.testclassification.MasterTests;
import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.apache.hadoop.hbase.util.Bytes;
@@ -52,6 +49,11 @@ import org.junit.experimental.categories.Category;
import org.junit.rules.TestName;
import org.mockito.Mockito;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledResponse;
+
/**
* Tests logic of {@link SimpleRegionNormalizer}.
*/
@@ -76,10 +78,10 @@ public class TestSimpleRegionNormalizer {
@Test
public void testNoNormalizationForMetaTable() throws HBaseIOException {
TableName testTable = TableName.META_TABLE_NAME;
- List<HRegionInfo> hris = new ArrayList<>();
+ List<RegionInfo> RegionInfo = new ArrayList<>();
Map<byte[], Integer> regionSizes = new HashMap<>();
- setupMocksForNormalizer(regionSizes, hris);
+ setupMocksForNormalizer(regionSizes, RegionInfo);
List<NormalizationPlan> plans = normalizer.computePlanForTable(testTable);
assertTrue(plans == null);
}
@@ -87,18 +89,23 @@ public class TestSimpleRegionNormalizer {
@Test
public void testNoNormalizationIfTooFewRegions() throws HBaseIOException {
final TableName tableName = TableName.valueOf(name.getMethodName());
- List<HRegionInfo> hris = new ArrayList<>();
+ List<RegionInfo> RegionInfo = new ArrayList<>();
Map<byte[], Integer> regionSizes = new HashMap<>();
-
- HRegionInfo hri1 = new HRegionInfo(tableName, Bytes.toBytes("aaa"), Bytes.toBytes("bbb"));
- hris.add(hri1);
+ RegionInfo hri1 = RegionInfoBuilder.newBuilder(tableName)
+ .setStartKey(Bytes.toBytes("aaa"))
+ .setEndKey(Bytes.toBytes("bbb"))
+ .build();
+ RegionInfo.add(hri1);
regionSizes.put(hri1.getRegionName(), 10);
- HRegionInfo hri2 = new HRegionInfo(tableName, Bytes.toBytes("bbb"), Bytes.toBytes("ccc"));
- hris.add(hri2);
+ RegionInfo hri2 = RegionInfoBuilder.newBuilder(tableName)
+ .setStartKey(Bytes.toBytes("bbb"))
+ .setEndKey(Bytes.toBytes("ccc"))
+ .build();
+ RegionInfo.add(hri2);
regionSizes.put(hri2.getRegionName(), 15);
- setupMocksForNormalizer(regionSizes, hris);
+ setupMocksForNormalizer(regionSizes, RegionInfo);
List<NormalizationPlan> plans = normalizer.computePlanForTable(tableName);
assertTrue(plans == null);
}
@@ -106,26 +113,37 @@ public class TestSimpleRegionNormalizer {
@Test
public void testNoNormalizationOnNormalizedCluster() throws HBaseIOException {
final TableName tableName = TableName.valueOf(name.getMethodName());
- List<HRegionInfo> hris = new ArrayList<>();
+ List<RegionInfo> RegionInfo = new ArrayList<>();
Map<byte[], Integer> regionSizes = new HashMap<>();
- HRegionInfo hri1 = new HRegionInfo(tableName, Bytes.toBytes("aaa"), Bytes.toBytes("bbb"));
- hris.add(hri1);
+ RegionInfo hri1 = RegionInfoBuilder.newBuilder(tableName)
+ .setStartKey(Bytes.toBytes("aaa"))
+ .setEndKey(Bytes.toBytes("bbb"))
+ .build();
+ RegionInfo.add(hri1);
regionSizes.put(hri1.getRegionName(), 10);
- HRegionInfo hri2 = new HRegionInfo(tableName, Bytes.toBytes("bbb"), Bytes.toBytes("ccc"));
- hris.add(hri2);
+ RegionInfo hri2 = RegionInfoBuilder.newBuilder(tableName)
+ .setStartKey(Bytes.toBytes("bbb"))
+ .setEndKey(Bytes.toBytes("ccc"))
+ .build();
+ RegionInfo.add(hri2);
regionSizes.put(hri2.getRegionName(), 15);
- HRegionInfo hri3 = new HRegionInfo(tableName, Bytes.toBytes("ccc"), Bytes.toBytes("ddd"));
- hris.add(hri3);
+ RegionInfo hri3 = RegionInfoBuilder.newBuilder(tableName)
+ .setStartKey(Bytes.toBytes("ccc"))
+ .setEndKey(Bytes.toBytes("ddd"))
+ .build();
+ RegionInfo.add(hri3);
regionSizes.put(hri3.getRegionName(), 8);
- HRegionInfo hri4 = new HRegionInfo(tableName, Bytes.toBytes("ddd"), Bytes.toBytes("eee"));
- hris.add(hri4);
+ RegionInfo hri4 = RegionInfoBuilder.newBuilder(tableName)
+ .setStartKey(Bytes.toBytes("ddd"))
+ .setEndKey(Bytes.toBytes("eee"))
+ .build();
regionSizes.put(hri4.getRegionName(), 10);
- setupMocksForNormalizer(regionSizes, hris);
+ setupMocksForNormalizer(regionSizes, RegionInfo);
List<NormalizationPlan> plans = normalizer.computePlanForTable(tableName);
assertTrue(plans == null);
}
@@ -133,30 +151,45 @@ public class TestSimpleRegionNormalizer {
@Test
public void testMergeOfSmallRegions() throws HBaseIOException {
final TableName tableName = TableName.valueOf(name.getMethodName());
- List<HRegionInfo> hris = new ArrayList<>();
+ List<RegionInfo> RegionInfo = new ArrayList<>();
Map<byte[], Integer> regionSizes = new HashMap<>();
- HRegionInfo hri1 = new HRegionInfo(tableName, Bytes.toBytes("aaa"), Bytes.toBytes("bbb"));
- hris.add(hri1);
+ RegionInfo hri1 = RegionInfoBuilder.newBuilder(tableName)
+ .setStartKey(Bytes.toBytes("aaa"))
+ .setEndKey(Bytes.toBytes("bbb"))
+ .build();
+ RegionInfo.add(hri1);
regionSizes.put(hri1.getRegionName(), 15);
- HRegionInfo hri2 = new HRegionInfo(tableName, Bytes.toBytes("bbb"), Bytes.toBytes("ccc"));
- hris.add(hri2);
+ RegionInfo hri2 = RegionInfoBuilder.newBuilder(tableName)
+ .setStartKey(Bytes.toBytes("bbb"))
+ .setEndKey(Bytes.toBytes("ccc"))
+ .build();
+ RegionInfo.add(hri2);
regionSizes.put(hri2.getRegionName(), 5);
- HRegionInfo hri3 = new HRegionInfo(tableName, Bytes.toBytes("ccc"), Bytes.toBytes("ddd"));
- hris.add(hri3);
+ RegionInfo hri3 = RegionInfoBuilder.newBuilder(tableName)
+ .setStartKey(Bytes.toBytes("ccc"))
+ .setEndKey(Bytes.toBytes("ddd"))
+ .build();
+ RegionInfo.add(hri3);
regionSizes.put(hri3.getRegionName(), 5);
- HRegionInfo hri4 = new HRegionInfo(tableName, Bytes.toBytes("ddd"), Bytes.toBytes("eee"));
- hris.add(hri4);
+ RegionInfo hri4 = RegionInfoBuilder.newBuilder(tableName)
+ .setStartKey(Bytes.toBytes("ddd"))
+ .setEndKey(Bytes.toBytes("eee"))
+ .build();
+ RegionInfo.add(hri4);
regionSizes.put(hri4.getRegionName(), 15);
- HRegionInfo hri5 = new HRegionInfo(tableName, Bytes.toBytes("eee"), Bytes.toBytes("fff"));
- hris.add(hri5);
+ RegionInfo hri5 = RegionInfoBuilder.newBuilder(tableName)
+ .setStartKey(Bytes.toBytes("eee"))
+ .setEndKey(Bytes.toBytes("fff"))
+ .build();
+ RegionInfo.add(hri5);
regionSizes.put(hri5.getRegionName(), 16);
- setupMocksForNormalizer(regionSizes, hris);
+ setupMocksForNormalizer(regionSizes, RegionInfo);
List<NormalizationPlan> plans = normalizer.computePlanForTable(tableName);
NormalizationPlan plan = plans.get(0);
@@ -169,34 +202,52 @@ public class TestSimpleRegionNormalizer {
@Test
public void testMergeOfSecondSmallestRegions() throws HBaseIOException {
final TableName tableName = TableName.valueOf(name.getMethodName());
- List<HRegionInfo> hris = new ArrayList<>();
+ List<RegionInfo> RegionInfo = new ArrayList<>();
Map<byte[], Integer> regionSizes = new HashMap<>();
- HRegionInfo hri1 = new HRegionInfo(tableName, Bytes.toBytes("aaa"), Bytes.toBytes("bbb"));
- hris.add(hri1);
+ RegionInfo hri1 = RegionInfoBuilder.newBuilder(tableName)
+ .setStartKey(Bytes.toBytes("aaa"))
+ .setEndKey(Bytes.toBytes("bbb"))
+ .build();
+ RegionInfo.add(hri1);
regionSizes.put(hri1.getRegionName(), 1);
- HRegionInfo hri2 = new HRegionInfo(tableName, Bytes.toBytes("bbb"), Bytes.toBytes("ccc"));
- hris.add(hri2);
+ RegionInfo hri2 = RegionInfoBuilder.newBuilder(tableName)
+ .setStartKey(Bytes.toBytes("bbb"))
+ .setEndKey(Bytes.toBytes("ccc"))
+ .build();
+ RegionInfo.add(hri2);
regionSizes.put(hri2.getRegionName(), 10000);
- HRegionInfo hri3 = new HRegionInfo(tableName, Bytes.toBytes("ccc"), Bytes.toBytes("ddd"));
- hris.add(hri3);
+ RegionInfo hri3 = RegionInfoBuilder.newBuilder(tableName)
+ .setStartKey(Bytes.toBytes("ccc"))
+ .setEndKey(Bytes.toBytes("ddd"))
+ .build();
+ RegionInfo.add(hri3);
regionSizes.put(hri3.getRegionName(), 10000);
- HRegionInfo hri4 = new HRegionInfo(tableName, Bytes.toBytes("ddd"), Bytes.toBytes("eee"));
- hris.add(hri4);
+ RegionInfo hri4 = RegionInfoBuilder.newBuilder(tableName)
+ .setStartKey(Bytes.toBytes("ddd"))
+ .setEndKey(Bytes.toBytes("eee"))
+ .build();
+ RegionInfo.add(hri4);
regionSizes.put(hri4.getRegionName(), 10000);
- HRegionInfo hri5 = new HRegionInfo(tableName, Bytes.toBytes("eee"), Bytes.toBytes("fff"));
- hris.add(hri5);
+ RegionInfo hri5 = RegionInfoBuilder.newBuilder(tableName)
+ .setStartKey(Bytes.toBytes("eee"))
+ .setEndKey(Bytes.toBytes("fff"))
+ .build();
+ RegionInfo.add(hri5);
regionSizes.put(hri5.getRegionName(), 2700);
- HRegionInfo hri6 = new HRegionInfo(tableName, Bytes.toBytes("fff"), Bytes.toBytes("ggg"));
- hris.add(hri6);
+ RegionInfo hri6 = RegionInfoBuilder.newBuilder(tableName)
+ .setStartKey(Bytes.toBytes("fff"))
+ .setEndKey(Bytes.toBytes("ggg"))
+ .build();
+ RegionInfo.add(hri6);
regionSizes.put(hri6.getRegionName(), 2700);
- setupMocksForNormalizer(regionSizes, hris);
+ setupMocksForNormalizer(regionSizes, RegionInfo);
List<NormalizationPlan> plans = normalizer.computePlanForTable(tableName);
NormalizationPlan plan = plans.get(0);
@@ -208,30 +259,45 @@ public class TestSimpleRegionNormalizer {
@Test
public void testMergeOfSmallNonAdjacentRegions() throws HBaseIOException {
final TableName tableName = TableName.valueOf(name.getMethodName());
- List<HRegionInfo> hris = new ArrayList<>();
+ List<RegionInfo> RegionInfo = new ArrayList<>();
Map<byte[], Integer> regionSizes = new HashMap<>();
- HRegionInfo hri1 = new HRegionInfo(tableName, Bytes.toBytes("aaa"), Bytes.toBytes("bbb"));
- hris.add(hri1);
+ RegionInfo hri1 = RegionInfoBuilder.newBuilder(tableName)
+ .setStartKey(Bytes.toBytes("aaa"))
+ .setEndKey(Bytes.toBytes("bbb"))
+ .build();
+ RegionInfo.add(hri1);
regionSizes.put(hri1.getRegionName(), 15);
- HRegionInfo hri2 = new HRegionInfo(tableName, Bytes.toBytes("bbb"), Bytes.toBytes("ccc"));
- hris.add(hri2);
+ RegionInfo hri2 = RegionInfoBuilder.newBuilder(tableName)
+ .setStartKey(Bytes.toBytes("bbb"))
+ .setEndKey(Bytes.toBytes("ccc"))
+ .build();
+ RegionInfo.add(hri2);
regionSizes.put(hri2.getRegionName(), 5);
- HRegionInfo hri3 = new HRegionInfo(tableName, Bytes.toBytes("ccc"), Bytes.toBytes("ddd"));
- hris.add(hri3);
+ RegionInfo hri3 = RegionInfoBuilder.newBuilder(tableName)
+ .setStartKey(Bytes.toBytes("ccc"))
+ .setEndKey(Bytes.toBytes("ddd"))
+ .build();
+ RegionInfo.add(hri3);
regionSizes.put(hri3.getRegionName(), 16);
- HRegionInfo hri4 = new HRegionInfo(tableName, Bytes.toBytes("ddd"), Bytes.toBytes("eee"));
- hris.add(hri4);
+ RegionInfo hri4 = RegionInfoBuilder.newBuilder(tableName)
+ .setStartKey(Bytes.toBytes("ddd"))
+ .setEndKey(Bytes.toBytes("eee"))
+ .build();
+ RegionInfo.add(hri4);
regionSizes.put(hri4.getRegionName(), 15);
- HRegionInfo hri5 = new HRegionInfo(tableName, Bytes.toBytes("ddd"), Bytes.toBytes("eee"));
- hris.add(hri4);
+ RegionInfo hri5 = RegionInfoBuilder.newBuilder(tableName)
+ .setStartKey(Bytes.toBytes("ddd"))
+ .setEndKey(Bytes.toBytes("eee"))
+ .build();
+ RegionInfo.add(hri4);
regionSizes.put(hri5.getRegionName(), 5);
- setupMocksForNormalizer(regionSizes, hris);
+ setupMocksForNormalizer(regionSizes, RegionInfo);
List<NormalizationPlan> plans = normalizer.computePlanForTable(tableName);
assertTrue(plans == null);
@@ -240,26 +306,38 @@ public class TestSimpleRegionNormalizer {
@Test
public void testSplitOfLargeRegion() throws HBaseIOException {
final TableName tableName = TableName.valueOf(name.getMethodName());
- List<HRegionInfo> hris = new ArrayList<>();
+ List<RegionInfo> RegionInfo = new ArrayList<>();
Map<byte[], Integer> regionSizes = new HashMap<>();
- HRegionInfo hri1 = new HRegionInfo(tableName, Bytes.toBytes("aaa"), Bytes.toBytes("bbb"));
- hris.add(hri1);
+ RegionInfo hri1 = RegionInfoBuilder.newBuilder(tableName)
+ .setStartKey(Bytes.toBytes("aaa"))
+ .setEndKey(Bytes.toBytes("bbb"))
+ .build();
+ RegionInfo.add(hri1);
regionSizes.put(hri1.getRegionName(), 8);
- HRegionInfo hri2 = new HRegionInfo(tableName, Bytes.toBytes("bbb"), Bytes.toBytes("ccc"));
- hris.add(hri2);
+ RegionInfo hri2 = RegionInfoBuilder.newBuilder(tableName)
+ .setStartKey(Bytes.toBytes("bbb"))
+ .setEndKey(Bytes.toBytes("ccc"))
+ .build();
+ RegionInfo.add(hri2);
regionSizes.put(hri2.getRegionName(), 6);
- HRegionInfo hri3 = new HRegionInfo(tableName, Bytes.toBytes("ccc"), Bytes.toBytes("ddd"));
- hris.add(hri3);
+ RegionInfo hri3 = RegionInfoBuilder.newBuilder(tableName)
+ .setStartKey(Bytes.toBytes("ccc"))
+ .setEndKey(Bytes.toBytes("ddd"))
+ .build();
+ RegionInfo.add(hri3);
regionSizes.put(hri3.getRegionName(), 10);
- HRegionInfo hri4 = new HRegionInfo(tableName, Bytes.toBytes("ddd"), Bytes.toBytes("eee"));
- hris.add(hri4);
+ RegionInfo hri4 = RegionInfoBuilder.newBuilder(tableName)
+ .setStartKey(Bytes.toBytes("ddd"))
+ .setEndKey(Bytes.toBytes("eee"))
+ .build();
+ RegionInfo.add(hri4);
regionSizes.put(hri4.getRegionName(), 30);
- setupMocksForNormalizer(regionSizes, hris);
+ setupMocksForNormalizer(regionSizes, RegionInfo);
List<NormalizationPlan> plans = normalizer.computePlanForTable(tableName);
NormalizationPlan plan = plans.get(0);
@@ -268,16 +346,16 @@ public class TestSimpleRegionNormalizer {
}
protected void setupMocksForNormalizer(Map<byte[], Integer> regionSizes,
- List<HRegionInfo> hris) {
+ List<RegionInfo> RegionInfo) {
masterServices = Mockito.mock(MasterServices.class, RETURNS_DEEP_STUBS);
masterRpcServices = Mockito.mock(MasterRpcServices.class, RETURNS_DEEP_STUBS);
// for simplicity all regions are assumed to be on one server; doesn't matter to us
ServerName sn = ServerName.valueOf("localhost", 0, 1L);
when(masterServices.getAssignmentManager().getRegionStates().
- getRegionsOfTable(any(TableName.class))).thenReturn(hris);
+ getRegionsOfTable(any(TableName.class))).thenReturn(RegionInfo);
when(masterServices.getAssignmentManager().getRegionStates().
- getRegionServerOfRegion(any(HRegionInfo.class))).thenReturn(sn);
+ getRegionServerOfRegion(any(RegionInfo.class))).thenReturn(sn);
for (Map.Entry<byte[], Integer> region : regionSizes.entrySet()) {
RegionLoad regionLoad = Mockito.mock(RegionLoad.class);
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizerOnCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizerOnCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizerOnCluster.java
index d733d2b..8fe53af 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizerOnCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizerOnCluster.java
@@ -18,6 +18,13 @@
*/
package org.apache.hadoop.hbase.master.normalizer;
+import static org.junit.Assert.assertEquals;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HBaseTestingUtility;
@@ -29,6 +36,7 @@ import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.master.TableNamespaceManager;
@@ -48,13 +56,6 @@ import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.junit.rules.TestName;
-import java.io.IOException;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.List;
-
-import static org.junit.Assert.assertEquals;
-
/**
* Testing {@link SimpleRegionNormalizer} on minicluster.
*/
@@ -112,13 +113,7 @@ public class TestSimpleRegionNormalizerOnCluster {
try (Table ht = TEST_UTIL.createMultiRegionTable(TABLENAME, FAMILYNAME, 5)) {
// Need to get sorted list of regions here
List<HRegion> generatedRegions = TEST_UTIL.getHBaseCluster().getRegions(TABLENAME);
- Collections.sort(generatedRegions, new Comparator<HRegion>() {
- @Override
- public int compare(HRegion o1, HRegion o2) {
- return o1.getRegionInfo().compareTo(o2.getRegionInfo());
- }
- });
-
+ Collections.sort(generatedRegions, Comparator.comparing(HRegion::getRegionInfo, RegionInfo.COMPARATOR));
HRegion region = generatedRegions.get(0);
generateTestData(region, 1);
region.flush(true);
@@ -189,12 +184,7 @@ public class TestSimpleRegionNormalizerOnCluster {
try (Table ht = TEST_UTIL.createMultiRegionTable(tableName, FAMILYNAME, 5)) {
// Need to get sorted list of regions here
List<HRegion> generatedRegions = TEST_UTIL.getHBaseCluster().getRegions(tableName);
- Collections.sort(generatedRegions, new Comparator<HRegion>() {
- @Override
- public int compare(HRegion o1, HRegion o2) {
- return o1.getRegionInfo().compareTo(o2.getRegionInfo());
- }
- });
+ Collections.sort(generatedRegions, Comparator.comparing(HRegion::getRegionInfo, RegionInfo.COMPARATOR));
HRegion region = generatedRegions.get(0);
generateTestData(region, 1);
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureTestingUtility.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureTestingUtility.java
index 1ca8518..b87c343 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureTestingUtility.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureTestingUtility.java
@@ -33,20 +33,19 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.RegionLocations;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
-import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.BufferedMutator;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
@@ -63,6 +62,7 @@ import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.MD5Hash;
import org.apache.hadoop.hbase.util.ModifyRegionUtils;
+import org.apache.yetus.audience.InterfaceAudience;
@InterfaceAudience.Private
public class MasterProcedureTestingUtility {
@@ -148,10 +148,10 @@ public class MasterProcedureTestingUtility {
return builder.build();
}
- public static HRegionInfo[] createTable(final ProcedureExecutor<MasterProcedureEnv> procExec,
+ public static RegionInfo[] createTable(final ProcedureExecutor<MasterProcedureEnv> procExec,
final TableName tableName, final byte[][] splitKeys, String... family) throws IOException {
TableDescriptor htd = createHTD(tableName, family);
- HRegionInfo[] regions = ModifyRegionUtils.createHRegionInfos(htd, splitKeys);
+ RegionInfo[] regions = ModifyRegionUtils.createRegionInfos(htd, splitKeys);
long procId = ProcedureTestingUtility.submitAndWait(procExec,
new CreateTableProcedure(procExec.getEnvironment(), htd, regions));
ProcedureTestingUtility.assertProcNotFailed(procExec.getResult(procId));
@@ -159,12 +159,12 @@ public class MasterProcedureTestingUtility {
}
public static void validateTableCreation(final HMaster master, final TableName tableName,
- final HRegionInfo[] regions, String... family) throws IOException {
+ final RegionInfo[] regions, String... family) throws IOException {
validateTableCreation(master, tableName, regions, true, family);
}
public static void validateTableCreation(final HMaster master, final TableName tableName,
- final HRegionInfo[] regions, boolean hasFamilyDirs, String... family) throws IOException {
+ final RegionInfo[] regions, boolean hasFamilyDirs, String... family) throws IOException {
// check filesystem
final FileSystem fs = master.getMasterFileSystem().getFileSystem();
final Path tableDir = FSUtils.getTableDir(master.getMasterFileSystem().getRootDir(), tableName);
@@ -230,7 +230,7 @@ public class MasterProcedureTestingUtility {
public boolean visit(Result rowResult) throws IOException {
RegionLocations list = MetaTableAccessor.getRegionLocations(rowResult);
if (list == null) {
- LOG.warn("No serialized HRegionInfo in " + rowResult);
+ LOG.warn("No serialized RegionInfo in " + rowResult);
return true;
}
HRegionLocation l = list.getRegionLocation();
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCreateTableProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCreateTableProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCreateTableProcedure.java
index 687f953..c57f210 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCreateTableProcedure.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCreateTableProcedure.java
@@ -18,13 +18,16 @@
package org.apache.hadoop.hbase.master.procedure;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.DoNotRetryIOException;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.TableExistsException;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
import org.apache.hadoop.hbase.procedure2.Procedure;
@@ -39,9 +42,6 @@ import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.junit.rules.TestName;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
@Category({MasterTests.class, MediumTests.class})
public class TestCreateTableProcedure extends TestTableDDLProcedureBase {
private static final Log LOG = LogFactory.getLog(TestCreateTableProcedure.class);
@@ -68,7 +68,7 @@ public class TestCreateTableProcedure extends TestTableDDLProcedureBase {
}
private void testSimpleCreate(final TableName tableName, byte[][] splitKeys) throws Exception {
- HRegionInfo[] regions = MasterProcedureTestingUtility.createTable(
+ RegionInfo[] regions = MasterProcedureTestingUtility.createTable(
getMasterProcedureExecutor(), tableName, splitKeys, F1, F2);
MasterProcedureTestingUtility.validateTableCreation(getMaster(), tableName, regions, F1, F2);
}
@@ -83,7 +83,7 @@ public class TestCreateTableProcedure extends TestTableDDLProcedureBase {
// disable sanity check
builder.setValue("hbase.table.sanity.checks", Boolean.FALSE.toString());
TableDescriptor htd = builder.build();
- final HRegionInfo[] regions = ModifyRegionUtils.createHRegionInfos(htd, null);
+ final RegionInfo[] regions = ModifyRegionUtils.createRegionInfos(htd, null);
long procId =
ProcedureTestingUtility.submitAndWait(procExec,
@@ -100,7 +100,7 @@ public class TestCreateTableProcedure extends TestTableDDLProcedureBase {
final TableName tableName = TableName.valueOf(name.getMethodName());
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
final TableDescriptor htd = MasterProcedureTestingUtility.createHTD(tableName, "f");
- final HRegionInfo[] regions = ModifyRegionUtils.createHRegionInfos(htd, null);
+ final RegionInfo[] regions = ModifyRegionUtils.createRegionInfos(htd, null);
// create the table
long procId1 = procExec.submitProcedure(
@@ -129,7 +129,7 @@ public class TestCreateTableProcedure extends TestTableDDLProcedureBase {
// Start the Create procedure && kill the executor
byte[][] splitKeys = null;
TableDescriptor htd = MasterProcedureTestingUtility.createHTD(tableName, "f1", "f2");
- HRegionInfo[] regions = ModifyRegionUtils.createHRegionInfos(htd, splitKeys);
+ RegionInfo[] regions = ModifyRegionUtils.createRegionInfos(htd, splitKeys);
long procId = procExec.submitProcedure(
new CreateTableProcedure(procExec.getEnvironment(), htd, regions));
@@ -166,7 +166,7 @@ public class TestCreateTableProcedure extends TestTableDDLProcedureBase {
};
builder.setRegionReplication(3);
TableDescriptor htd = builder.build();
- HRegionInfo[] regions = ModifyRegionUtils.createHRegionInfos(htd, splitKeys);
+ RegionInfo[] regions = ModifyRegionUtils.createRegionInfos(htd, splitKeys);
long procId = procExec.submitProcedure(
new CreateTableProcedure(procExec.getEnvironment(), htd, regions));
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDeleteColumnFamilyProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDeleteColumnFamilyProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDeleteColumnFamilyProcedure.java
index d126251..9747da6 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDeleteColumnFamilyProcedure.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDeleteColumnFamilyProcedure.java
@@ -23,9 +23,9 @@ import static org.junit.Assert.assertTrue;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.CategoryBasedTimeout;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.InvalidFamilyOperationException;
import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.procedure2.Procedure;
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
@@ -193,7 +193,7 @@ public class TestDeleteColumnFamilyProcedure extends TestTableDDLProcedureBase {
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
// create the table
- HRegionInfo[] regions = MasterProcedureTestingUtility.createTable(
+ RegionInfo[] regions = MasterProcedureTestingUtility.createTable(
procExec, tableName, null, "f1", "f2", "f3", cf5);
ProcedureTestingUtility.waitNoProcedureRunning(procExec);
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
[06/19] hbase git commit: HBASE-18839 Apply RegionInfo to code base
Posted by ch...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncClusterAdminApi.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncClusterAdminApi.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncClusterAdminApi.java
index 1acfcde..6307210 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncClusterAdminApi.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncClusterAdminApi.java
@@ -34,14 +34,13 @@ import java.util.Optional;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.ClusterStatus;
+import org.apache.hadoop.hbase.ClusterStatus.Option;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.RegionLoad;
import org.apache.hadoop.hbase.ServerLoad;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.ClusterStatus.Option;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.testclassification.ClientTests;
@@ -227,13 +226,13 @@ public class TestAsyncClusterAdminApi extends TestAsyncAdminBase {
// Check if regions match with the regionLoad from the server
Collection<ServerName> servers = admin.getRegionServers().get();
for (ServerName serverName : servers) {
- List<HRegionInfo> regions = admin.getOnlineRegions(serverName).get();
+ List<RegionInfo> regions = admin.getOnlineRegions(serverName).get();
checkRegionsAndRegionLoads(regions, admin.getRegionLoads(serverName).get());
}
// Check if regionLoad matches the table's regions and nothing is missed
for (TableName table : tables) {
- List<HRegionInfo> tableRegions = admin.getTableRegions(table).get();
+ List<RegionInfo> tableRegions = admin.getTableRegions(table).get();
List<RegionLoad> regionLoads = Lists.newArrayList();
for (ServerName serverName : servers) {
regionLoads.addAll(admin.getRegionLoads(serverName, Optional.of(table)).get());
@@ -268,7 +267,7 @@ public class TestAsyncClusterAdminApi extends TestAsyncAdminBase {
}
}
- private void checkRegionsAndRegionLoads(Collection<HRegionInfo> regions,
+ private void checkRegionsAndRegionLoads(Collection<RegionInfo> regions,
Collection<RegionLoad> regionLoads) {
assertEquals("No of regions and regionloads doesn't match", regions.size(), regionLoads.size());
@@ -277,7 +276,7 @@ public class TestAsyncClusterAdminApi extends TestAsyncAdminBase {
for (RegionLoad regionLoad : regionLoads) {
regionLoadMap.put(regionLoad.getName(), regionLoad);
}
- for (HRegionInfo info : regions) {
+ for (RegionInfo info : regions) {
assertTrue("Region not in regionLoadMap region:" + info.getRegionNameAsString()
+ " regionMap: " + regionLoadMap, regionLoadMap.containsKey(info.getRegionName()));
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionAdminApi.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionAdminApi.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionAdminApi.java
index 6292b10..9775b86 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionAdminApi.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionAdminApi.java
@@ -33,7 +33,6 @@ import java.util.stream.Collectors;
import org.apache.hadoop.hbase.AsyncMetaTableAccessor;
import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
@@ -45,7 +44,6 @@ import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
import org.apache.hadoop.hbase.master.assignment.RegionStates;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.regionserver.Region;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.testclassification.ClientTests;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.util.Bytes;
@@ -58,6 +56,8 @@ import org.junit.experimental.categories.Category;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+
/**
* Class to test asynchronous region admin operations.
*/
@@ -69,10 +69,10 @@ public class TestAsyncRegionAdminApi extends TestAsyncAdminBase {
public void testCloseRegion() throws Exception {
createTableWithDefaultConf(tableName);
- HRegionInfo info = null;
+ RegionInfo info = null;
HRegionServer rs = TEST_UTIL.getRSForFirstRegionInTable(tableName);
- List<HRegionInfo> onlineRegions = ProtobufUtil.getOnlineRegions(rs.getRSRpcServices());
- for (HRegionInfo regionInfo : onlineRegions) {
+ List<RegionInfo> onlineRegions = ProtobufUtil.getOnlineRegions(rs.getRSRpcServices());
+ for (RegionInfo regionInfo : onlineRegions) {
if (!regionInfo.getTable().isSystemTable()) {
info = regionInfo;
boolean closed = admin.closeRegion(regionInfo.getRegionName(),
@@ -94,10 +94,10 @@ public class TestAsyncRegionAdminApi extends TestAsyncAdminBase {
public void testCloseRegionIfInvalidRegionNameIsPassed() throws Exception {
createTableWithDefaultConf(tableName);
- HRegionInfo info = null;
+ RegionInfo info = null;
HRegionServer rs = TEST_UTIL.getRSForFirstRegionInTable(tableName);
- List<HRegionInfo> onlineRegions = ProtobufUtil.getOnlineRegions(rs.getRSRpcServices());
- for (HRegionInfo regionInfo : onlineRegions) {
+ List<RegionInfo> onlineRegions = ProtobufUtil.getOnlineRegions(rs.getRSRpcServices());
+ for (RegionInfo regionInfo : onlineRegions) {
if (!regionInfo.isMetaTable()) {
if (regionInfo.getRegionNameAsString().contains(tableName.getNameAsString())) {
info = regionInfo;
@@ -123,8 +123,8 @@ public class TestAsyncRegionAdminApi extends TestAsyncAdminBase {
createTableWithDefaultConf(tableName);
HRegionServer rs = TEST_UTIL.getRSForFirstRegionInTable(tableName);
- List<HRegionInfo> onlineRegions = ProtobufUtil.getOnlineRegions(rs.getRSRpcServices());
- for (HRegionInfo regionInfo : onlineRegions) {
+ List<RegionInfo> onlineRegions = ProtobufUtil.getOnlineRegions(rs.getRSRpcServices());
+ for (RegionInfo regionInfo : onlineRegions) {
if (!regionInfo.isMetaTable()) {
if (regionInfo.getRegionNameAsString().contains("TestHBACloseRegionWhenServerNameIsEmpty")) {
admin.closeRegion(regionInfo.getRegionName(), Optional.empty()).get();
@@ -139,7 +139,7 @@ public class TestAsyncRegionAdminApi extends TestAsyncAdminBase {
TEST_UTIL.createMultiRegionTable(tableName, HConstants.CATALOG_FAMILY);
AsyncTableRegionLocator locator = ASYNC_CONN.getRegionLocator(tableName);
HRegionLocation regionLocation = locator.getRegionLocation(Bytes.toBytes("mmm")).get();
- HRegionInfo region = regionLocation.getRegionInfo();
+ RegionInfo region = regionLocation.getRegionInfo();
byte[] regionName = regionLocation.getRegionInfo().getRegionName();
HRegionLocation location = rawAdmin.getRegionLocation(regionName).get();
assertTrue(Bytes.equals(regionName, location.getRegionInfo().getRegionName()));
@@ -154,7 +154,7 @@ public class TestAsyncRegionAdminApi extends TestAsyncAdminBase {
// assign region.
HMaster master = TEST_UTIL.getHBaseCluster().getMaster();
AssignmentManager am = master.getAssignmentManager();
- HRegionInfo hri = am.getRegionStates().getRegionsOfTable(tableName).get(0);
+ RegionInfo hri = am.getRegionStates().getRegionsOfTable(tableName).get(0);
// assert region on server
RegionStates regionStates = am.getRegionStates();
@@ -184,7 +184,7 @@ public class TestAsyncRegionAdminApi extends TestAsyncAdminBase {
assertTrue(regionStates.getRegionState(hri).isClosed());
}
- HRegionInfo createTableAndGetOneRegion(final TableName tableName)
+ RegionInfo createTableAndGetOneRegion(final TableName tableName)
throws IOException, InterruptedException, ExecutionException {
TableDescriptor desc =
TableDescriptorBuilder.newBuilder(tableName)
@@ -195,7 +195,7 @@ public class TestAsyncRegionAdminApi extends TestAsyncAdminBase {
HMaster master = TEST_UTIL.getHBaseCluster().getMaster();
long timeoutTime = System.currentTimeMillis() + 3000;
while (true) {
- List<HRegionInfo> regions =
+ List<RegionInfo> regions =
master.getAssignmentManager().getRegionStates().getRegionsOfTable(tableName);
if (regions.size() > 3) {
return regions.get(2);
@@ -215,7 +215,7 @@ public class TestAsyncRegionAdminApi extends TestAsyncAdminBase {
// Will cause the Master to tell the regionserver to shut itself down because
// regionserver is reporting the state as OPEN.
public void testOfflineRegion() throws Exception {
- HRegionInfo hri = createTableAndGetOneRegion(tableName);
+ RegionInfo hri = createTableAndGetOneRegion(tableName);
RegionStates regionStates =
TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager().getRegionStates();
@@ -224,7 +224,7 @@ public class TestAsyncRegionAdminApi extends TestAsyncAdminBase {
long timeoutTime = System.currentTimeMillis() + 3000;
while (true) {
if (regionStates.getRegionByStateOfTable(tableName).get(RegionState.State.OFFLINE)
- .contains(hri)) break;
+ .stream().anyMatch(r -> RegionInfo.COMPARATOR.compare(r, hri) == 0)) break;
long now = System.currentTimeMillis();
if (now > timeoutTime) {
fail("Failed to offline the region in time");
@@ -238,21 +238,21 @@ public class TestAsyncRegionAdminApi extends TestAsyncAdminBase {
@Test
public void testGetRegionByStateOfTable() throws Exception {
- HRegionInfo hri = createTableAndGetOneRegion(tableName);
+ RegionInfo hri = createTableAndGetOneRegion(tableName);
RegionStates regionStates =
TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager().getRegionStates();
assertTrue(regionStates.getRegionByStateOfTable(tableName).get(RegionState.State.OPEN)
- .contains(hri));
+ .stream().anyMatch(r -> RegionInfo.COMPARATOR.compare(r, hri) == 0));
assertFalse(regionStates.getRegionByStateOfTable(TableName.valueOf("I_am_the_phantom"))
- .get(RegionState.State.OPEN).contains(hri));
+ .get(RegionState.State.OPEN).stream().anyMatch(r -> RegionInfo.COMPARATOR.compare(r, hri) == 0));
}
@Test
public void testMoveRegion() throws Exception {
admin.setBalancerOn(false).join();
- HRegionInfo hri = createTableAndGetOneRegion(tableName);
+ RegionInfo hri = createTableAndGetOneRegion(tableName);
RawAsyncHBaseAdmin rawAdmin = (RawAsyncHBaseAdmin) ASYNC_CONN.getAdmin();
ServerName serverName = rawAdmin.getRegionLocation(hri.getRegionName()).get().getServerName();
@@ -312,7 +312,7 @@ public class TestAsyncRegionAdminApi extends TestAsyncAdminBase {
@Test
public void testFlushTableAndRegion() throws Exception {
- HRegionInfo hri = createTableAndGetOneRegion(tableName);
+ RegionInfo hri = createTableAndGetOneRegion(tableName);
ServerName serverName =
TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager().getRegionStates()
.getRegionServerOfRegion(hri);
@@ -405,7 +405,7 @@ public class TestAsyncRegionAdminApi extends TestAsyncAdminBase {
// Merge switch is off so merge should NOT succeed.
assertTrue(admin.setMergeOn(false).get());
- List<HRegionInfo> regions = admin.getTableRegions(tableName).get();
+ List<RegionInfo> regions = admin.getTableRegions(tableName).get();
assertTrue(regions.size() > 1);
admin.mergeRegions(regions.get(0).getRegionName(), regions.get(1).getRegionName(), true).join();
int count = admin.getTableRegions(tableName).get().size();
@@ -437,8 +437,8 @@ public class TestAsyncRegionAdminApi extends TestAsyncAdminBase {
RawAsyncTable metaTable = ASYNC_CONN.getRawTable(META_TABLE_NAME);
List<HRegionLocation> regionLocations =
AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, Optional.of(tableName)).get();
- HRegionInfo regionA;
- HRegionInfo regionB;
+ RegionInfo regionA;
+ RegionInfo regionB;
// merge with full name
assertEquals(3, regionLocations.size());
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestBlockEvictionFromClient.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestBlockEvictionFromClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestBlockEvictionFromClient.java
index dec28f3..b4af697 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestBlockEvictionFromClient.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestBlockEvictionFromClient.java
@@ -39,7 +39,6 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
import org.apache.hadoop.hbase.coprocessor.MultiRowMutationEndpoint;
@@ -598,10 +597,10 @@ public class TestBlockEvictionFromClient {
region.flush(true);
LOG.info("About to SPLIT on " + Bytes.toString(ROW1));
TEST_UTIL.getAdmin().split(tableName, ROW1);
- List<HRegionInfo> tableRegions = TEST_UTIL.getAdmin().getTableRegions(tableName);
+ List<RegionInfo> tableRegions = TEST_UTIL.getAdmin().getRegions(tableName);
// Wait for splits
while (tableRegions.size() != 2) {
- tableRegions = TEST_UTIL.getAdmin().getTableRegions(tableName);
+ tableRegions = TEST_UTIL.getAdmin().getRegions(tableName);
Thread.sleep(100);
LOG.info("Waiting on SPLIT to complete...");
}
@@ -1520,7 +1519,7 @@ public class TestBlockEvictionFromClient {
}
@Override
- public HRegionInfo getRegionInfo() {
+ public RegionInfo getRegionInfo() {
return delegate.getRegionInfo();
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java
index 6f590d1..98d864b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java
@@ -38,7 +38,6 @@ import org.apache.hadoop.hbase.CategoryBasedTimeout;
import org.apache.hadoop.hbase.ClusterStatus.Option;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.RegionLocations;
@@ -107,7 +106,7 @@ public class TestMetaWithReplicas {
});
l.setBalancerOn(false);
for (int replicaId = 1; replicaId < 3; replicaId ++) {
- HRegionInfo h = RegionReplicaUtil.getRegionInfoForReplica(HRegionInfo.FIRST_META_REGIONINFO,
+ RegionInfo h = RegionReplicaUtil.getRegionInfoForReplica(RegionInfoBuilder.FIRST_META_REGIONINFO,
replicaId);
try {
TEST_UTIL.getMiniHBaseCluster().getMaster().getAssignmentManager().waitForAssignment(h);
@@ -189,7 +188,7 @@ public class TestMetaWithReplicas {
util.getAdmin().flush(TableName.META_TABLE_NAME);
Thread.sleep(conf.getInt(StorefileRefresherChore.REGIONSERVER_STOREFILE_REFRESH_PERIOD,
30000) * 6);
- List<HRegionInfo> regions = MetaTableAccessor.getTableRegions(c, TABLE);
+ List<RegionInfo> regions = MetaTableAccessor.getTableRegions(c, TABLE);
HRegionLocation hrl = MetaTableAccessor.getRegionLocation(c, regions.get(0));
// Ensure that the primary server for test table is not the same one as the primary
// of the meta region since we will be killing the srv holding the meta's primary...
@@ -420,7 +419,7 @@ public class TestMetaWithReplicas {
final TableName tableName = TableName.valueOf(name.getMethodName());
TEST_UTIL.createTable(tableName, "f");
assertTrue(TEST_UTIL.getAdmin().tableExists(tableName));
- TEST_UTIL.getAdmin().move(HRegionInfo.FIRST_META_REGIONINFO.getEncodedNameAsBytes(),
+ TEST_UTIL.getAdmin().move(RegionInfoBuilder.FIRST_META_REGIONINFO.getEncodedNameAsBytes(),
Bytes.toBytes(moveToServer.getServerName()));
int i = 0;
assert !moveToServer.equals(currentServer);
@@ -463,8 +462,8 @@ public class TestMetaWithReplicas {
@Ignore @Test // Disabled because fsck and this needs work for AMv2
public void testHBaseFsckWithExcessMetaReplicas() throws Exception {
// Create a meta replica (this will be the 4th one) and assign it
- HRegionInfo h = RegionReplicaUtil.getRegionInfoForReplica(
- HRegionInfo.FIRST_META_REGIONINFO, 3);
+ RegionInfo h = RegionReplicaUtil.getRegionInfoForReplica(
+ RegionInfoBuilder.FIRST_META_REGIONINFO, 3);
TEST_UTIL.assignRegion(h);
HBaseFsckRepair.waitUntilAssigned(TEST_UTIL.getAdmin(), h);
// check that problem exists
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMvccConsistentScanner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMvccConsistentScanner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMvccConsistentScanner.java
index eccb1f7..82f4952 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMvccConsistentScanner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMvccConsistentScanner.java
@@ -24,7 +24,6 @@ import static org.junit.Assert.assertNull;
import java.io.IOException;
import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.testclassification.ClientTests;
@@ -83,7 +82,7 @@ public class TestMvccConsistentScanner {
}
private void move() throws IOException, InterruptedException {
- HRegionInfo region =
+ RegionInfo region =
UTIL.getHBaseCluster().getRegions(tableName).stream().findAny().get().getRegionInfo();
HRegionServer rs =
UTIL.getHBaseCluster().getRegionServerThreads().stream().map(t -> t.getRegionServer())
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorInterface.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorInterface.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorInterface.java
index 3ff0f1f..8c5dbae 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorInterface.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorInterface.java
@@ -46,10 +46,11 @@ import org.apache.hadoop.hbase.HBaseTestCase;
import org.apache.hadoop.hbase.HBaseTestingUtility;
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.TableName;
import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.regionserver.ChunkCreator;
import org.apache.hadoop.hbase.regionserver.HRegion;
@@ -114,7 +115,7 @@ public class TestCoprocessorInterface {
}
@Override
- public HRegionInfo getRegionInfo() {
+ public RegionInfo getRegionInfo() {
return delegate.getRegionInfo();
}
@@ -376,7 +377,7 @@ public class TestCoprocessorInterface {
Region reopenRegion(final Region closedRegion, Class<?> ... implClasses)
throws IOException {
- //HRegionInfo info = new HRegionInfo(tableName, null, null, false);
+ //RegionInfo info = new RegionInfo(tableName, null, null, false);
Region r = HRegion.openHRegion(closedRegion, null);
// this following piece is a hack. currently a coprocessorHost
@@ -409,7 +410,11 @@ public class TestCoprocessorInterface {
htd.addFamily(new HColumnDescriptor(family));
}
ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
- HRegionInfo info = new HRegionInfo(tableName, null, null, false);
+ RegionInfo info = RegionInfoBuilder.newBuilder(tableName)
+ .setStartKey(null)
+ .setEndKey(null)
+ .setSplit(false)
+ .build();
Path path = new Path(DIR + callingMethod);
Region r = HBaseTestingUtility.createRegionAndWAL(info, path, conf, htd);
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java
index b038d9d..2759a68 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java
@@ -19,6 +19,11 @@
package org.apache.hadoop.hbase.coprocessor;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
import java.io.IOException;
import java.util.Arrays;
import java.util.Collection;
@@ -33,7 +38,6 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.CoprocessorEnvironment;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MiniHBaseCluster;
@@ -64,10 +68,6 @@ import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
import org.apache.hadoop.hbase.quotas.GlobalQuotaSettings;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesRequest;
import org.apache.hadoop.hbase.testclassification.CoprocessorTests;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.util.Bytes;
@@ -79,10 +79,10 @@ import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.junit.rules.TestName;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesRequest;
/**
@@ -1927,10 +1927,10 @@ public class TestMasterObserver {
byte[] destRS = Bytes.toBytes(cluster.getRegionServer(1).getServerName().toString());
//Make sure no regions are in transition now
UTIL.waitUntilNoRegionsInTransition();
- List<HRegionInfo> openRegions = ProtobufUtil.getOnlineRegions(rs.getRSRpcServices());
+ List<RegionInfo> openRegions = ProtobufUtil.getOnlineRegions(rs.getRSRpcServices());
int moveCnt = openRegions.size()/2;
for (int i=0; i<moveCnt; i++) {
- HRegionInfo info = openRegions.get(i);
+ RegionInfo info = openRegions.get(i);
if (!info.isMetaTable()) {
master.getMasterRpcServices().moveRegion(null, RequestConverter.buildMoveRegionRequest(
openRegions.get(i).getEncodedNameAsBytes(), destRS));
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverInterface.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverInterface.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverInterface.java
index 2666340..88e548a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverInterface.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverInterface.java
@@ -40,7 +40,6 @@ import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.Coprocessor;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.MiniHBaseCluster;
@@ -53,6 +52,7 @@ import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Increment;
import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.RegionLocator;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
@@ -668,7 +668,7 @@ public class TestRegionObserverInterface {
if (!t.isAlive() || t.getRegionServer().isAborted() || t.getRegionServer().isStopping()) {
continue;
}
- for (HRegionInfo r : ProtobufUtil
+ for (RegionInfo r : ProtobufUtil
.getOnlineRegions(t.getRegionServer().getRSRpcServices())) {
if (!r.getTable().equals(tableName)) {
continue;
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java
index 0446f61..2c9ab20 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java
@@ -43,6 +43,7 @@ import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Table;
@@ -262,7 +263,7 @@ public class TestRegionObserverScannerOpenHook {
@SuppressWarnings("deprecation")
public CompactionCompletionNotifyingRegion(Path tableDir, WAL log,
- FileSystem fs, Configuration confParam, HRegionInfo info,
+ FileSystem fs, Configuration confParam, RegionInfo info,
TableDescriptor htd, RegionServerServices rsServices) {
super(tableDir, log, fs, confParam, info, htd, rsServices);
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/favored/TestFavoredNodeAssignmentHelper.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/favored/TestFavoredNodeAssignmentHelper.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/favored/TestFavoredNodeAssignmentHelper.java
index 8d0fe19..24bb4bd 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/favored/TestFavoredNodeAssignmentHelper.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/favored/TestFavoredNodeAssignmentHelper.java
@@ -35,9 +35,10 @@ import java.util.TreeMap;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
import org.apache.hadoop.hbase.master.RackManager;
import org.apache.hadoop.hbase.testclassification.MasterTests;
import org.apache.hadoop.hbase.testclassification.SmallTests;
@@ -160,16 +161,16 @@ public class TestFavoredNodeAssignmentHelper {
Map<String,Integer> rackToServerCount = new HashMap<>();
rackToServerCount.put("rack1", 10);
// have lots of regions to test with
- Triple<Map<HRegionInfo, ServerName>, FavoredNodeAssignmentHelper, List<HRegionInfo>>
+ Triple<Map<RegionInfo, ServerName>, FavoredNodeAssignmentHelper, List<RegionInfo>>
primaryRSMapAndHelper = secondaryAndTertiaryRSPlacementHelper(60000, rackToServerCount);
FavoredNodeAssignmentHelper helper = primaryRSMapAndHelper.getSecond();
- Map<HRegionInfo, ServerName> primaryRSMap = primaryRSMapAndHelper.getFirst();
- List<HRegionInfo> regions = primaryRSMapAndHelper.getThird();
- Map<HRegionInfo, ServerName[]> secondaryAndTertiaryMap =
+ Map<RegionInfo, ServerName> primaryRSMap = primaryRSMapAndHelper.getFirst();
+ List<RegionInfo> regions = primaryRSMapAndHelper.getThird();
+ Map<RegionInfo, ServerName[]> secondaryAndTertiaryMap =
helper.placeSecondaryAndTertiaryRS(primaryRSMap);
// although we created lots of regions we should have no overlap on the
// primary/secondary/tertiary for any given region
- for (HRegionInfo region : regions) {
+ for (RegionInfo region : regions) {
ServerName[] secondaryAndTertiaryServers = secondaryAndTertiaryMap.get(region);
assertNotNull(secondaryAndTertiaryServers);
assertTrue(primaryRSMap.containsKey(region));
@@ -185,13 +186,13 @@ public class TestFavoredNodeAssignmentHelper {
// the primary can be assigned but the secondary/tertiary would be null
Map<String,Integer> rackToServerCount = new HashMap<>();
rackToServerCount.put("rack1", 1);
- Triple<Map<HRegionInfo, ServerName>, FavoredNodeAssignmentHelper, List<HRegionInfo>>
+ Triple<Map<RegionInfo, ServerName>, FavoredNodeAssignmentHelper, List<RegionInfo>>
primaryRSMapAndHelper = secondaryAndTertiaryRSPlacementHelper(1, rackToServerCount);
FavoredNodeAssignmentHelper helper = primaryRSMapAndHelper.getSecond();
- Map<HRegionInfo, ServerName> primaryRSMap = primaryRSMapAndHelper.getFirst();
- List<HRegionInfo> regions = primaryRSMapAndHelper.getThird();
+ Map<RegionInfo, ServerName> primaryRSMap = primaryRSMapAndHelper.getFirst();
+ List<RegionInfo> regions = primaryRSMapAndHelper.getThird();
- Map<HRegionInfo, ServerName[]> secondaryAndTertiaryMap =
+ Map<RegionInfo, ServerName[]> secondaryAndTertiaryMap =
helper.placeSecondaryAndTertiaryRS(primaryRSMap);
// no secondary/tertiary placement in case of a single RegionServer
assertTrue(secondaryAndTertiaryMap.get(regions.get(0)) == null);
@@ -205,18 +206,18 @@ public class TestFavoredNodeAssignmentHelper {
rackToServerCount.put("rack1", 10);
rackToServerCount.put("rack2", 10);
- Triple<Map<HRegionInfo, ServerName>, FavoredNodeAssignmentHelper, List<HRegionInfo>>
+ Triple<Map<RegionInfo, ServerName>, FavoredNodeAssignmentHelper, List<RegionInfo>>
primaryRSMapAndHelper = secondaryAndTertiaryRSPlacementHelper(60000, rackToServerCount);
FavoredNodeAssignmentHelper helper = primaryRSMapAndHelper.getSecond();
- Map<HRegionInfo, ServerName> primaryRSMap = primaryRSMapAndHelper.getFirst();
+ Map<RegionInfo, ServerName> primaryRSMap = primaryRSMapAndHelper.getFirst();
assertTrue(primaryRSMap.size() == 60000);
- Map<HRegionInfo, ServerName[]> secondaryAndTertiaryMap =
+ Map<RegionInfo, ServerName[]> secondaryAndTertiaryMap =
helper.placeSecondaryAndTertiaryRS(primaryRSMap);
assertTrue(secondaryAndTertiaryMap.size() == 60000);
// for every region, the primary should be on one rack and the secondary/tertiary
// on another (we create a lot of regions just to increase probability of failure)
- for (Map.Entry<HRegionInfo, ServerName[]> entry : secondaryAndTertiaryMap.entrySet()) {
+ for (Map.Entry<RegionInfo, ServerName[]> entry : secondaryAndTertiaryMap.entrySet()) {
ServerName[] allServersForRegion = entry.getValue();
String primaryRSRack = rackManager.getRack(primaryRSMap.get(entry.getKey()));
String secondaryRSRack = rackManager.getRack(allServersForRegion[0]);
@@ -235,15 +236,15 @@ public class TestFavoredNodeAssignmentHelper {
Map<String,Integer> rackToServerCount = new HashMap<>();
rackToServerCount.put("rack1", 1);
rackToServerCount.put("rack2", 1);
- Triple<Map<HRegionInfo, ServerName>, FavoredNodeAssignmentHelper, List<HRegionInfo>>
+ Triple<Map<RegionInfo, ServerName>, FavoredNodeAssignmentHelper, List<RegionInfo>>
primaryRSMapAndHelper = secondaryAndTertiaryRSPlacementHelper(6, rackToServerCount);
FavoredNodeAssignmentHelper helper = primaryRSMapAndHelper.getSecond();
- Map<HRegionInfo, ServerName> primaryRSMap = primaryRSMapAndHelper.getFirst();
- List<HRegionInfo> regions = primaryRSMapAndHelper.getThird();
+ Map<RegionInfo, ServerName> primaryRSMap = primaryRSMapAndHelper.getFirst();
+ List<RegionInfo> regions = primaryRSMapAndHelper.getThird();
assertTrue(primaryRSMap.size() == 6);
- Map<HRegionInfo, ServerName[]> secondaryAndTertiaryMap =
+ Map<RegionInfo, ServerName[]> secondaryAndTertiaryMap =
helper.placeSecondaryAndTertiaryRS(primaryRSMap);
- for (HRegionInfo region : regions) {
+ for (RegionInfo region : regions) {
// not enough secondary/tertiary room to place the regions
assertTrue(secondaryAndTertiaryMap.get(region) == null);
}
@@ -259,16 +260,16 @@ public class TestFavoredNodeAssignmentHelper {
Map<String,Integer> rackToServerCount = new HashMap<>();
rackToServerCount.put("rack1", 2);
rackToServerCount.put("rack2", 1);
- Triple<Map<HRegionInfo, ServerName>, FavoredNodeAssignmentHelper, List<HRegionInfo>>
+ Triple<Map<RegionInfo, ServerName>, FavoredNodeAssignmentHelper, List<RegionInfo>>
primaryRSMapAndHelper = secondaryAndTertiaryRSPlacementHelper(6, rackToServerCount);
FavoredNodeAssignmentHelper helper = primaryRSMapAndHelper.getSecond();
- Map<HRegionInfo, ServerName> primaryRSMap = primaryRSMapAndHelper.getFirst();
- List<HRegionInfo> regions = primaryRSMapAndHelper.getThird();
+ Map<RegionInfo, ServerName> primaryRSMap = primaryRSMapAndHelper.getFirst();
+ List<RegionInfo> regions = primaryRSMapAndHelper.getThird();
assertTrue(primaryRSMap.size() == 6);
- Map<HRegionInfo, ServerName[]> secondaryAndTertiaryMap =
+ Map<RegionInfo, ServerName[]> secondaryAndTertiaryMap =
helper.placeSecondaryAndTertiaryRS(primaryRSMap);
assertTrue(secondaryAndTertiaryMap.size() == regions.size());
- for (HRegionInfo region : regions) {
+ for (RegionInfo region : regions) {
ServerName s = primaryRSMap.get(region);
ServerName secondaryRS = secondaryAndTertiaryMap.get(region)[0];
ServerName tertiaryRS = secondaryAndTertiaryMap.get(region)[1];
@@ -279,28 +280,29 @@ public class TestFavoredNodeAssignmentHelper {
}
}
- private Triple<Map<HRegionInfo, ServerName>, FavoredNodeAssignmentHelper, List<HRegionInfo>>
+ private Triple<Map<RegionInfo, ServerName>, FavoredNodeAssignmentHelper, List<RegionInfo>>
secondaryAndTertiaryRSPlacementHelper(
int regionCount, Map<String, Integer> rackToServerCount) {
- Map<HRegionInfo, ServerName> primaryRSMap = new HashMap<HRegionInfo, ServerName>();
+ Map<RegionInfo, ServerName> primaryRSMap = new HashMap<RegionInfo, ServerName>();
List<ServerName> servers = getServersFromRack(rackToServerCount);
FavoredNodeAssignmentHelper helper = new FavoredNodeAssignmentHelper(servers, rackManager);
- Map<ServerName, List<HRegionInfo>> assignmentMap =
- new HashMap<ServerName, List<HRegionInfo>>();
+ Map<ServerName, List<RegionInfo>> assignmentMap =
+ new HashMap<ServerName, List<RegionInfo>>();
helper.initialize();
// create regions
- List<HRegionInfo> regions = new ArrayList<>(regionCount);
+ List<RegionInfo> regions = new ArrayList<>(regionCount);
for (int i = 0; i < regionCount; i++) {
- HRegionInfo region = new HRegionInfo(TableName.valueOf(name.getMethodName()),
- Bytes.toBytes(i), Bytes.toBytes(i + 1));
- regions.add(region);
+ regions.add(RegionInfoBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
+ .setStartKey(Bytes.toBytes(i))
+ .setEndKey(Bytes.toBytes(i + 1))
+ .build());
}
// place the regions
helper.placePrimaryRSAsRoundRobin(assignmentMap, primaryRSMap, regions);
return new Triple<>(primaryRSMap, helper, regions);
}
- private void primaryRSPlacement(int regionCount, Map<HRegionInfo, ServerName> primaryRSMap,
+ private void primaryRSPlacement(int regionCount, Map<RegionInfo, ServerName> primaryRSMap,
int firstRackSize, int secondRackSize, int thirdRackSize) {
Map<String,Integer> rackToServerCount = new HashMap<>();
rackToServerCount.put("rack1", firstRackSize);
@@ -313,14 +315,15 @@ public class TestFavoredNodeAssignmentHelper {
assertTrue(helper.canPlaceFavoredNodes());
- Map<ServerName, List<HRegionInfo>> assignmentMap = new HashMap<>();
+ Map<ServerName, List<RegionInfo>> assignmentMap = new HashMap<>();
if (primaryRSMap == null) primaryRSMap = new HashMap<>();
// create some regions
- List<HRegionInfo> regions = new ArrayList<>(regionCount);
+ List<RegionInfo> regions = new ArrayList<>(regionCount);
for (int i = 0; i < regionCount; i++) {
- HRegionInfo region = new HRegionInfo(TableName.valueOf("foobar"),
- Bytes.toBytes(i), Bytes.toBytes(i + 1));
- regions.add(region);
+ regions.add(RegionInfoBuilder.newBuilder(TableName.valueOf("foobar"))
+ .setStartKey(Bytes.toBytes(i))
+ .setEndKey(Bytes.toBytes(i + 1))
+ .build());
}
// place those regions in primary RSs
helper.placePrimaryRSAsRoundRobin(assignmentMap, primaryRSMap, regions);
@@ -329,7 +332,7 @@ public class TestFavoredNodeAssignmentHelper {
int regionsOnRack1 = 0;
int regionsOnRack2 = 0;
int regionsOnRack3 = 0;
- for (HRegionInfo region : regions) {
+ for (RegionInfo region : regions) {
if (rackManager.getRack(primaryRSMap.get(region)).equals("rack1")) {
regionsOnRack1++;
} else if (rackManager.getRack(primaryRSMap.get(region)).equals("rack2")) {
@@ -346,7 +349,7 @@ public class TestFavoredNodeAssignmentHelper {
private void checkNumRegions(int regionCount, int firstRackSize, int secondRackSize,
int thirdRackSize, int regionsOnRack1, int regionsOnRack2, int regionsOnRack3,
- Map<ServerName, List<HRegionInfo>> assignmentMap) {
+ Map<ServerName, List<RegionInfo>> assignmentMap) {
//The regions should be distributed proportionately to the racksizes
//Verify the ordering was as expected by inserting the racks and regions
//in sorted maps. The keys being the racksize and numregions; values are
@@ -387,18 +390,19 @@ public class TestFavoredNodeAssignmentHelper {
helper.initialize();
assertTrue(helper.canPlaceFavoredNodes());
- List<HRegionInfo> regions = new ArrayList<>(20);
+ List<RegionInfo> regions = new ArrayList<>(20);
for (int i = 0; i < 20; i++) {
- HRegionInfo region = new HRegionInfo(TableName.valueOf(name.getMethodName()),
- Bytes.toBytes(i), Bytes.toBytes(i + 1));
- regions.add(region);
+ regions.add(RegionInfoBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
+ .setStartKey(Bytes.toBytes(i))
+ .setEndKey(Bytes.toBytes(i + 1))
+ .build());
}
- Map<ServerName, List<HRegionInfo>> assignmentMap =
- new HashMap<ServerName, List<HRegionInfo>>();
- Map<HRegionInfo, ServerName> primaryRSMap = new HashMap<HRegionInfo, ServerName>();
+ Map<ServerName, List<RegionInfo>> assignmentMap =
+ new HashMap<ServerName, List<RegionInfo>>();
+ Map<RegionInfo, ServerName> primaryRSMap = new HashMap<RegionInfo, ServerName>();
helper.placePrimaryRSAsRoundRobin(assignmentMap, primaryRSMap, regions);
assertTrue(primaryRSMap.size() == regions.size());
- Map<HRegionInfo, ServerName[]> secondaryAndTertiary =
+ Map<RegionInfo, ServerName[]> secondaryAndTertiary =
helper.placeSecondaryAndTertiaryRS(primaryRSMap);
assertEquals(regions.size(), secondaryAndTertiary.size());
}
@@ -534,8 +538,10 @@ public class TestFavoredNodeAssignmentHelper {
helper.initialize();
assertTrue(helper.canPlaceFavoredNodes());
- HRegionInfo region = new HRegionInfo(TableName.valueOf(name.getMethodName()),
- HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW);
+ RegionInfo region = RegionInfoBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
+ .setStartKey(HConstants.EMPTY_START_ROW)
+ .setEndKey(HConstants.EMPTY_END_ROW)
+ .build();
for (int maxattempts = 0; maxattempts < MAX_ATTEMPTS; maxattempts++) {
List<ServerName> fn = helper.generateFavoredNodes(region);
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
index e96ce6d..fda3563 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
@@ -17,13 +17,14 @@
*/
package org.apache.hadoop.hbase.master;
+import static org.mockito.Mockito.mock;
+
import java.io.IOException;
import java.util.List;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.ChoreService;
import org.apache.hadoop.hbase.CoordinatedStateManager;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableDescriptors;
@@ -31,6 +32,7 @@ import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.ClusterConnection;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
import org.apache.hadoop.hbase.client.MasterSwitchType;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.executor.ExecutorService;
import org.apache.hadoop.hbase.favored.FavoredNodesManager;
@@ -53,8 +55,6 @@ import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import com.google.protobuf.Service;
-import static org.mockito.Mockito.mock;
-
public class MockNoopMasterServices implements MasterServices, Server {
private final Configuration conf;
private final MetricsMaster metricsMaster;
@@ -309,7 +309,7 @@ public class MockNoopMasterServices implements MasterServices, Server {
@Override
public long mergeRegions(
- final HRegionInfo[] regionsToMerge,
+ final RegionInfo[] regionsToMerge,
final boolean forcible,
final long nonceGroup,
final long nonce) throws IOException {
@@ -318,7 +318,7 @@ public class MockNoopMasterServices implements MasterServices, Server {
@Override
public long splitRegion(
- final HRegionInfo regionInfo,
+ final RegionInfo regionInfo,
final byte[] splitRow,
final long nonceGroup,
final long nonce) throws IOException {
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
index 450bf8e..f777067 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
@@ -36,11 +36,12 @@ import org.apache.hadoop.hbase.CellScannable;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.ChoreService;
import org.apache.hadoop.hbase.CoordinatedStateManager;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
import org.apache.hadoop.hbase.client.ClusterConnection;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.locking.EntityLock;
@@ -436,7 +437,7 @@ ClientProtos.ClientService.BlockingInterface, RegionServerServices {
public GetRegionInfoResponse getRegionInfo(RpcController controller,
GetRegionInfoRequest request) throws ServiceException {
GetRegionInfoResponse.Builder builder = GetRegionInfoResponse.newBuilder();
- builder.setRegionInfo(HRegionInfo.convert(HRegionInfo.FIRST_META_REGIONINFO));
+ builder.setRegionInfo(ProtobufUtil.toRegionInfo(RegionInfoBuilder.FIRST_META_REGIONINFO));
return builder.build();
}
@@ -558,7 +559,7 @@ ClientProtos.ClientService.BlockingInterface, RegionServerServices {
}
@Override
- public WAL getWAL(HRegionInfo regionInfo) throws IOException {
+ public WAL getWAL(RegionInfo regionInfo) throws IOException {
return null;
}
@@ -608,13 +609,13 @@ ClientProtos.ClientService.BlockingInterface, RegionServerServices {
}
@Override
- public boolean reportRegionStateTransition(TransitionCode code, HRegionInfo... hris) {
+ public boolean reportRegionStateTransition(TransitionCode code, RegionInfo... hris) {
return false;
}
@Override
public boolean reportRegionStateTransition(TransitionCode code, long openSeqNum,
- HRegionInfo... hris) {
+ RegionInfo... hris) {
return false;
}
@@ -674,7 +675,7 @@ ClientProtos.ClientService.BlockingInterface, RegionServerServices {
}
@Override
- public EntityLock regionLock(List<HRegionInfo> regionInfos, String description, Abortable abort)
+ public EntityLock regionLock(List<RegionInfo> regionInfos, String description, Abortable abort)
throws IOException {
return null;
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentListener.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentListener.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentListener.java
index eaa2edb..4c4a8ed 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentListener.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentListener.java
@@ -32,13 +32,13 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Abortable;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.ServerLoad;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
import org.apache.hadoop.hbase.regionserver.HRegion;
@@ -97,14 +97,14 @@ public class TestAssignmentListener {
}
@Override
- public void regionOpened(final HRegionInfo regionInfo, final ServerName serverName) {
+ public void regionOpened(final RegionInfo regionInfo, final ServerName serverName) {
LOG.info("Assignment open region=" + regionInfo + " server=" + serverName);
openCount.incrementAndGet();
modified.incrementAndGet();
}
@Override
- public void regionClosed(final HRegionInfo regionInfo) {
+ public void regionClosed(final RegionInfo regionInfo) {
LOG.info("Assignment close region=" + regionInfo);
closeCount.incrementAndGet();
modified.incrementAndGet();
@@ -267,7 +267,7 @@ public class TestAssignmentListener {
// Merge the two regions
LOG.info("Merge Regions");
listener.reset();
- List<HRegionInfo> regions = admin.getTableRegions(tableName);
+ List<RegionInfo> regions = admin.getRegions(tableName);
assertEquals(2, regions.size());
boolean sameServer = areAllRegionsLocatedOnSameServer(tableName);
// If the regions are located by different server, we need to move
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitorInMemoryStates.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitorInMemoryStates.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitorInMemoryStates.java
index 19415bd..28ed6a8 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitorInMemoryStates.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitorInMemoryStates.java
@@ -18,40 +18,45 @@
*/
package org.apache.hadoop.hbase.master;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.*;
-import org.apache.hadoop.hbase.client.*;
+import org.apache.hadoop.hbase.CategoryBasedTimeout;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.MetaMockingUtil;
+import org.apache.hadoop.hbase.MetaTableAccessor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionLocator;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
-import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.testclassification.MasterTests;
import org.apache.hadoop.hbase.testclassification.MediumTests;
-import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.PairOfSameType;
import org.apache.hadoop.hbase.util.Threads;
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Rule;
import org.junit.Test;
import org.junit.experimental.categories.Category;
-
import org.junit.rules.TestName;
import org.junit.rules.TestRule;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.assertNotNull;
-
@Category({MasterTests.class, MediumTests.class})
public class TestCatalogJanitorInMemoryStates {
private static final Log LOG = LogFactory.getLog(TestCatalogJanitorInMemoryStates.class);
@@ -129,7 +134,7 @@ public class TestCatalogJanitorInMemoryStates {
* @return List of region locations
* @throws IOException, InterruptedException
*/
- private List<HRegionLocation> splitRegion(final HRegionInfo r)
+ private List<HRegionLocation> splitRegion(final RegionInfo r)
throws IOException, InterruptedException {
List<HRegionLocation> locations = new ArrayList<>();
// Split this table in two.
@@ -137,7 +142,7 @@ public class TestCatalogJanitorInMemoryStates {
Connection connection = TEST_UTIL.getConnection();
admin.splitRegion(r.getEncodedNameAsBytes());
admin.close();
- PairOfSameType<HRegionInfo> regions = waitOnDaughters(r);
+ PairOfSameType<RegionInfo> regions = waitOnDaughters(r);
if (regions != null) {
try (RegionLocator rl = connection.getRegionLocator(r.getTable())) {
locations.add(rl.getRegionLocation(regions.getFirst().getEncodedNameAsBytes()));
@@ -154,20 +159,20 @@ public class TestCatalogJanitorInMemoryStates {
* @param r
* @return Daughter regions; caller needs to check table actually split.
*/
- private PairOfSameType<HRegionInfo> waitOnDaughters(final HRegionInfo r)
+ private PairOfSameType<RegionInfo> waitOnDaughters(final RegionInfo r)
throws IOException {
long start = System.currentTimeMillis();
- PairOfSameType<HRegionInfo> pair = null;
+ PairOfSameType<RegionInfo> pair = null;
try (Connection conn = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration());
Table metaTable = conn.getTable(TableName.META_TABLE_NAME)) {
Result result = null;
- HRegionInfo region = null;
+ RegionInfo region = null;
while ((System.currentTimeMillis() - start) < 60000) {
result = metaTable.get(new Get(r.getRegionName()));
if (result == null) {
break;
}
- region = MetaTableAccessor.getHRegionInfo(result);
+ region = MetaTableAccessor.getRegionInfo(result);
if (region.isSplitParent()) {
LOG.debug(region.toString() + " IS a parent!");
pair = MetaTableAccessor.getDaughterRegions(result);
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java
index 4763169..8fb76c7 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java
@@ -59,7 +59,6 @@ import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
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.KeyValue;
import org.apache.hadoop.hbase.MiniHBaseCluster;
@@ -77,6 +76,8 @@ import org.apache.hadoop.hbase.client.Increment;
import org.apache.hadoop.hbase.client.NonceGenerator;
import org.apache.hadoop.hbase.client.PerClientRandomNonceGenerator;
import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
import org.apache.hadoop.hbase.client.RegionLocator;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.RetriesExhaustedWithDetailsException;
@@ -85,13 +86,11 @@ import org.apache.hadoop.hbase.coordination.BaseCoordinatedStateManager;
import org.apache.hadoop.hbase.coordination.ZKSplitLogManagerCoordination;
import org.apache.hadoop.hbase.exceptions.RegionInRecoveryException;
import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
-import org.apache.hadoop.hbase.master.assignment.RegionStates;
import org.apache.hadoop.hbase.master.SplitLogManager.TaskBatch;
+import org.apache.hadoop.hbase.master.assignment.RegionStates;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.regionserver.Region;
-import org.apache.hadoop.hbase.wal.WALEdit;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.testclassification.MasterTests;
import org.apache.hadoop.hbase.util.Bytes;
@@ -102,6 +101,7 @@ import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
import org.apache.hadoop.hbase.wal.WAL;
+import org.apache.hadoop.hbase.wal.WALEdit;
import org.apache.hadoop.hbase.wal.WALFactory;
import org.apache.hadoop.hbase.wal.WALKey;
import org.apache.hadoop.hbase.wal.WALSplitter;
@@ -120,6 +120,8 @@ import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.junit.rules.TestName;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+
@Category({MasterTests.class, LargeTests.class})
@SuppressWarnings("deprecation")
public class TestDistributedLogSplitting {
@@ -232,13 +234,13 @@ public class TestDistributedLogSplitting {
"table", "family", 40);
try {
TableName table = t.getName();
- List<HRegionInfo> regions = null;
+ List<RegionInfo> regions = null;
HRegionServer hrs = null;
for (int i = 0; i < NUM_RS; i++) {
boolean foundRs = false;
hrs = rsts.get(i).getRegionServer();
regions = ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices());
- for (HRegionInfo region : regions) {
+ for (RegionInfo region : regions) {
if (region.getTable().getNameAsString().equalsIgnoreCase("table")) {
foundRs = true;
break;
@@ -250,9 +252,9 @@ public class TestDistributedLogSplitting {
.getServerName().toString()));
LOG.info("#regions = " + regions.size());
- Iterator<HRegionInfo> it = regions.iterator();
+ Iterator<RegionInfo> it = regions.iterator();
while (it.hasNext()) {
- HRegionInfo region = it.next();
+ RegionInfo region = it.next();
if (region.getTable().getNamespaceAsString()
.equals(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR)) {
it.remove();
@@ -264,7 +266,7 @@ public class TestDistributedLogSplitting {
slm.splitLogDistributed(logDir);
int count = 0;
- for (HRegionInfo hri : regions) {
+ for (RegionInfo hri : regions) {
Path tdir = FSUtils.getTableDir(rootdir, table);
Path editsdir =
@@ -315,7 +317,7 @@ public class TestDistributedLogSplitting {
Table ht = installTable(zkw, "table", "family", NUM_REGIONS_TO_CREATE);
try {
HRegionServer hrs = findRSToKill(false, "table");
- List<HRegionInfo> regions = ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices());
+ List<RegionInfo> regions = ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices());
makeWAL(hrs, regions, "table", "family", NUM_LOG_LINES, 100);
// wait for abort completes
@@ -374,8 +376,8 @@ public class TestDistributedLogSplitting {
List<Increment> reqs = new ArrayList<>();
for (RegionServerThread rst : cluster.getLiveRegionServerThreads()) {
HRegionServer hrs = rst.getRegionServer();
- List<HRegionInfo> hris = ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices());
- for (HRegionInfo hri : hris) {
+ List<RegionInfo> hris = ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices());
+ for (RegionInfo hri : hris) {
if (TABLE_NAME.equalsIgnoreCase(hri.getTable().getNameAsString())) {
byte[] key = hri.getStartKey();
if (key == null || key.length == 0) {
@@ -424,7 +426,7 @@ public class TestDistributedLogSplitting {
Table ht = installTable(zkw, "table", "family", NUM_REGIONS_TO_CREATE);
try {
HRegionServer hrs = findRSToKill(true, "table");
- List<HRegionInfo> regions = ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices());
+ List<RegionInfo> regions = ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices());
makeWAL(hrs, regions, "table", "family", NUM_LOG_LINES, 100);
this.abortRSAndVerifyRecovery(hrs, ht, zkw, NUM_REGIONS_TO_CREATE, NUM_LOG_LINES);
@@ -494,7 +496,7 @@ public class TestDistributedLogSplitting {
Table ht = installTable(zkw, "table", "family", NUM_REGIONS_TO_CREATE);
try {
HRegionServer hrs = findRSToKill(false, "table");
- List<HRegionInfo> regions = ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices());
+ List<RegionInfo> regions = ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices());
makeWAL(hrs, regions, "table", "family", NUM_LOG_LINES, 100);
// abort master
@@ -552,7 +554,7 @@ public class TestDistributedLogSplitting {
Table ht = installTable(zkw, "table", "family", NUM_REGIONS_TO_CREATE);
try {
HRegionServer hrs = findRSToKill(false, "table");
- List<HRegionInfo> regions = ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices());
+ List<RegionInfo> regions = ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices());
makeWAL(hrs, regions, "table", "family", NUM_LOG_LINES, 100);
// abort master
@@ -613,7 +615,7 @@ public class TestDistributedLogSplitting {
final ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "table-creation", null);
Table ht = installTable(zkw, "table", "family", NUM_REGIONS_TO_CREATE);
try {
- List<HRegionInfo> regions = null;
+ List<RegionInfo> regions = null;
HRegionServer hrs1 = findRSToKill(false, "table");
regions = ProtobufUtil.getOnlineRegions(hrs1.getRSRpcServices());
@@ -694,14 +696,14 @@ public class TestDistributedLogSplitting {
try {
final SplitLogManager slm = master.getMasterWalManager().getSplitLogManager();
- Set<HRegionInfo> regionSet = new HashSet<>();
- HRegionInfo region = null;
+ Set<RegionInfo> regionSet = new HashSet<>();
+ RegionInfo region = null;
HRegionServer hrs = null;
ServerName firstFailedServer = null;
ServerName secondFailedServer = null;
for (int i = 0; i < NUM_RS; i++) {
hrs = rsts.get(i).getRegionServer();
- List<HRegionInfo> regions = ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices());
+ List<RegionInfo> regions = ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices());
if (regions.isEmpty()) continue;
region = regions.get(0);
regionSet.add(region);
@@ -746,13 +748,13 @@ public class TestDistributedLogSplitting {
final ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "table-creation", null);
Table ht = installTable(zkw, "table", "family", NUM_REGIONS_TO_CREATE);
try {
- List<HRegionInfo> regions = null;
+ List<RegionInfo> regions = null;
HRegionServer hrs = null;
for (int i = 0; i < NUM_RS; i++) {
boolean isCarryingMeta = false;
hrs = rsts.get(i).getRegionServer();
regions = ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices());
- for (HRegionInfo region : regions) {
+ for (RegionInfo region : regions) {
if (region.isMetaRegion()) {
isCarryingMeta = true;
break;
@@ -795,7 +797,7 @@ public class TestDistributedLogSplitting {
// they will consume recovered.edits
master.balanceSwitch(false);
- List<HRegionInfo> regions = null;
+ List<RegionInfo> regions = null;
HRegionServer hrs = null;
boolean hasRegionsForBothTables = false;
String tableName = null;
@@ -805,7 +807,7 @@ public class TestDistributedLogSplitting {
boolean isCarryingSystem = false;
hrs = rsts.get(i).getRegionServer();
regions = ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices());
- for (HRegionInfo region : regions) {
+ for (RegionInfo region : regions) {
if (region.getTable().isSystemTable()) {
isCarryingSystem = true;
break;
@@ -831,9 +833,9 @@ public class TestDistributedLogSplitting {
Assert.assertTrue(hasRegionsForBothTables);
LOG.info("#regions = " + regions.size());
- Iterator<HRegionInfo> it = regions.iterator();
+ Iterator<RegionInfo> it = regions.iterator();
while (it.hasNext()) {
- HRegionInfo region = it.next();
+ RegionInfo region = it.next();
if (region.isMetaTable()) {
it.remove();
}
@@ -882,7 +884,7 @@ public class TestDistributedLogSplitting {
FileSystem fs = master.getMasterFileSystem().getFileSystem();
Path rootdir = FSUtils.getRootDir(conf);
Path tdir = FSUtils.getTableDir(rootdir, TableName.valueOf(name.getMethodName()));
- for (HRegionInfo hri : regions) {
+ for (RegionInfo hri : regions) {
Path editsdir =
WALSplitter.getRegionDirRecoveredEditsDir(
HRegion.getRegionDir(tdir, hri.getEncodedName()));
@@ -912,7 +914,7 @@ public class TestDistributedLogSplitting {
assertEquals(NUM_LOG_LINES, TEST_UTIL.countRows(ht));
// clean up
- for (HRegionInfo hri : regions) {
+ for (RegionInfo hri : regions) {
Path editsdir =
WALSplitter.getRegionDirRecoveredEditsDir(
HRegion.getRegionDir(tdir, hri.getEncodedName()));
@@ -943,13 +945,13 @@ public class TestDistributedLogSplitting {
try {
final SplitLogManager slm = master.getMasterWalManager().getSplitLogManager();
- Set<HRegionInfo> regionSet = new HashSet<>();
- HRegionInfo region = null;
+ Set<RegionInfo> regionSet = new HashSet<>();
+ RegionInfo region = null;
HRegionServer hrs = null;
HRegionServer dstRS = null;
for (int i = 0; i < NUM_RS; i++) {
hrs = rsts.get(i).getRegionServer();
- List<HRegionInfo> regions = ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices());
+ List<RegionInfo> regions = ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices());
if (regions.isEmpty()) continue;
region = regions.get(0);
regionSet.add(region);
@@ -959,7 +961,7 @@ public class TestDistributedLogSplitting {
slm.markRegionsRecovering(hrs.getServerName(), regionSet);
// move region in order for the region opened in recovering state
- final HRegionInfo hri = region;
+ final RegionInfo hri = region;
final HRegionServer tmpRS = dstRS;
TEST_UTIL.getAdmin().move(region.getEncodedNameAsBytes(),
Bytes.toBytes(dstRS.getServerName().getServerName()));
@@ -1212,20 +1214,20 @@ public class TestDistributedLogSplitting {
// only testing meta recovery in ZK operation
HRegionServer hrs = findRSToKill(true, null);
- List<HRegionInfo> regions = ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices());
+ List<RegionInfo> regions = ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices());
LOG.info("#regions = " + regions.size());
- Set<HRegionInfo> tmpRegions = new HashSet<>();
- tmpRegions.add(HRegionInfo.FIRST_META_REGIONINFO);
+ Set<RegionInfo> tmpRegions = new HashSet<>();
+ tmpRegions.add(RegionInfoBuilder.FIRST_META_REGIONINFO);
master.getMasterWalManager().prepareLogReplay(hrs.getServerName(), tmpRegions);
- Set<HRegionInfo> userRegionSet = new HashSet<>();
+ Set<RegionInfo> userRegionSet = new HashSet<>();
userRegionSet.addAll(regions);
master.getMasterWalManager().prepareLogReplay(hrs.getServerName(), userRegionSet);
boolean isMetaRegionInRecovery = false;
List<String> recoveringRegions =
zkw.getRecoverableZooKeeper().getChildren(zkw.znodePaths.recoveringRegionsZNode, false);
for (String curEncodedRegionName : recoveringRegions) {
- if (curEncodedRegionName.equals(HRegionInfo.FIRST_META_REGIONINFO.getEncodedName())) {
+ if (curEncodedRegionName.equals(RegionInfoBuilder.FIRST_META_REGIONINFO.getEncodedName())) {
isMetaRegionInRecovery = true;
break;
}
@@ -1238,7 +1240,7 @@ public class TestDistributedLogSplitting {
recoveringRegions =
zkw.getRecoverableZooKeeper().getChildren(zkw.znodePaths.recoveringRegionsZNode, false);
for (String curEncodedRegionName : recoveringRegions) {
- if (curEncodedRegionName.equals(HRegionInfo.FIRST_META_REGIONINFO.getEncodedName())) {
+ if (curEncodedRegionName.equals(RegionInfoBuilder.FIRST_META_REGIONINFO.getEncodedName())) {
isMetaRegionInRecovery = true;
break;
}
@@ -1265,13 +1267,13 @@ public class TestDistributedLogSplitting {
final ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "table-creation", null);
Table ht = installTable(zkw, name.getMethodName(), "family", NUM_REGIONS_TO_CREATE);
try {
- List<HRegionInfo> regions = null;
+ List<RegionInfo> regions = null;
HRegionServer hrs = null;
for (int i = 0; i < NUM_RS; i++) {
boolean isCarryingMeta = false;
hrs = rsts.get(i).getRegionServer();
regions = ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices());
- for (HRegionInfo region : regions) {
+ for (RegionInfo region : regions) {
if (region.isMetaRegion()) {
isCarryingMeta = true;
break;
@@ -1284,17 +1286,17 @@ public class TestDistributedLogSplitting {
}
LOG.info("#regions = " + regions.size());
- Iterator<HRegionInfo> it = regions.iterator();
+ Iterator<RegionInfo> it = regions.iterator();
while (it.hasNext()) {
- HRegionInfo region = it.next();
+ RegionInfo region = it.next();
if (region.isMetaTable()
|| region.getEncodedName().equals(
- HRegionInfo.FIRST_META_REGIONINFO.getEncodedName())) {
+ RegionInfoBuilder.FIRST_META_REGIONINFO.getEncodedName())) {
it.remove();
}
}
if (regions.isEmpty()) return;
- HRegionInfo curRegionInfo = regions.get(0);
+ RegionInfo curRegionInfo = regions.get(0);
byte[] startRow = curRegionInfo.getStartKey();
if (startRow == null || startRow.length == 0) {
startRow = new byte[] { 0, 0, 0, 0, 1 };
@@ -1361,13 +1363,13 @@ public class TestDistributedLogSplitting {
final ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "table-creation", null);
Table ht = installTable(zkw, "table", "family", NUM_REGIONS_TO_CREATE);
try {
- List<HRegionInfo> regions = null;
+ List<RegionInfo> regions = null;
HRegionServer hrs = null;
for (int i = 0; i < NUM_RS; i++) {
boolean isCarryingMeta = false;
hrs = rsts.get(i).getRegionServer();
regions = ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices());
- for (HRegionInfo region : regions) {
+ for (RegionInfo region : regions) {
if (region.isMetaRegion()) {
isCarryingMeta = true;
break;
@@ -1380,16 +1382,16 @@ public class TestDistributedLogSplitting {
}
LOG.info("#regions = " + regions.size());
- Iterator<HRegionInfo> it = regions.iterator();
+ Iterator<RegionInfo> it = regions.iterator();
while (it.hasNext()) {
- HRegionInfo region = it.next();
+ RegionInfo region = it.next();
if (region.isMetaTable()
- || region.getEncodedName().equals(HRegionInfo.FIRST_META_REGIONINFO.getEncodedName())) {
+ || region.getEncodedName().equals(RegionInfoBuilder.FIRST_META_REGIONINFO.getEncodedName())) {
it.remove();
}
}
if (regions.isEmpty()) return;
- HRegionInfo curRegionInfo = regions.get(0);
+ RegionInfo curRegionInfo = regions.get(0);
byte[] startRow = curRegionInfo.getStartKey();
if (startRow == null || startRow.length == 0) {
startRow = new byte[] { 0, 0, 0, 0, 1 };
@@ -1532,8 +1534,8 @@ public class TestDistributedLogSplitting {
for (RegionServerThread rst : rsts) {
HRegionServer hrs = rst.getRegionServer();
- List<HRegionInfo> hris = ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices());
- for (HRegionInfo hri : hris) {
+ List<RegionInfo> hris = ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices());
+ for (RegionInfo hri : hris) {
if (hri.getTable().isSystemTable()) {
continue;
}
@@ -1547,14 +1549,14 @@ public class TestDistributedLogSplitting {
for (MasterThread mt : cluster.getLiveMasterThreads()) {
HRegionServer hrs = mt.getMaster();
- List<HRegionInfo> hris;
+ List<RegionInfo> hris;
try {
hris = ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices());
} catch (ServerNotRunningYetException e) {
// It's ok: this master may be a backup. Ignored.
continue;
}
- for (HRegionInfo hri : hris) {
+ for (RegionInfo hri : hris) {
if (hri.getTable().isSystemTable()) {
continue;
}
@@ -1567,22 +1569,22 @@ public class TestDistributedLogSplitting {
}
}
- public void makeWAL(HRegionServer hrs, List<HRegionInfo> regions, String tname, String fname,
+ public void makeWAL(HRegionServer hrs, List<RegionInfo> regions, String tname, String fname,
int num_edits, int edit_size) throws IOException {
makeWAL(hrs, regions, tname, fname, num_edits, edit_size, true);
}
- public void makeWAL(HRegionServer hrs, List<HRegionInfo> regions, String tname, String fname,
+ public void makeWAL(HRegionServer hrs, List<RegionInfo> regions, String tname, String fname,
int num_edits, int edit_size, boolean cleanShutdown) throws IOException {
TableName fullTName = TableName.valueOf(tname);
// remove root and meta region
- regions.remove(HRegionInfo.FIRST_META_REGIONINFO);
+ regions.remove(RegionInfoBuilder.FIRST_META_REGIONINFO);
// using one sequenceId for edits across all regions is ok.
final AtomicLong sequenceId = new AtomicLong(10);
- for(Iterator<HRegionInfo> iter = regions.iterator(); iter.hasNext(); ) {
- HRegionInfo regionInfo = iter.next();
+ for(Iterator<RegionInfo> iter = regions.iterator(); iter.hasNext(); ) {
+ RegionInfo regionInfo = iter.next();
if(regionInfo.getTable().isSystemTable()) {
iter.remove();
}
@@ -1592,8 +1594,8 @@ public class TestDistributedLogSplitting {
htd.addFamily(new HColumnDescriptor(family));
byte[] value = new byte[edit_size];
- List<HRegionInfo> hris = new ArrayList<>();
- for (HRegionInfo region : regions) {
+ List<RegionInfo> hris = new ArrayList<>();
+ for (RegionInfo region : regions) {
if (!region.getTable().getNameAsString().equalsIgnoreCase(tname)) {
continue;
}
@@ -1610,7 +1612,7 @@ public class TestDistributedLogSplitting {
if (n > 0) {
for (int i = 0; i < num_edits; i += 1) {
WALEdit e = new WALEdit();
- HRegionInfo curRegionInfo = hris.get(i % n);
+ RegionInfo curRegionInfo = hris.get(i % n);
final WAL log = hrs.getWAL(curRegionInfo);
byte[] startRow = curRegionInfo.getStartKey();
if (startRow == null || startRow.length == 0) {
@@ -1633,12 +1635,12 @@ public class TestDistributedLogSplitting {
}
// done as two passes because the regions might share logs. shutdown is idempotent, but sync
// will cause errors if done after.
- for (HRegionInfo info : hris) {
+ for (RegionInfo info : hris) {
final WAL log = hrs.getWAL(info);
log.sync();
}
if (cleanShutdown) {
- for (HRegionInfo info : hris) {
+ for (RegionInfo info : hris) {
final WAL log = hrs.getWAL(info);
log.shutdown();
}
@@ -1754,7 +1756,7 @@ public class TestDistributedLogSplitting {
*/
private HRegionServer findRSToKill(boolean hasMetaRegion, String tableName) throws Exception {
List<RegionServerThread> rsts = cluster.getLiveRegionServerThreads();
- List<HRegionInfo> regions = null;
+ List<RegionInfo> regions = null;
HRegionServer hrs = null;
for (RegionServerThread rst: rsts) {
@@ -1768,7 +1770,7 @@ public class TestDistributedLogSplitting {
boolean isCarryingMeta = false;
boolean foundTableRegion = false;
regions = ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices());
- for (HRegionInfo region : regions) {
+ for (RegionInfo region : regions) {
if (region.isMetaRegion()) {
isCarryingMeta = true;
}
@@ -1784,9 +1786,9 @@ public class TestDistributedLogSplitting {
if (!foundTableRegion) {
final HRegionServer destRS = hrs;
// the RS doesn't have regions of the specified table so we need move one to this RS
- List<HRegionInfo> tableRegions =
- TEST_UTIL.getAdmin().getTableRegions(TableName.valueOf(tableName));
- final HRegionInfo hri = tableRegions.get(0);
+ List<RegionInfo> tableRegions =
+ TEST_UTIL.getAdmin().getRegions(TableName.valueOf(tableName));
+ final RegionInfo hri = tableRegions.get(0);
TEST_UTIL.getAdmin().move(hri.getEncodedNameAsBytes(),
Bytes.toBytes(destRS.getServerName().getServerName()));
// wait for region move completes
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMaster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMaster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMaster.java
index 1c31acd..c4163d5 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMaster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMaster.java
@@ -31,7 +31,6 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HBaseTestingUtility;
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.MetaTableAccessor;
import org.apache.hadoop.hbase.MiniHBaseCluster;
@@ -40,6 +39,8 @@ import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.UnknownRegionException;
import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.master.assignment.RegionStates;
@@ -53,9 +54,9 @@ import org.junit.BeforeClass;
import org.junit.Rule;
import org.junit.Test;
import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
import org.apache.hadoop.hbase.shaded.com.google.common.base.Joiner;
-import org.junit.rules.TestName;
@Category({MasterTests.class, MediumTests.class})
public class TestMaster {
@@ -94,7 +95,7 @@ public class TestMaster {
TEST_UTIL.loadTable(ht, FAMILYNAME, false);
}
- List<Pair<HRegionInfo, ServerName>> tableRegions = MetaTableAccessor.getTableRegionsAndLocations(
+ List<Pair<RegionInfo, ServerName>> tableRegions = MetaTableAccessor.getTableRegionsAndLocations(
m.getConnection(), TABLENAME);
LOG.info("Regions after load: " + Joiner.on(',').join(tableRegions));
assertEquals(1, tableRegions.size());
@@ -119,13 +120,13 @@ public class TestMaster {
// We have three regions because one is split-in-progress
assertEquals(3, tableRegions.size());
LOG.info("Making sure we can call getTableRegionClosest while opening");
- Pair<HRegionInfo, ServerName> pair =
+ Pair<RegionInfo, ServerName> pair =
m.getTableRegionForRow(TABLENAME, Bytes.toBytes("cde"));
LOG.info("Result is: " + pair);
- Pair<HRegionInfo, ServerName> tableRegionFromName =
+ Pair<RegionInfo, ServerName> tableRegionFromName =
MetaTableAccessor.getRegion(m.getConnection(),
pair.getFirst().getRegionName());
- assertEquals(tableRegionFromName.getFirst(), pair.getFirst());
+ assertTrue(RegionInfo.COMPARATOR.compare(tableRegionFromName.getFirst(), pair.getFirst()) == 0);
}
@Test
@@ -134,7 +135,7 @@ public class TestMaster {
HMaster m = cluster.getMaster();
try {
m.setInitialized(false); // fake it, set back later
- HRegionInfo meta = HRegionInfo.FIRST_META_REGIONINFO;
+ RegionInfo meta = RegionInfoBuilder.FIRST_META_REGIONINFO;
m.move(meta.getEncodedNameAsBytes(), null);
fail("Region should not be moved since master is not initialized");
} catch (IOException ioe) {
@@ -153,8 +154,10 @@ public class TestMaster {
admin.createTable(htd, null);
try {
- HRegionInfo hri = new HRegionInfo(
- tableName, Bytes.toBytes("A"), Bytes.toBytes("Z"));
+ RegionInfo hri = RegionInfoBuilder.newBuilder(tableName)
+ .setStartKey(Bytes.toBytes("A"))
+ .setEndKey(Bytes.toBytes("Z"))
+ .build();
admin.move(hri.getEncodedNameAsBytes(), null);
fail("Region should not be moved since it is fake");
} catch (IOException ioe) {
@@ -174,7 +177,7 @@ public class TestMaster {
admin.createTable(htd, null);
try {
- List<HRegionInfo> tableRegions = admin.getTableRegions(tableName);
+ List<RegionInfo> tableRegions = admin.getRegions(tableName);
master.setInitialized(false); // fake it, set back later
admin.move(tableRegions.get(0).getEncodedNameAsBytes(), null);
[16/19] hbase git commit: HBASE-18839 Apply RegionInfo to code base
Posted by ch...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupBasedLoadBalancer.java
----------------------------------------------------------------------
diff --git a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupBasedLoadBalancer.java b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupBasedLoadBalancer.java
index 169e42f..3f1373f 100644
--- a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupBasedLoadBalancer.java
+++ b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupBasedLoadBalancer.java
@@ -18,13 +18,6 @@
package org.apache.hadoop.hbase.rsgroup;
-import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.ArrayListMultimap;
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.LinkedListMultimap;
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.ListMultimap;
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.Maps;
-
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
@@ -43,10 +36,9 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.ClusterStatus;
import org.apache.hadoop.hbase.HBaseIOException;
import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.constraint.ConstraintException;
import org.apache.hadoop.hbase.master.LoadBalancer;
import org.apache.hadoop.hbase.master.MasterServices;
@@ -54,6 +46,14 @@ import org.apache.hadoop.hbase.master.RegionPlan;
import org.apache.hadoop.hbase.master.balancer.StochasticLoadBalancer;
import org.apache.hadoop.hbase.net.Address;
import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.ArrayListMultimap;
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.LinkedListMultimap;
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.ListMultimap;
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.Maps;
/**
* GroupBasedLoadBalancer, used when Region Server Grouping is configured (HBase-6721)
@@ -106,31 +106,31 @@ public class RSGroupBasedLoadBalancer implements RSGroupableBalancer {
}
@Override
- public List<RegionPlan> balanceCluster(TableName tableName, Map<ServerName, List<HRegionInfo>>
+ public List<RegionPlan> balanceCluster(TableName tableName, Map<ServerName, List<RegionInfo>>
clusterState) throws HBaseIOException {
return balanceCluster(clusterState);
}
@Override
- public List<RegionPlan> balanceCluster(Map<ServerName, List<HRegionInfo>> clusterState)
+ public List<RegionPlan> balanceCluster(Map<ServerName, List<RegionInfo>> clusterState)
throws HBaseIOException {
if (!isOnline()) {
throw new ConstraintException(RSGroupInfoManager.RSGROUP_TABLE_NAME +
" is not online, unable to perform balance");
}
- Map<ServerName,List<HRegionInfo>> correctedState = correctAssignments(clusterState);
+ Map<ServerName,List<RegionInfo>> correctedState = correctAssignments(clusterState);
List<RegionPlan> regionPlans = new ArrayList<>();
- List<HRegionInfo> misplacedRegions = correctedState.get(LoadBalancer.BOGUS_SERVER_NAME);
- for (HRegionInfo regionInfo : misplacedRegions) {
+ List<RegionInfo> misplacedRegions = correctedState.get(LoadBalancer.BOGUS_SERVER_NAME);
+ for (RegionInfo regionInfo : misplacedRegions) {
regionPlans.add(new RegionPlan(regionInfo, null, null));
}
try {
List<RSGroupInfo> rsgi = rsGroupInfoManager.listRSGroups();
for (RSGroupInfo info: rsgi) {
- Map<ServerName, List<HRegionInfo>> groupClusterState = new HashMap<>();
- Map<TableName, Map<ServerName, List<HRegionInfo>>> groupClusterLoad = new HashMap<>();
+ Map<ServerName, List<RegionInfo>> groupClusterState = new HashMap<>();
+ Map<TableName, Map<ServerName, List<RegionInfo>>> groupClusterLoad = new HashMap<>();
for (Address sName : info.getServers()) {
for(ServerName curr: clusterState.keySet()) {
if(curr.getAddress().equals(sName)) {
@@ -154,15 +154,15 @@ public class RSGroupBasedLoadBalancer implements RSGroupableBalancer {
}
@Override
- public Map<ServerName, List<HRegionInfo>> roundRobinAssignment(
- List<HRegionInfo> regions, List<ServerName> servers) throws HBaseIOException {
- Map<ServerName, List<HRegionInfo>> assignments = Maps.newHashMap();
- ListMultimap<String,HRegionInfo> regionMap = ArrayListMultimap.create();
+ public Map<ServerName, List<RegionInfo>> roundRobinAssignment(
+ List<RegionInfo> regions, List<ServerName> servers) throws HBaseIOException {
+ Map<ServerName, List<RegionInfo>> assignments = Maps.newHashMap();
+ ListMultimap<String,RegionInfo> regionMap = ArrayListMultimap.create();
ListMultimap<String,ServerName> serverMap = ArrayListMultimap.create();
generateGroupMaps(regions, servers, regionMap, serverMap);
for(String groupKey : regionMap.keySet()) {
if (regionMap.get(groupKey).size() > 0) {
- Map<ServerName, List<HRegionInfo>> result =
+ Map<ServerName, List<RegionInfo>> result =
this.internalBalancer.roundRobinAssignment(
regionMap.get(groupKey),
serverMap.get(groupKey));
@@ -181,13 +181,13 @@ public class RSGroupBasedLoadBalancer implements RSGroupableBalancer {
}
@Override
- public Map<ServerName, List<HRegionInfo>> retainAssignment(
- Map<HRegionInfo, ServerName> regions, List<ServerName> servers) throws HBaseIOException {
+ public Map<ServerName, List<RegionInfo>> retainAssignment(
+ Map<RegionInfo, ServerName> regions, List<ServerName> servers) throws HBaseIOException {
try {
- Map<ServerName, List<HRegionInfo>> assignments = new TreeMap<>();
- ListMultimap<String, HRegionInfo> groupToRegion = ArrayListMultimap.create();
- Set<HRegionInfo> misplacedRegions = getMisplacedRegions(regions);
- for (HRegionInfo region : regions.keySet()) {
+ Map<ServerName, List<RegionInfo>> assignments = new TreeMap<>();
+ ListMultimap<String, RegionInfo> groupToRegion = ArrayListMultimap.create();
+ Set<RegionInfo> misplacedRegions = getMisplacedRegions(regions);
+ for (RegionInfo region : regions.keySet()) {
if (!misplacedRegions.contains(region)) {
String groupName = rsGroupInfoManager.getRSGroupOfTable(region.getTable());
groupToRegion.put(groupName, region);
@@ -196,11 +196,11 @@ public class RSGroupBasedLoadBalancer implements RSGroupableBalancer {
// Now the "groupToRegion" map has only the regions which have correct
// assignments.
for (String key : groupToRegion.keySet()) {
- Map<HRegionInfo, ServerName> currentAssignmentMap = new TreeMap<HRegionInfo, ServerName>();
- List<HRegionInfo> regionList = groupToRegion.get(key);
+ Map<RegionInfo, ServerName> currentAssignmentMap = new TreeMap<RegionInfo, ServerName>();
+ List<RegionInfo> regionList = groupToRegion.get(key);
RSGroupInfo info = rsGroupInfoManager.getRSGroup(key);
List<ServerName> candidateList = filterOfflineServers(info, servers);
- for (HRegionInfo region : regionList) {
+ for (RegionInfo region : regionList) {
currentAssignmentMap.put(region, regions.get(region));
}
if(candidateList.size() > 0) {
@@ -209,7 +209,7 @@ public class RSGroupBasedLoadBalancer implements RSGroupableBalancer {
}
}
- for (HRegionInfo region : misplacedRegions) {
+ for (RegionInfo region : misplacedRegions) {
String groupName = rsGroupInfoManager.getRSGroupOfTable(region.getTable());;
RSGroupInfo info = rsGroupInfoManager.getRSGroup(groupName);
List<ServerName> candidateList = filterOfflineServers(info, servers);
@@ -235,9 +235,9 @@ public class RSGroupBasedLoadBalancer implements RSGroupableBalancer {
}
@Override
- public ServerName randomAssignment(HRegionInfo region,
+ public ServerName randomAssignment(RegionInfo region,
List<ServerName> servers) throws HBaseIOException {
- ListMultimap<String,HRegionInfo> regionMap = LinkedListMultimap.create();
+ ListMultimap<String,RegionInfo> regionMap = LinkedListMultimap.create();
ListMultimap<String,ServerName> serverMap = LinkedListMultimap.create();
generateGroupMaps(Lists.newArrayList(region), servers, regionMap, serverMap);
List<ServerName> filteredServers = serverMap.get(regionMap.keySet().iterator().next());
@@ -245,12 +245,12 @@ public class RSGroupBasedLoadBalancer implements RSGroupableBalancer {
}
private void generateGroupMaps(
- List<HRegionInfo> regions,
+ List<RegionInfo> regions,
List<ServerName> servers,
- ListMultimap<String, HRegionInfo> regionMap,
+ ListMultimap<String, RegionInfo> regionMap,
ListMultimap<String, ServerName> serverMap) throws HBaseIOException {
try {
- for (HRegionInfo region : regions) {
+ for (RegionInfo region : regions) {
String groupName = rsGroupInfoManager.getRSGroupOfTable(region.getTable());
if (groupName == null) {
LOG.warn("Group for table "+region.getTable()+" is null");
@@ -301,11 +301,11 @@ public class RSGroupBasedLoadBalancer implements RSGroupableBalancer {
return finalList;
}
- private Set<HRegionInfo> getMisplacedRegions(
- Map<HRegionInfo, ServerName> regions) throws IOException {
- Set<HRegionInfo> misplacedRegions = new HashSet<>();
- for(Map.Entry<HRegionInfo, ServerName> region : regions.entrySet()) {
- HRegionInfo regionInfo = region.getKey();
+ private Set<RegionInfo> getMisplacedRegions(
+ Map<RegionInfo, ServerName> regions) throws IOException {
+ Set<RegionInfo> misplacedRegions = new HashSet<>();
+ for(Map.Entry<RegionInfo, ServerName> region : regions.entrySet()) {
+ RegionInfo regionInfo = region.getKey();
ServerName assignedServer = region.getValue();
RSGroupInfo info = rsGroupInfoManager.getRSGroup(rsGroupInfoManager.
getRSGroupOfTable(regionInfo.getTable()));
@@ -323,17 +323,17 @@ public class RSGroupBasedLoadBalancer implements RSGroupableBalancer {
return misplacedRegions;
}
- private Map<ServerName, List<HRegionInfo>> correctAssignments(
- Map<ServerName, List<HRegionInfo>> existingAssignments)
+ private Map<ServerName, List<RegionInfo>> correctAssignments(
+ Map<ServerName, List<RegionInfo>> existingAssignments)
throws HBaseIOException{
- Map<ServerName, List<HRegionInfo>> correctAssignments = new TreeMap<>();
- List<HRegionInfo> misplacedRegions = new LinkedList<>();
+ Map<ServerName, List<RegionInfo>> correctAssignments = new TreeMap<>();
+ List<RegionInfo> misplacedRegions = new LinkedList<>();
correctAssignments.put(LoadBalancer.BOGUS_SERVER_NAME, new LinkedList<>());
- for (Map.Entry<ServerName, List<HRegionInfo>> assignments : existingAssignments.entrySet()){
+ for (Map.Entry<ServerName, List<RegionInfo>> assignments : existingAssignments.entrySet()){
ServerName sName = assignments.getKey();
correctAssignments.put(sName, new LinkedList<>());
- List<HRegionInfo> regions = assignments.getValue();
- for (HRegionInfo region : regions) {
+ List<RegionInfo> regions = assignments.getValue();
+ for (RegionInfo region : regions) {
RSGroupInfo info = null;
try {
info = rsGroupInfoManager.getRSGroup(
@@ -352,7 +352,7 @@ public class RSGroupBasedLoadBalancer implements RSGroupableBalancer {
//TODO bulk unassign?
//unassign misplaced regions, so that they are assigned to correct groups.
- for(HRegionInfo info: misplacedRegions) {
+ for(RegionInfo info: misplacedRegions) {
try {
this.masterServices.getAssignmentManager().unassign(info);
} catch (IOException e) {
@@ -395,15 +395,15 @@ public class RSGroupBasedLoadBalancer implements RSGroupableBalancer {
}
@Override
- public void setClusterLoad(Map<TableName, Map<ServerName, List<HRegionInfo>>> clusterLoad) {
+ public void setClusterLoad(Map<TableName, Map<ServerName, List<RegionInfo>>> clusterLoad) {
}
@Override
- public void regionOnline(HRegionInfo regionInfo, ServerName sn) {
+ public void regionOnline(RegionInfo regionInfo, ServerName sn) {
}
@Override
- public void regionOffline(HRegionInfo regionInfo) {
+ public void regionOffline(RegionInfo regionInfo) {
}
@Override
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManagerImpl.java
----------------------------------------------------------------------
diff --git a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManagerImpl.java b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManagerImpl.java
index 4a3747d..e116f58 100644
--- a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManagerImpl.java
+++ b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManagerImpl.java
@@ -42,19 +42,18 @@ import org.apache.hadoop.hbase.Coprocessor;
import org.apache.hadoop.hbase.DoNotRetryIOException;
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.MetaTableAccessor;
import org.apache.hadoop.hbase.MetaTableAccessor.DefaultVisitorBase;
import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
-import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.ClusterConnection;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Table;
@@ -74,16 +73,18 @@ import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos;
import org.apache.hadoop.hbase.protobuf.generated.RSGroupProtos;
import org.apache.hadoop.hbase.regionserver.DisabledRegionSplitPolicy;
import org.apache.hadoop.hbase.security.access.AccessControlLists;
-import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.yetus.audience.InterfaceAudience;
import org.apache.zookeeper.KeeperException;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Maps;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Sets;
+import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
+
import com.google.protobuf.ServiceException;
/**
@@ -639,8 +640,8 @@ class RSGroupInfoManagerImpl implements RSGroupInfoManager {
}
private boolean waitForGroupTableOnline() {
- final List<HRegionInfo> foundRegions = new LinkedList<>();
- final List<HRegionInfo> assignedRegions = new LinkedList<>();
+ final List<RegionInfo> foundRegions = new LinkedList<>();
+ final List<RegionInfo> assignedRegions = new LinkedList<>();
final AtomicBoolean found = new AtomicBoolean(false);
final TableStateManager tsm = masterServices.getTableStateManager();
boolean createSent = false;
@@ -659,7 +660,7 @@ class RSGroupInfoManagerImpl implements RSGroupInfoManager {
MetaTableAccessor.Visitor visitor = new DefaultVisitorBase() {
@Override
public boolean visitInternal(Result row) throws IOException {
- HRegionInfo info = MetaTableAccessor.getHRegionInfo(row);
+ RegionInfo info = MetaTableAccessor.getRegionInfo(row);
if (info != null) {
Cell serverCell =
row.getColumnLatestCell(HConstants.CATALOG_FAMILY,
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRSGroupBasedLoadBalancer.java
----------------------------------------------------------------------
diff --git a/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRSGroupBasedLoadBalancer.java b/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRSGroupBasedLoadBalancer.java
index fe62d16..db7cf4d 100644
--- a/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRSGroupBasedLoadBalancer.java
+++ b/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRSGroupBasedLoadBalancer.java
@@ -17,27 +17,44 @@
*/
package org.apache.hadoop.hbase.master.balancer;
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.ArrayListMultimap;
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.security.SecureRandom;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.TreeSet;
+
import org.apache.commons.lang3.StringUtils;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableDescriptors;
import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.rsgroup.RSGroupBasedLoadBalancer;
-import org.apache.hadoop.hbase.rsgroup.RSGroupInfo;
-import org.apache.hadoop.hbase.rsgroup.RSGroupInfoManager;
-import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.master.LoadBalancer;
import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.master.RegionPlan;
+import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
import org.apache.hadoop.hbase.net.Address;
+import org.apache.hadoop.hbase.rsgroup.RSGroupBasedLoadBalancer;
+import org.apache.hadoop.hbase.rsgroup.RSGroupInfo;
+import org.apache.hadoop.hbase.rsgroup.RSGroupInfoManager;
import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.BeforeClass;
@@ -47,23 +64,8 @@ import org.mockito.Mockito;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.security.SecureRandom;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.TreeMap;
-import java.util.TreeSet;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.ArrayListMultimap;
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
//TODO use stochastic based load balancer instead
@Category(SmallTests.class)
@@ -114,7 +116,7 @@ public class TestRSGroupBasedLoadBalancer {
*/
@Test
public void testBalanceCluster() throws Exception {
- Map<ServerName, List<HRegionInfo>> servers = mockClusterServers();
+ Map<ServerName, List<RegionInfo>> servers = mockClusterServers();
ArrayListMultimap<String, ServerAndLoad> list = convertToGroupBasedMap(servers);
LOG.info("Mock Cluster : " + printStats(list));
List<RegionPlan> plans = loadBalancer.balanceCluster(servers);
@@ -169,11 +171,11 @@ public class TestRSGroupBasedLoadBalancer {
* @throws java.io.IOException
* @throws java.io.FileNotFoundException
*/
- private void assertImmediateAssignment(List<HRegionInfo> regions,
+ private void assertImmediateAssignment(List<RegionInfo> regions,
List<ServerName> servers,
- Map<HRegionInfo, ServerName> assignments)
+ Map<RegionInfo, ServerName> assignments)
throws IOException {
- for (HRegionInfo region : regions) {
+ for (RegionInfo region : regions) {
assertTrue(assignments.containsKey(region));
ServerName server = assignments.get(region);
TableName tableName = region.getTable();
@@ -197,8 +199,8 @@ public class TestRSGroupBasedLoadBalancer {
*/
@Test
public void testBulkAssignment() throws Exception {
- List<HRegionInfo> regions = randomRegions(25);
- Map<ServerName, List<HRegionInfo>> assignments = loadBalancer
+ List<RegionInfo> regions = randomRegions(25);
+ Map<ServerName, List<RegionInfo>> assignments = loadBalancer
.roundRobinAssignment(regions, servers);
//test empty region/servers scenario
//this should not throw an NPE
@@ -207,8 +209,8 @@ public class TestRSGroupBasedLoadBalancer {
//test regular scenario
assertTrue(assignments.keySet().size() == servers.size());
for (ServerName sn : assignments.keySet()) {
- List<HRegionInfo> regionAssigned = assignments.get(sn);
- for (HRegionInfo region : regionAssigned) {
+ List<RegionInfo> regionAssigned = assignments.get(sn);
+ for (RegionInfo region : regionAssigned) {
TableName tableName = region.getTable();
String groupName =
getMockedGroupInfoManager().getRSGroupOfTable(tableName);
@@ -233,16 +235,16 @@ public class TestRSGroupBasedLoadBalancer {
@Test
public void testRetainAssignment() throws Exception {
// Test simple case where all same servers are there
- Map<ServerName, List<HRegionInfo>> currentAssignments = mockClusterServers();
- Map<HRegionInfo, ServerName> inputForTest = new HashMap<>();
+ Map<ServerName, List<RegionInfo>> currentAssignments = mockClusterServers();
+ Map<RegionInfo, ServerName> inputForTest = new HashMap<>();
for (ServerName sn : currentAssignments.keySet()) {
- for (HRegionInfo region : currentAssignments.get(sn)) {
+ for (RegionInfo region : currentAssignments.get(sn)) {
inputForTest.put(region, sn);
}
}
//verify region->null server assignment is handled
inputForTest.put(randomRegions(1).get(0), null);
- Map<ServerName, List<HRegionInfo>> newAssignment = loadBalancer
+ Map<ServerName, List<RegionInfo>> newAssignment = loadBalancer
.retainAssignment(inputForTest, servers);
assertRetainedAssignment(inputForTest, servers, newAssignment);
}
@@ -255,9 +257,9 @@ public class TestRSGroupBasedLoadBalancer {
public void testRoundRobinAssignment() throws Exception {
List<ServerName> onlineServers = new ArrayList<ServerName>(servers.size());
onlineServers.addAll(servers);
- List<HRegionInfo> regions = randomRegions(25);
+ List<RegionInfo> regions = randomRegions(25);
int bogusRegion = 0;
- for(HRegionInfo region : regions){
+ for(RegionInfo region : regions){
String group = tableMap.get(region.getTable());
if("dg3".equals(group) || "dg4".equals(group)){
bogusRegion++;
@@ -273,7 +275,7 @@ public class TestRSGroupBasedLoadBalancer {
it.remove();
}
}
- Map<ServerName, List<HRegionInfo>> assignments = loadBalancer
+ Map<ServerName, List<RegionInfo>> assignments = loadBalancer
.roundRobinAssignment(regions, onlineServers);
assertEquals(bogusRegion, assignments.get(LoadBalancer.BOGUS_SERVER_NAME).size());
}
@@ -294,17 +296,17 @@ public class TestRSGroupBasedLoadBalancer {
* @throws java.io.FileNotFoundException
*/
private void assertRetainedAssignment(
- Map<HRegionInfo, ServerName> existing, List<ServerName> servers,
- Map<ServerName, List<HRegionInfo>> assignment)
+ Map<RegionInfo, ServerName> existing, List<ServerName> servers,
+ Map<ServerName, List<RegionInfo>> assignment)
throws FileNotFoundException, IOException {
// Verify condition 1, every region assigned, and to online server
Set<ServerName> onlineServerSet = new TreeSet<>(servers);
- Set<HRegionInfo> assignedRegions = new TreeSet<>();
- for (Map.Entry<ServerName, List<HRegionInfo>> a : assignment.entrySet()) {
+ Set<RegionInfo> assignedRegions = new TreeSet<>(RegionInfo.COMPARATOR);
+ for (Map.Entry<ServerName, List<RegionInfo>> a : assignment.entrySet()) {
assertTrue(
"Region assigned to server that was not listed as online",
onlineServerSet.contains(a.getKey()));
- for (HRegionInfo r : a.getValue())
+ for (RegionInfo r : a.getValue())
assignedRegions.add(r);
}
assertEquals(existing.size(), assignedRegions.size());
@@ -315,9 +317,9 @@ public class TestRSGroupBasedLoadBalancer {
onlineHostNames.add(s.getHostname());
}
- for (Map.Entry<ServerName, List<HRegionInfo>> a : assignment.entrySet()) {
+ for (Map.Entry<ServerName, List<RegionInfo>> a : assignment.entrySet()) {
ServerName currentServer = a.getKey();
- for (HRegionInfo r : a.getValue()) {
+ for (RegionInfo r : a.getValue()) {
ServerName oldAssignedServer = existing.get(r);
TableName tableName = r.getTable();
String groupName =
@@ -374,7 +376,7 @@ public class TestRSGroupBasedLoadBalancer {
}
private ArrayListMultimap<String, ServerAndLoad> convertToGroupBasedMap(
- final Map<ServerName, List<HRegionInfo>> serversMap) throws IOException {
+ final Map<ServerName, List<RegionInfo>> serversMap) throws IOException {
ArrayListMultimap<String, ServerAndLoad> loadMap = ArrayListMultimap
.create();
for (RSGroupInfo gInfo : getMockedGroupInfoManager().listRSGroups()) {
@@ -387,7 +389,7 @@ public class TestRSGroupBasedLoadBalancer {
break;
}
}
- List<HRegionInfo> regions = serversMap.get(actual);
+ List<RegionInfo> regions = serversMap.get(actual);
assertTrue("No load for " + actual, regions != null);
loadMap.put(gInfo.getName(),
new ServerAndLoad(actual, regions.size()));
@@ -434,12 +436,12 @@ public class TestRSGroupBasedLoadBalancer {
}
}
- private Map<ServerName, List<HRegionInfo>> mockClusterServers() throws IOException {
+ private Map<ServerName, List<RegionInfo>> mockClusterServers() throws IOException {
assertTrue(servers.size() == regionAssignment.length);
- Map<ServerName, List<HRegionInfo>> assignment = new TreeMap<>();
+ Map<ServerName, List<RegionInfo>> assignment = new TreeMap<>();
for (int i = 0; i < servers.size(); i++) {
int numRegions = regionAssignment[i];
- List<HRegionInfo> regions = assignedRegions(numRegions, servers.get(i));
+ List<RegionInfo> regions = assignedRegions(numRegions, servers.get(i));
assignment.put(servers.get(i), regions);
}
return assignment;
@@ -449,10 +451,10 @@ public class TestRSGroupBasedLoadBalancer {
* Generate a list of regions evenly distributed between the tables.
*
* @param numRegions The number of regions to be generated.
- * @return List of HRegionInfo.
+ * @return List of RegionInfo.
*/
- private List<HRegionInfo> randomRegions(int numRegions) {
- List<HRegionInfo> regions = new ArrayList<>(numRegions);
+ private List<RegionInfo> randomRegions(int numRegions) {
+ List<RegionInfo> regions = new ArrayList<>(numRegions);
byte[] start = new byte[16];
byte[] end = new byte[16];
rand.nextBytes(start);
@@ -462,9 +464,12 @@ public class TestRSGroupBasedLoadBalancer {
Bytes.putInt(start, 0, numRegions << 1);
Bytes.putInt(end, 0, (numRegions << 1) + 1);
int tableIndex = (i + regionIdx) % tables.length;
- HRegionInfo hri = new HRegionInfo(
- tables[tableIndex], start, end, false, regionId++);
- regions.add(hri);
+ regions.add(RegionInfoBuilder.newBuilder(tables[tableIndex])
+ .setStartKey(start)
+ .setEndKey(end)
+ .setSplit(false)
+ .setRegionId(regionId++)
+ .build());
}
return regions;
}
@@ -477,18 +482,20 @@ public class TestRSGroupBasedLoadBalancer {
* @return the list of regions
* @throws java.io.IOException Signals that an I/O exception has occurred.
*/
- private List<HRegionInfo> assignedRegions(int numRegions, ServerName sn) throws IOException {
- List<HRegionInfo> regions = new ArrayList<>(numRegions);
+ private List<RegionInfo> assignedRegions(int numRegions, ServerName sn) throws IOException {
+ List<RegionInfo> regions = new ArrayList<>(numRegions);
byte[] start = new byte[16];
byte[] end = new byte[16];
Bytes.putInt(start, 0, numRegions << 1);
Bytes.putInt(end, 0, (numRegions << 1) + 1);
for (int i = 0; i < numRegions; i++) {
TableName tableName = getTableName(sn);
- HRegionInfo hri = new HRegionInfo(
- tableName, start, end, false,
- regionId++);
- regions.add(hri);
+ regions.add(RegionInfoBuilder.newBuilder(tableName)
+ .setStartKey(start)
+ .setEndKey(end)
+ .setSplit(false)
+ .setRegionId(regionId++)
+ .build());
}
return regions;
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsBase.java
----------------------------------------------------------------------
diff --git a/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsBase.java b/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsBase.java
index 6a5d68b..f0291fa 100644
--- a/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsBase.java
+++ b/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsBase.java
@@ -36,32 +36,32 @@ import java.util.TreeMap;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.ClusterStatus;
+import org.apache.hadoop.hbase.ClusterStatus.Option;
import org.apache.hadoop.hbase.HBaseCluster;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.RegionLoad;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.Waiter;
-import org.apache.hadoop.hbase.ClusterStatus.Option;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.ClusterConnection;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.constraint.ConstraintException;
import org.apache.hadoop.hbase.net.Address;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Rule;
import org.junit.Test;
+import org.junit.rules.TestName;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Maps;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Sets;
-import org.junit.rules.TestName;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
public abstract class TestRSGroupsBase {
protected static final Log LOG = LogFactory.getLog(TestRSGroupsBase.class);
@@ -166,7 +166,7 @@ public abstract class TestRSGroupsBase {
for(RegionLoad rl : status.getLoad(serverName).getRegionsLoad().values()) {
TableName tableName = null;
try {
- tableName = HRegionInfo.getTable(rl.getName());
+ tableName = RegionInfo.getTable(rl.getName());
} catch (IllegalArgumentException e) {
LOG.warn("Failed parse a table name from regionname=" +
Bytes.toStringBinary(rl.getName()));
@@ -417,7 +417,7 @@ public abstract class TestRSGroupsBase {
Map<ServerName,List<String>> assignMap =
getTableServerRegionMap().get(tableName);
final ServerName first = assignMap.entrySet().iterator().next().getKey();
- for(HRegionInfo region: admin.getTableRegions(tableName)) {
+ for(RegionInfo region: admin.getTableRegions(tableName)) {
if(!assignMap.get(first).contains(region)) {
admin.move(region.getEncodedNameAsBytes(), Bytes.toBytes(first.getServerName()));
}
@@ -514,7 +514,7 @@ public abstract class TestRSGroupsBase {
});
// Lets move this region to the new group.
- TEST_UTIL.getAdmin().move(Bytes.toBytes(HRegionInfo.encodeRegionName(Bytes.toBytes(targetRegion))),
+ TEST_UTIL.getAdmin().move(Bytes.toBytes(RegionInfo.encodeRegionName(Bytes.toBytes(targetRegion))),
Bytes.toBytes(targetServer.getServerName()));
TEST_UTIL.waitFor(WAIT_TIMEOUT, new Waiter.Predicate<Exception>() {
@Override
@@ -587,7 +587,7 @@ public abstract class TestRSGroupsBase {
appInfo.getServers().iterator().next().toString());
AdminProtos.AdminService.BlockingInterface targetRS =
((ClusterConnection) admin.getConnection()).getAdmin(targetServer);
- HRegionInfo targetRegion = ProtobufUtil.getOnlineRegions(targetRS).get(0);
+ RegionInfo targetRegion = ProtobufUtil.getOnlineRegions(targetRS).get(0);
Assert.assertEquals(1, ProtobufUtil.getOnlineRegions(targetRS).size());
try {
@@ -728,7 +728,7 @@ public abstract class TestRSGroupsBase {
//get server which is not a member of new group
ServerName targetServer = null;
for(ServerName server : admin.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS)).getServers()) {
- if(!newGroup.containsServer(server.getAddress()) &&
+ if(!newGroup.containsServer(server.getAddress()) &&
!rsGroupAdmin.getRSGroupInfo("master").containsServer(server.getAddress())) {
targetServer = server;
break;
@@ -780,7 +780,7 @@ public abstract class TestRSGroupsBase {
List<String> regionList = getTableRegionMap().get(tableName);
for(String region : regionList) {
// Lets move this region to the targetServer
- TEST_UTIL.getAdmin().move(Bytes.toBytes(HRegionInfo.encodeRegionName(Bytes.toBytes(region))),
+ TEST_UTIL.getAdmin().move(Bytes.toBytes(RegionInfo.encodeRegionName(Bytes.toBytes(region))),
Bytes.toBytes(targetServer.getServerName()));
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/MasterStatusTmpl.jamon
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/MasterStatusTmpl.jamon b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/MasterStatusTmpl.jamon
index bf3b623..292a668 100644
--- a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/MasterStatusTmpl.jamon
+++ b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/MasterStatusTmpl.jamon
@@ -30,9 +30,9 @@ AssignmentManager assignmentManager = null;
</%args>
<%import>
java.util.*;
+org.apache.hadoop.hbase.client.RegionInfo;
org.apache.hadoop.hbase.HBaseConfiguration;
org.apache.hadoop.hbase.HConstants;
-org.apache.hadoop.hbase.HRegionInfo;
org.apache.hadoop.hbase.HTableDescriptor;
org.apache.hadoop.hbase.NamespaceDescriptor;
org.apache.hadoop.hbase.ServerLoad;
@@ -435,7 +435,7 @@ AssignmentManager assignmentManager = master.getAssignmentManager();
<%for HTableDescriptor htDesc : tables%>
<%java>
TableName tableName = htDesc.getTableName();
- Map<RegionState.State, List<HRegionInfo>> tableRegions =
+ Map<RegionState.State, List<RegionInfo>> tableRegions =
master.getAssignmentManager().getRegionStates()
.getRegionByStateOfTable(tableName);
int openRegionsCount = tableRegions.get(RegionState.State.OPEN).size();
@@ -444,7 +444,7 @@ AssignmentManager assignmentManager = master.getAssignmentManager();
int failedRegionsCount = tableRegions.get(RegionState.State.FAILED_OPEN).size()
+ tableRegions.get(RegionState.State.FAILED_CLOSE).size();
int otherRegionsCount = 0;
- for (List<HRegionInfo> list: tableRegions.values()) {
+ for (List<RegionInfo> list: tableRegions.values()) {
otherRegionsCount += list.size();
}
// now subtract known states
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/RSStatusTmpl.jamon
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/RSStatusTmpl.jamon b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/RSStatusTmpl.jamon
index 6b403b2..c9bfcc9 100644
--- a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/RSStatusTmpl.jamon
+++ b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/RSStatusTmpl.jamon
@@ -26,7 +26,7 @@ String bcv = "";
<%import>
java.util.*;
org.apache.hadoop.hbase.regionserver.HRegionServer;
-org.apache.hadoop.hbase.HRegionInfo;
+org.apache.hadoop.hbase.client.RegionInfo;
org.apache.hadoop.hbase.ServerName;
org.apache.hadoop.hbase.HBaseConfiguration;
org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
@@ -44,7 +44,7 @@ org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
<%java>
ServerInfo serverInfo = ProtobufUtil.getServerInfo(null, regionServer.getRSRpcServices());
ServerName serverName = ProtobufUtil.toServerName(serverInfo.getServerName());
- List<HRegionInfo> onlineRegions = ProtobufUtil.getOnlineRegions(regionServer.getRSRpcServices());
+ List<RegionInfo> onlineRegions = ProtobufUtil.getOnlineRegions(regionServer.getRSRpcServices());
MasterAddressTracker masterAddressTracker = regionServer.getMasterAddressTracker();
ServerName masterServerName = masterAddressTracker == null ? null
: masterAddressTracker.getMasterAddress();
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/RegionListTmpl.jamon
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/RegionListTmpl.jamon b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/RegionListTmpl.jamon
index 285dbe6..cf0e8ad 100644
--- a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/RegionListTmpl.jamon
+++ b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/RegionListTmpl.jamon
@@ -18,14 +18,15 @@
</%doc>
<%args>
HRegionServer regionServer;
- List<HRegionInfo> onlineRegions;
+ List<RegionInfo> onlineRegions;
</%args>
<%import>
java.util.*;
org.apache.commons.lang3.time.FastDateFormat;
org.apache.hadoop.hbase.regionserver.HRegionServer;
org.apache.hadoop.hbase.util.Bytes;
- org.apache.hadoop.hbase.HRegionInfo;
+ org.apache.hadoop.hbase.client.RegionInfo;
+ org.apache.hadoop.hbase.client.RegionInfoDisplay;
org.apache.hadoop.hbase.regionserver.Region;
org.apache.hadoop.hbase.ServerName;
org.apache.hadoop.hbase.HBaseConfiguration;
@@ -39,7 +40,7 @@
<%if (onlineRegions != null && onlineRegions.size() > 0) %>
<%java>
- Collections.sort(onlineRegions);
+ Collections.sort(onlineRegions, RegionInfo.COMPARATOR);
</%java>
<div class="tabbable">
@@ -85,7 +86,7 @@
<%def baseInfo>
<%args>
- List<HRegionInfo> onlineRegions;
+ List<RegionInfo> onlineRegions;
</%args>
<table class="table table-striped">
<tr>
@@ -95,16 +96,16 @@
<th>ReplicaID</th>
</tr>
- <%for HRegionInfo r: onlineRegions %>
+ <%for RegionInfo r: onlineRegions %>
<tr>
<%java>
- String displayName = HRegionInfo.getRegionNameAsStringForDisplay(r,
+ String displayName = RegionInfoDisplay.getRegionNameAsStringForDisplay(r,
regionServer.getConfiguration());
</%java>
<td><a href="region.jsp?name=<% r.getEncodedName() %>"><% displayName %></a></td>
- <td><% Bytes.toStringBinary(HRegionInfo.getStartKeyForDisplay(r,
+ <td><% Bytes.toStringBinary(RegionInfoDisplay.getStartKeyForDisplay(r,
regionServer.getConfiguration())) %></td>
- <td><% Bytes.toStringBinary(HRegionInfo.getEndKeyForDisplay(r,
+ <td><% Bytes.toStringBinary(RegionInfoDisplay.getEndKeyForDisplay(r,
regionServer.getConfiguration())) %></td>
<td><% r.getReplicaId() %></td>
</tr>
@@ -114,7 +115,7 @@
<%def requestStats>
<%args>
- List<HRegionInfo> onlineRegions;
+ List<RegionInfo> onlineRegions;
</%args>
<table class="table table-striped">
<tr>
@@ -124,12 +125,12 @@
<th>Write Request Count</th>
</tr>
- <%for HRegionInfo r: onlineRegions %>
+ <%for RegionInfo r: onlineRegions %>
<tr>
<%java>
RegionLoad load = regionServer.createRegionLoad(r.getEncodedName());
- String displayName = HRegionInfo.getRegionNameAsStringForDisplay(r,
+ String displayName = RegionInfoDisplay.getRegionNameAsStringForDisplay(r,
regionServer.getConfiguration());
</%java>
<td><a href="region.jsp?name=<% r.getEncodedName() %>"><% displayName %></a></td>
@@ -146,7 +147,7 @@
<%def storeStats>
<%args>
- List<HRegionInfo> onlineRegions;
+ List<RegionInfo> onlineRegions;
</%args>
<table class="table table-striped">
<tr>
@@ -160,12 +161,12 @@
<th>Data Locality</th>
</tr>
- <%for HRegionInfo r: onlineRegions %>
+ <%for RegionInfo r: onlineRegions %>
<tr>
<%java>
RegionLoad load = regionServer.createRegionLoad(r.getEncodedName());
- String displayName = HRegionInfo.getRegionNameAsStringForDisplay(r,
+ String displayName = RegionInfoDisplay.getRegionNameAsStringForDisplay(r,
regionServer.getConfiguration());
</%java>
<td><a href="region.jsp?name=<% r.getEncodedName() %>"><% displayName %></a></td>
@@ -190,7 +191,7 @@
<%def compactStats>
<%args>
- List<HRegionInfo> onlineRegions;
+ List<RegionInfo> onlineRegions;
</%args>
<table class="table table-striped">
<tr>
@@ -201,7 +202,7 @@
<th>Last Major Compaction</th>
</tr>
- <%for HRegionInfo r: onlineRegions %>
+ <%for RegionInfo r: onlineRegions %>
<tr>
<%java>
@@ -218,7 +219,7 @@
compactTime = fdf.format(load.getLastMajorCompactionTs());
}
}
- String displayName = HRegionInfo.getRegionNameAsStringForDisplay(r,
+ String displayName = RegionInfoDisplay.getRegionNameAsStringForDisplay(r,
regionServer.getConfiguration());
</%java>
<td><a href="region.jsp?name=<% r.getEncodedName() %>"><% displayName %></a></td>
@@ -235,7 +236,7 @@
<%def memstoreStats>
<%args>
- List<HRegionInfo> onlineRegions;
+ List<RegionInfo> onlineRegions;
</%args>
<table class="table table-striped">
<tr>
@@ -243,12 +244,12 @@
<th>Memstore Size</th>
</tr>
- <%for HRegionInfo r: onlineRegions %>
+ <%for RegionInfo r: onlineRegions %>
<tr>
<%java>
RegionLoad load = regionServer.createRegionLoad(r.getEncodedName());
- String displayName = HRegionInfo.getRegionNameAsStringForDisplay(r,
+ String displayName = RegionInfoDisplay.getRegionNameAsStringForDisplay(r,
regionServer.getConfiguration());
</%java>
<td><a href="region.jsp?name=<% r.getEncodedName() %>"><% displayName %></a></td>
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/RegionStateListener.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/RegionStateListener.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/RegionStateListener.java
index 43685ae..e57471a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/RegionStateListener.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/RegionStateListener.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.hbase;
import java.io.IOException;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.yetus.audience.InterfaceAudience;
/**
@@ -31,22 +32,22 @@ public interface RegionStateListener {
/**
* Process region split event.
*
- * @param hri An instance of HRegionInfo
+ * @param hri An instance of RegionInfo
* @throws IOException
*/
- void onRegionSplit(HRegionInfo hri) throws IOException;
+ void onRegionSplit(RegionInfo hri) throws IOException;
/**
* Process region split reverted event.
*
- * @param hri An instance of HRegionInfo
+ * @param hri An instance of RegionInfo
* @throws IOException Signals that an I/O exception has occurred.
*/
- void onRegionSplitReverted(HRegionInfo hri) throws IOException;
+ void onRegionSplitReverted(RegionInfo hri) throws IOException;
/**
* Process region merge event.
* @throws IOException
*/
- void onRegionMerged(HRegionInfo mergedRegion) throws IOException;
+ void onRegionMerged(RegionInfo mergedRegion) throws IOException;
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/HFileArchiver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/HFileArchiver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/HFileArchiver.java
index 4321dc8..4da1235 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/HFileArchiver.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/HFileArchiver.java
@@ -32,7 +32,7 @@ import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.PathFilter;
-import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HStoreFile;
import org.apache.hadoop.hbase.util.Bytes;
@@ -75,7 +75,7 @@ public class HFileArchiver {
/**
* @return True if the Region exits in the filesystem.
*/
- public static boolean exists(Configuration conf, FileSystem fs, HRegionInfo info)
+ public static boolean exists(Configuration conf, FileSystem fs, RegionInfo info)
throws IOException {
Path rootDir = FSUtils.getRootDir(conf);
Path regionDir = HRegion.getRegionDir(rootDir, info);
@@ -87,10 +87,10 @@ public class HFileArchiver {
* archive directory
* @param conf the configuration to use
* @param fs the file system object
- * @param info HRegionInfo for region to be deleted
+ * @param info RegionInfo for region to be deleted
* @throws IOException
*/
- public static void archiveRegion(Configuration conf, FileSystem fs, HRegionInfo info)
+ public static void archiveRegion(Configuration conf, FileSystem fs, RegionInfo info)
throws IOException {
Path rootDir = FSUtils.getRootDir(conf);
archiveRegion(fs, rootDir, FSUtils.getTableDir(rootDir, info.getTable()),
@@ -176,7 +176,7 @@ public class HFileArchiver {
* @throws IOException if the files could not be correctly disposed.
*/
public static void archiveFamily(FileSystem fs, Configuration conf,
- HRegionInfo parent, Path tableDir, byte[] family) throws IOException {
+ RegionInfo parent, Path tableDir, byte[] family) throws IOException {
Path familyDir = new Path(tableDir, new Path(parent.getEncodedName(), Bytes.toString(family)));
archiveFamilyByFamilyDir(fs, conf, parent, familyDir, family);
}
@@ -192,7 +192,7 @@ public class HFileArchiver {
* @throws IOException if the files could not be correctly disposed.
*/
public static void archiveFamilyByFamilyDir(FileSystem fs, Configuration conf,
- HRegionInfo parent, Path familyDir, byte[] family) throws IOException {
+ RegionInfo parent, Path familyDir, byte[] family) throws IOException {
FileStatus[] storeFiles = FSUtils.listStatus(fs, familyDir);
if (storeFiles == null) {
LOG.debug("No store files to dispose for region=" + parent.getRegionNameAsString() +
@@ -219,13 +219,13 @@ public class HFileArchiver {
* Remove the store files, either by archiving them or outright deletion
* @param conf {@link Configuration} to examine to determine the archive directory
* @param fs the filesystem where the store files live
- * @param regionInfo {@link HRegionInfo} of the region hosting the store files
+ * @param regionInfo {@link RegionInfo} of the region hosting the store files
* @param family the family hosting the store files
* @param compactedFiles files to be disposed of. No further reading of these files should be
* attempted; otherwise likely to cause an {@link IOException}
* @throws IOException if the files could not be correctly disposed.
*/
- public static void archiveStoreFiles(Configuration conf, FileSystem fs, HRegionInfo regionInfo,
+ public static void archiveStoreFiles(Configuration conf, FileSystem fs, RegionInfo regionInfo,
Path tableDir, byte[] family, Collection<HStoreFile> compactedFiles)
throws IOException, FailedArchiveException {
@@ -284,7 +284,7 @@ public class HFileArchiver {
* @param storeFile file to be archived
* @throws IOException if the files could not be correctly disposed.
*/
- public static void archiveStoreFile(Configuration conf, FileSystem fs, HRegionInfo regionInfo,
+ public static void archiveStoreFile(Configuration conf, FileSystem fs, RegionInfo regionInfo,
Path tableDir, byte[] family, Path storeFile) throws IOException {
Path storeArchiveDir = HFileArchiveUtil.getStoreArchivePath(conf, regionInfo, tableDir, family);
// make sure we don't archive if we can't and that the archive dir exists
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/client/ClientSideRegionScanner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/ClientSideRegionScanner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/ClientSideRegionScanner.java
index efdf8e5..72a2ea4 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/ClientSideRegionScanner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/ClientSideRegionScanner.java
@@ -29,11 +29,10 @@ import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.RegionScanner;
+import org.apache.yetus.audience.InterfaceAudience;
/**
* A client scanner for a region opened for read-only on the client side. Assumes region data
@@ -49,7 +48,7 @@ public class ClientSideRegionScanner extends AbstractClientScanner {
List<Cell> values;
public ClientSideRegionScanner(Configuration conf, FileSystem fs,
- Path rootDir, TableDescriptor htd, HRegionInfo hri, Scan scan, ScanMetrics scanMetrics)
+ Path rootDir, TableDescriptor htd, RegionInfo hri, Scan scan, ScanMetrics scanMetrics)
throws IOException {
// region is immutable, set isolation level
scan.setIsolationLevel(IsolationLevel.READ_UNCOMMITTED);
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/client/TableSnapshotScanner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/TableSnapshotScanner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/TableSnapshotScanner.java
index 9244ced..ab361c1 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/TableSnapshotScanner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/TableSnapshotScanner.java
@@ -30,10 +30,9 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.CellUtil;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.snapshot.RestoreSnapshotHelper;
import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.yetus.audience.InterfaceAudience;
/**
* A Scanner which performs a scan over snapshot files. Using this class requires copying the
@@ -73,7 +72,7 @@ public class TableSnapshotScanner extends AbstractClientScanner {
private Path rootDir;
private Path restoreDir;
private Scan scan;
- private ArrayList<HRegionInfo> regions;
+ private ArrayList<RegionInfo> regions;
private TableDescriptor htd;
private ClientSideRegionScanner currentRegionScanner = null;
@@ -121,11 +120,11 @@ public class TableSnapshotScanner extends AbstractClientScanner {
final RestoreSnapshotHelper.RestoreMetaChanges meta =
RestoreSnapshotHelper.copySnapshotForScanner(
conf, fs, rootDir, restoreDir, snapshotName);
- final List<HRegionInfo> restoredRegions = meta.getRegionsToAdd();
+ final List<RegionInfo> restoredRegions = meta.getRegionsToAdd();
htd = meta.getTableDescriptor();
regions = new ArrayList<>(restoredRegions.size());
- for (HRegionInfo hri : restoredRegions) {
+ for (RegionInfo hri : restoredRegions) {
if (hri.isOffline() && (hri.isSplit() || hri.isSplitParent())) {
continue;
}
@@ -136,7 +135,7 @@ public class TableSnapshotScanner extends AbstractClientScanner {
}
// sort for regions according to startKey.
- Collections.sort(regions);
+ Collections.sort(regions, RegionInfo.COMPARATOR);
initScanMetrics(scan);
}
@@ -150,7 +149,7 @@ public class TableSnapshotScanner extends AbstractClientScanner {
return null;
}
- HRegionInfo hri = regions.get(currentRegion);
+ RegionInfo hri = regions.get(currentRegion);
currentRegionScanner = new ClientSideRegionScanner(conf, fs,
restoreDir, htd, hri, scan, scanMetrics);
if (this.scanMetrics != null) {
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/client/locking/LockServiceClient.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/locking/LockServiceClient.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/locking/LockServiceClient.java
index 2a77d4c..8694a4c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/locking/LockServiceClient.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/locking/LockServiceClient.java
@@ -19,22 +19,22 @@
package org.apache.hadoop.hbase.client.locking;
-import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
-
import java.util.List;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Abortable;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.NonceGenerator;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability;
-import org.apache.hadoop.hbase.client.NonceGenerator;
+
+import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockType;
import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockService;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockType;
/**
* Helper class to create "master locks" for namespaces, tables and regions.
@@ -83,7 +83,7 @@ public class LockServiceClient {
* Create a new EntityLock object to acquire exclusive lock on multiple regions of same tables.
* Internally, the table and its namespace will also be locked in shared mode.
*/
- public EntityLock regionLock(List<HRegionInfo> regionInfos, String description, Abortable abort) {
+ public EntityLock regionLock(List<RegionInfo> regionInfos, String description, Abortable abort) {
LockRequest lockRequest = buildLockRequest(LockType.EXCLUSIVE,
null, null, regionInfos, description, ng.getNonceGroup(), ng.newNonce());
return new EntityLock(conf, stub, lockRequest, abort);
@@ -91,15 +91,15 @@ public class LockServiceClient {
@VisibleForTesting
public static LockRequest buildLockRequest(final LockType type,
- final String namespace, final TableName tableName, final List<HRegionInfo> regionInfos,
+ final String namespace, final TableName tableName, final List<RegionInfo> regionInfos,
final String description, final long nonceGroup, final long nonce) {
final LockRequest.Builder builder = LockRequest.newBuilder()
.setLockType(type)
.setNonceGroup(nonceGroup)
.setNonce(nonce);
if (regionInfos != null) {
- for (HRegionInfo hri: regionInfos) {
- builder.addRegionInfo(HRegionInfo.convert(hri));
+ for (RegionInfo hri: regionInfos) {
+ builder.addRegionInfo(ProtobufUtil.toRegionInfo(hri));
}
} else if (namespace != null) {
builder.setNamespace(namespace);
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/SplitLogManagerCoordination.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/SplitLogManagerCoordination.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/SplitLogManagerCoordination.java
index 288721a..df8103b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/SplitLogManagerCoordination.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/SplitLogManagerCoordination.java
@@ -24,15 +24,15 @@ import java.io.InterruptedIOException;
import java.util.Set;
import java.util.concurrent.ConcurrentMap;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.master.SplitLogManager.ResubmitDirective;
import org.apache.hadoop.hbase.master.SplitLogManager.Task;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
+import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
/**
* Coordination for SplitLogManager. It creates and works with tasks for split log operations<BR>
@@ -123,7 +123,7 @@ public interface SplitLogManagerCoordination {
* @throws IOException in case of failure
* @throws InterruptedIOException
*/
- void markRegionsRecovering(final ServerName serverName, Set<HRegionInfo> userRegions)
+ void markRegionsRecovering(final ServerName serverName, Set<RegionInfo> userRegions)
throws IOException, InterruptedIOException;
/**
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZKSplitLogManagerCoordination.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZKSplitLogManagerCoordination.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZKSplitLogManagerCoordination.java
index ef6ef62..5fd20e8 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZKSplitLogManagerCoordination.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZKSplitLogManagerCoordination.java
@@ -18,13 +18,13 @@
package org.apache.hadoop.hbase.coordination;
-import static org.apache.hadoop.hbase.util.CollectionUtils.*;
import static org.apache.hadoop.hbase.master.SplitLogManager.ResubmitDirective.CHECK;
import static org.apache.hadoop.hbase.master.SplitLogManager.ResubmitDirective.FORCE;
import static org.apache.hadoop.hbase.master.SplitLogManager.TerminationStatus.DELETED;
import static org.apache.hadoop.hbase.master.SplitLogManager.TerminationStatus.FAILURE;
import static org.apache.hadoop.hbase.master.SplitLogManager.TerminationStatus.IN_PROGRESS;
import static org.apache.hadoop.hbase.master.SplitLogManager.TerminationStatus.SUCCESS;
+import static org.apache.hadoop.hbase.util.CollectionUtils.computeIfAbsent;
import java.io.IOException;
import java.io.InterruptedIOException;
@@ -39,17 +39,16 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.CoordinatedStateManager;
import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.SplitLogCounters;
import org.apache.hadoop.hbase.SplitLogTask;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
import org.apache.hadoop.hbase.coordination.ZKSplitLogManagerCoordination.TaskFinisher.Status;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.master.SplitLogManager.ResubmitDirective;
import org.apache.hadoop.hbase.master.SplitLogManager.Task;
import org.apache.hadoop.hbase.master.SplitLogManager.TerminationStatus;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
import org.apache.hadoop.hbase.wal.WALSplitter;
@@ -59,6 +58,7 @@ import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.hadoop.util.StringUtils;
+import org.apache.yetus.audience.InterfaceAudience;
import org.apache.zookeeper.AsyncCallback;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
@@ -66,6 +66,8 @@ import org.apache.zookeeper.KeeperException.NoNodeException;
import org.apache.zookeeper.ZooDefs.Ids;
import org.apache.zookeeper.data.Stat;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
+
/**
* ZooKeeper based implementation of
* {@link SplitLogManagerCoordination}
@@ -287,7 +289,7 @@ public class ZKSplitLogManagerCoordination extends ZooKeeperListener implements
public void removeRecoveringRegions(final Set<String> recoveredServerNameSet,
Boolean isMetaRecovery)
throws IOException {
- final String metaEncodeRegionName = HRegionInfo.FIRST_META_REGIONINFO.getEncodedName();
+ final String metaEncodeRegionName = RegionInfoBuilder.FIRST_META_REGIONINFO.getEncodedName();
int count = 0;
try {
List<String> tasks = ZKUtil.listChildrenNoWatch(watcher, watcher.znodePaths.splitLogZNode);
@@ -594,10 +596,10 @@ public class ZKSplitLogManagerCoordination extends ZooKeeperListener implements
* @param userRegions user regiones assigned on the region server
*/
@Override
- public void markRegionsRecovering(final ServerName serverName, Set<HRegionInfo> userRegions)
+ public void markRegionsRecovering(final ServerName serverName, Set<RegionInfo> userRegions)
throws IOException, InterruptedIOException {
this.lastRecoveringNodeCreationTime = EnvironmentEdgeManager.currentTime();
- for (HRegionInfo region : userRegions) {
+ for (RegionInfo region : userRegions) {
String regionEncodeName = region.getEncodedName();
long retries = this.zkretries;
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MultiRowMutationEndpoint.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MultiRowMutationEndpoint.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MultiRowMutationEndpoint.java
index c4fb440..fbb4101 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MultiRowMutationEndpoint.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MultiRowMutationEndpoint.java
@@ -24,22 +24,22 @@ import java.util.Optional;
import java.util.SortedSet;
import java.util.TreeSet;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.yetus.audience.InterfaceStability;
import org.apache.hadoop.hbase.CoprocessorEnvironment;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils;
-import org.apache.hadoop.hbase.regionserver.HRegion;
-import org.apache.hadoop.hbase.regionserver.WrongRegionException;
-import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto;
+import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos.MultiRowMutationService;
import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos.MutateRowsRequest;
import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos.MutateRowsResponse;
-import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos.MultiRowMutationService;
+import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.regionserver.WrongRegionException;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.yetus.audience.InterfaceStability;
import com.google.protobuf.RpcCallback;
import com.google.protobuf.RpcController;
@@ -92,7 +92,7 @@ public class MultiRowMutationEndpoint extends MultiRowMutationService implements
mutations.add(ProtobufUtil.toMutation(m));
}
- HRegionInfo regionInfo = env.getRegion().getRegionInfo();
+ RegionInfo regionInfo = env.getRegion().getRegionInfo();
for (Mutation m : mutations) {
// check whether rows are in range for this region
if (!HRegion.rowIsInRange(regionInfo, m.getRow())) {
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/favored/FavoredNodeAssignmentHelper.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/favored/FavoredNodeAssignmentHelper.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/favored/FavoredNodeAssignmentHelper.java
index ef5c650..f0afad9 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/favored/FavoredNodeAssignmentHelper.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/favored/FavoredNodeAssignmentHelper.java
@@ -37,29 +37,29 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseIOException;
import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
-import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.master.RackManager;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.FavoredNodes;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Sets;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.FavoredNodes;
/**
* Helper class for {@link FavoredNodeLoadBalancer} that has all the intelligence for racks,
* meta scans, etc. Instantiated by the {@link FavoredNodeLoadBalancer} when needed (from
- * within calls like {@link FavoredNodeLoadBalancer#randomAssignment(HRegionInfo, List)}).
+ * within calls like {@link FavoredNodeLoadBalancer#randomAssignment(RegionInfo, List)}).
* All updates to favored nodes should only be done from {@link FavoredNodesManager} and not
* through this helper class (except for tests).
*/
@@ -116,15 +116,15 @@ public class FavoredNodeAssignmentHelper {
/**
* Update meta table with favored nodes info
- * @param regionToFavoredNodes map of HRegionInfo's to their favored nodes
+ * @param regionToFavoredNodes map of RegionInfo's to their favored nodes
* @param connection connection to be used
* @throws IOException
*/
public static void updateMetaWithFavoredNodesInfo(
- Map<HRegionInfo, List<ServerName>> regionToFavoredNodes,
+ Map<RegionInfo, List<ServerName>> regionToFavoredNodes,
Connection connection) throws IOException {
List<Put> puts = new ArrayList<>();
- for (Map.Entry<HRegionInfo, List<ServerName>> entry : regionToFavoredNodes.entrySet()) {
+ for (Map.Entry<RegionInfo, List<ServerName>> entry : regionToFavoredNodes.entrySet()) {
Put put = makePutFromRegionInfo(entry.getKey(), entry.getValue());
if (put != null) {
puts.add(put);
@@ -141,10 +141,10 @@ public class FavoredNodeAssignmentHelper {
* @throws IOException
*/
public static void updateMetaWithFavoredNodesInfo(
- Map<HRegionInfo, List<ServerName>> regionToFavoredNodes,
+ Map<RegionInfo, List<ServerName>> regionToFavoredNodes,
Configuration conf) throws IOException {
List<Put> puts = new ArrayList<>();
- for (Map.Entry<HRegionInfo, List<ServerName>> entry : regionToFavoredNodes.entrySet()) {
+ for (Map.Entry<RegionInfo, List<ServerName>> entry : regionToFavoredNodes.entrySet()) {
Put put = makePutFromRegionInfo(entry.getKey(), entry.getValue());
if (put != null) {
puts.add(put);
@@ -170,7 +170,7 @@ public class FavoredNodeAssignmentHelper {
* @param favoredNodeList
* @return Put object
*/
- static Put makePutFromRegionInfo(HRegionInfo regionInfo, List<ServerName>favoredNodeList)
+ static Put makePutFromRegionInfo(RegionInfo regionInfo, List<ServerName>favoredNodeList)
throws IOException {
Put put = null;
if (favoredNodeList != null) {
@@ -225,8 +225,8 @@ public class FavoredNodeAssignmentHelper {
// If there were fewer servers in one rack, say r3, which had 3 servers, one possible
// placement could be r2:s5, <skip-r3>, r4:s5, r1:s5, r2:s6, <skip-r3> ...
// The regions should be distributed proportionately to the racksizes
- public void placePrimaryRSAsRoundRobin(Map<ServerName, List<HRegionInfo>> assignmentMap,
- Map<HRegionInfo, ServerName> primaryRSMap, List<HRegionInfo> regions) {
+ public void placePrimaryRSAsRoundRobin(Map<ServerName, List<RegionInfo>> assignmentMap,
+ Map<RegionInfo, ServerName> primaryRSMap, List<RegionInfo> regions) {
List<String> rackList = new ArrayList<>(rackToRegionServerMap.size());
rackList.addAll(rackToRegionServerMap.keySet());
int rackIndex = random.nextInt(rackList.size());
@@ -239,7 +239,7 @@ public class FavoredNodeAssignmentHelper {
int numIterations = 0;
// Initialize the current processing host index.
int serverIndex = random.nextInt(maxRackSize);
- for (HRegionInfo regionInfo : regions) {
+ for (RegionInfo regionInfo : regions) {
List<ServerName> currentServerList;
String rackName;
while (true) {
@@ -264,7 +264,7 @@ public class FavoredNodeAssignmentHelper {
// Place the current region with the current primary region server
primaryRSMap.put(regionInfo, currentServer);
if (assignmentMap != null) {
- List<HRegionInfo> regionsForServer = assignmentMap.get(currentServer);
+ List<RegionInfo> regionsForServer = assignmentMap.get(currentServer);
if (regionsForServer == null) {
regionsForServer = new ArrayList<>();
assignmentMap.put(currentServer, regionsForServer);
@@ -282,12 +282,12 @@ public class FavoredNodeAssignmentHelper {
}
}
- public Map<HRegionInfo, ServerName[]> placeSecondaryAndTertiaryRS(
- Map<HRegionInfo, ServerName> primaryRSMap) {
- Map<HRegionInfo, ServerName[]> secondaryAndTertiaryMap = new HashMap<>();
- for (Map.Entry<HRegionInfo, ServerName> entry : primaryRSMap.entrySet()) {
+ public Map<RegionInfo, ServerName[]> placeSecondaryAndTertiaryRS(
+ Map<RegionInfo, ServerName> primaryRSMap) {
+ Map<RegionInfo, ServerName[]> secondaryAndTertiaryMap = new HashMap<>();
+ for (Map.Entry<RegionInfo, ServerName> entry : primaryRSMap.entrySet()) {
// Get the target region and its primary region server rack
- HRegionInfo regionInfo = entry.getKey();
+ RegionInfo regionInfo = entry.getKey();
ServerName primaryRS = entry.getValue();
try {
// Create the secondary and tertiary region server pair object.
@@ -306,7 +306,7 @@ public class FavoredNodeAssignmentHelper {
return secondaryAndTertiaryMap;
}
- public ServerName[] getSecondaryAndTertiary(HRegionInfo regionInfo, ServerName primaryRS)
+ public ServerName[] getSecondaryAndTertiary(RegionInfo regionInfo, ServerName primaryRS)
throws IOException {
ServerName[] favoredNodes;// Get the rack for the primary region server
@@ -320,11 +320,11 @@ public class FavoredNodeAssignmentHelper {
return favoredNodes;
}
- private Map<ServerName, Set<HRegionInfo>> mapRSToPrimaries(
- Map<HRegionInfo, ServerName> primaryRSMap) {
- Map<ServerName, Set<HRegionInfo>> primaryServerMap = new HashMap<>();
- for (Entry<HRegionInfo, ServerName> e : primaryRSMap.entrySet()) {
- Set<HRegionInfo> currentSet = primaryServerMap.get(e.getValue());
+ private Map<ServerName, Set<RegionInfo>> mapRSToPrimaries(
+ Map<RegionInfo, ServerName> primaryRSMap) {
+ Map<ServerName, Set<RegionInfo>> primaryServerMap = new HashMap<>();
+ for (Entry<RegionInfo, ServerName> e : primaryRSMap.entrySet()) {
+ Set<RegionInfo> currentSet = primaryServerMap.get(e.getValue());
if (currentSet == null) {
currentSet = new HashSet<>();
}
@@ -341,15 +341,15 @@ public class FavoredNodeAssignmentHelper {
* @param primaryRSMap
* @return the map of regions to the servers the region-files should be hosted on
*/
- public Map<HRegionInfo, ServerName[]> placeSecondaryAndTertiaryWithRestrictions(
- Map<HRegionInfo, ServerName> primaryRSMap) {
- Map<ServerName, Set<HRegionInfo>> serverToPrimaries =
+ public Map<RegionInfo, ServerName[]> placeSecondaryAndTertiaryWithRestrictions(
+ Map<RegionInfo, ServerName> primaryRSMap) {
+ Map<ServerName, Set<RegionInfo>> serverToPrimaries =
mapRSToPrimaries(primaryRSMap);
- Map<HRegionInfo, ServerName[]> secondaryAndTertiaryMap = new HashMap<>();
+ Map<RegionInfo, ServerName[]> secondaryAndTertiaryMap = new HashMap<>();
- for (Entry<HRegionInfo, ServerName> entry : primaryRSMap.entrySet()) {
+ for (Entry<RegionInfo, ServerName> entry : primaryRSMap.entrySet()) {
// Get the target region and its primary region server rack
- HRegionInfo regionInfo = entry.getKey();
+ RegionInfo regionInfo = entry.getKey();
ServerName primaryRS = entry.getValue();
try {
// Get the rack for the primary region server
@@ -378,9 +378,9 @@ public class FavoredNodeAssignmentHelper {
}
private ServerName[] multiRackCaseWithRestrictions(
- Map<ServerName, Set<HRegionInfo>> serverToPrimaries,
- Map<HRegionInfo, ServerName[]> secondaryAndTertiaryMap,
- String primaryRack, ServerName primaryRS, HRegionInfo regionInfo) throws IOException {
+ Map<ServerName, Set<RegionInfo>> serverToPrimaries,
+ Map<RegionInfo, ServerName[]> secondaryAndTertiaryMap,
+ String primaryRack, ServerName primaryRS, RegionInfo regionInfo) throws IOException {
// Random to choose the secondary and tertiary region server
// from another rack to place the secondary and tertiary
// Random to choose one rack except for the current rack
@@ -395,13 +395,13 @@ public class FavoredNodeAssignmentHelper {
// Randomly pick up two servers from this secondary rack
// Skip the secondary for the tertiary placement
// skip the servers which share the primary already
- Set<HRegionInfo> primaries = serverToPrimaries.get(primaryRS);
+ Set<RegionInfo> primaries = serverToPrimaries.get(primaryRS);
Set<ServerName> skipServerSet = new HashSet<>();
while (true) {
ServerName[] secondaryAndTertiary = null;
if (primaries.size() > 1) {
// check where his tertiary and secondary are
- for (HRegionInfo primary : primaries) {
+ for (RegionInfo primary : primaries) {
secondaryAndTertiary = secondaryAndTertiaryMap.get(primary);
if (secondaryAndTertiary != null) {
if (getRackOfServer(secondaryAndTertiary[0]).equals(secondaryRack)) {
@@ -468,7 +468,7 @@ public class FavoredNodeAssignmentHelper {
return favoredNodes;
}
- private ServerName[] singleRackCase(HRegionInfo regionInfo,
+ private ServerName[] singleRackCase(RegionInfo regionInfo,
ServerName primaryRS,
String primaryRack) throws IOException {
// Single rack case: have to pick the secondary and tertiary
@@ -516,7 +516,7 @@ public class FavoredNodeAssignmentHelper {
* @return Array containing secondary and tertiary favored nodes.
* @throws IOException Signals that an I/O exception has occurred.
*/
- private ServerName[] multiRackCase(HRegionInfo regionInfo, ServerName primaryRS,
+ private ServerName[] multiRackCase(RegionInfo regionInfo, ServerName primaryRS,
String primaryRack) throws IOException {
List<ServerName>favoredNodes = Lists.newArrayList(primaryRS);
@@ -764,15 +764,15 @@ public class FavoredNodeAssignmentHelper {
* Choose a random server as primary and then choose secondary and tertiary FN so its spread
* across two racks.
*/
- public List<ServerName> generateFavoredNodes(HRegionInfo hri) throws IOException {
+ public List<ServerName> generateFavoredNodes(RegionInfo hri) throws IOException {
List<ServerName> favoredNodesForRegion = new ArrayList<>(FAVORED_NODES_NUM);
ServerName primary = servers.get(random.nextInt(servers.size()));
favoredNodesForRegion.add(ServerName.valueOf(primary.getHostAndPort(), ServerName.NON_STARTCODE));
- Map<HRegionInfo, ServerName> primaryRSMap = new HashMap<>(1);
+ Map<RegionInfo, ServerName> primaryRSMap = new HashMap<>(1);
primaryRSMap.put(hri, primary);
- Map<HRegionInfo, ServerName[]> secondaryAndTertiaryRSMap =
+ Map<RegionInfo, ServerName[]> secondaryAndTertiaryRSMap =
placeSecondaryAndTertiaryRS(primaryRSMap);
ServerName[] secondaryAndTertiaryNodes = secondaryAndTertiaryRSMap.get(hri);
if (secondaryAndTertiaryNodes != null && secondaryAndTertiaryNodes.length == 2) {
@@ -785,13 +785,13 @@ public class FavoredNodeAssignmentHelper {
}
}
- public Map<HRegionInfo, List<ServerName>> generateFavoredNodesRoundRobin(
- Map<ServerName, List<HRegionInfo>> assignmentMap, List<HRegionInfo> regions)
+ public Map<RegionInfo, List<ServerName>> generateFavoredNodesRoundRobin(
+ Map<ServerName, List<RegionInfo>> assignmentMap, List<RegionInfo> regions)
throws IOException {
if (regions.size() > 0) {
if (canPlaceFavoredNodes()) {
- Map<HRegionInfo, ServerName> primaryRSMap = new HashMap<>();
+ Map<RegionInfo, ServerName> primaryRSMap = new HashMap<>();
// Lets try to have an equal distribution for primary favored node
placePrimaryRSAsRoundRobin(assignmentMap, primaryRSMap, regions);
return generateFavoredNodes(primaryRSMap);
@@ -806,16 +806,16 @@ public class FavoredNodeAssignmentHelper {
/*
* Generate favored nodes for a set of regions when we know where they are currently hosted.
*/
- private Map<HRegionInfo, List<ServerName>> generateFavoredNodes(
- Map<HRegionInfo, ServerName> primaryRSMap) {
+ private Map<RegionInfo, List<ServerName>> generateFavoredNodes(
+ Map<RegionInfo, ServerName> primaryRSMap) {
- Map<HRegionInfo, List<ServerName>> generatedFavNodes = new HashMap<>();
- Map<HRegionInfo, ServerName[]> secondaryAndTertiaryRSMap =
+ Map<RegionInfo, List<ServerName>> generatedFavNodes = new HashMap<>();
+ Map<RegionInfo, ServerName[]> secondaryAndTertiaryRSMap =
placeSecondaryAndTertiaryRS(primaryRSMap);
- for (Entry<HRegionInfo, ServerName> entry : primaryRSMap.entrySet()) {
+ for (Entry<RegionInfo, ServerName> entry : primaryRSMap.entrySet()) {
List<ServerName> favoredNodesForRegion = new ArrayList<>(FAVORED_NODES_NUM);
- HRegionInfo region = entry.getKey();
+ RegionInfo region = entry.getKey();
ServerName primarySN = entry.getValue();
favoredNodesForRegion.add(ServerName.valueOf(primarySN.getHostname(), primarySN.getPort(),
NON_STARTCODE));
[10/19] hbase git commit: HBASE-18839 Apply RegionInfo to code base
Posted by ch...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/flush/MasterFlushTableProcedureManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/flush/MasterFlushTableProcedureManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/flush/MasterFlushTableProcedureManager.java
index 03141a3..66f9240 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/flush/MasterFlushTableProcedureManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/flush/MasterFlushTableProcedureManager.java
@@ -24,15 +24,15 @@ import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ThreadPoolExecutor;
+
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
-import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
-import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.MetaTableAccessor;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.errorhandling.ForeignException;
import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
@@ -43,12 +43,13 @@ import org.apache.hadoop.hbase.procedure.Procedure;
import org.apache.hadoop.hbase.procedure.ProcedureCoordinator;
import org.apache.hadoop.hbase.procedure.ProcedureCoordinatorRpcs;
import org.apache.hadoop.hbase.procedure.ZKProcedureCoordinator;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ProcedureDescription;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
+import org.apache.yetus.audience.InterfaceAudience;
import org.apache.zookeeper.KeeperException;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ProcedureDescription;
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
public class MasterFlushTableProcedureManager extends MasterProcedureManager {
@@ -125,7 +126,7 @@ public class MasterFlushTableProcedureManager extends MasterProcedureManager {
// It is possible that regions may move after we get the region server list.
// Each region server will get its own online regions for the table.
// We may still miss regions that need to be flushed.
- List<Pair<HRegionInfo, ServerName>> regionsAndLocations;
+ List<Pair<RegionInfo, ServerName>> regionsAndLocations;
if (TableName.META_TABLE_NAME.equals(tableName)) {
regionsAndLocations = new MetaTableLocator().getMetaRegionsAndLocations(
@@ -136,9 +137,9 @@ public class MasterFlushTableProcedureManager extends MasterProcedureManager {
}
Set<String> regionServers = new HashSet<>(regionsAndLocations.size());
- for (Pair<HRegionInfo, ServerName> region : regionsAndLocations) {
+ for (Pair<RegionInfo, ServerName> region : regionsAndLocations) {
if (region != null && region.getFirst() != null && region.getSecond() != null) {
- HRegionInfo hri = region.getFirst();
+ RegionInfo hri = region.getFirst();
if (hri.isOffline() && (hri.isSplit() || hri.isSplitParent())) continue;
regionServers.add(region.getSecond().toString());
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/FileSystemUtilizationChore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/FileSystemUtilizationChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/FileSystemUtilizationChore.java
index 89d7e14..a76e9c1 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/FileSystemUtilizationChore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/FileSystemUtilizationChore.java
@@ -26,16 +26,14 @@ import java.util.concurrent.TimeUnit;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ScheduledChore;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.regionserver.Store;
-import org.apache.hadoop.hbase.regionserver.StoreFile;
-import org.apache.hadoop.hbase.regionserver.StoreFileReader;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.yetus.audience.InterfaceAudience;
/**
* A chore which computes the size of each {@link HRegion} on the FileSystem hosted by the given {@link HRegionServer}.
@@ -76,7 +74,7 @@ public class FileSystemUtilizationChore extends ScheduledChore {
numberOfCyclesToSkip--;
return;
}
- final Map<HRegionInfo,Long> onlineRegionSizes = new HashMap<>();
+ final Map<RegionInfo, Long> onlineRegionSizes = new HashMap<>();
final Set<Region> onlineRegions = new HashSet<>(rs.getRegions());
// Process the regions from the last run if we have any. If we are somehow having difficulty
// processing the Regions, we want to avoid creating a backlog in memory of Region objs.
@@ -121,7 +119,7 @@ public class FileSystemUtilizationChore extends ScheduledChore {
continue;
}
// Avoid computing the size of region replicas.
- if (HRegionInfo.DEFAULT_REPLICA_ID != region.getRegionInfo().getReplicaId()) {
+ if (RegionInfo.DEFAULT_REPLICA_ID != region.getRegionInfo().getReplicaId()) {
skippedRegionReplicas++;
continue;
}
@@ -185,7 +183,7 @@ public class FileSystemUtilizationChore extends ScheduledChore {
* @return {@code false} if FileSystemUtilizationChore should pause reporting to master,
* {@code true} otherwise.
*/
- boolean reportRegionSizesToMaster(Map<HRegionInfo,Long> onlineRegionSizes) {
+ boolean reportRegionSizesToMaster(Map<RegionInfo,Long> onlineRegionSizes) {
return this.rs.reportRegionSizesForQuotas(onlineRegionSizes);
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterQuotaManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterQuotaManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterQuotaManager.java
index 4f2b51d..0587cc7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterQuotaManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterQuotaManager.java
@@ -31,22 +31,22 @@ import org.apache.commons.lang3.builder.HashCodeBuilder;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.DoNotRetryIOException;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.RegionStateListener;
import org.apache.hadoop.hbase.TableName;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.yetus.audience.InterfaceStability;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.namespace.NamespaceAuditor;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaResponse;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.yetus.audience.InterfaceStability;
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.TextFormat;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaResponse;
/**
* Master Quota Manager.
@@ -60,7 +60,7 @@ import org.apache.hadoop.hbase.shaded.com.google.protobuf.TextFormat;
@InterfaceStability.Evolving
public class MasterQuotaManager implements RegionStateListener {
private static final Log LOG = LogFactory.getLog(MasterQuotaManager.class);
- private static final Map<HRegionInfo, Long> EMPTY_MAP = Collections.unmodifiableMap(
+ private static final Map<RegionInfo, Long> EMPTY_MAP = Collections.unmodifiableMap(
new HashMap<>());
private final MasterServices masterServices;
@@ -69,7 +69,7 @@ public class MasterQuotaManager implements RegionStateListener {
private NamedLock<String> userLocks;
private boolean initialized = false;
private NamespaceAuditor namespaceQuotaManager;
- private ConcurrentHashMap<HRegionInfo, SizeSnapshotWithTimestamp> regionSizes;
+ private ConcurrentHashMap<RegionInfo, SizeSnapshotWithTimestamp> regionSizes;
public MasterQuotaManager(final MasterServices masterServices) {
this.masterServices = masterServices;
@@ -367,14 +367,14 @@ public class MasterQuotaManager implements RegionStateListener {
}
@Override
- public void onRegionMerged(HRegionInfo mergedRegion) throws IOException {
+ public void onRegionMerged(RegionInfo mergedRegion) throws IOException {
if (initialized) {
namespaceQuotaManager.updateQuotaForRegionMerge(mergedRegion);
}
}
@Override
- public void onRegionSplit(HRegionInfo hri) throws IOException {
+ public void onRegionSplit(RegionInfo hri) throws IOException {
if (initialized) {
namespaceQuotaManager.checkQuotaToSplitRegion(hri);
}
@@ -476,7 +476,7 @@ public class MasterQuotaManager implements RegionStateListener {
}
@Override
- public void onRegionSplitReverted(HRegionInfo hri) throws IOException {
+ public void onRegionSplitReverted(RegionInfo hri) throws IOException {
if (initialized) {
this.namespaceQuotaManager.removeRegionFromNamespaceUsage(hri);
}
@@ -530,20 +530,20 @@ public class MasterQuotaManager implements RegionStateListener {
this.regionSizes = new ConcurrentHashMap<>();
}
- public void addRegionSize(HRegionInfo hri, long size, long time) {
+ public void addRegionSize(RegionInfo hri, long size, long time) {
if (regionSizes == null) {
return;
}
regionSizes.put(hri, new SizeSnapshotWithTimestamp(size, time));
}
- public Map<HRegionInfo, Long> snapshotRegionSizes() {
+ public Map<RegionInfo, Long> snapshotRegionSizes() {
if (regionSizes == null) {
return EMPTY_MAP;
}
- Map<HRegionInfo, Long> copy = new HashMap<>();
- for (Entry<HRegionInfo, SizeSnapshotWithTimestamp> entry : regionSizes.entrySet()) {
+ Map<RegionInfo, Long> copy = new HashMap<>();
+ for (Entry<RegionInfo, SizeSnapshotWithTimestamp> entry : regionSizes.entrySet()) {
copy.put(entry.getKey(), entry.getValue().getSize());
}
return copy;
@@ -554,7 +554,7 @@ public class MasterQuotaManager implements RegionStateListener {
return 0;
}
int numEntriesRemoved = 0;
- Iterator<Entry<HRegionInfo,SizeSnapshotWithTimestamp>> iterator =
+ Iterator<Entry<RegionInfo,SizeSnapshotWithTimestamp>> iterator =
regionSizes.entrySet().iterator();
while (iterator.hasNext()) {
long currentEntryTime = iterator.next().getValue().getTime();
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/NamespaceQuotaSnapshotStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/NamespaceQuotaSnapshotStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/NamespaceQuotaSnapshotStore.java
index 5f71705..8d0a27e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/NamespaceQuotaSnapshotStore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/NamespaceQuotaSnapshotStore.java
@@ -24,16 +24,16 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
import java.util.concurrent.locks.ReentrantReadWriteLock.ReadLock;
import java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot.SpaceQuotaStatus;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota;
+import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.common.base.Predicate;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Iterables;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota;
/**
* {@link QuotaSnapshotStore} implementation for namespaces.
@@ -46,9 +46,9 @@ public class NamespaceQuotaSnapshotStore implements QuotaSnapshotStore<String> {
private final Connection conn;
private final QuotaObserverChore chore;
- private Map<HRegionInfo,Long> regionUsage;
+ private Map<RegionInfo,Long> regionUsage;
- public NamespaceQuotaSnapshotStore(Connection conn, QuotaObserverChore chore, Map<HRegionInfo,Long> regionUsage) {
+ public NamespaceQuotaSnapshotStore(Connection conn, QuotaObserverChore chore, Map<RegionInfo,Long> regionUsage) {
this.conn = Objects.requireNonNull(conn);
this.chore = Objects.requireNonNull(chore);
this.regionUsage = Objects.requireNonNull(regionUsage);
@@ -83,7 +83,7 @@ public class NamespaceQuotaSnapshotStore implements QuotaSnapshotStore<String> {
try {
final long sizeLimitInBytes = spaceQuota.getSoftLimit();
long sum = 0L;
- for (Entry<HRegionInfo,Long> entry : filterBySubject(subject)) {
+ for (Entry<RegionInfo,Long> entry : filterBySubject(subject)) {
sum += entry.getValue();
}
// Add in the size for any snapshots against this table
@@ -98,12 +98,12 @@ public class NamespaceQuotaSnapshotStore implements QuotaSnapshotStore<String> {
}
@Override
- public Iterable<Entry<HRegionInfo,Long>> filterBySubject(String namespace) {
+ public Iterable<Entry<RegionInfo, Long>> filterBySubject(String namespace) {
rlock.lock();
try {
- return Iterables.filter(regionUsage.entrySet(), new Predicate<Entry<HRegionInfo,Long>>() {
+ return Iterables.filter(regionUsage.entrySet(), new Predicate<Entry<RegionInfo,Long>>() {
@Override
- public boolean apply(Entry<HRegionInfo,Long> input) {
+ public boolean apply(Entry<RegionInfo,Long> input) {
return namespace.equals(input.getKey().getTable().getNamespaceAsString());
}
});
@@ -119,7 +119,7 @@ public class NamespaceQuotaSnapshotStore implements QuotaSnapshotStore<String> {
}
@Override
- public void setRegionUsage(Map<HRegionInfo,Long> regionUsage) {
+ public void setRegionUsage(Map<RegionInfo,Long> regionUsage) {
wlock.lock();
try {
this.regionUsage = Objects.requireNonNull(regionUsage);
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/QuotaObserverChore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/QuotaObserverChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/QuotaObserverChore.java
index 3eb36c5..bfbda35 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/QuotaObserverChore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/QuotaObserverChore.java
@@ -29,24 +29,23 @@ import java.util.concurrent.TimeUnit;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ScheduledChore;
import org.apache.hadoop.hbase.Stoppable;
import org.apache.hadoop.hbase.TableName;
-import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.master.MetricsMaster;
-import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot;
import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot.SpaceQuotaStatus;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.HashMultimap;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Iterables;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Multimap;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota;
/**
* Reads the currently received Region filesystem-space use reports and acts on those which
@@ -159,7 +158,7 @@ public class QuotaObserverChore extends ScheduledChore {
}
// The current "view" of region space use. Used henceforth.
- final Map<HRegionInfo,Long> reportedRegionSpaceUse = quotaManager.snapshotRegionSizes();
+ final Map<RegionInfo,Long> reportedRegionSpaceUse = quotaManager.snapshotRegionSizes();
if (LOG.isTraceEnabled()) {
LOG.trace(
"Using " + reportedRegionSpaceUse.size() + " region space use reports: " +
@@ -216,8 +215,8 @@ public class QuotaObserverChore extends ScheduledChore {
processNamespacesWithQuotas(namespacesWithQuotas, tablesByNamespace);
}
- void initializeSnapshotStores(Map<HRegionInfo,Long> regionSizes) {
- Map<HRegionInfo,Long> immutableRegionSpaceUse = Collections.unmodifiableMap(regionSizes);
+ void initializeSnapshotStores(Map<RegionInfo,Long> regionSizes) {
+ Map<RegionInfo,Long> immutableRegionSpaceUse = Collections.unmodifiableMap(regionSizes);
if (tableSnapshotStore == null) {
tableSnapshotStore = new TableQuotaSnapshotStore(conn, this, immutableRegionSpaceUse);
} else {
@@ -729,7 +728,7 @@ public class QuotaObserverChore extends ScheduledChore {
* Computes the total number of regions in a table.
*/
int getNumRegions(TableName table) throws IOException {
- List<HRegionInfo> regions = this.conn.getAdmin().getTableRegions(table);
+ List<RegionInfo> regions = this.conn.getAdmin().getRegions(table);
if (regions == null) {
return 0;
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/QuotaSnapshotStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/QuotaSnapshotStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/QuotaSnapshotStore.java
index d7c5a00..929eeb6 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/QuotaSnapshotStore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/QuotaSnapshotStore.java
@@ -20,9 +20,10 @@ import java.io.IOException;
import java.util.Map;
import java.util.Map.Entry;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot.SpaceQuotaStatus;
+import org.apache.yetus.audience.InterfaceAudience;
+
import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota;
/**
@@ -68,7 +69,7 @@ public interface QuotaSnapshotStore<T> {
*
* @param subject The filter criteria. Only regions belonging to this parameter will be returned
*/
- Iterable<Entry<HRegionInfo,Long>> filterBySubject(T subject);
+ Iterable<Entry<RegionInfo,Long>> filterBySubject(T subject);
/**
* Persists the current {@link SpaceQuotaSnapshot} for the {@code subject}.
@@ -81,7 +82,7 @@ public interface QuotaSnapshotStore<T> {
/**
* Updates {@code this} with the latest snapshot of filesystem use by region.
*
- * @param regionUsage A map of {@code HRegionInfo} objects to their filesystem usage in bytes
+ * @param regionUsage A map of {@code RegionInfo} objects to their filesystem usage in bytes
*/
- void setRegionUsage(Map<HRegionInfo,Long> regionUsage);
+ void setRegionUsage(Map<RegionInfo,Long> regionUsage);
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/TableQuotaSnapshotStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/TableQuotaSnapshotStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/TableQuotaSnapshotStore.java
index 33f748c..dfaabec 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/TableQuotaSnapshotStore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/TableQuotaSnapshotStore.java
@@ -28,23 +28,23 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellScanner;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.TableName;
-import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot.SpaceQuotaStatus;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hadoop.hbase.shaded.com.google.common.base.Predicate;
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.Iterables;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas;
import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota;
-import org.apache.hadoop.hbase.shaded.com.google.common.base.Predicate;
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.Iterables;
-
/**
* {@link QuotaSnapshotStore} for tables.
*/
@@ -58,9 +58,9 @@ public class TableQuotaSnapshotStore implements QuotaSnapshotStore<TableName> {
private final Connection conn;
private final QuotaObserverChore chore;
- private Map<HRegionInfo,Long> regionUsage;
+ private Map<RegionInfo,Long> regionUsage;
- public TableQuotaSnapshotStore(Connection conn, QuotaObserverChore chore, Map<HRegionInfo,Long> regionUsage) {
+ public TableQuotaSnapshotStore(Connection conn, QuotaObserverChore chore, Map<RegionInfo,Long> regionUsage) {
this.conn = Objects.requireNonNull(conn);
this.chore = Objects.requireNonNull(chore);
this.regionUsage = Objects.requireNonNull(regionUsage);
@@ -94,7 +94,7 @@ public class TableQuotaSnapshotStore implements QuotaSnapshotStore<TableName> {
try {
final long sizeLimitInBytes = spaceQuota.getSoftLimit();
long sum = 0L;
- for (Entry<HRegionInfo,Long> entry : filterBySubject(table)) {
+ for (Entry<RegionInfo,Long> entry : filterBySubject(table)) {
sum += entry.getValue();
}
// Add in the size for any snapshots against this table
@@ -145,12 +145,12 @@ public class TableQuotaSnapshotStore implements QuotaSnapshotStore<TableName> {
}
@Override
- public Iterable<Entry<HRegionInfo,Long>> filterBySubject(TableName table) {
+ public Iterable<Entry<RegionInfo,Long>> filterBySubject(TableName table) {
rlock.lock();
try {
- return Iterables.filter(regionUsage.entrySet(), new Predicate<Entry<HRegionInfo,Long>>() {
+ return Iterables.filter(regionUsage.entrySet(), new Predicate<Entry<RegionInfo,Long>>() {
@Override
- public boolean apply(Entry<HRegionInfo,Long> input) {
+ public boolean apply(Entry<RegionInfo,Long> input) {
return table.equals(input.getKey().getTable());
}
});
@@ -166,7 +166,7 @@ public class TableQuotaSnapshotStore implements QuotaSnapshotStore<TableName> {
}
@Override
- public void setRegionUsage(Map<HRegionInfo,Long> regionUsage) {
+ public void setRegionUsage(Map<RegionInfo,Long> regionUsage) {
wlock.lock();
try {
this.regionUsage = Objects.requireNonNull(regionUsage);
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/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 9552f43..d059977 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
@@ -91,7 +91,6 @@ import org.apache.hadoop.hbase.ExtendedCellBuilderFactory;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HConstants.OperationStatusCode;
import org.apache.hadoop.hbase.HDFSBlocksDistribution;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.NamespaceDescriptor;
@@ -111,6 +110,7 @@ import org.apache.hadoop.hbase.client.IsolationLevel;
import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.client.PackagePrivateFieldAccessor;
import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.RowMutations;
@@ -668,7 +668,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
* the supplied path.
* @param fs is the filesystem.
* @param confParam is global configuration settings.
- * @param regionInfo - HRegionInfo that describes the region
+ * @param regionInfo - RegionInfo that describes the region
* is new), then read them from the supplied path.
* @param htd the table descriptor
* @param rsServices reference to {@link RegionServerServices} or null
@@ -677,7 +677,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
@Deprecated
@VisibleForTesting
public HRegion(final Path tableDir, final WAL wal, final FileSystem fs,
- final Configuration confParam, final HRegionInfo regionInfo,
+ final Configuration confParam, final RegionInfo regionInfo,
final TableDescriptor htd, final RegionServerServices rsServices) {
this(new HRegionFileSystem(confParam, fs, tableDir, regionInfo),
wal, confParam, htd, rsServices);
@@ -880,7 +880,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
// Write HRI to a file in case we need to recover hbase:meta
// Only the primary replica should write .regioninfo
- if (this.getRegionInfo().getReplicaId() == HRegionInfo.DEFAULT_REPLICA_ID) {
+ if (this.getRegionInfo().getReplicaId() == RegionInfo.DEFAULT_REPLICA_ID) {
status.setStatus("Writing region info on filesystem");
fs.checkRegionInfoOnFilesystem();
} else {
@@ -1140,7 +1140,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
* @throws IOException
*/
public static HDFSBlocksDistribution computeHDFSBlocksDistribution(Configuration conf,
- TableDescriptor tableDescriptor, HRegionInfo regionInfo) throws IOException {
+ TableDescriptor tableDescriptor, RegionInfo regionInfo) throws IOException {
Path tablePath = FSUtils.getTableDir(FSUtils.getRootDir(conf), tableDescriptor.getTableName());
return computeHDFSBlocksDistribution(conf, tableDescriptor, regionInfo, tablePath);
}
@@ -1155,7 +1155,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
* @throws IOException
*/
public static HDFSBlocksDistribution computeHDFSBlocksDistribution(Configuration conf,
- TableDescriptor tableDescriptor, HRegionInfo regionInfo, Path tablePath) throws IOException {
+ TableDescriptor tableDescriptor, RegionInfo regionInfo, Path tablePath) throws IOException {
HDFSBlocksDistribution hdfsBlocksDistribution = new HDFSBlocksDistribution();
FileSystem fs = tablePath.getFileSystem(conf);
@@ -1222,7 +1222,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
}
@Override
- public HRegionInfo getRegionInfo() {
+ public RegionInfo getRegionInfo() {
return this.fs.getRegionInfo();
}
@@ -2307,7 +2307,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
}
long modifiedFlushCheckInterval = flushCheckInterval;
if (getRegionInfo().isSystemTable() &&
- getRegionInfo().getReplicaId() == HRegionInfo.DEFAULT_REPLICA_ID) {
+ getRegionInfo().getReplicaId() == RegionInfo.DEFAULT_REPLICA_ID) {
modifiedFlushCheckInterval = SYSTEM_CACHE_FLUSH_INTERVAL;
}
if (modifiedFlushCheckInterval <= 0) { //disabled
@@ -2558,7 +2558,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
/**
* Sync unflushed WAL changes. See HBASE-8208 for details
*/
- private static void doSyncOfUnflushedWALChanges(final WAL wal, final HRegionInfo hri)
+ private static void doSyncOfUnflushedWALChanges(final WAL wal, final RegionInfo hri)
throws IOException {
if (wal == null) {
return;
@@ -5798,7 +5798,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
private final FilterWrapper filter;
@Override
- public HRegionInfo getRegionInfo() {
+ public RegionInfo getRegionInfo() {
return region.getRegionInfo();
}
@@ -6435,13 +6435,13 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
* the supplied path.
* @param fs is the filesystem.
* @param conf is global configuration settings.
- * @param regionInfo - HRegionInfo that describes the region
+ * @param regionInfo - RegionInfo that describes the region
* is new), then read them from the supplied path.
* @param htd the table descriptor
* @return the new instance
*/
static HRegion newHRegion(Path tableDir, WAL wal, FileSystem fs,
- Configuration conf, HRegionInfo regionInfo, final TableDescriptor htd,
+ Configuration conf, RegionInfo regionInfo, final TableDescriptor htd,
RegionServerServices rsServices) {
try {
@SuppressWarnings("unchecked")
@@ -6450,7 +6450,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
Constructor<? extends HRegion> c =
regionClass.getConstructor(Path.class, WAL.class, FileSystem.class,
- Configuration.class, HRegionInfo.class, TableDescriptor.class,
+ Configuration.class, RegionInfo.class, TableDescriptor.class,
RegionServerServices.class);
return c.newInstance(tableDir, wal, fs, conf, regionInfo, htd, rsServices);
@@ -6470,7 +6470,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
* @return new HRegion
* @throws IOException
*/
- public static HRegion createHRegion(final HRegionInfo info, final Path rootDir,
+ public static HRegion createHRegion(final RegionInfo info, final Path rootDir,
final Configuration conf, final TableDescriptor hTableDescriptor,
final WAL wal, final boolean initialize)
throws IOException {
@@ -6485,7 +6485,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
return region;
}
- public static HRegion createHRegion(final HRegionInfo info, final Path rootDir,
+ public static HRegion createHRegion(final RegionInfo info, final Path rootDir,
final Configuration conf,
final TableDescriptor hTableDescriptor,
final WAL wal)
@@ -6505,7 +6505,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
*
* @throws IOException
*/
- public static HRegion openHRegion(final HRegionInfo info,
+ public static HRegion openHRegion(final RegionInfo info,
final TableDescriptor htd, final WAL wal,
final Configuration conf)
throws IOException {
@@ -6527,7 +6527,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
*
* @throws IOException
*/
- public static HRegion openHRegion(final HRegionInfo info,
+ public static HRegion openHRegion(final RegionInfo info,
final TableDescriptor htd, final WAL wal, final Configuration conf,
final RegionServerServices rsServices,
final CancelableProgressable reporter)
@@ -6548,7 +6548,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
* @return new HRegion
* @throws IOException
*/
- public static HRegion openHRegion(Path rootDir, final HRegionInfo info,
+ public static HRegion openHRegion(Path rootDir, final RegionInfo info,
final TableDescriptor htd, final WAL wal, final Configuration conf)
throws IOException {
return openHRegion(rootDir, info, htd, wal, conf, null, null);
@@ -6569,7 +6569,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
* @return new HRegion
* @throws IOException
*/
- public static HRegion openHRegion(final Path rootDir, final HRegionInfo info,
+ public static HRegion openHRegion(final Path rootDir, final RegionInfo info,
final TableDescriptor htd, final WAL wal, final Configuration conf,
final RegionServerServices rsServices,
final CancelableProgressable reporter)
@@ -6599,7 +6599,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
* @throws IOException
*/
public static HRegion openHRegion(final Configuration conf, final FileSystem fs,
- final Path rootDir, final HRegionInfo info, final TableDescriptor htd, final WAL wal)
+ final Path rootDir, final RegionInfo info, final TableDescriptor htd, final WAL wal)
throws IOException {
return openHRegion(conf, fs, rootDir, info, htd, wal, null, null);
}
@@ -6621,7 +6621,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
* @throws IOException
*/
public static HRegion openHRegion(final Configuration conf, final FileSystem fs,
- final Path rootDir, final HRegionInfo info, final TableDescriptor htd, final WAL wal,
+ final Path rootDir, final RegionInfo info, final TableDescriptor htd, final WAL wal,
final RegionServerServices rsServices, final CancelableProgressable reporter)
throws IOException {
Path tableDir = FSUtils.getTableDir(rootDir, info.getTable());
@@ -6645,7 +6645,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
* @throws IOException
*/
public static HRegion openHRegion(final Configuration conf, final FileSystem fs,
- final Path rootDir, final Path tableDir, final HRegionInfo info, final TableDescriptor htd,
+ final Path rootDir, final Path tableDir, final RegionInfo info, final TableDescriptor htd,
final WAL wal, final RegionServerServices rsServices,
final CancelableProgressable reporter)
throws IOException {
@@ -6709,7 +6709,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
return this;
}
- public static void warmupHRegion(final HRegionInfo info,
+ public static void warmupHRegion(final RegionInfo info,
final TableDescriptor htd, final WAL wal, final Configuration conf,
final RegionServerServices rsServices,
final CancelableProgressable reporter)
@@ -6760,7 +6760,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
* @param hri Spec. for daughter region to open.
* @throws IOException
*/
- public HRegion createDaughterRegionFromSplits(final HRegionInfo hri) throws IOException {
+ public HRegion createDaughterRegionFromSplits(final RegionInfo hri) throws IOException {
// Move the files from the temporary .splits to the final /table/region directory
fs.commitDaughterRegion(hri);
@@ -6779,7 +6779,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
* @return merged HRegion
* @throws IOException
*/
- HRegion createMergedRegionFromMerges(final HRegionInfo mergedRegionInfo,
+ HRegion createMergedRegionFromMerges(final RegionInfo mergedRegionInfo,
final HRegion region_b) throws IOException {
HRegion r = HRegion.newHRegion(this.fs.getTableDir(), this.getWAL(),
fs.getFileSystem(), this.getBaseConf(), mergedRegionInfo,
@@ -6812,33 +6812,33 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
* Computes the Path of the HRegion
*
* @param rootdir qualified path of HBase root directory
- * @param info HRegionInfo for the region
+ * @param info RegionInfo for the region
* @return qualified path of region directory
* @deprecated For tests only; to be removed.
*/
@Deprecated
@VisibleForTesting
- public static Path getRegionDir(final Path rootdir, final HRegionInfo info) {
+ public static Path getRegionDir(final Path rootdir, final RegionInfo info) {
return new Path(
FSUtils.getTableDir(rootdir, info.getTable()), info.getEncodedName());
}
/**
* Determines if the specified row is within the row range specified by the
- * specified HRegionInfo
+ * specified RegionInfo
*
- * @param info HRegionInfo that specifies the row range
+ * @param info RegionInfo that specifies the row range
* @param row row to be checked
- * @return true if the row is within the range specified by the HRegionInfo
+ * @return true if the row is within the range specified by the RegionInfo
*/
- public static boolean rowIsInRange(HRegionInfo info, final byte [] row) {
+ public static boolean rowIsInRange(RegionInfo info, final byte [] row) {
return ((info.getStartKey().length == 0) ||
(Bytes.compareTo(info.getStartKey(), row) <= 0)) &&
((info.getEndKey().length == 0) ||
(Bytes.compareTo(info.getEndKey(), row) > 0));
}
- public static boolean rowIsInRange(HRegionInfo info, final byte [] row, final int offset,
+ public static boolean rowIsInRange(RegionInfo info, final byte [] row, final int offset,
final short length) {
return ((info.getStartKey().length == 0) ||
(Bytes.compareTo(info.getStartKey(), 0, info.getStartKey().length,
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java
index 3f42466..fcd6c57 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java
@@ -42,9 +42,9 @@ import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.backup.HFileArchiver;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.fs.HFileSystem;
import org.apache.hadoop.hbase.io.Reference;
@@ -79,9 +79,9 @@ public class HRegionFileSystem {
/** Temporary subdirectory of the region directory used for compaction output. */
private static final String REGION_TEMP_DIR = ".tmp";
- private final HRegionInfo regionInfo;
+ private final RegionInfo regionInfo;
//regionInfo for interacting with FS (getting encodedName, etc)
- private final HRegionInfo regionInfoForFs;
+ private final RegionInfo regionInfoForFs;
private final Configuration conf;
private final Path tableDir;
private final FileSystem fs;
@@ -100,10 +100,10 @@ public class HRegionFileSystem {
* @param conf the {@link Configuration} to use
* @param fs {@link FileSystem} that contains the region
* @param tableDir {@link Path} to where the table is being stored
- * @param regionInfo {@link HRegionInfo} for region
+ * @param regionInfo {@link RegionInfo} for region
*/
HRegionFileSystem(final Configuration conf, final FileSystem fs, final Path tableDir,
- final HRegionInfo regionInfo) {
+ final RegionInfo regionInfo) {
this.fs = fs;
this.conf = conf;
this.tableDir = tableDir;
@@ -120,12 +120,12 @@ public class HRegionFileSystem {
return this.fs;
}
- /** @return the {@link HRegionInfo} that describe this on-disk region view */
- public HRegionInfo getRegionInfo() {
+ /** @return the {@link RegionInfo} that describe this on-disk region view */
+ public RegionInfo getRegionInfo() {
return this.regionInfo;
}
- public HRegionInfo getRegionInfoForFS() {
+ public RegionInfo getRegionInfoForFS() {
return this.regionInfoForFs;
}
@@ -547,7 +547,7 @@ public class HRegionFileSystem {
return new Path(getRegionDir(), REGION_SPLITS_DIR);
}
- public Path getSplitsDir(final HRegionInfo hri) {
+ public Path getSplitsDir(final RegionInfo hri) {
return new Path(getSplitsDir(), hri.getEncodedName());
}
@@ -589,10 +589,10 @@ public class HRegionFileSystem {
/**
* Remove daughter region
- * @param regionInfo daughter {@link HRegionInfo}
+ * @param regionInfo daughter {@link RegionInfo}
* @throws IOException
*/
- void cleanupDaughterRegion(final HRegionInfo regionInfo) throws IOException {
+ void cleanupDaughterRegion(final RegionInfo regionInfo) throws IOException {
Path regionDir = new Path(this.tableDir, regionInfo.getEncodedName());
if (this.fs.exists(regionDir) && !deleteDir(regionDir)) {
throw new IOException("Failed delete of " + regionDir);
@@ -603,10 +603,10 @@ public class HRegionFileSystem {
* Commit a daughter region, moving it from the split temporary directory
* to the proper location in the filesystem.
*
- * @param regionInfo daughter {@link org.apache.hadoop.hbase.HRegionInfo}
+ * @param regionInfo daughter {@link org.apache.hadoop.hbase.client.RegionInfo}
* @throws IOException
*/
- public Path commitDaughterRegion(final HRegionInfo regionInfo)
+ public Path commitDaughterRegion(final RegionInfo regionInfo)
throws IOException {
Path regionDir = new Path(this.tableDir, regionInfo.getEncodedName());
Path daughterTmpDir = this.getSplitsDir(regionInfo);
@@ -648,7 +648,7 @@ public class HRegionFileSystem {
/**
* Write out a split reference. Package local so it doesnt leak out of
* regionserver.
- * @param hri {@link HRegionInfo} of the destination
+ * @param hri {@link RegionInfo} of the destination
* @param familyName Column Family Name
* @param f File to split.
* @param splitRow Split Row
@@ -657,7 +657,7 @@ public class HRegionFileSystem {
* @return Path to created reference.
* @throws IOException
*/
- public Path splitStoreFile(HRegionInfo hri, String familyName, HStoreFile f, byte[] splitRow,
+ public Path splitStoreFile(RegionInfo hri, String familyName, HStoreFile f, byte[] splitRow,
boolean top, RegionSplitPolicy splitPolicy) throws IOException {
if (splitPolicy == null || !splitPolicy.skipStoreFileRangeCheck(familyName)) {
// Check whether the split row lies in the range of the store file
@@ -715,7 +715,7 @@ public class HRegionFileSystem {
return new Path(getRegionDir(), REGION_MERGES_DIR);
}
- Path getMergesDir(final HRegionInfo hri) {
+ Path getMergesDir(final RegionInfo hri) {
return new Path(getMergesDir(), hri.getEncodedName());
}
@@ -728,10 +728,10 @@ public class HRegionFileSystem {
/**
* Remove merged region
- * @param mergedRegion {@link HRegionInfo}
+ * @param mergedRegion {@link RegionInfo}
* @throws IOException
*/
- public void cleanupMergedRegion(final HRegionInfo mergedRegion) throws IOException {
+ public void cleanupMergedRegion(final RegionInfo mergedRegion) throws IOException {
Path regionDir = new Path(this.tableDir, mergedRegion.getEncodedName());
if (this.fs.exists(regionDir) && !this.fs.delete(regionDir, true)) {
throw new IOException("Failed delete of " + regionDir);
@@ -769,14 +769,14 @@ public class HRegionFileSystem {
/**
* Write out a merge reference under the given merges directory. Package local
* so it doesnt leak out of regionserver.
- * @param mergedRegion {@link HRegionInfo} of the merged region
+ * @param mergedRegion {@link RegionInfo} of the merged region
* @param familyName Column Family Name
* @param f File to create reference.
* @param mergedDir
* @return Path to created reference.
* @throws IOException
*/
- public Path mergeStoreFile(HRegionInfo mergedRegion, String familyName, HStoreFile f,
+ public Path mergeStoreFile(RegionInfo mergedRegion, String familyName, HStoreFile f,
Path mergedDir) throws IOException {
Path referenceDir = new Path(new Path(mergedDir,
mergedRegion.getEncodedName()), familyName);
@@ -797,10 +797,10 @@ public class HRegionFileSystem {
/**
* Commit a merged region, moving it from the merges temporary directory to
* the proper location in the filesystem.
- * @param mergedRegionInfo merged region {@link HRegionInfo}
+ * @param mergedRegionInfo merged region {@link RegionInfo}
* @throws IOException
*/
- public void commitMergedRegion(final HRegionInfo mergedRegionInfo) throws IOException {
+ public void commitMergedRegion(final RegionInfo mergedRegionInfo) throws IOException {
Path regionDir = new Path(this.tableDir, mergedRegionInfo.getEncodedName());
Path mergedRegionTmpDir = this.getMergesDir(mergedRegionInfo);
// Move the tmp dir in the expected location
@@ -829,22 +829,22 @@ public class HRegionFileSystem {
* @return Content of the file we write out to the filesystem under a region
* @throws IOException
*/
- private static byte[] getRegionInfoFileContent(final HRegionInfo hri) throws IOException {
- return hri.toDelimitedByteArray();
+ private static byte[] getRegionInfoFileContent(final RegionInfo hri) throws IOException {
+ return RegionInfo.toDelimitedByteArray(hri);
}
/**
- * Create a {@link HRegionInfo} from the serialized version on-disk.
+ * Create a {@link RegionInfo} from the serialized version on-disk.
* @param fs {@link FileSystem} that contains the Region Info file
* @param regionDir {@link Path} to the Region Directory that contains the Info file
- * @return An {@link HRegionInfo} instance gotten from the Region Info file.
+ * @return An {@link RegionInfo} instance gotten from the Region Info file.
* @throws IOException if an error occurred during file open/read operation.
*/
- public static HRegionInfo loadRegionInfoFileContent(final FileSystem fs, final Path regionDir)
+ public static RegionInfo loadRegionInfoFileContent(final FileSystem fs, final Path regionDir)
throws IOException {
FSDataInputStream in = fs.open(new Path(regionDir, REGION_INFO_FILE));
try {
- return HRegionInfo.parseFrom(in);
+ return RegionInfo.parseFrom(in);
} finally {
in.close();
}
@@ -921,7 +921,7 @@ public class HRegionFileSystem {
/**
* Write out an info file under the region directory. Useful recovering mangled regions.
- * @param regionInfoContent serialized version of the {@link HRegionInfo}
+ * @param regionInfoContent serialized version of the {@link RegionInfo}
* @param useTempDir indicate whether or not using the region .tmp dir for a safer file creation.
*/
private void writeRegionInfoOnFilesystem(final byte[] regionInfoContent,
@@ -962,11 +962,11 @@ public class HRegionFileSystem {
* @param conf the {@link Configuration} to use
* @param fs {@link FileSystem} from which to add the region
* @param tableDir {@link Path} to where the table is being stored
- * @param regionInfo {@link HRegionInfo} for region to be added
+ * @param regionInfo {@link RegionInfo} for region to be added
* @throws IOException if the region creation fails due to a FileSystem exception.
*/
public static HRegionFileSystem createRegionOnFileSystem(final Configuration conf,
- final FileSystem fs, final Path tableDir, final HRegionInfo regionInfo) throws IOException {
+ final FileSystem fs, final Path tableDir, final RegionInfo regionInfo) throws IOException {
HRegionFileSystem regionFs = new HRegionFileSystem(conf, fs, tableDir, regionInfo);
Path regionDir = regionFs.getRegionDir();
@@ -983,7 +983,7 @@ public class HRegionFileSystem {
// Write HRI to a file in case we need to recover hbase:meta
// Only primary replicas should write region info
- if (regionInfo.getReplicaId() == HRegionInfo.DEFAULT_REPLICA_ID) {
+ if (regionInfo.getReplicaId() == RegionInfo.DEFAULT_REPLICA_ID) {
regionFs.writeRegionInfoOnFilesystem(false);
} else {
if (LOG.isDebugEnabled())
@@ -997,12 +997,12 @@ public class HRegionFileSystem {
* @param conf the {@link Configuration} to use
* @param fs {@link FileSystem} from which to add the region
* @param tableDir {@link Path} to where the table is being stored
- * @param regionInfo {@link HRegionInfo} for region to be added
+ * @param regionInfo {@link RegionInfo} for region to be added
* @param readOnly True if you don't want to edit the region data
* @throws IOException if the region creation fails due to a FileSystem exception.
*/
public static HRegionFileSystem openRegionFromFileSystem(final Configuration conf,
- final FileSystem fs, final Path tableDir, final HRegionInfo regionInfo, boolean readOnly)
+ final FileSystem fs, final Path tableDir, final RegionInfo regionInfo, boolean readOnly)
throws IOException {
HRegionFileSystem regionFs = new HRegionFileSystem(conf, fs, tableDir, regionInfo);
Path regionDir = regionFs.getRegionDir();
@@ -1020,7 +1020,7 @@ public class HRegionFileSystem {
// If it doesn't exists, Write HRI to a file, in case we need to recover hbase:meta
// Only create HRI if we are the default replica
- if (regionInfo.getReplicaId() == HRegionInfo.DEFAULT_REPLICA_ID) {
+ if (regionInfo.getReplicaId() == RegionInfo.DEFAULT_REPLICA_ID) {
regionFs.checkRegionInfoOnFilesystem();
} else {
if (LOG.isDebugEnabled()) {
@@ -1037,11 +1037,11 @@ public class HRegionFileSystem {
* @param conf the {@link Configuration} to use
* @param fs {@link FileSystem} from which to remove the region
* @param tableDir {@link Path} to where the table is being stored
- * @param regionInfo {@link HRegionInfo} for region to be deleted
+ * @param regionInfo {@link RegionInfo} for region to be deleted
* @throws IOException if the request cannot be completed
*/
public static void deleteRegionFromFileSystem(final Configuration conf,
- final FileSystem fs, final Path tableDir, final HRegionInfo regionInfo) throws IOException {
+ final FileSystem fs, final Path tableDir, final RegionInfo regionInfo) throws IOException {
HRegionFileSystem regionFs = new HRegionFileSystem(conf, fs, tableDir, regionInfo);
Path regionDir = regionFs.getRegionDir();
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/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 53f4445..5ef0358 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
@@ -18,6 +18,9 @@
*/
package org.apache.hadoop.hbase.regionserver;
+import javax.management.MalformedObjectNameException;
+import javax.management.ObjectName;
+import javax.servlet.http.HttpServlet;
import java.io.IOException;
import java.io.InterruptedIOException;
import java.lang.Thread.UncaughtExceptionHandler;
@@ -51,10 +54,6 @@ import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import java.util.function.Function;
-import javax.management.MalformedObjectNameException;
-import javax.management.ObjectName;
-import javax.servlet.http.HttpServlet;
-
import org.apache.commons.lang3.RandomUtils;
import org.apache.commons.lang3.SystemUtils;
import org.apache.commons.logging.Log;
@@ -71,7 +70,6 @@ import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HealthCheckChore;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.NotServingRegionException;
@@ -87,6 +85,8 @@ import org.apache.hadoop.hbase.client.ClusterConnection;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionUtils;
import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
import org.apache.hadoop.hbase.client.RpcRetryingCallerFactory;
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
import org.apache.hadoop.hbase.client.locking.EntityLock;
@@ -1229,7 +1229,7 @@ public class HRegionServer extends HasThread implements
}
private boolean containsMetaTableRegions() {
- return onlineRegions.containsKey(HRegionInfo.FIRST_META_REGIONINFO.getEncodedName());
+ return onlineRegions.containsKey(RegionInfoBuilder.FIRST_META_REGIONINFO.getEncodedName());
}
private boolean areAllUserRegionsOffline() {
@@ -1292,7 +1292,7 @@ public class HRegionServer extends HasThread implements
* @param onlineRegionSizes A map of region info to size in bytes
* @return false if FileSystemUtilizationChore should pause reporting to master. true otherwise
*/
- public boolean reportRegionSizesForQuotas(final Map<HRegionInfo, Long> onlineRegionSizes) {
+ public boolean reportRegionSizesForQuotas(final Map<RegionInfo, Long> onlineRegionSizes) {
RegionServerStatusService.BlockingInterface rss = rssStub;
if (rss == null) {
// the current server could be stopping.
@@ -1336,25 +1336,25 @@ public class HRegionServer extends HasThread implements
* @param regionSizes Map of region info to size in bytes.
* @return The corresponding protocol buffer message.
*/
- RegionSpaceUseReportRequest buildRegionSpaceUseReportRequest(Map<HRegionInfo,Long> regionSizes) {
+ RegionSpaceUseReportRequest buildRegionSpaceUseReportRequest(Map<RegionInfo,Long> regionSizes) {
RegionSpaceUseReportRequest.Builder request = RegionSpaceUseReportRequest.newBuilder();
- for (Entry<HRegionInfo, Long> entry : Objects.requireNonNull(regionSizes).entrySet()) {
+ for (Entry<RegionInfo, Long> entry : Objects.requireNonNull(regionSizes).entrySet()) {
request.addSpaceUse(convertRegionSize(entry.getKey(), entry.getValue()));
}
return request.build();
}
/**
- * Converts a pair of {@link HRegionInfo} and {@code long} into a {@link RegionSpaceUse}
+ * Converts a pair of {@link RegionInfo} and {@code long} into a {@link RegionSpaceUse}
* protobuf message.
*
- * @param regionInfo The HRegionInfo
+ * @param regionInfo The RegionInfo
* @param sizeInBytes The size in bytes of the Region
* @return The protocol buffer
*/
- RegionSpaceUse convertRegionSize(HRegionInfo regionInfo, Long sizeInBytes) {
+ RegionSpaceUse convertRegionSize(RegionInfo regionInfo, Long sizeInBytes) {
return RegionSpaceUse.newBuilder()
- .setRegionInfo(HRegionInfo.convert(Objects.requireNonNull(regionInfo)))
+ .setRegionInfo(ProtobufUtil.toRegionInfo(Objects.requireNonNull(regionInfo)))
.setRegionSize(Objects.requireNonNull(sizeInBytes))
.build();
}
@@ -1470,7 +1470,7 @@ public class HRegionServer extends HasThread implements
// protect against the case where an open comes in after we start the
// iterator of onlineRegions to close all user regions.
for (Map.Entry<String, Region> e : this.onlineRegions.entrySet()) {
- HRegionInfo hri = e.getValue().getRegionInfo();
+ RegionInfo hri = e.getValue().getRegionInfo();
if (!this.regionsInTransitionInRS.containsKey(hri.getEncodedNameAsBytes())
&& !closedRegions.contains(hri.getEncodedName())) {
closedRegions.add(hri.getEncodedName());
@@ -2075,11 +2075,11 @@ public class HRegionServer extends HasThread implements
}
@Override
- public WAL getWAL(HRegionInfo regionInfo) throws IOException {
+ public WAL getWAL(RegionInfo regionInfo) throws IOException {
WAL wal;
// _ROOT_ and hbase:meta regions have separate WAL.
if (regionInfo != null && regionInfo.isMetaTable()
- && regionInfo.getReplicaId() == HRegionInfo.DEFAULT_REPLICA_ID) {
+ && regionInfo.getReplicaId() == RegionInfo.DEFAULT_REPLICA_ID) {
wal = walFactory.getMetaWAL(regionInfo.getEncodedNameAsBytes());
} else if (regionInfo == null) {
wal = walFactory.getWAL(UNSPECIFIED_REGION, null);
@@ -2198,13 +2198,13 @@ public class HRegionServer extends HasThread implements
}
@Override
- public boolean reportRegionStateTransition(TransitionCode code, HRegionInfo... hris) {
+ public boolean reportRegionStateTransition(TransitionCode code, RegionInfo... hris) {
return reportRegionStateTransition(code, HConstants.NO_SEQNUM, hris);
}
@Override
public boolean reportRegionStateTransition(
- TransitionCode code, long openSeqNum, HRegionInfo... hris) {
+ TransitionCode code, long openSeqNum, RegionInfo... hris) {
return reportRegionStateTransition(
new RegionStateTransitionContext(code, HConstants.NO_SEQNUM, -1, hris));
}
@@ -2214,7 +2214,7 @@ public class HRegionServer extends HasThread implements
TransitionCode code = context.getCode();
long openSeqNum = context.getOpenSeqNum();
long masterSystemTime = context.getMasterSystemTime();
- HRegionInfo[] hris = context.getHris();
+ RegionInfo[] hris = context.getHris();
if (TEST_SKIP_REPORTING_TRANSITION) {
// This is for testing only in case there is no master
@@ -2250,8 +2250,8 @@ public class HRegionServer extends HasThread implements
if (code == TransitionCode.OPENED && openSeqNum >= 0) {
transition.setOpenSeqNum(openSeqNum);
}
- for (HRegionInfo hri: hris) {
- transition.addRegionInfo(HRegionInfo.convert(hri));
+ for (RegionInfo hri: hris) {
+ transition.addRegionInfo(ProtobufUtil.toRegionInfo(hri));
}
ReportRegionStateTransitionRequest request = builder.build();
int tries = 0;
@@ -2668,7 +2668,7 @@ public class HRegionServer extends HasThread implements
this.lock.writeLock().lock();
try {
for (Map.Entry<String, Region> e: onlineRegions.entrySet()) {
- HRegionInfo hri = e.getValue().getRegionInfo();
+ RegionInfo hri = e.getValue().getRegionInfo();
if (hri.isMetaRegion()) {
meta = e.getValue();
}
@@ -2802,8 +2802,8 @@ public class HRegionServer extends HasThread implements
* calculate which regions are most loaded. (Right now, we're just grabbing
* the first N regions being served regardless of load.)
*/
- protected HRegionInfo[] getMostLoadedRegions() {
- ArrayList<HRegionInfo> regions = new ArrayList<>();
+ protected RegionInfo[] getMostLoadedRegions() {
+ ArrayList<RegionInfo> regions = new ArrayList<>();
for (Region r : onlineRegions.values()) {
if (!r.isAvailable()) {
continue;
@@ -2814,7 +2814,7 @@ public class HRegionServer extends HasThread implements
break;
}
}
- return regions.toArray(new HRegionInfo[regions.size()]);
+ return regions.toArray(new RegionInfo[regions.size()]);
}
@Override
@@ -3007,7 +3007,7 @@ public class HRegionServer extends HasThread implements
List<Region> tableRegions = new ArrayList<>();
synchronized (this.onlineRegions) {
for (Region region: this.onlineRegions.values()) {
- HRegionInfo regionInfo = region.getRegionInfo();
+ RegionInfo regionInfo = region.getRegionInfo();
if(regionInfo.getTable().equals(tableName)) {
tableRegions.add(region);
}
@@ -3070,7 +3070,7 @@ public class HRegionServer extends HasThread implements
* Try to close the region, logs a warning on failure but continues.
* @param region Region to close
*/
- private void closeRegionIgnoreErrors(HRegionInfo region, final boolean abort) {
+ private void closeRegionIgnoreErrors(RegionInfo region, final boolean abort) {
try {
if (!closeRegion(region.getEncodedName(), abort, null)) {
LOG.warn("Failed to close " + region.getRegionNameAsString() +
@@ -3150,7 +3150,7 @@ public class HRegionServer extends HasThread implements
}
CloseRegionHandler crh;
- final HRegionInfo hri = actualRegion.getRegionInfo();
+ final RegionInfo hri = actualRegion.getRegionInfo();
if (hri.isMetaRegion()) {
crh = new CloseMetaHandler(this, this, hri, abort);
} else {
@@ -3223,7 +3223,7 @@ public class HRegionServer extends HasThread implements
* named region is not member of the online regions.
*/
public Region getOnlineRegion(final byte[] regionName) {
- String encodedRegionName = HRegionInfo.encodeRegionName(regionName);
+ String encodedRegionName = RegionInfo.encodeRegionName(regionName);
return this.onlineRegions.get(encodedRegionName);
}
@@ -3263,7 +3263,7 @@ public class HRegionServer extends HasThread implements
*/
protected Region getRegion(final byte[] regionName)
throws NotServingRegionException {
- String encodedRegionName = HRegionInfo.encodeRegionName(regionName);
+ String encodedRegionName = RegionInfo.encodeRegionName(regionName);
return getRegionByEncodedName(regionName, encodedRegionName);
}
@@ -3537,7 +3537,7 @@ public class HRegionServer extends HasThread implements
return;
}
- HRegionInfo regionInfo = r.getRegionInfo();
+ RegionInfo regionInfo = r.getRegionInfo();
ZooKeeperWatcher zkw = getZooKeeper();
String previousRSName = this.getLastFailedRSFromZK(regionInfo.getEncodedName());
Map<byte[], Long> maxSeqIdInStores = r.getMaxStoreSeqId();
@@ -3747,7 +3747,7 @@ public class HRegionServer extends HasThread implements
}
@Override
- public EntityLock regionLock(List<HRegionInfo> regionInfos, String description,
+ public EntityLock regionLock(List<RegionInfo> regionInfos, String description,
Abortable abort) throws IOException {
return new LockServiceClient(conf, lockStub, clusterConnection.getNonceGenerator())
.regionLock(regionInfos, description, abort);
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
index 8c55c42..f232ddd 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
@@ -59,11 +59,11 @@ import org.apache.hadoop.hbase.CellComparator;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.CompoundConfiguration;
import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.MemoryCompactionPolicy;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.backup.FailedArchiveException;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.conf.ConfigurationManager;
import org.apache.hadoop.hbase.conf.PropagatingConfigurationObserver;
@@ -467,13 +467,13 @@ public class HStore implements Store, HeapSize, StoreConfigInformation, Propagat
/**
* @param tabledir {@link Path} to where the table is being stored
- * @param hri {@link HRegionInfo} for the region.
+ * @param hri {@link RegionInfo} for the region.
* @param family {@link ColumnFamilyDescriptor} describing the column family
* @return Path to family/Store home directory.
*/
@Deprecated
public static Path getStoreHomedir(final Path tabledir,
- final HRegionInfo hri, final byte[] family) {
+ final RegionInfo hri, final byte[] family) {
return getStoreHomedir(tabledir, hri.getEncodedName(), family);
}
@@ -1454,7 +1454,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation, Propagat
filesCompacted.stream().map(HStoreFile::getPath).collect(Collectors.toList());
List<Path> outputPaths =
newFiles.stream().map(HStoreFile::getPath).collect(Collectors.toList());
- HRegionInfo info = this.region.getRegionInfo();
+ RegionInfo info = this.region.getRegionInfo();
CompactionDescriptor compactionDescriptor = ProtobufUtil.toCompactionDescriptor(info,
family.getName(), inputPaths, outputPaths, fs.getStoreDir(getColumnFamilyDescriptor().getNameAsString()));
// Fix reaching into Region to get the maxWaitForSeqId.
@@ -2142,7 +2142,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation, Propagat
}
@Override
- public HRegionInfo getRegionInfo() {
+ public RegionInfo getRegionInfo() {
return this.fs.getRegionInfo();
}
@@ -2452,7 +2452,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation, Propagat
@Override
public boolean isPrimaryReplicaStore() {
- return getRegionInfo().getReplicaId() == HRegionInfo.DEFAULT_REPLICA_ID;
+ return getRegionInfo().getReplicaId() == RegionInfo.DEFAULT_REPLICA_ID;
}
/**
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapperImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapperImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapperImpl.java
index efdf712..24e1ae5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapperImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapperImpl.java
@@ -30,7 +30,7 @@ import java.util.concurrent.TimeUnit;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
-import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.metrics2.MetricsExecutor;
@@ -88,7 +88,7 @@ public class MetricsRegionWrapperImpl implements MetricsRegionWrapper, Closeable
@Override
public String getRegionName() {
- HRegionInfo regionInfo = this.region.getRegionInfo();
+ RegionInfo regionInfo = this.region.getRegionInfo();
if (regionInfo == null) {
return UNKNOWN;
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
index e71c1eb..dba314d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
@@ -18,6 +18,31 @@
*/
package org.apache.hadoop.hbase.regionserver;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InterruptedIOException;
+import java.net.BindException;
+import java.net.InetSocketAddress;
+import java.net.UnknownHostException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.NavigableMap;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.LongAdder;
+
import org.apache.commons.lang3.mutable.MutableObject;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@@ -33,13 +58,11 @@ import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.DroppedSnapshotException;
import org.apache.hadoop.hbase.HBaseIOException;
import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.MultiActionResultTooLarge;
import org.apache.hadoop.hbase.NotServingRegionException;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.UnknownScannerException;
-import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.Append;
import org.apache.hadoop.hbase.client.ConnectionUtils;
import org.apache.hadoop.hbase.client.Delete;
@@ -48,6 +71,7 @@ import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Increment;
import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.RowMutations;
@@ -88,9 +112,22 @@ import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTrack
import org.apache.hadoop.hbase.regionserver.handler.OpenMetaHandler;
import org.apache.hadoop.hbase.regionserver.handler.OpenPriorityRegionHandler;
import org.apache.hadoop.hbase.regionserver.handler.OpenRegionHandler;
-import org.apache.hadoop.hbase.wal.WALEdit;
import org.apache.hadoop.hbase.security.Superusers;
import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.DNS;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil;
+import org.apache.hadoop.hbase.util.Strings;
+import org.apache.hadoop.hbase.wal.WAL;
+import org.apache.hadoop.hbase.wal.WALEdit;
+import org.apache.hadoop.hbase.wal.WALKey;
+import org.apache.hadoop.hbase.wal.WALSplitter;
+import org.apache.hadoop.hbase.zookeeper.ZKSplitLog;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.zookeeper.KeeperException;
+
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.hbase.shaded.com.google.common.cache.Cache;
import org.apache.hadoop.hbase.shaded.com.google.common.cache.CacheBuilder;
@@ -172,7 +209,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionLoad;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPair;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameInt64Pair;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MapReduceProtos.ScanMetrics;
@@ -184,42 +220,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.BulkLoadDescr
import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.CompactionDescriptor;
import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor;
import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.RegionEventDescriptor;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.DNS;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.hadoop.hbase.util.Pair;
-import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil;
-import org.apache.hadoop.hbase.util.Strings;
-import org.apache.hadoop.hbase.wal.WAL;
-import org.apache.hadoop.hbase.wal.WALKey;
-import org.apache.hadoop.hbase.wal.WALSplitter;
-import org.apache.hadoop.hbase.zookeeper.ZKSplitLog;
-import org.apache.zookeeper.KeeperException;
-
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.io.InterruptedIOException;
-import java.net.BindException;
-import java.net.InetSocketAddress;
-import java.net.UnknownHostException;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.NavigableMap;
-import java.util.Set;
-import java.util.TreeSet;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.atomic.LongAdder;
/**
* Implements the regionserver RPC services.
@@ -1370,7 +1370,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
switch (type) {
case REGION_NAME:
byte[] regionName = value.toByteArray();
- String encodedRegionName = HRegionInfo.encodeRegionName(regionName);
+ String encodedRegionName = RegionInfo.encodeRegionName(regionName);
return regionServer.getRegionByEncodedName(regionName, encodedRegionName);
case ENCODED_REGION_NAME:
return regionServer.getRegionByEncodedName(value.toStringUtf8());
@@ -1653,11 +1653,11 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
checkOpen();
requestCount.increment();
Map<String, Region> onlineRegions = regionServer.onlineRegions;
- List<HRegionInfo> list = new ArrayList<>(onlineRegions.size());
+ List<RegionInfo> list = new ArrayList<>(onlineRegions.size());
for (Region region: onlineRegions.values()) {
list.add(region.getRegionInfo());
}
- Collections.sort(list);
+ Collections.sort(list, RegionInfo.COMPARATOR);
return ResponseConverter.buildGetOnlineRegionResponse(list);
} catch (IOException ie) {
throw new ServiceException(ie);
@@ -1672,7 +1672,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
checkOpen();
requestCount.increment();
Region region = getRegion(request.getRegion());
- HRegionInfo info = region.getRegionInfo();
+ RegionInfo info = region.getRegionInfo();
byte[] bestSplitRow = null;
if (request.hasBestSplitRow() && request.getBestSplitRow()) {
HRegion r = (HRegion) region;
@@ -1688,7 +1688,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
r.clearSplit();
}
GetRegionInfoResponse.Builder builder = GetRegionInfoResponse.newBuilder();
- builder.setRegionInfo(HRegionInfo.convert(info));
+ builder.setRegionInfo(ProtobufUtil.toRegionInfo(info));
if (request.hasCompactionState() && request.getCompactionState()) {
builder.setCompactionState(region.getCompactionState());
}
@@ -1866,7 +1866,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
} catch (IOException ie) {
TableName tableName = null;
if (regionCount == 1) {
- RegionInfo ri = request.getOpenInfo(0).getRegion();
+ org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo ri = request.getOpenInfo(0).getRegion();
if (ri != null) {
tableName = ProtobufUtil.toTableName(ri.getTableName());
}
@@ -1897,7 +1897,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
long masterSystemTime = request.hasMasterSystemTime() ? request.getMasterSystemTime() : -1;
for (RegionOpenInfo regionOpenInfo : request.getOpenInfoList()) {
- final HRegionInfo region = HRegionInfo.convert(regionOpenInfo.getRegion());
+ final RegionInfo region = ProtobufUtil.toRegionInfo(regionOpenInfo.getRegion());
TableDescriptor htd;
try {
String encodedName = region.getEncodedName();
@@ -2018,8 +2018,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
public WarmupRegionResponse warmupRegion(final RpcController controller,
final WarmupRegionRequest request) throws ServiceException {
- RegionInfo regionInfo = request.getRegionInfo();
- final HRegionInfo region = HRegionInfo.convert(regionInfo);
+ final RegionInfo region = ProtobufUtil.toRegionInfo(request.getRegionInfo());
TableDescriptor htd;
WarmupRegionResponse response = WarmupRegionResponse.getDefaultInstance();
@@ -2237,7 +2236,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
List<UpdateFavoredNodesRequest.RegionUpdateInfo> openInfoList = request.getUpdateInfoList();
UpdateFavoredNodesResponse.Builder respBuilder = UpdateFavoredNodesResponse.newBuilder();
for (UpdateFavoredNodesRequest.RegionUpdateInfo regionUpdateInfo : openInfoList) {
- HRegionInfo hri = HRegionInfo.convert(regionUpdateInfo.getRegion());
+ RegionInfo hri = ProtobufUtil.toRegionInfo(regionUpdateInfo.getRegion());
if (regionUpdateInfo.getFavoredNodesCount() > 0) {
regionServer.updateRegionFavoredNodesMapping(hri.getEncodedName(),
regionUpdateInfo.getFavoredNodesList());
@@ -2868,7 +2867,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
"'hbase.client.scanner.timeout.period' configuration.");
}
}
- HRegionInfo hri = rsh.s.getRegionInfo();
+ RegionInfo hri = rsh.s.getRegionInfo();
// Yes, should be the same instance
if (regionServer.getOnlineRegion(hri.getRegionName()) != rsh.r) {
String msg = "Region has changed on the scanner " + scannerName + ": regionName="
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Region.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Region.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Region.java
index 073c25f..f410e57 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Region.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Region.java
@@ -28,7 +28,6 @@ import org.apache.hadoop.hbase.CellComparator;
import org.apache.hadoop.hbase.CompareOperator;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.HDFSBlocksDistribution;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.client.Append;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get;
@@ -36,6 +35,7 @@ import org.apache.hadoop.hbase.client.Increment;
import org.apache.hadoop.hbase.client.IsolationLevel;
import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.RowMutations;
import org.apache.hadoop.hbase.client.Scan;
@@ -78,7 +78,7 @@ public interface Region extends ConfigurationObserver {
// Region state
/** @return region information for this region */
- HRegionInfo getRegionInfo();
+ RegionInfo getRegionInfo();
/** @return table descriptor for this region */
TableDescriptor getTableDescriptor();
@@ -308,11 +308,11 @@ public interface Region extends ConfigurationObserver {
*
* Before calling this function make sure that a region operation has already been
* started (the calling thread has already acquired the region-close-guard lock).
- *
+ *
* NOTE: the boolean passed here has changed. It used to be a boolean that
* stated whether or not to wait on the lock. Now it is whether it an exclusive
* lock is requested.
- *
+ *
* @param row The row actions will be performed against
* @param readLock is the lock reader or writer. True indicates that a non-exclusive
* lock is requested
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionMergeRequest.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionMergeRequest.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionMergeRequest.java
index c3c78bc..1779e5c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionMergeRequest.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionMergeRequest.java
@@ -22,23 +22,24 @@ import java.security.PrivilegedAction;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.TableName;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
import org.apache.hadoop.hbase.security.User;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
+import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
/**
* Handles processing region merges. Put in a queue, owned by HRegionServer.
*/
-// UNUSED: REMOVE!!!
+// TODO:UNUSED: REMOVE!!!
@InterfaceAudience.Private
class RegionMergeRequest implements Runnable {
private static final Log LOG = LogFactory.getLog(RegionMergeRequest.class);
- private final HRegionInfo region_a;
- private final HRegionInfo region_b;
+ private final RegionInfo region_a;
+ private final RegionInfo region_b;
private final HRegionServer server;
private final boolean forcible;
private final User user;
@@ -84,7 +85,7 @@ class RegionMergeRequest implements Runnable {
}
// TODO: fake merged region for compat with the report protocol
- final HRegionInfo merged = new HRegionInfo(table);
+ final RegionInfo merged = RegionInfoBuilder.newBuilder(table).build();
// Send the split request to the master. the master will do the validation on the split-key.
// The parent region will be unassigned and the two new regions will be assigned.
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionScanner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionScanner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionScanner.java
index ad5608c..159f13b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionScanner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionScanner.java
@@ -23,7 +23,7 @@ import java.util.List;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
-import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability;
@@ -36,7 +36,7 @@ public interface RegionScanner extends InternalScanner, Shipper {
/**
* @return The RegionInfo for this scanner.
*/
- HRegionInfo getRegionInfo();
+ RegionInfo getRegionInfo();
/**
* @return True if a filter indicates that this scanner will return no further rows.
@@ -57,7 +57,7 @@ public interface RegionScanner extends InternalScanner, Shipper {
boolean reseek(byte[] row) throws IOException;
/**
- * @return The preferred max buffersize. See
+ * @return The preferred max buffersize. See
* {@link org.apache.hadoop.hbase.client.Scan#setMaxResultSize(long)}
*/
long getMaxResultSize();
@@ -83,7 +83,7 @@ public interface RegionScanner extends InternalScanner, Shipper {
* @throws IOException e
*/
boolean nextRaw(List<Cell> result) throws IOException;
-
+
/**
* Grab the next row's worth of values. The {@link ScannerContext} is used to enforce and track
* any limits associated with this call. Any progress that exists in the {@link ScannerContext}
[03/19] hbase git commit: HBASE-18839 Apply RegionInfo to code base
Posted by ch...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDeleteTableProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDeleteTableProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDeleteTableProcedure.java
index 3ad8ec8..6f109e5 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDeleteTableProcedure.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDeleteTableProcedure.java
@@ -23,10 +23,10 @@ import static org.junit.Assert.assertTrue;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.CategoryBasedTimeout;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotDisabledException;
import org.apache.hadoop.hbase.TableNotFoundException;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.procedure2.Procedure;
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
@@ -75,7 +75,7 @@ public class TestDeleteTableProcedure extends TestTableDDLProcedureBase {
final TableName tableName = TableName.valueOf(name.getMethodName());
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
- HRegionInfo[] regions = MasterProcedureTestingUtility.createTable(
+ RegionInfo[] regions = MasterProcedureTestingUtility.createTable(
procExec, tableName, null, "f");
UTIL.getAdmin().disableTable(tableName);
@@ -118,7 +118,7 @@ public class TestDeleteTableProcedure extends TestTableDDLProcedureBase {
}
private void testSimpleDelete(final TableName tableName, byte[][] splitKeys) throws Exception {
- HRegionInfo[] regions = MasterProcedureTestingUtility.createTable(
+ RegionInfo[] regions = MasterProcedureTestingUtility.createTable(
getMasterProcedureExecutor(), tableName, splitKeys, "f1", "f2");
UTIL.getAdmin().disableTable(tableName);
@@ -136,7 +136,7 @@ public class TestDeleteTableProcedure extends TestTableDDLProcedureBase {
// create the table
byte[][] splitKeys = null;
- HRegionInfo[] regions = MasterProcedureTestingUtility.createTable(
+ RegionInfo[] regions = MasterProcedureTestingUtility.createTable(
getMasterProcedureExecutor(), tableName, splitKeys, "f1", "f2");
UTIL.getAdmin().disableTable(tableName);
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterFailoverWithProcedures.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterFailoverWithProcedures.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterFailoverWithProcedures.java
index 4e96cea..3eeb382 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterFailoverWithProcedures.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterFailoverWithProcedures.java
@@ -26,18 +26,13 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.CategoryBasedTimeout;
import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.procedure2.Procedure;
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
import org.apache.hadoop.hbase.procedure2.store.wal.WALProcedureStore;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.CreateTableState;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DeleteTableState;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DisableTableState;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.EnableTableState;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.TruncateTableState;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.testclassification.MasterTests;
import org.apache.hadoop.hbase.util.Bytes;
@@ -50,6 +45,12 @@ import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.junit.rules.TestRule;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.CreateTableState;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DeleteTableState;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DisableTableState;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.EnableTableState;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.TruncateTableState;
+
@Category({MasterTests.class, LargeTests.class})
public class TestMasterFailoverWithProcedures {
private static final Log LOG = LogFactory.getLog(TestMasterFailoverWithProcedures.class);
@@ -113,7 +114,7 @@ public class TestMasterFailoverWithProcedures {
// Start the Create procedure && kill the executor
byte[][] splitKeys = null;
TableDescriptor htd = MasterProcedureTestingUtility.createHTD(tableName, "f1", "f2");
- HRegionInfo[] regions = ModifyRegionUtils.createHRegionInfos(htd, splitKeys);
+ RegionInfo[] regions = ModifyRegionUtils.createRegionInfos(htd, splitKeys);
long procId = procExec.submitProcedure(
new CreateTableProcedure(procExec.getEnvironment(), htd, regions));
testRecoveryAndDoubleExecution(UTIL, procId, step);
@@ -140,7 +141,7 @@ public class TestMasterFailoverWithProcedures {
// create the table
byte[][] splitKeys = null;
- HRegionInfo[] regions = MasterProcedureTestingUtility.createTable(
+ RegionInfo[] regions = MasterProcedureTestingUtility.createTable(
getMasterProcedureExecutor(), tableName, splitKeys, "f1", "f2");
Path tableDir = FSUtils.getTableDir(getRootDir(), tableName);
MasterProcedureTestingUtility.validateTableCreation(
@@ -182,7 +183,7 @@ public class TestMasterFailoverWithProcedures {
final byte[][] splitKeys = new byte[][] {
Bytes.toBytes("a"), Bytes.toBytes("b"), Bytes.toBytes("c")
};
- HRegionInfo[] regions = MasterProcedureTestingUtility.createTable(
+ RegionInfo[] regions = MasterProcedureTestingUtility.createTable(
getMasterProcedureExecutor(), tableName, splitKeys, families);
// load and verify that there are rows in the table
MasterProcedureTestingUtility.loadData(
@@ -203,7 +204,7 @@ public class TestMasterFailoverWithProcedures {
UTIL.waitUntilAllRegionsAssigned(tableName);
// validate the table regions and layout
- regions = UTIL.getAdmin().getTableRegions(tableName).toArray(new HRegionInfo[0]);
+ regions = UTIL.getAdmin().getTableRegions(tableName).toArray(new RegionInfo[0]);
if (preserveSplits) {
assertEquals(1 + splitKeys.length, regions.length);
} else {
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureWalLease.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureWalLease.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureWalLease.java
index 68013fb..86f0abc 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureWalLease.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureWalLease.java
@@ -30,8 +30,8 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.CategoryBasedTimeout;
import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.procedure2.Procedure;
@@ -45,9 +45,9 @@ import org.apache.hadoop.hbase.util.ModifyRegionUtils;
import org.junit.After;
import org.junit.Before;
import org.junit.ClassRule;
+import org.junit.Ignore;
import org.junit.Rule;
import org.junit.Test;
-import org.junit.Ignore;
import org.junit.experimental.categories.Category;
import org.junit.rules.TestName;
import org.junit.rules.TestRule;
@@ -136,7 +136,7 @@ public class TestMasterProcedureWalLease {
// Try to trigger a command on the master (WAL lease expired on the active one)
TableDescriptor htd = MasterProcedureTestingUtility.createHTD(TableName.valueOf(name.getMethodName()), "f");
- HRegionInfo[] regions = ModifyRegionUtils.createHRegionInfos(htd, null);
+ RegionInfo[] regions = ModifyRegionUtils.createRegionInfos(htd, null);
LOG.debug("submit proc");
try {
getMasterProcedureExecutor().submitProcedure(
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestModifyTableProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestModifyTableProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestModifyTableProcedure.java
index 8e77985..8b58646 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestModifyTableProcedure.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestModifyTableProcedure.java
@@ -25,9 +25,9 @@ import static org.junit.Assert.assertTrue;
import org.apache.hadoop.hbase.CategoryBasedTimeout;
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.procedure2.Procedure;
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
@@ -190,7 +190,7 @@ public class TestModifyTableProcedure extends TestTableDDLProcedureBase {
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
// create the table
- HRegionInfo[] regions = MasterProcedureTestingUtility.createTable(
+ RegionInfo[] regions = MasterProcedureTestingUtility.createTable(
procExec, tableName, null, "cf1", cf3);
UTIL.getAdmin().disableTable(tableName);
@@ -230,7 +230,7 @@ public class TestModifyTableProcedure extends TestTableDDLProcedureBase {
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
// create the table
- HRegionInfo[] regions = MasterProcedureTestingUtility.createTable(
+ RegionInfo[] regions = MasterProcedureTestingUtility.createTable(
procExec, tableName, null, "cf1", cf3);
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
@@ -268,7 +268,7 @@ public class TestModifyTableProcedure extends TestTableDDLProcedureBase {
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
// create the table
- HRegionInfo[] regions = MasterProcedureTestingUtility.createTable(
+ RegionInfo[] regions = MasterProcedureTestingUtility.createTable(
procExec, tableName, null, "cf1");
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
@@ -297,7 +297,7 @@ public class TestModifyTableProcedure extends TestTableDDLProcedureBase {
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
// create the table
- HRegionInfo[] regions = MasterProcedureTestingUtility.createTable(
+ RegionInfo[] regions = MasterProcedureTestingUtility.createTable(
procExec, tableName, null, "cf1");
UTIL.getAdmin().disableTable(tableName);
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestProcedureAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestProcedureAdmin.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestProcedureAdmin.java
index 38a12e8..bb531ce 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestProcedureAdmin.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestProcedureAdmin.java
@@ -29,9 +29,9 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.CategoryBasedTimeout;
import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.procedure2.Procedure;
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
@@ -123,7 +123,7 @@ public class TestProcedureAdmin {
final TableName tableName = TableName.valueOf(name.getMethodName());
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
- HRegionInfo[] regions =
+ RegionInfo[] regions =
MasterProcedureTestingUtility.createTable(procExec, tableName, null, "f");
UTIL.getAdmin().disableTable(tableName);
ProcedureTestingUtility.waitNoProcedureRunning(procExec);
@@ -155,7 +155,7 @@ public class TestProcedureAdmin {
final TableName tableName = TableName.valueOf(name.getMethodName());
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
- HRegionInfo[] regions =
+ RegionInfo[] regions =
MasterProcedureTestingUtility.createTable(procExec, tableName, null, "f");
ProcedureTestingUtility.waitNoProcedureRunning(procExec);
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestSafemodeBringsDownMaster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestSafemodeBringsDownMaster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestSafemodeBringsDownMaster.java
index c255843..08070ad 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestSafemodeBringsDownMaster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestSafemodeBringsDownMaster.java
@@ -26,16 +26,15 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.Waiter;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer;
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.JVMClusterUtil;
-import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread;
import org.apache.hadoop.hdfs.DistributedFileSystem;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
@@ -96,7 +95,7 @@ public class TestSafemodeBringsDownMaster {
final byte[][] splitKeys = new byte[][] {
Bytes.toBytes("a"), Bytes.toBytes("b"), Bytes.toBytes("c")
};
- HRegionInfo[] regions = MasterProcedureTestingUtility.createTable(
+ RegionInfo[] regions = MasterProcedureTestingUtility.createTable(
getMasterProcedureExecutor(), tableName, splitKeys, "f1", "f2");
MiniDFSCluster dfsCluster = UTIL.getDFSCluster();
DistributedFileSystem dfs = (DistributedFileSystem) dfsCluster.getFileSystem();
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTruncateTableProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTruncateTableProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTruncateTableProcedure.java
index cebee98..c5a17a0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTruncateTableProcedure.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTruncateTableProcedure.java
@@ -24,10 +24,10 @@ import static org.junit.Assert.assertTrue;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.CategoryBasedTimeout;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotDisabledException;
import org.apache.hadoop.hbase.TableNotFoundException;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.procedure2.Procedure;
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
@@ -101,7 +101,7 @@ public class TestTruncateTableProcedure extends TestTableDDLProcedureBase {
Bytes.toBytes("a"), Bytes.toBytes("b"), Bytes.toBytes("c")
};
- HRegionInfo[] regions = MasterProcedureTestingUtility.createTable(
+ RegionInfo[] regions = MasterProcedureTestingUtility.createTable(
getMasterProcedureExecutor(), tableName, splitKeys, families);
// load and verify that there are rows in the table
MasterProcedureTestingUtility.loadData(
@@ -119,7 +119,7 @@ public class TestTruncateTableProcedure extends TestTableDDLProcedureBase {
UTIL.waitUntilAllRegionsAssigned(tableName);
// validate the table regions and layout
- regions = UTIL.getAdmin().getTableRegions(tableName).toArray(new HRegionInfo[0]);
+ regions = UTIL.getAdmin().getTableRegions(tableName).toArray(new RegionInfo[0]);
if (preserveSplits) {
assertEquals(1 + splitKeys.length, regions.length);
} else {
@@ -157,7 +157,7 @@ public class TestTruncateTableProcedure extends TestTableDDLProcedureBase {
final byte[][] splitKeys = new byte[][] {
Bytes.toBytes("a"), Bytes.toBytes("b"), Bytes.toBytes("c")
};
- HRegionInfo[] regions = MasterProcedureTestingUtility.createTable(
+ RegionInfo[] regions = MasterProcedureTestingUtility.createTable(
getMasterProcedureExecutor(), tableName, splitKeys, families);
// load and verify that there are rows in the table
MasterProcedureTestingUtility.loadData(
@@ -181,7 +181,7 @@ public class TestTruncateTableProcedure extends TestTableDDLProcedureBase {
UTIL.waitUntilAllRegionsAssigned(tableName);
// validate the table regions and layout
- regions = UTIL.getAdmin().getTableRegions(tableName).toArray(new HRegionInfo[0]);
+ regions = UTIL.getAdmin().getTableRegions(tableName).toArray(new RegionInfo[0]);
if (preserveSplits) {
assertEquals(1 + splitKeys.length, regions.length);
} else {
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/master/snapshot/TestSnapshotFileCache.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/snapshot/TestSnapshotFileCache.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/snapshot/TestSnapshotFileCache.java
index 19e7137..15e3c9a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/snapshot/TestSnapshotFileCache.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/snapshot/TestSnapshotFileCache.java
@@ -24,24 +24,26 @@ import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
import java.io.IOException;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.UUID;
import java.util.concurrent.atomic.AtomicInteger;
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.Iterables;
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos;
-import org.apache.hadoop.hbase.testclassification.MediumTests;
-import org.apache.hadoop.hbase.testclassification.MasterTests;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
import org.apache.hadoop.hbase.snapshot.SnapshotReferenceUtil;
import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils.SnapshotMock;
+import org.apache.hadoop.hbase.testclassification.MasterTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.util.FSUtils;
import org.junit.After;
import org.junit.AfterClass;
@@ -49,6 +51,10 @@ import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.Iterables;
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos;
+
/**
* Test that we correctly reload the cache, filter directories, etc.
*/
@@ -180,7 +186,7 @@ public class TestSnapshotFileCache {
SnapshotReferenceUtil
.visitReferencedFiles(UTIL.getConfiguration(), fs, builder.getSnapshotsDir(),
new SnapshotReferenceUtil.SnapshotVisitor() {
- @Override public void storeFile(HRegionInfo regionInfo, String familyName,
+ @Override public void storeFile(RegionInfo regionInfo, String familyName,
SnapshotProtos.SnapshotRegionManifest.StoreFile storeFile) throws IOException {
FileStatus status = mockStoreFile(storeFile.getName());
allStoreFiles.add(status);
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestFileSystemUtilizationChore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestFileSystemUtilizationChore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestFileSystemUtilizationChore.java
index 60f2467..719b5e1 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestFileSystemUtilizationChore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestFileSystemUtilizationChore.java
@@ -34,9 +34,8 @@ import java.util.concurrent.TimeUnit;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
-import org.apache.hadoop.hbase.regionserver.HStore;
import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.regionserver.Store;
import org.apache.hadoop.hbase.testclassification.SmallTests;
@@ -61,7 +60,7 @@ public class TestFileSystemUtilizationChore {
final FileSystemUtilizationChore chore = new FileSystemUtilizationChore(rs);
doAnswer(new ExpectedRegionSizeSummationAnswer(sum(regionSizes)))
.when(rs)
- .reportRegionSizesForQuotas((Map<HRegionInfo,Long>) any(Map.class));
+ .reportRegionSizesForQuotas((Map<RegionInfo,Long>) any(Map.class));
final Region region = mockRegionWithSize(regionSizes);
when(rs.getRegions()).thenReturn(Arrays.asList(region));
@@ -78,7 +77,7 @@ public class TestFileSystemUtilizationChore {
final FileSystemUtilizationChore chore = new FileSystemUtilizationChore(rs);
doAnswer(new ExpectedRegionSizeSummationAnswer(sum(regionSizes)))
.when(rs)
- .reportRegionSizesForQuotas((Map<HRegionInfo,Long>) any(Map.class));
+ .reportRegionSizesForQuotas((Map<RegionInfo,Long>) any(Map.class));
final Region region = mockRegionWithSize(regionSizes);
when(rs.getRegions()).thenReturn(Arrays.asList(region));
@@ -102,7 +101,7 @@ public class TestFileSystemUtilizationChore {
final FileSystemUtilizationChore chore = new FileSystemUtilizationChore(rs);
doAnswer(new ExpectedRegionSizeSummationAnswer(sum(Arrays.asList(r1Sum, r2Sum, r3Sum))))
.when(rs)
- .reportRegionSizesForQuotas((Map<HRegionInfo,Long>) any(Map.class));
+ .reportRegionSizesForQuotas((Map<RegionInfo,Long>) any(Map.class));
final Region r1 = mockRegionWithSize(r1Sizes);
final Region r2 = mockRegionWithSize(r2Sizes);
@@ -167,7 +166,7 @@ public class TestFileSystemUtilizationChore {
};
doAnswer(new ExpectedRegionSizeSummationAnswer(sum(Arrays.asList(leftover1Sum, leftover2Sum))))
.when(rs)
- .reportRegionSizesForQuotas((Map<HRegionInfo,Long>) any(Map.class));
+ .reportRegionSizesForQuotas((Map<RegionInfo,Long>) any(Map.class));
// We shouldn't compute all of these region sizes, just the leftovers
final Region r1 = mockRegionWithSize(Arrays.asList(1024L, 2048L));
@@ -199,7 +198,7 @@ public class TestFileSystemUtilizationChore {
};
doAnswer(new ExpectedRegionSizeSummationAnswer(sum(Arrays.asList(leftover1Sum))))
.when(rs)
- .reportRegionSizesForQuotas((Map<HRegionInfo,Long>) any(Map.class));
+ .reportRegionSizesForQuotas((Map<RegionInfo,Long>) any(Map.class));
// We shouldn't compute all of these region sizes, just the leftovers
final Region r1 = mockRegionWithSize(Arrays.asList(1024L, 2048L));
@@ -225,7 +224,7 @@ public class TestFileSystemUtilizationChore {
final FileSystemUtilizationChore chore = new FileSystemUtilizationChore(rs);
doAnswer(new ExpectedRegionSizeSummationAnswer(sum(Arrays.asList(r1Sum))))
.when(rs)
- .reportRegionSizesForQuotas((Map<HRegionInfo,Long>) any(Map.class));
+ .reportRegionSizesForQuotas((Map<RegionInfo,Long>) any(Map.class));
final Region r1 = mockRegionWithSize(r1Sizes);
final Region r2 = mockSplitParentRegionWithSize(r2Sizes);
@@ -247,7 +246,7 @@ public class TestFileSystemUtilizationChore {
final FileSystemUtilizationChore chore = new FileSystemUtilizationChore(rs);
doAnswer(new ExpectedRegionSizeSummationAnswer(r1Sum))
.when(rs)
- .reportRegionSizesForQuotas((Map<HRegionInfo,Long>) any(Map.class));
+ .reportRegionSizesForQuotas((Map<RegionInfo,Long>) any(Map.class));
final Region r1 = mockRegionWithSize(r1Sizes);
final Region r2 = mockRegionReplicaWithSize(r2Sizes);
@@ -274,7 +273,7 @@ public class TestFileSystemUtilizationChore {
final FileSystemUtilizationChore chore = new FileSystemUtilizationChore(rs);
doAnswer(new ExpectedRegionSizeSummationAnswer(
sum(Arrays.asList(r1HFileSizeSum, r2HFileSizeSum))))
- .when(rs).reportRegionSizesForQuotas((Map<HRegionInfo,Long>) any(Map.class));
+ .when(rs).reportRegionSizesForQuotas((Map<RegionInfo,Long>) any(Map.class));
final Region r1 = mockRegionWithHFileLinks(r1StoreFileSizes, r1HFileSizes);
final Region r2 = mockRegionWithHFileLinks(r2StoreFileSizes, r2HFileSizes);
@@ -321,7 +320,7 @@ public class TestFileSystemUtilizationChore {
*/
private Region mockRegionWithSize(Collection<Long> storeSizes) {
final Region r = mock(Region.class);
- final HRegionInfo info = mock(HRegionInfo.class);
+ final RegionInfo info = mock(RegionInfo.class);
when(r.getRegionInfo()).thenReturn(info);
List<Store> stores = new ArrayList<>();
when(r.getStores()).thenReturn((List) stores);
@@ -335,7 +334,7 @@ public class TestFileSystemUtilizationChore {
private Region mockRegionWithHFileLinks(Collection<Long> storeSizes, Collection<Long> hfileSizes) {
final Region r = mock(Region.class);
- final HRegionInfo info = mock(HRegionInfo.class);
+ final RegionInfo info = mock(RegionInfo.class);
when(r.getRegionInfo()).thenReturn(info);
List<Store> stores = new ArrayList<>();
when(r.getStores()).thenReturn((List) stores);
@@ -363,7 +362,7 @@ public class TestFileSystemUtilizationChore {
*/
private Region mockSplitParentRegionWithSize(Collection<Long> storeSizes) {
final Region r = mockRegionWithSize(storeSizes);
- final HRegionInfo info = r.getRegionInfo();
+ final RegionInfo info = r.getRegionInfo();
when(info.isSplitParent()).thenReturn(true);
return r;
}
@@ -376,7 +375,7 @@ public class TestFileSystemUtilizationChore {
*/
private Region mockRegionReplicaWithSize(Collection<Long> storeSizes) {
final Region r = mockRegionWithSize(storeSizes);
- final HRegionInfo info = r.getRegionInfo();
+ final RegionInfo info = r.getRegionInfo();
when(info.getReplicaId()).thenReturn(1);
return r;
}
@@ -396,7 +395,7 @@ public class TestFileSystemUtilizationChore {
Object[] args = invocation.getArguments();
assertEquals(1, args.length);
@SuppressWarnings("unchecked")
- Map<HRegionInfo,Long> regionSizes = (Map<HRegionInfo,Long>) args[0];
+ Map<RegionInfo,Long> regionSizes = (Map<RegionInfo,Long>) args[0];
long sum = 0L;
for (Long regionSize : regionSizes.values()) {
sum += regionSize;
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestNamespaceQuotaViolationStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestNamespaceQuotaViolationStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestNamespaceQuotaViolationStore.java
index a673bcb..303dad0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestNamespaceQuotaViolationStore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestNamespaceQuotaViolationStore.java
@@ -33,12 +33,10 @@ import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Table;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota;
import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.Before;
@@ -47,6 +45,11 @@ import org.junit.experimental.categories.Category;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota;
+
/**
* Test class for {@link NamespaceQuotaSnapshotStore}.
*/
@@ -56,7 +59,7 @@ public class TestNamespaceQuotaViolationStore {
private Connection conn;
private QuotaObserverChore chore;
- private Map<HRegionInfo, Long> regionReports;
+ private Map<RegionInfo, Long> regionReports;
private NamespaceQuotaSnapshotStore store;
@Before
@@ -109,22 +112,38 @@ public class TestNamespaceQuotaViolationStore {
// Create some junk data to filter. Makes sure it's so large that it would
// immediately violate the quota.
for (int i = 0; i < 3; i++) {
- regionReports.put(new HRegionInfo(tn3, Bytes.toBytes(i), Bytes.toBytes(i + 1)),
+
+ regionReports.put(RegionInfoBuilder.newBuilder(tn3)
+ .setStartKey(Bytes.toBytes(i))
+ .setEndKey(Bytes.toBytes(i + 1))
+ .build(),
5L * ONE_MEGABYTE);
}
- regionReports.put(new HRegionInfo(tn1, Bytes.toBytes(0), Bytes.toBytes(1)), 1024L * 512L);
- regionReports.put(new HRegionInfo(tn1, Bytes.toBytes(1), Bytes.toBytes(2)), 1024L * 256L);
+ regionReports.put(RegionInfoBuilder.newBuilder(tn1)
+ .setStartKey(Bytes.toBytes(0))
+ .setEndKey(Bytes.toBytes(1))
+ .build(), 1024L * 512L);
+ regionReports.put(RegionInfoBuilder.newBuilder(tn1)
+ .setStartKey(Bytes.toBytes(1))
+ .setEndKey(Bytes.toBytes(2))
+ .build(), 1024L * 256L);
// Below the quota
assertEquals(false, store.getTargetState(NS, quota).getQuotaStatus().isInViolation());
- regionReports.put(new HRegionInfo(tn2, Bytes.toBytes(2), Bytes.toBytes(3)), 1024L * 256L);
+ regionReports.put(RegionInfoBuilder.newBuilder(tn2)
+ .setStartKey(Bytes.toBytes(2))
+ .setEndKey(Bytes.toBytes(3))
+ .build(), 1024L * 256L);
// Equal to the quota is still in observance
assertEquals(false, store.getTargetState(NS, quota).getQuotaStatus().isInViolation());
- regionReports.put(new HRegionInfo(tn2, Bytes.toBytes(3), Bytes.toBytes(4)), 1024L);
+ regionReports.put(RegionInfoBuilder.newBuilder(tn2)
+ .setStartKey(Bytes.toBytes(3))
+ .setEndKey(Bytes.toBytes(4))
+ .build(), 1024L);
// Exceeds the quota, should be in violation
assertEquals(true, store.getTargetState(NS, quota).getQuotaStatus().isInViolation());
@@ -142,16 +161,28 @@ public class TestNamespaceQuotaViolationStore {
assertEquals(0, size(store.filterBySubject("asdf")));
for (int i = 0; i < 5; i++) {
- regionReports.put(new HRegionInfo(tn1, Bytes.toBytes(i), Bytes.toBytes(i+1)), 0L);
+ regionReports.put(RegionInfoBuilder.newBuilder(tn1)
+ .setStartKey(Bytes.toBytes(i))
+ .setEndKey(Bytes.toBytes(i + 1))
+ .build(), 0L);
}
for (int i = 0; i < 3; i++) {
- regionReports.put(new HRegionInfo(tn2, Bytes.toBytes(i), Bytes.toBytes(i+1)), 0L);
+ regionReports.put(RegionInfoBuilder.newBuilder(tn2)
+ .setStartKey(Bytes.toBytes(i))
+ .setEndKey(Bytes.toBytes(i + 1))
+ .build(), 0L);
}
for (int i = 0; i < 10; i++) {
- regionReports.put(new HRegionInfo(tn3, Bytes.toBytes(i), Bytes.toBytes(i+1)), 0L);
+ regionReports.put(RegionInfoBuilder.newBuilder(tn3)
+ .setStartKey(Bytes.toBytes(i))
+ .setEndKey(Bytes.toBytes(i + 1))
+ .build(), 0L);
}
for (int i = 0; i < 8; i++) {
- regionReports.put(new HRegionInfo(tn4, Bytes.toBytes(i), Bytes.toBytes(i+1)), 0L);
+ regionReports.put(RegionInfoBuilder.newBuilder(tn4)
+ .setStartKey(Bytes.toBytes(i))
+ .setEndKey(Bytes.toBytes(i + 1))
+ .build(), 0L);
}
assertEquals(26, regionReports.size());
assertEquals(5, size(store.filterBySubject(NamespaceDescriptor.DEFAULT_NAMESPACE_NAME_STR)));
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaObserverChore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaObserverChore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaObserverChore.java
index 74511f1..e82aee2 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaObserverChore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaObserverChore.java
@@ -23,9 +23,10 @@ import static org.mockito.Mockito.when;
import java.util.HashMap;
import java.util.Map;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.Before;
@@ -57,17 +58,26 @@ public class TestQuotaObserverChore {
final int numTable1Regions = 10;
final int numTable2Regions = 15;
final int numTable3Regions = 8;
- Map<HRegionInfo,Long> regionReports = new HashMap<>();
+ Map<RegionInfo,Long> regionReports = new HashMap<>();
for (int i = 0; i < numTable1Regions; i++) {
- regionReports.put(new HRegionInfo(tn1, Bytes.toBytes(i), Bytes.toBytes(i + 1)), 0L);
+ regionReports.put(RegionInfoBuilder.newBuilder(tn1)
+ .setStartKey(Bytes.toBytes(i))
+ .setEndKey(Bytes.toBytes(i + 1))
+ .build(), 0L);
}
for (int i = 0; i < numTable2Regions; i++) {
- regionReports.put(new HRegionInfo(tn2, Bytes.toBytes(i), Bytes.toBytes(i + 1)), 0L);
+ regionReports.put(RegionInfoBuilder.newBuilder(tn2)
+ .setStartKey(Bytes.toBytes(i))
+ .setEndKey(Bytes.toBytes(i + 1))
+ .build(), 0L);
}
for (int i = 0; i < numTable3Regions; i++) {
- regionReports.put(new HRegionInfo(tn3, Bytes.toBytes(i), Bytes.toBytes(i + 1)), 0L);
+ regionReports.put(RegionInfoBuilder.newBuilder(tn3)
+ .setStartKey(Bytes.toBytes(i))
+ .setEndKey(Bytes.toBytes(i + 1))
+ .build(), 0L);
}
TableQuotaSnapshotStore store = new TableQuotaSnapshotStore(conn, chore, regionReports);
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaObserverChoreRegionReports.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaObserverChoreRegionReports.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaObserverChoreRegionReports.java
index 62c6b53..c57a89f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaObserverChoreRegionReports.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaObserverChoreRegionReports.java
@@ -31,13 +31,13 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.Waiter;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Table;
@@ -175,9 +175,9 @@ public class TestQuotaObserverChoreRegionReports {
});
// Close the region, prevent the server from sending new status reports.
- List<HRegionInfo> regions = admin.getTableRegions(tn);
+ List<RegionInfo> regions = admin.getRegions(tn);
assertEquals(1, regions.size());
- HRegionInfo hri = regions.get(0);
+ RegionInfo hri = regions.get(0);
admin.unassign(hri.getRegionName(), true);
// We should see this table move out of violation after the report expires.
@@ -218,9 +218,9 @@ public class TestQuotaObserverChoreRegionReports {
}
}
- private int getRegionReportsForTable(Map<HRegionInfo,Long> reports, TableName tn) {
+ private int getRegionReportsForTable(Map<RegionInfo,Long> reports, TableName tn) {
int numReports = 0;
- for (Entry<HRegionInfo,Long> entry : reports.entrySet()) {
+ for (Entry<RegionInfo,Long> entry : reports.entrySet()) {
if (tn.equals(entry.getKey().getTable())) {
numReports++;
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaStatusRPCs.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaStatusRPCs.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaStatusRPCs.java
index aeae80a..a5fe406 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaStatusRPCs.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaStatusRPCs.java
@@ -30,11 +30,11 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.Waiter;
import org.apache.hadoop.hbase.Waiter.Predicate;
import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.RetriesExhaustedWithDetailsException;
import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot.SpaceQuotaStatus;
@@ -94,7 +94,7 @@ public class TestQuotaStatusRPCs {
Waiter.waitFor(TEST_UTIL.getConfiguration(), 30 * 1000, new Predicate<Exception>() {
@Override
public boolean evaluate() throws Exception {
- Map<HRegionInfo,Long> regionSizes = quotaManager.snapshotRegionSizes();
+ Map<RegionInfo,Long> regionSizes = quotaManager.snapshotRegionSizes();
LOG.trace("Region sizes=" + regionSizes);
return numRegions == countRegionsForTable(tn, regionSizes) &&
tableSize <= getTableSize(tn, regionSizes);
@@ -271,9 +271,9 @@ public class TestQuotaStatusRPCs {
});
}
- private int countRegionsForTable(TableName tn, Map<HRegionInfo,Long> regionSizes) {
+ private int countRegionsForTable(TableName tn, Map<RegionInfo,Long> regionSizes) {
int size = 0;
- for (HRegionInfo regionInfo : regionSizes.keySet()) {
+ for (RegionInfo regionInfo : regionSizes.keySet()) {
if (tn.equals(regionInfo.getTable())) {
size++;
}
@@ -281,10 +281,10 @@ public class TestQuotaStatusRPCs {
return size;
}
- private int getTableSize(TableName tn, Map<HRegionInfo,Long> regionSizes) {
+ private int getTableSize(TableName tn, Map<RegionInfo,Long> regionSizes) {
int tableSize = 0;
- for (Entry<HRegionInfo,Long> entry : regionSizes.entrySet()) {
- HRegionInfo regionInfo = entry.getKey();
+ for (Entry<RegionInfo,Long> entry : regionSizes.entrySet()) {
+ RegionInfo regionInfo = entry.getKey();
long regionSize = entry.getValue();
if (tn.equals(regionInfo.getTable())) {
tableSize += regionSize;
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestRegionSizeUse.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestRegionSizeUse.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestRegionSizeUse.java
index 8584d55..035216c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestRegionSizeUse.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestRegionSizeUse.java
@@ -30,13 +30,13 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.testclassification.MediumTests;
@@ -87,11 +87,11 @@ public class TestRegionSizeUse {
admin.flush(tn);
LOG.debug("Data flushed to disk");
// Get the final region distribution
- final List<HRegionInfo> regions = TEST_UTIL.getAdmin().getTableRegions(tn);
+ final List<RegionInfo> regions = TEST_UTIL.getAdmin().getRegions(tn);
HMaster master = cluster.getMaster();
MasterQuotaManager quotaManager = master.getMasterQuotaManager();
- Map<HRegionInfo,Long> regionSizes = quotaManager.snapshotRegionSizes();
+ Map<RegionInfo,Long> regionSizes = quotaManager.snapshotRegionSizes();
// Wait until we get all of the region reports for our table
// The table may split, so make sure we have at least as many as expected right after we
// finished writing the data.
@@ -181,9 +181,9 @@ public class TestRegionSizeUse {
* @param regions A collection of region sizes
* @return The number of regions for the given table.
*/
- private int numRegionsForTable(TableName tn, Map<HRegionInfo,Long> regions) {
+ private int numRegionsForTable(TableName tn, Map<RegionInfo,Long> regions) {
int sum = 0;
- for (Entry<HRegionInfo,Long> entry : regions.entrySet()) {
+ for (Entry<RegionInfo,Long> entry : regions.entrySet()) {
if (tn.equals(entry.getKey().getTable()) && 0 < entry.getValue()) {
sum++;
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestSpaceQuotas.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestSpaceQuotas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestSpaceQuotas.java
index 9f6c9f8..e923cc6 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestSpaceQuotas.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestSpaceQuotas.java
@@ -38,7 +38,6 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.Append;
@@ -48,6 +47,7 @@ import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Increment;
import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.RpcRetryingCaller;
@@ -265,7 +265,7 @@ public class TestSpaceQuotas {
HRegionServer rs = TEST_UTIL.getMiniHBaseCluster().getRegionServer(0);
RegionServerSpaceQuotaManager spaceQuotaManager = rs.getRegionServerSpaceQuotaManager();
Map<TableName,SpaceQuotaSnapshot> snapshots = spaceQuotaManager.copyQuotaSnapshots();
- Map<HRegionInfo,Long> regionSizes = getReportedSizesForTable(tn);
+ Map<RegionInfo,Long> regionSizes = getReportedSizesForTable(tn);
while (true) {
SpaceQuotaSnapshot snapshot = snapshots.get(tn);
if (snapshot != null && snapshot.getLimit() > 0) {
@@ -348,11 +348,11 @@ public class TestSpaceQuotas {
verifyViolation(policy, tn, p);
}
- private Map<HRegionInfo,Long> getReportedSizesForTable(TableName tn) {
+ private Map<RegionInfo,Long> getReportedSizesForTable(TableName tn) {
HMaster master = TEST_UTIL.getMiniHBaseCluster().getMaster();
MasterQuotaManager quotaManager = master.getMasterQuotaManager();
- Map<HRegionInfo,Long> filteredRegionSizes = new HashMap<>();
- for (Entry<HRegionInfo,Long> entry : quotaManager.snapshotRegionSizes().entrySet()) {
+ Map<RegionInfo,Long> filteredRegionSizes = new HashMap<>();
+ for (Entry<RegionInfo,Long> entry : quotaManager.snapshotRegionSizes().entrySet()) {
if (entry.getKey().getTable().equals(tn)) {
filteredRegionSizes.put(entry.getKey(), entry.getValue());
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestTableQuotaViolationStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestTableQuotaViolationStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestTableQuotaViolationStore.java
index fe7500d..5ba830d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestTableQuotaViolationStore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestTableQuotaViolationStore.java
@@ -29,18 +29,15 @@ import java.util.HashMap;
import java.util.Map;
import java.util.concurrent.atomic.AtomicReference;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot.SpaceQuotaStatus;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota;
import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.Before;
@@ -49,6 +46,11 @@ import org.junit.experimental.categories.Category;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota;
+
/**
* Test class for {@link TableQuotaSnapshotStore}.
*/
@@ -58,7 +60,7 @@ public class TestTableQuotaViolationStore {
private Connection conn;
private QuotaObserverChore chore;
- private Map<HRegionInfo, Long> regionReports;
+ private Map<RegionInfo, Long> regionReports;
private TableQuotaSnapshotStore store;
@Before
@@ -78,13 +80,22 @@ public class TestTableQuotaViolationStore {
assertEquals(0, size(store.filterBySubject(tn1)));
for (int i = 0; i < 5; i++) {
- regionReports.put(new HRegionInfo(tn1, Bytes.toBytes(i), Bytes.toBytes(i+1)), 0L);
+ regionReports.put(RegionInfoBuilder.newBuilder(tn1)
+ .setStartKey(Bytes.toBytes(i))
+ .setEndKey(Bytes.toBytes(i + 1))
+ .build(), 0L);
}
for (int i = 0; i < 3; i++) {
- regionReports.put(new HRegionInfo(tn2, Bytes.toBytes(i), Bytes.toBytes(i+1)), 0L);
+ regionReports.put(RegionInfoBuilder.newBuilder(tn2)
+ .setStartKey(Bytes.toBytes(i))
+ .setEndKey(Bytes.toBytes(i + 1))
+ .build(), 0L);
}
for (int i = 0; i < 10; i++) {
- regionReports.put(new HRegionInfo(tn3, Bytes.toBytes(i), Bytes.toBytes(i+1)), 0L);
+ regionReports.put(RegionInfoBuilder.newBuilder(tn3)
+ .setStartKey(Bytes.toBytes(i))
+ .setEndKey(Bytes.toBytes(i + 1))
+ .build(), 0L);
}
assertEquals(18, regionReports.size());
assertEquals(5, size(store.filterBySubject(tn1)));
@@ -106,14 +117,23 @@ public class TestTableQuotaViolationStore {
// Create some junk data to filter. Makes sure it's so large that it would
// immediately violate the quota.
for (int i = 0; i < 3; i++) {
- regionReports.put(new HRegionInfo(tn2, Bytes.toBytes(i), Bytes.toBytes(i + 1)),
- 5L * ONE_MEGABYTE);
- regionReports.put(new HRegionInfo(tn3, Bytes.toBytes(i), Bytes.toBytes(i + 1)),
- 5L * ONE_MEGABYTE);
+ regionReports.put(RegionInfoBuilder.newBuilder(tn2)
+ .setStartKey(Bytes.toBytes(i))
+ .setEndKey(Bytes.toBytes(i + 1))
+ .build(), 5L * ONE_MEGABYTE);
+ regionReports.put(RegionInfoBuilder.newBuilder(tn3)
+ .setStartKey(Bytes.toBytes(i))
+ .setEndKey(Bytes.toBytes(i + 1))
+ .build(), 5L * ONE_MEGABYTE);
}
-
- regionReports.put(new HRegionInfo(tn1, Bytes.toBytes(0), Bytes.toBytes(1)), 1024L * 512L);
- regionReports.put(new HRegionInfo(tn1, Bytes.toBytes(1), Bytes.toBytes(2)), 1024L * 256L);
+ regionReports.put(RegionInfoBuilder.newBuilder(tn1)
+ .setStartKey(Bytes.toBytes(0))
+ .setEndKey(Bytes.toBytes(1))
+ .build(), 1024L * 512L);
+ regionReports.put(RegionInfoBuilder.newBuilder(tn1)
+ .setStartKey(Bytes.toBytes(1))
+ .setEndKey(Bytes.toBytes(2))
+ .build(), 1024L * 256L);
SpaceQuotaSnapshot tn1Snapshot = new SpaceQuotaSnapshot(
SpaceQuotaStatus.notInViolation(), 1024L * 768L, 1024L * 1024L);
@@ -121,13 +141,20 @@ public class TestTableQuotaViolationStore {
// Below the quota
assertEquals(tn1Snapshot, store.getTargetState(tn1, quota));
- regionReports.put(new HRegionInfo(tn1, Bytes.toBytes(2), Bytes.toBytes(3)), 1024L * 256L);
+
+ regionReports.put(RegionInfoBuilder.newBuilder(tn1)
+ .setStartKey(Bytes.toBytes(2))
+ .setEndKey(Bytes.toBytes(3))
+ .build(), 1024L * 256L);
tn1Snapshot = new SpaceQuotaSnapshot(SpaceQuotaStatus.notInViolation(), 1024L * 1024L, 1024L * 1024L);
// Equal to the quota is still in observance
assertEquals(tn1Snapshot, store.getTargetState(tn1, quota));
- regionReports.put(new HRegionInfo(tn1, Bytes.toBytes(3), Bytes.toBytes(4)), 1024L);
+ regionReports.put(RegionInfoBuilder.newBuilder(tn1)
+ .setStartKey(Bytes.toBytes(3))
+ .setEndKey(Bytes.toBytes(4))
+ .build(), 1024L);
tn1Snapshot = new SpaceQuotaSnapshot(
new SpaceQuotaStatus(SpaceViolationPolicy.DISABLE), 1024L * 1024L + 1024L, 1024L * 1024L);
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestTablesWithQuotas.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestTablesWithQuotas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestTablesWithQuotas.java
index 2de649b..6fcbf77 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestTablesWithQuotas.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestTablesWithQuotas.java
@@ -32,10 +32,10 @@ import java.util.Set;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.quotas.QuotaObserverChore.TablesWithQuotas;
import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.junit.Before;
@@ -176,7 +176,7 @@ public class TestTablesWithQuotas {
when(admin.getTableRegions(missingTable)).thenReturn(null);
QuotaObserverChore chore = mock(QuotaObserverChore.class);
- Map<HRegionInfo,Long> regionUsage = new HashMap<>();
+ Map<RegionInfo,Long> regionUsage = new HashMap<>();
TableQuotaSnapshotStore store = new TableQuotaSnapshotStore(conn, chore, regionUsage);
// A super dirty hack to verify that, after getting no regions for our table,
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java
index 6a7e98b..674c3e8 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java
@@ -57,6 +57,7 @@ import org.apache.hadoop.hbase.client.Increment;
import org.apache.hadoop.hbase.client.IsolationLevel;
import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.RowMutations;
import org.apache.hadoop.hbase.client.Scan;
@@ -97,7 +98,7 @@ public class TestAtomicOperation {
static final byte [] row = Bytes.toBytes("rowA");
static final byte [] row2 = Bytes.toBytes("rowB");
- @Before
+ @Before
public void setup() {
tableName = Bytes.toBytes(name.getMethodName());
}
@@ -115,7 +116,7 @@ public class TestAtomicOperation {
}
//////////////////////////////////////////////////////////////////////////////
// New tests that doesn't spin up a mini cluster but rather just test the
- // individual code pieces in the HRegion.
+ // individual code pieces in the HRegion.
//////////////////////////////////////////////////////////////////////////////
/**
@@ -289,7 +290,7 @@ public class TestAtomicOperation {
Result result = region.increment(inc, HConstants.NO_NONCE, HConstants.NO_NONCE);
if (result != null) {
assertEquals(Bytes.toLong(result.getValue(fam1, qual1))*2,
- Bytes.toLong(result.getValue(fam1, qual2)));
+ Bytes.toLong(result.getValue(fam1, qual2)));
assertTrue(result.getValue(fam2, qual3) != null);
assertEquals(Bytes.toLong(result.getValue(fam1, qual1))*3,
Bytes.toLong(result.getValue(fam2, qual3)));
@@ -335,8 +336,8 @@ public class TestAtomicOperation {
Get g = new Get(row);
Result result = region.get(g);
- assertEquals(result.getValue(fam1, qual1).length, result.getValue(fam1, qual2).length);
- assertEquals(result.getValue(fam1, qual1).length, result.getValue(fam2, qual3).length);
+ assertEquals(result.getValue(fam1, qual1).length, result.getValue(fam1, qual2).length);
+ assertEquals(result.getValue(fam1, qual1).length, result.getValue(fam2, qual3).length);
} catch (IOException e) {
e.printStackTrace();
failures.incrementAndGet();
@@ -568,7 +569,7 @@ public class TestAtomicOperation {
this.failures = failures;
}
}
-
+
private static CountDownLatch latch = new CountDownLatch(1);
private enum TestStep {
INIT, // initial put of 10 to set value of the cell
@@ -580,11 +581,11 @@ public class TestAtomicOperation {
}
private static volatile TestStep testStep = TestStep.INIT;
private final String family = "f1";
-
+
/**
* Test written as a verifier for HBASE-7051, CheckAndPut should properly read
- * MVCC.
- *
+ * MVCC.
+ *
* Moved into TestAtomicOperation from its original location, TestHBase7051
*/
@Test
@@ -598,7 +599,7 @@ public class TestAtomicOperation {
Put put = new Put(Bytes.toBytes("r1"));
put.addColumn(Bytes.toBytes(family), Bytes.toBytes("q1"), Bytes.toBytes("10"));
puts[0] = put;
-
+
region.batchMutate(puts, HConstants.NO_NONCE, HConstants.NO_NONCE);
MultithreadedTestUtil.TestContext ctx =
new MultithreadedTestUtil.TestContext(conf);
@@ -661,7 +662,7 @@ public class TestAtomicOperation {
public static class MockHRegion extends HRegion {
public MockHRegion(Path tableDir, WAL log, FileSystem fs, Configuration conf,
- final HRegionInfo regionInfo, final TableDescriptor htd, RegionServerServices rsServices) {
+ final RegionInfo regionInfo, final TableDescriptor htd, RegionServerServices rsServices) {
super(tableDir, log, fs, conf, regionInfo, htd, rsServices);
}
@@ -672,7 +673,7 @@ public class TestAtomicOperation {
}
return new WrappedRowLock(super.getRowLockInternal(row, readLock));
}
-
+
public class WrappedRowLock implements RowLock {
private final RowLock rowLock;
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java
index b36b8fe..e40ff8e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java
@@ -50,6 +50,7 @@ import org.apache.hadoop.hbase.KeyValueTestUtil;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.exceptions.UnexpectedStateException;
import org.apache.hadoop.hbase.testclassification.MediumTests;
@@ -988,8 +989,7 @@ public class TestDefaultMemStore {
final long now = EnvironmentEdgeManager.currentTime();
final List<Cell> cells = new ArrayList<>(2);
cells.add(new KeyValue(row, HConstants.CATALOG_FAMILY,
- HConstants.REGIONINFO_QUALIFIER, now,
- r.getRegionInfo().toByteArray()));
+ HConstants.REGIONINFO_QUALIFIER, now, RegionInfo.toByteArray(r.getRegionInfo())));
// Set into the root table the version of the meta table.
cells.add(new KeyValue(row, HConstants.CATALOG_FAMILY,
HConstants.META_VERSION_QUALIFIER, now,
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java
index ab60d9d..6a41742 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java
@@ -19,7 +19,6 @@ package org.apache.hadoop.hbase.regionserver;
import static org.junit.Assert.assertArrayEquals;
import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import java.io.IOException;
@@ -28,6 +27,7 @@ import java.util.List;
import java.util.Random;
import java.util.Set;
import java.util.TreeSet;
+import java.util.stream.Collectors;
import org.apache.commons.io.IOUtils;
import org.apache.commons.logging.Log;
@@ -36,7 +36,6 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.ChoreService;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.NotServingRegionException;
@@ -48,16 +47,10 @@ import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.RegionLocator;
import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Table;
-import org.apache.hadoop.hbase.ipc.HBaseRpcControllerImpl;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair;
@@ -69,10 +62,10 @@ import org.junit.BeforeClass;
import org.junit.Rule;
import org.junit.Test;
import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Iterators;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Sets;
-import org.junit.rules.TestName;
@Category(LargeTests.class)
public class TestEndToEndSplitTransaction {
@@ -154,7 +147,7 @@ public class TestEndToEndSplitTransaction {
try {
Random random = new Random();
for (int i= 0; i< 5; i++) {
- List<HRegionInfo> regions =
+ List<RegionInfo> regions =
MetaTableAccessor.getTableRegions(connection, tableName, true);
if (regions.isEmpty()) {
continue;
@@ -162,7 +155,7 @@ public class TestEndToEndSplitTransaction {
int regionIndex = random.nextInt(regions.size());
//pick a random region and split it into two
- HRegionInfo region = Iterators.get(regions.iterator(), regionIndex);
+ RegionInfo region = Iterators.get(regions.iterator(), regionIndex);
//pick the mid split point
int start = 0, end = Integer.MAX_VALUE;
@@ -227,10 +220,10 @@ public class TestEndToEndSplitTransaction {
/** verify region boundaries obtained from MetaScanner */
void verifyRegionsUsingMetaTableAccessor() throws Exception {
- List<HRegionInfo> regionList = MetaTableAccessor.getTableRegions(connection, tableName, true);
- verifyTableRegions(Sets.newTreeSet(regionList));
+ List<RegionInfo> regionList = MetaTableAccessor.getTableRegions(connection, tableName, true);
+ verifyTableRegions(regionList.stream().collect(Collectors.toCollection(() -> new TreeSet<>(RegionInfo.COMPARATOR))));
regionList = MetaTableAccessor.getAllRegions(connection, true);
- verifyTableRegions(Sets.newTreeSet(regionList));
+ verifyTableRegions(regionList.stream().collect(Collectors.toCollection(() -> new TreeSet<>(RegionInfo.COMPARATOR))));
}
/** verify region boundaries obtained from HTable.getStartEndKeys() */
@@ -244,8 +237,7 @@ public class TestEndToEndSplitTransaction {
Pair<byte[][], byte[][]> keys = rl.getStartEndKeys();
verifyStartEndKeys(keys);
- //HTable.getRegionsInfo()
- Set<HRegionInfo> regions = new TreeSet<>();
+ Set<RegionInfo> regions = new TreeSet<>(RegionInfo.COMPARATOR);
for (HRegionLocation loc : rl.getAllRegionLocations()) {
regions.add(loc.getRegionInfo());
}
@@ -262,14 +254,14 @@ public class TestEndToEndSplitTransaction {
verifyRegionsUsingHTable();
}
- void verifyTableRegions(Set<HRegionInfo> regions) {
+ void verifyTableRegions(Set<RegionInfo> regions) {
log("Verifying " + regions.size() + " regions: " + regions);
byte[][] startKeys = new byte[regions.size()][];
byte[][] endKeys = new byte[regions.size()][];
int i=0;
- for (HRegionInfo region : regions) {
+ for (RegionInfo region : regions) {
startKeys[i] = region.getStartKey();
endKeys[i] = region.getEndKey();
i++;
@@ -352,21 +344,21 @@ public class TestEndToEndSplitTransaction {
throws IOException, InterruptedException {
long start = System.currentTimeMillis();
log("blocking until region is split:" + Bytes.toStringBinary(regionName));
- HRegionInfo daughterA = null, daughterB = null;
+ RegionInfo daughterA = null, daughterB = null;
try (Connection conn = ConnectionFactory.createConnection(conf);
Table metaTable = conn.getTable(TableName.META_TABLE_NAME)) {
Result result = null;
- HRegionInfo region = null;
+ RegionInfo region = null;
while ((System.currentTimeMillis() - start) < timeout) {
result = metaTable.get(new Get(regionName));
if (result == null) {
break;
}
- region = MetaTableAccessor.getHRegionInfo(result);
+ region = MetaTableAccessor.getRegionInfo(result);
if (region.isSplitParent()) {
log("found parent region: " + region.toString());
- PairOfSameType<HRegionInfo> pair = MetaTableAccessor.getDaughterRegions(result);
+ PairOfSameType<RegionInfo> pair = MetaTableAccessor.getDaughterRegions(result);
daughterA = pair.getFirst();
daughterB = pair.getSecond();
break;
@@ -396,7 +388,7 @@ public class TestEndToEndSplitTransaction {
}
}
- public static void blockUntilRegionIsInMeta(Connection conn, long timeout, HRegionInfo hri)
+ public static void blockUntilRegionIsInMeta(Connection conn, long timeout, RegionInfo hri)
throws IOException, InterruptedException {
log("blocking until region is in META: " + hri.getRegionNameAsString());
long start = System.currentTimeMillis();
@@ -410,7 +402,7 @@ public class TestEndToEndSplitTransaction {
}
}
- public static void blockUntilRegionIsOpened(Configuration conf, long timeout, HRegionInfo hri)
+ public static void blockUntilRegionIsOpened(Configuration conf, long timeout, RegionInfo hri)
throws IOException, InterruptedException {
log("blocking until region is opened for reading:" + hri.getRegionNameAsString());
long start = System.currentTimeMillis();
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
index 9fcdf56..4d557b9 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
@@ -105,6 +105,7 @@ import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Increment;
import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.RowMutations;
import org.apache.hadoop.hbase.client.Scan;
@@ -5822,7 +5823,7 @@ public class TestHRegion {
// Helper for test testOpenRegionWrittenToWALForLogReplay
static class HRegionWithSeqId extends HRegion {
public HRegionWithSeqId(final Path tableDir, final WAL wal, final FileSystem fs,
- final Configuration confParam, final HRegionInfo regionInfo,
+ final Configuration confParam, final RegionInfo regionInfo,
final TableDescriptor htd, final RegionServerServices rsServices) {
super(tableDir, wal, fs, confParam, regionInfo, htd, rsServices);
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionFileSystem.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionFileSystem.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionFileSystem.java
index 477c870..829b488 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionFileSystem.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionFileSystem.java
@@ -41,11 +41,12 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
import org.apache.hadoop.hbase.fs.HFileSystem;
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
import org.apache.hadoop.hbase.testclassification.SmallTests;
@@ -182,7 +183,7 @@ public class TestHRegionFileSystem {
assertEquals(1, regionDirs.size());
List<Path> familyDirs = FSUtils.getFamilyDirs(fs, regionDirs.get(0));
assertEquals(2, familyDirs.size());
- HRegionInfo hri = table.getRegionLocator().getAllRegionLocations().get(0).getRegionInfo();
+ RegionInfo hri = table.getRegionLocator().getAllRegionLocations().get(0).getRegionInfo();
HRegionFileSystem regionFs = new HRegionFileSystem(conf, new HFileSystem(fs), tableDir, hri);
return regionFs;
}
@@ -194,7 +195,7 @@ public class TestHRegionFileSystem {
Configuration conf = TEST_UTIL.getConfiguration();
// Create a Region
- HRegionInfo hri = new HRegionInfo(TableName.valueOf(name.getMethodName()));
+ RegionInfo hri = RegionInfoBuilder.newBuilder(TableName.valueOf(name.getMethodName())).build();
HRegionFileSystem regionFs = HRegionFileSystem.createRegionOnFileSystem(conf, fs,
FSUtils.getTableDir(rootDir, hri.getTable()), hri);
@@ -203,7 +204,7 @@ public class TestHRegionFileSystem {
assertTrue("The region folder should be created", fs.exists(regionDir));
// Verify the .regioninfo
- HRegionInfo hriVerify = HRegionFileSystem.loadRegionInfoFileContent(fs, regionDir);
+ RegionInfo hriVerify = HRegionFileSystem.loadRegionInfoFileContent(fs, regionDir);
assertEquals(hri, hriVerify);
// Open the region
@@ -226,7 +227,7 @@ public class TestHRegionFileSystem {
Configuration conf = TEST_UTIL.getConfiguration();
// Create a Region
- HRegionInfo hri = new HRegionInfo(TableName.valueOf(name.getMethodName()));
+ RegionInfo hri = RegionInfoBuilder.newBuilder(TableName.valueOf(name.getMethodName())).build();
HRegionFileSystem regionFs = HRegionFileSystem.createRegionOnFileSystem(conf, fs, rootDir, hri);
assertTrue(fs.exists(regionFs.getRegionDir()));
@@ -351,7 +352,8 @@ public class TestHRegionFileSystem {
// Create a Region
String familyName = "cf";
- HRegionInfo hri = new HRegionInfo(TableName.valueOf(name.getMethodName()));
+ ;
+ RegionInfo hri = RegionInfoBuilder.newBuilder(TableName.valueOf(name.getMethodName())).build();
HRegionFileSystem regionFs = HRegionFileSystem.createRegionOnFileSystem(conf, fs, rootDir, hri);
// New region, no store files
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionInfo.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionInfo.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionInfo.java
index 1b5ee08..a20fccb 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionInfo.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionInfo.java
@@ -81,9 +81,9 @@ public class TestHRegionInfo {
long modtime2 = getModTime(r);
assertEquals(modtime, modtime2);
// Now load the file.
- HRegionInfo deserializedHri = HRegionFileSystem.loadRegionInfoFileContent(
+ org.apache.hadoop.hbase.client.RegionInfo deserializedHri = HRegionFileSystem.loadRegionInfoFileContent(
r.getRegionFileSystem().getFileSystem(), r.getRegionFileSystem().getRegionDir());
- assertTrue(hri.equals(deserializedHri));
+ assertTrue(org.apache.hadoop.hbase.client.RegionInfo.COMPARATOR.compare(hri, deserializedHri) == 0);
HBaseTestingUtility.closeRegionAndWAL(r);
}
@@ -300,12 +300,12 @@ public class TestHRegionInfo {
String firstPart = descriptiveNameForDisplay.substring(0,
descriptiveNameForDisplay.indexOf(new String(HRegionInfo.HIDDEN_START_KEY)));
String secondPart = descriptiveNameForDisplay.substring(
- descriptiveNameForDisplay.indexOf(new String(HRegionInfo.HIDDEN_START_KEY)) +
+ descriptiveNameForDisplay.indexOf(new String(HRegionInfo.HIDDEN_START_KEY)) +
HRegionInfo.HIDDEN_START_KEY.length);
String firstPartOrig = origDesc.substring(0,
origDesc.indexOf(Bytes.toStringBinary(startKey)));
String secondPartOrig = origDesc.substring(
- origDesc.indexOf(Bytes.toStringBinary(startKey)) +
+ origDesc.indexOf(Bytes.toStringBinary(startKey)) +
Bytes.toStringBinary(startKey).length());
assert(firstPart.equals(firstPartOrig));
assert(secondPart.equals(secondPartOrig));
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMobStoreScanner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMobStoreScanner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMobStoreScanner.java
index 9305bbc..59e4ded 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMobStoreScanner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMobStoreScanner.java
@@ -30,7 +30,6 @@ import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Admin;
@@ -38,6 +37,7 @@ import org.apache.hadoop.hbase.client.ConnectionConfiguration;
import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
@@ -407,7 +407,7 @@ public class TestMobStoreScanner {
// Get the archive path
Path rootDir = FSUtils.getRootDir(TEST_UTIL.getConfiguration());
Path tableDir = FSUtils.getTableDir(rootDir, tn);
- HRegionInfo regionInfo = MobUtils.getMobRegionInfo(tn);
+ RegionInfo regionInfo = MobUtils.getMobRegionInfo(tn);
Path storeArchiveDir = HFileArchiveUtil.getStoreArchivePath(TEST_UTIL.getConfiguration(),
regionInfo, tableDir, family);
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRSKilledWhenInitializing.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRSKilledWhenInitializing.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRSKilledWhenInitializing.java
index 3cba15c..edd5673 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRSKilledWhenInitializing.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRSKilledWhenInitializing.java
@@ -34,16 +34,14 @@ import org.apache.hadoop.hbase.CategoryBasedTimeout;
import org.apache.hadoop.hbase.CoordinatedStateManager;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.LocalHBaseCluster;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.master.LoadBalancer;
import org.apache.hadoop.hbase.master.ServerListener;
import org.apache.hadoop.hbase.master.ServerManager;
-import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStartupResponse;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
import org.apache.hadoop.hbase.util.Bytes;
@@ -55,6 +53,8 @@ import org.junit.experimental.categories.Category;
import org.junit.rules.TestName;
import org.junit.rules.TestRule;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStartupResponse;
+
/**
* Tests that a regionserver that dies after reporting for duty gets removed
* from list of online regions. See HBASE-9593.
@@ -124,12 +124,12 @@ public class TestRSKilledWhenInitializing {
// showing still. The downed RegionServer should still be showing as registered.
assertTrue(master.getMaster().getServerManager().isServerOnline(killedRS.get()));
// Find non-meta region (namespace?) and assign to the killed server. That'll trigger cleanup.
- Map<HRegionInfo, ServerName> assignments = null;
+ Map<RegionInfo, ServerName> assignments = null;
do {
assignments = master.getMaster().getAssignmentManager().getRegionStates().getRegionAssignments();
} while (assignments == null || assignments.size() < 2);
- HRegionInfo hri = null;
- for (Map.Entry<HRegionInfo, ServerName> e: assignments.entrySet()) {
+ RegionInfo hri = null;
+ for (Map.Entry<RegionInfo, ServerName> e: assignments.entrySet()) {
if (e.getKey().isMetaRegion()) continue;
hri = e.getKey();
break;
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRSStatusServlet.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRSStatusServlet.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRSStatusServlet.java
index 3a404e9..cfecd9f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRSStatusServlet.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRSStatusServlet.java
@@ -22,15 +22,19 @@ import java.io.IOException;
import java.io.StringWriter;
import java.util.List;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.shaded.protobuf.ResponseConverter;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetServerInfoRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetServerInfoResponse;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.io.hfile.CacheConfig;
+import org.apache.hadoop.hbase.ipc.MetricsHBaseServer;
+import org.apache.hadoop.hbase.ipc.MetricsHBaseServerWrapperStub;
+import org.apache.hadoop.hbase.ipc.RpcServerInterface;
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.apache.hadoop.hbase.tmpl.regionserver.RSStatusTmpl;
@@ -45,16 +49,12 @@ import org.junit.rules.TestName;
import org.mockito.Mockito;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
-
import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.io.hfile.CacheConfig;
-import org.apache.hadoop.hbase.ipc.MetricsHBaseServer;
-import org.apache.hadoop.hbase.ipc.MetricsHBaseServerWrapperStub;
-import org.apache.hadoop.hbase.ipc.RpcServerInterface;
+import org.apache.hadoop.hbase.shaded.protobuf.ResponseConverter;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetServerInfoRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetServerInfoResponse;
/**
* Tests for the region server status page and its template.
@@ -101,7 +101,7 @@ public class TestRSStatusServlet {
CacheConfig cacheConf = Mockito.mock(CacheConfig.class);
Mockito.doReturn(null).when(cacheConf).getBlockCache();
Mockito.doReturn(cacheConf).when(rs).getCacheConfig();
-
+
// Fake MasterAddressTracker
MasterAddressTracker mat = Mockito.mock(MasterAddressTracker.class);
Mockito.doReturn(fakeMasterAddress).when(mat).getMasterAddress();
@@ -115,23 +115,28 @@ public class TestRSStatusServlet {
Mockito.doReturn(new MetricsHBaseServerWrapperStub()).when(ms).getHBaseServerWrapper();
Mockito.doReturn(ms).when(rpcServer).getMetrics();
}
-
+
@Test
public void testBasic() throws IOException, ServiceException {
new RSStatusTmpl().render(new StringWriter(), rs);
}
-
+
@Test
public void testWithRegions() throws IOException, ServiceException {
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name.getMethodName()));
- List<HRegionInfo> regions = Lists.newArrayList(
- new HRegionInfo(htd.getTableName(), Bytes.toBytes("a"), Bytes.toBytes("d")),
- new HRegionInfo(htd.getTableName(), Bytes.toBytes("d"), Bytes.toBytes("z"))
+ List<RegionInfo> regions = Lists.newArrayList(
+ RegionInfoBuilder.newBuilder(htd.getTableName())
+ .setStartKey(Bytes.toBytes("a"))
+ .setEndKey(Bytes.toBytes("d"))
+ .build(),
+ RegionInfoBuilder.newBuilder(htd.getTableName())
+ .setStartKey(Bytes.toBytes("d"))
+ .setEndKey(Bytes.toBytes("z"))
+ .build()
);
Mockito.doReturn(ResponseConverter.buildGetOnlineRegionResponse(
regions)).when(rpcServices).getOnlineRegion((RpcController)Mockito.any(),
(GetOnlineRegionRequest)Mockito.any());
-
new RSStatusTmpl().render(new StringWriter(), rs);
}
}
[08/19] hbase git commit: HBASE-18839 Apply RegionInfo to code base
Posted by ch...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java
index b4e5007..e942a02 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java
@@ -22,8 +22,6 @@ package org.apache.hadoop.hbase.tool;
import static org.apache.hadoop.hbase.HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT;
import static org.apache.hadoop.hbase.HConstants.ZOOKEEPER_ZNODE_PARENT;
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
-
import java.io.Closeable;
import java.io.IOException;
import java.net.InetSocketAddress;
@@ -62,7 +60,6 @@ import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MetaTableAccessor;
@@ -72,13 +69,13 @@ import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotEnabledException;
import org.apache.hadoop.hbase.TableNotFoundException;
-import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.RegionLocator;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
@@ -96,11 +93,14 @@ import org.apache.hadoop.hbase.zookeeper.ZKConfig;
import org.apache.hadoop.util.GenericOptionsParser;
import org.apache.hadoop.util.Tool;
import org.apache.hadoop.util.ToolRunner;
+import org.apache.yetus.audience.InterfaceAudience;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.ZooKeeper;
import org.apache.zookeeper.client.ConnectStringParser;
import org.apache.zookeeper.data.Stat;
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
+
/**
* HBase Canary Tool, that that can be used to do
* "canary monitoring" of a running HBase cluster.
@@ -210,34 +210,34 @@ public final class Canary implements Tool {
private Map<String, LongAdder> perTableReadLatency = new HashMap<>();
private LongAdder writeLatency = new LongAdder();
- public void publishReadFailure(ServerName serverName, HRegionInfo region, Exception e) {
+ public void publishReadFailure(ServerName serverName, RegionInfo region, Exception e) {
incReadFailureCount();
LOG.error(String.format("read from region %s on regionserver %s failed", region.getRegionNameAsString(), serverName), e);
}
- public void publishReadFailure(ServerName serverName, HRegionInfo region, ColumnFamilyDescriptor column, Exception e) {
+ public void publishReadFailure(ServerName serverName, RegionInfo region, ColumnFamilyDescriptor column, Exception e) {
incReadFailureCount();
LOG.error(String.format("read from region %s on regionserver %s column family %s failed",
region.getRegionNameAsString(), serverName, column.getNameAsString()), e);
}
- public void publishReadTiming(ServerName serverName, HRegionInfo region, ColumnFamilyDescriptor column, long msTime) {
+ public void publishReadTiming(ServerName serverName, RegionInfo region, ColumnFamilyDescriptor column, long msTime) {
LOG.info(String.format("read from region %s on regionserver %s column family %s in %dms",
region.getRegionNameAsString(), serverName, column.getNameAsString(), msTime));
}
- public void publishWriteFailure(ServerName serverName, HRegionInfo region, Exception e) {
+ public void publishWriteFailure(ServerName serverName, RegionInfo region, Exception e) {
incWriteFailureCount();
LOG.error(String.format("write to region %s on regionserver %s failed", region.getRegionNameAsString(), serverName), e);
}
- public void publishWriteFailure(ServerName serverName, HRegionInfo region, ColumnFamilyDescriptor column, Exception e) {
+ public void publishWriteFailure(ServerName serverName, RegionInfo region, ColumnFamilyDescriptor column, Exception e) {
incWriteFailureCount();
LOG.error(String.format("write to region %s on regionserver %s column family %s failed",
region.getRegionNameAsString(), serverName, column.getNameAsString()), e);
}
- public void publishWriteTiming(ServerName serverName, HRegionInfo region, ColumnFamilyDescriptor column, long msTime) {
+ public void publishWriteTiming(ServerName serverName, RegionInfo region, ColumnFamilyDescriptor column, long msTime) {
LOG.info(String.format("write to region %s on regionserver %s column family %s in %dms",
region.getRegionNameAsString(), serverName, column.getNameAsString(), msTime));
}
@@ -307,14 +307,14 @@ public final class Canary implements Tool {
READ, WRITE
}
private Connection connection;
- private HRegionInfo region;
+ private RegionInfo region;
private RegionStdOutSink sink;
private TaskType taskType;
private boolean rawScanEnabled;
private ServerName serverName;
private LongAdder readWriteLatency;
- RegionTask(Connection connection, HRegionInfo region, ServerName serverName, RegionStdOutSink sink,
+ RegionTask(Connection connection, RegionInfo region, ServerName serverName, RegionStdOutSink sink,
TaskType taskType, boolean rawScanEnabled, LongAdder rwLatency) {
this.connection = connection;
this.region = region;
@@ -476,11 +476,11 @@ public final class Canary implements Tool {
static class RegionServerTask implements Callable<Void> {
private Connection connection;
private String serverName;
- private HRegionInfo region;
+ private RegionInfo region;
private RegionServerStdOutSink sink;
private AtomicLong successes;
- RegionServerTask(Connection connection, String serverName, HRegionInfo region,
+ RegionServerTask(Connection connection, String serverName, RegionInfo region,
RegionServerStdOutSink sink, AtomicLong successes) {
this.connection = connection;
this.serverName = serverName;
@@ -1188,7 +1188,7 @@ public final class Canary implements Tool {
numberOfServers -= 1;
}
- List<Pair<HRegionInfo, ServerName>> pairs =
+ List<Pair<RegionInfo, ServerName>> pairs =
MetaTableAccessor.getTableRegionsAndLocations(connection, writeTableName);
int numberOfRegions = pairs.size();
if (numberOfRegions < numberOfServers * regionsLowerLimit
@@ -1198,7 +1198,7 @@ public final class Canary implements Tool {
createWriteTable(numberOfServers);
}
HashSet<ServerName> serverSet = new HashSet<>();
- for (Pair<HRegionInfo, ServerName> pair : pairs) {
+ for (Pair<RegionInfo, ServerName> pair : pairs) {
serverSet.add(pair.getSecond());
}
int numberOfCoveredServers = serverSet.size();
@@ -1273,7 +1273,7 @@ public final class Canary implements Tool {
regionLocator = admin.getConnection().getRegionLocator(tableDesc.getTableName());
for (HRegionLocation location : regionLocator.getAllRegionLocations()) {
ServerName rs = location.getServerName();
- HRegionInfo region = location.getRegionInfo();
+ RegionInfo region = location.getRegionInfo();
tasks.add(new RegionTask(admin.getConnection(), region, rs, (RegionStdOutSink) sink, taskType, rawScanEnabled,
rwLatency));
}
@@ -1376,7 +1376,7 @@ public final class Canary implements Tool {
LOG.error("Run RegionServerMonitor failed!", e);
this.errorCode = ERROR_EXIT_CODE;
}
- Map<String, List<HRegionInfo>> rsAndRMap = this.filterRegionServerByName();
+ Map<String, List<RegionInfo>> rsAndRMap = this.filterRegionServerByName();
this.initialized = true;
this.monitorRegionServers(rsAndRMap, regionServerSink);
}
@@ -1416,18 +1416,18 @@ public final class Canary implements Tool {
return foundTableNames.isEmpty();
}
- private void monitorRegionServers(Map<String, List<HRegionInfo>> rsAndRMap, RegionServerStdOutSink regionServerSink) {
+ private void monitorRegionServers(Map<String, List<RegionInfo>> rsAndRMap, RegionServerStdOutSink regionServerSink) {
List<RegionServerTask> tasks = new ArrayList<>();
Map<String, AtomicLong> successMap = new HashMap<>();
Random rand = new Random();
- for (Map.Entry<String, List<HRegionInfo>> entry : rsAndRMap.entrySet()) {
+ for (Map.Entry<String, List<RegionInfo>> entry : rsAndRMap.entrySet()) {
String serverName = entry.getKey();
AtomicLong successes = new AtomicLong(0);
successMap.put(serverName, successes);
if (entry.getValue().isEmpty()) {
LOG.error(String.format("Regionserver not serving any regions - %s", serverName));
} else if (this.allRegions) {
- for (HRegionInfo region : entry.getValue()) {
+ for (RegionInfo region : entry.getValue()) {
tasks.add(new RegionServerTask(this.connection,
serverName,
region,
@@ -1436,7 +1436,7 @@ public final class Canary implements Tool {
}
} else {
// random select a region if flag not set
- HRegionInfo region = entry.getValue().get(rand.nextInt(entry.getValue().size()));
+ RegionInfo region = entry.getValue().get(rand.nextInt(entry.getValue().size()));
tasks.add(new RegionServerTask(this.connection,
serverName,
region,
@@ -1454,7 +1454,7 @@ public final class Canary implements Tool {
}
}
if (this.allRegions) {
- for (Map.Entry<String, List<HRegionInfo>> entry : rsAndRMap.entrySet()) {
+ for (Map.Entry<String, List<RegionInfo>> entry : rsAndRMap.entrySet()) {
String serverName = entry.getKey();
LOG.info("Successfully read " + successMap.get(serverName) + " regions out of "
+ entry.getValue().size() + " on regionserver:" + serverName);
@@ -1466,14 +1466,14 @@ public final class Canary implements Tool {
}
}
- private Map<String, List<HRegionInfo>> filterRegionServerByName() {
- Map<String, List<HRegionInfo>> regionServerAndRegionsMap = this.getAllRegionServerByName();
+ private Map<String, List<RegionInfo>> filterRegionServerByName() {
+ Map<String, List<RegionInfo>> regionServerAndRegionsMap = this.getAllRegionServerByName();
regionServerAndRegionsMap = this.doFilterRegionServerByName(regionServerAndRegionsMap);
return regionServerAndRegionsMap;
}
- private Map<String, List<HRegionInfo>> getAllRegionServerByName() {
- Map<String, List<HRegionInfo>> rsAndRMap = new HashMap<>();
+ private Map<String, List<RegionInfo>> getAllRegionServerByName() {
+ Map<String, List<RegionInfo>> rsAndRMap = new HashMap<>();
Table table = null;
RegionLocator regionLocator = null;
try {
@@ -1481,7 +1481,7 @@ public final class Canary implements Tool {
LOG.debug(String.format("reading list of tables and locations"));
}
HTableDescriptor[] tableDescs = this.admin.listTables();
- List<HRegionInfo> regions = null;
+ List<RegionInfo> regions = null;
for (HTableDescriptor tableDesc : tableDescs) {
table = this.admin.getConnection().getTable(tableDesc.getTableName());
regionLocator = this.admin.getConnection().getRegionLocator(tableDesc.getTableName());
@@ -1489,7 +1489,7 @@ public final class Canary implements Tool {
for (HRegionLocation location : regionLocator.getAllRegionLocations()) {
ServerName rs = location.getServerName();
String rsName = rs.getHostname();
- HRegionInfo r = location.getRegionInfo();
+ RegionInfo r = location.getRegionInfo();
if (rsAndRMap.containsKey(rsName)) {
regions = rsAndRMap.get(rsName);
@@ -1507,7 +1507,7 @@ public final class Canary implements Tool {
.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS)).getServers()) {
String rsName = rs.getHostname();
if (!rsAndRMap.containsKey(rsName)) {
- rsAndRMap.put(rsName, Collections.<HRegionInfo> emptyList());
+ rsAndRMap.put(rsName, Collections.<RegionInfo> emptyList());
}
}
} catch (IOException e) {
@@ -1527,10 +1527,10 @@ public final class Canary implements Tool {
return rsAndRMap;
}
- private Map<String, List<HRegionInfo>> doFilterRegionServerByName(
- Map<String, List<HRegionInfo>> fullRsAndRMap) {
+ private Map<String, List<RegionInfo>> doFilterRegionServerByName(
+ Map<String, List<RegionInfo>> fullRsAndRMap) {
- Map<String, List<HRegionInfo>> filteredRsAndRMap = null;
+ Map<String, List<RegionInfo>> filteredRsAndRMap = null;
if (this.targets != null && this.targets.length > 0) {
filteredRsAndRMap = new HashMap<>();
@@ -1541,7 +1541,7 @@ public final class Canary implements Tool {
if (this.useRegExp) {
regExpFound = false;
pattern = Pattern.compile(rsName);
- for (Map.Entry<String, List<HRegionInfo>> entry : fullRsAndRMap.entrySet()) {
+ for (Map.Entry<String, List<RegionInfo>> entry : fullRsAndRMap.entrySet()) {
matcher = pattern.matcher(entry.getKey());
if (matcher.matches()) {
filteredRsAndRMap.put(entry.getKey(), entry.getValue());
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
index 0c95e7e..4afdcb9 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
@@ -17,18 +17,6 @@
*/
package org.apache.hadoop.hbase.util;
-import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
-import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
-import org.apache.hadoop.hbase.client.TableDescriptor;
-import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
-import org.apache.hadoop.hbase.shaded.com.google.common.base.Joiner;
-import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableList;
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.Multimap;
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.Ordering;
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.TreeMultimap;
-
import java.io.Closeable;
import java.io.FileNotFoundException;
import java.io.IOException;
@@ -100,19 +88,23 @@ import org.apache.hadoop.hbase.RegionLocations;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.yetus.audience.InterfaceStability;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.ClusterConnection;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.RowMutations;
import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.io.FileLink;
import org.apache.hadoop.hbase.io.HFileLink;
@@ -120,8 +112,6 @@ import org.apache.hadoop.hbase.io.hfile.CacheConfig;
import org.apache.hadoop.hbase.io.hfile.HFile;
import org.apache.hadoop.hbase.master.MasterFileSystem;
import org.apache.hadoop.hbase.master.RegionState;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
@@ -147,8 +137,20 @@ import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.util.ReflectionUtils;
import org.apache.hadoop.util.Tool;
import org.apache.hadoop.util.ToolRunner;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.yetus.audience.InterfaceStability;
import org.apache.zookeeper.KeeperException;
+import org.apache.hadoop.hbase.shaded.com.google.common.base.Joiner;
+import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableList;
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.Multimap;
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.Ordering;
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.TreeMultimap;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface;
+
/**
* HBaseFsck (hbck) is a tool for checking and repairing region consistency and
* table integrity problems in a corrupted HBase.
@@ -833,11 +835,11 @@ public class HBaseFsck extends Configured implements Closeable {
public void checkRegionBoundaries() {
try {
ByteArrayComparator comparator = new ByteArrayComparator();
- List<HRegionInfo> regions = MetaTableAccessor.getAllRegions(connection, true);
+ List<RegionInfo> regions = MetaTableAccessor.getAllRegions(connection, true);
final RegionBoundariesInformation currentRegionBoundariesInformation =
new RegionBoundariesInformation();
Path hbaseRoot = FSUtils.getRootDir(getConf());
- for (HRegionInfo regionInfo : regions) {
+ for (RegionInfo regionInfo : regions) {
Path tableDir = FSUtils.getTableDir(hbaseRoot, regionInfo.getTable());
currentRegionBoundariesInformation.regionName = regionInfo.getRegionName();
// For each region, get the start and stop key from the META and compare them to the
@@ -1005,10 +1007,12 @@ public class HBaseFsck extends Configured implements Closeable {
Bytes.toString(orphanRegionRange.getSecond()) + ")");
// create new region on hdfs. move data into place.
- HRegionInfo hri = new HRegionInfo(template.getTableName(), orphanRegionRange.getFirst(),
- Bytes.add(orphanRegionRange.getSecond(), new byte[1]));
- LOG.info("Creating new region : " + hri);
- HRegion region = HBaseFsckRepair.createHDFSRegionDir(getConf(), hri, template);
+ RegionInfo regionInfo = RegionInfoBuilder.newBuilder(template.getTableName())
+ .setStartKey(orphanRegionRange.getFirst())
+ .setEndKey(Bytes.add(orphanRegionRange.getSecond(), new byte[1]))
+ .build();
+ LOG.info("Creating new region : " + regionInfo);
+ HRegion region = HBaseFsckRepair.createHDFSRegionDir(getConf(), regionInfo, template);
Path target = region.getRegionFileSystem().getRegionDir();
// rename all the data to new region
@@ -1232,7 +1236,7 @@ public class HBaseFsck extends Configured implements Closeable {
private void loadHdfsRegioninfo(HbckInfo hbi) throws IOException {
Path regionDir = hbi.getHdfsRegionDir();
if (regionDir == null) {
- if (hbi.getReplicaId() == HRegionInfo.DEFAULT_REPLICA_ID) {
+ if (hbi.getReplicaId() == RegionInfo.DEFAULT_REPLICA_ID) {
// Log warning only for default/ primary replica with no region dir
LOG.warn("No HDFS region dir found: " + hbi + " meta=" + hbi.metaEntry);
}
@@ -1245,8 +1249,8 @@ public class HBaseFsck extends Configured implements Closeable {
}
FileSystem fs = FileSystem.get(getConf());
- HRegionInfo hri = HRegionFileSystem.loadRegionInfoFileContent(fs, regionDir);
- LOG.debug("HRegionInfo read: " + hri.toString());
+ RegionInfo hri = HRegionFileSystem.loadRegionInfoFileContent(fs, regionDir);
+ LOG.debug("RegionInfo read: " + hri.toString());
hbi.hdfsEntry.hri = hri;
}
@@ -1468,7 +1472,7 @@ public class HBaseFsck extends Configured implements Closeable {
private HRegion createNewMeta(String walFactoryID) throws IOException {
Path rootdir = FSUtils.getRootDir(getConf());
Configuration c = getConf();
- HRegionInfo metaHRI = new HRegionInfo(HRegionInfo.FIRST_META_REGIONINFO);
+ RegionInfo metaHRI = RegionInfoBuilder.FIRST_META_REGIONINFO;
TableDescriptor metaDescriptor = new FSTableDescriptors(c).get(TableName.META_TABLE_NAME);
MasterFileSystem.setInfoFamilyCachingForMeta(metaDescriptor, false);
// The WAL subsystem will use the default rootDir rather than the passed in rootDir
@@ -1518,7 +1522,7 @@ public class HBaseFsck extends Configured implements Closeable {
// add the row directly to meta.
HbckInfo hi = his.iterator().next();
- HRegionInfo hri = hi.getHdfsHRI(); // hi.metaEntry;
+ RegionInfo hri = hi.getHdfsHRI(); // hi.metaEntry;
Put p = MetaTableAccessor.makePutFromRegionInfo(hri);
puts.add(p);
}
@@ -1974,7 +1978,7 @@ public class HBaseFsck extends Configured implements Closeable {
List<CheckRegionConsistencyWorkItem> workItems = new ArrayList<>(regionInfoMap.size());
for (java.util.Map.Entry<String, HbckInfo> e: regionInfoMap.entrySet()) {
- if (e.getValue().getReplicaId() == HRegionInfo.DEFAULT_REPLICA_ID) {
+ if (e.getValue().getReplicaId() == RegionInfo.DEFAULT_REPLICA_ID) {
workItems.add(new CheckRegionConsistencyWorkItem(e.getKey(), e.getValue()));
}
}
@@ -1986,7 +1990,7 @@ public class HBaseFsck extends Configured implements Closeable {
// deployed/undeployed replicas.
List<CheckRegionConsistencyWorkItem> replicaWorkItems = new ArrayList<>(regionInfoMap.size());
for (java.util.Map.Entry<String, HbckInfo> e: regionInfoMap.entrySet()) {
- if (e.getValue().getReplicaId() != HRegionInfo.DEFAULT_REPLICA_ID) {
+ if (e.getValue().getReplicaId() != RegionInfo.DEFAULT_REPLICA_ID) {
replicaWorkItems.add(new CheckRegionConsistencyWorkItem(e.getKey(), e.getValue()));
}
}
@@ -2169,9 +2173,10 @@ public class HBaseFsck extends Configured implements Closeable {
d.addColumn(HConstants.CATALOG_FAMILY, HConstants.SPLITB_QUALIFIER);
mutations.add(d);
- HRegionInfo hri = new HRegionInfo(hi.metaEntry);
- hri.setOffline(false);
- hri.setSplit(false);
+ RegionInfo hri = RegionInfoBuilder.newBuilder(hi.metaEntry)
+ .setOffline(false)
+ .setSplit(false)
+ .build();
Put p = MetaTableAccessor.makePutFromRegionInfo(hri);
mutations.add(p);
@@ -2219,13 +2224,13 @@ public class HBaseFsck extends Configured implements Closeable {
private void undeployRegions(HbckInfo hi) throws IOException, InterruptedException {
undeployRegionsForHbi(hi);
// undeploy replicas of the region (but only if the method is invoked for the primary)
- if (hi.getReplicaId() != HRegionInfo.DEFAULT_REPLICA_ID) {
+ if (hi.getReplicaId() != RegionInfo.DEFAULT_REPLICA_ID) {
return;
}
int numReplicas = admin.getTableDescriptor(hi.getTableName()).getRegionReplication();
for (int i = 1; i < numReplicas; i++) {
if (hi.getPrimaryHRIForDeployedReplica() == null) continue;
- HRegionInfo hri = RegionReplicaUtil.getRegionInfoForReplica(
+ RegionInfo hri = RegionReplicaUtil.getRegionInfoForReplica(
hi.getPrimaryHRIForDeployedReplica(), i);
HbckInfo h = regionInfoMap.get(hri.getEncodedName());
if (h != null) {
@@ -2274,7 +2279,7 @@ public class HBaseFsck extends Configured implements Closeable {
get.addColumn(HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER);
get.addColumn(HConstants.CATALOG_FAMILY, HConstants.STARTCODE_QUALIFIER);
// also get the locations of the replicas to close if the primary region is being closed
- if (hi.getReplicaId() == HRegionInfo.DEFAULT_REPLICA_ID) {
+ if (hi.getReplicaId() == RegionInfo.DEFAULT_REPLICA_ID) {
int numReplicas = admin.getTableDescriptor(hi.getTableName()).getRegionReplication();
for (int i = 0; i < numReplicas; i++) {
get.addColumn(HConstants.CATALOG_FAMILY, MetaTableAccessor.getServerColumn(i));
@@ -2296,7 +2301,7 @@ public class HBaseFsck extends Configured implements Closeable {
+ "have handle to reach it.");
continue;
}
- HRegionInfo hri = h.getRegionInfo();
+ RegionInfo hri = h.getRegionInfo();
if (hri == null) {
LOG.warn("Unable to close region " + hi.getRegionNameAsString()
+ " because hbase:meta had invalid or missing "
@@ -2317,7 +2322,7 @@ public class HBaseFsck extends Configured implements Closeable {
errors.print(msg);
undeployRegions(hbi);
setShouldRerun();
- HRegionInfo hri = hbi.getHdfsHRI();
+ RegionInfo hri = hbi.getHdfsHRI();
if (hri == null) {
hri = hbi.metaEntry;
}
@@ -2325,7 +2330,7 @@ public class HBaseFsck extends Configured implements Closeable {
HBaseFsckRepair.waitUntilAssigned(admin, hri);
// also assign replicas if needed (do it only when this call operates on a primary replica)
- if (hbi.getReplicaId() != HRegionInfo.DEFAULT_REPLICA_ID) return;
+ if (hbi.getReplicaId() != RegionInfo.DEFAULT_REPLICA_ID) return;
int replicationCount = admin.getTableDescriptor(hri.getTable()).getRegionReplication();
for (int i = 1; i < replicationCount; i++) {
hri = RegionReplicaUtil.getRegionInfoForReplica(hri, i);
@@ -2413,10 +2418,10 @@ public class HBaseFsck extends Configured implements Closeable {
return;
}
- HRegionInfo hri = hbi.getHdfsHRI();
+ RegionInfo hri = hbi.getHdfsHRI();
TableInfo tableInfo = tablesInfo.get(hri.getTable());
- for (HRegionInfo region : tableInfo.getRegionsFromMeta()) {
+ for (RegionInfo region : tableInfo.getRegionsFromMeta()) {
if (Bytes.compareTo(region.getStartKey(), hri.getStartKey()) <= 0
&& (region.getEndKey().length == 0 || Bytes.compareTo(region.getEndKey(),
hri.getEndKey()) >= 0)
@@ -2454,7 +2459,7 @@ public class HBaseFsck extends Configured implements Closeable {
errors.reportError(ERROR_CODE.NOT_IN_META, "Region " + descriptiveName
+ " not in META, but deployed on " + Joiner.on(", ").join(hbi.deployedOn));
debugLsr(hbi.getHdfsRegionDir());
- if (hbi.getReplicaId() != HRegionInfo.DEFAULT_REPLICA_ID) {
+ if (hbi.getReplicaId() != RegionInfo.DEFAULT_REPLICA_ID) {
// for replicas, this means that we should undeploy the region (we would have
// gone over the primaries and fixed meta holes in first phase under
// checkAndFixConsistency; we shouldn't get the condition !inMeta at
@@ -2463,7 +2468,7 @@ public class HBaseFsck extends Configured implements Closeable {
undeployRegionsForHbi(hbi);
}
}
- if (shouldFixMeta() && hbi.getReplicaId() == HRegionInfo.DEFAULT_REPLICA_ID) {
+ if (shouldFixMeta() && hbi.getReplicaId() == RegionInfo.DEFAULT_REPLICA_ID) {
if (!hbi.isHdfsRegioninfoPresent()) {
LOG.error("This should have been repaired in table integrity repair phase");
return;
@@ -2493,7 +2498,7 @@ public class HBaseFsck extends Configured implements Closeable {
// For Replica region, we need to do a similar check. If replica is not split successfully,
// error is going to be reported against primary daughter region.
- if (hbi.getReplicaId() != HRegionInfo.DEFAULT_REPLICA_ID) {
+ if (hbi.getReplicaId() != RegionInfo.DEFAULT_REPLICA_ID) {
LOG.info("Region " + descriptiveName + " is a split parent in META, in HDFS, "
+ "and not deployed on any region server. This may be transient.");
hbi.setSkipChecks(true);
@@ -2784,7 +2789,7 @@ public class HBaseFsck extends Configured implements Closeable {
TreeMultimap.create(RegionSplitCalculator.BYTES_COMPARATOR, cmp);
// list of regions derived from meta entries.
- private ImmutableList<HRegionInfo> regionsFromMeta = null;
+ private ImmutableList<RegionInfo> regionsFromMeta = null;
TableInfo(TableName name) {
this.tableName = name;
@@ -2808,7 +2813,7 @@ public class HBaseFsck extends Configured implements Closeable {
if (Bytes.equals(hir.getEndKey(), HConstants.EMPTY_END_ROW)) {
// end key is absolute end key, just add it.
// ignore replicas other than primary for these checks
- if (hir.getReplicaId() == HRegionInfo.DEFAULT_REPLICA_ID) sc.add(hir);
+ if (hir.getReplicaId() == RegionInfo.DEFAULT_REPLICA_ID) sc.add(hir);
return;
}
@@ -2826,7 +2831,7 @@ public class HBaseFsck extends Configured implements Closeable {
// main case, add to split calculator
// ignore replicas other than primary for these checks
- if (hir.getReplicaId() == HRegionInfo.DEFAULT_REPLICA_ID) sc.add(hir);
+ if (hir.getReplicaId() == RegionInfo.DEFAULT_REPLICA_ID) sc.add(hir);
}
public void addServer(ServerName server) {
@@ -2841,18 +2846,18 @@ public class HBaseFsck extends Configured implements Closeable {
return sc.getStarts().size() + backwards.size();
}
- public synchronized ImmutableList<HRegionInfo> getRegionsFromMeta() {
+ public synchronized ImmutableList<RegionInfo> getRegionsFromMeta() {
// lazy loaded, synchronized to ensure a single load
if (regionsFromMeta == null) {
- List<HRegionInfo> regions = new ArrayList<>();
+ List<RegionInfo> regions = new ArrayList<>();
for (HbckInfo h : HBaseFsck.this.regionInfoMap.values()) {
if (tableName.equals(h.getTableName())) {
if (h.metaEntry != null) {
- regions.add((HRegionInfo) h.metaEntry);
+ regions.add(h.metaEntry);
}
}
}
- regionsFromMeta = Ordering.natural().immutableSortedCopy(regions);
+ regionsFromMeta = Ordering.from(RegionInfo.COMPARATOR).immutableSortedCopy(regions);
}
return regionsFromMeta;
@@ -2968,8 +2973,10 @@ public class HBaseFsck extends Configured implements Closeable {
getTableInfo(), next);
TableDescriptor htd = getTableInfo().getHTD();
// from special EMPTY_START_ROW to next region's startKey
- HRegionInfo newRegion = new HRegionInfo(htd.getTableName(),
- HConstants.EMPTY_START_ROW, next.getStartKey());
+ RegionInfo newRegion = RegionInfoBuilder.newBuilder(htd.getTableName())
+ .setStartKey(HConstants.EMPTY_START_ROW)
+ .setEndKey(next.getStartKey())
+ .build();
// TODO test
HRegion region = HBaseFsckRepair.createHDFSRegionDir(conf, newRegion, htd);
@@ -2985,8 +2992,10 @@ public class HBaseFsck extends Configured implements Closeable {
+ "region and regioninfo in HDFS to plug the hole.", getTableInfo());
TableDescriptor htd = getTableInfo().getHTD();
// from curEndKey to EMPTY_START_ROW
- HRegionInfo newRegion = new HRegionInfo(htd.getTableName(), curEndKey,
- HConstants.EMPTY_START_ROW);
+ RegionInfo newRegion = RegionInfoBuilder.newBuilder(htd.getTableName())
+ .setStartKey(curEndKey)
+ .setEndKey(HConstants.EMPTY_START_ROW)
+ .build();
HRegion region = HBaseFsckRepair.createHDFSRegionDir(conf, newRegion, htd);
LOG.info("Table region end key was not empty. Created new empty region: " + newRegion
@@ -3008,7 +3017,10 @@ public class HBaseFsck extends Configured implements Closeable {
+ ". Creating a new regioninfo and region "
+ "dir in hdfs to plug the hole.");
TableDescriptor htd = getTableInfo().getHTD();
- HRegionInfo newRegion = new HRegionInfo(htd.getTableName(), holeStartKey, holeStopKey);
+ RegionInfo newRegion = RegionInfoBuilder.newBuilder(htd.getTableName())
+ .setStartKey(holeStartKey)
+ .setEndKey(holeStopKey)
+ .build();
HRegion region = HBaseFsckRepair.createHDFSRegionDir(conf, newRegion, htd);
LOG.info("Plugged hole by creating new empty region: "+ newRegion + " " +region);
fixes++;
@@ -3210,8 +3222,10 @@ public class HBaseFsck extends Configured implements Closeable {
// create new empty container region.
TableDescriptor htd = getTableInfo().getHTD();
// from start key to end Key
- HRegionInfo newRegion = new HRegionInfo(htd.getTableName(), range.getFirst(),
- range.getSecond());
+ RegionInfo newRegion = RegionInfoBuilder.newBuilder(htd.getTableName())
+ .setStartKey(range.getFirst())
+ .setEndKey(range.getSecond())
+ .build();
HRegion region = HBaseFsckRepair.createHDFSRegionDir(conf, newRegion, htd);
LOG.info("[" + thread + "] Created new empty container region: " +
newRegion + " to contain regions: " + Joiner.on(",").join(overlap));
@@ -3344,10 +3358,10 @@ public class HBaseFsck extends Configured implements Closeable {
ArrayList<HbckInfo> subRange = new ArrayList<>(ranges);
// this dumb and n^2 but this shouldn't happen often
for (HbckInfo r1 : ranges) {
- if (r1.getReplicaId() != HRegionInfo.DEFAULT_REPLICA_ID) continue;
+ if (r1.getReplicaId() != RegionInfo.DEFAULT_REPLICA_ID) continue;
subRange.remove(r1);
for (HbckInfo r2 : subRange) {
- if (r2.getReplicaId() != HRegionInfo.DEFAULT_REPLICA_ID) continue;
+ if (r2.getReplicaId() != RegionInfo.DEFAULT_REPLICA_ID) continue;
// general case of same start key
if (Bytes.compareTo(r1.getStartKey(), r2.getStartKey())==0) {
handler.handleDuplicateStartKeys(r1,r2);
@@ -3642,8 +3656,8 @@ public class HBaseFsck extends Configured implements Closeable {
errors.print("Trying to fix a problem with hbase:meta..");
setShouldRerun();
// try to fix it (treat it as unassigned region)
- HRegionInfo h = RegionReplicaUtil.getRegionInfoForReplica(
- HRegionInfo.FIRST_META_REGIONINFO, replicaId);
+ RegionInfo h = RegionReplicaUtil.getRegionInfoForReplica(
+ RegionInfoBuilder.FIRST_META_REGIONINFO, replicaId);
HBaseFsckRepair.fixUnassigned(admin, h);
HBaseFsckRepair.waitUntilAssigned(admin, h);
}
@@ -3679,19 +3693,19 @@ public class HBaseFsck extends Configured implements Closeable {
return true;
}
ServerName sn = null;
- if (rl.getRegionLocation(HRegionInfo.DEFAULT_REPLICA_ID) == null ||
- rl.getRegionLocation(HRegionInfo.DEFAULT_REPLICA_ID).getRegionInfo() == null) {
+ if (rl.getRegionLocation(RegionInfo.DEFAULT_REPLICA_ID) == null ||
+ rl.getRegionLocation(RegionInfo.DEFAULT_REPLICA_ID).getRegionInfo() == null) {
emptyRegionInfoQualifiers.add(result);
errors.reportError(ERROR_CODE.EMPTY_META_CELL,
"Empty REGIONINFO_QUALIFIER found in hbase:meta");
return true;
}
- HRegionInfo hri = rl.getRegionLocation(HRegionInfo.DEFAULT_REPLICA_ID).getRegionInfo();
+ RegionInfo hri = rl.getRegionLocation(RegionInfo.DEFAULT_REPLICA_ID).getRegionInfo();
if (!(isTableIncluded(hri.getTable())
|| hri.isMetaRegion())) {
return true;
}
- PairOfSameType<HRegionInfo> daughters = MetaTableAccessor.getDaughterRegions(result);
+ PairOfSameType<RegionInfo> daughters = MetaTableAccessor.getDaughterRegions(result);
for (HRegionLocation h : rl.getRegionLocations()) {
if (h == null || h.getRegionInfo() == null) {
continue;
@@ -3700,7 +3714,7 @@ public class HBaseFsck extends Configured implements Closeable {
hri = h.getRegionInfo();
MetaEntry m = null;
- if (hri.getReplicaId() == HRegionInfo.DEFAULT_REPLICA_ID) {
+ if (hri.getReplicaId() == RegionInfo.DEFAULT_REPLICA_ID) {
m = new MetaEntry(hri, sn, ts, daughters.getFirst(), daughters.getSecond());
} else {
m = new MetaEntry(hri, sn, ts, null, null);
@@ -3714,8 +3728,8 @@ public class HBaseFsck extends Configured implements Closeable {
throw new IOException("Two entries in hbase:meta are same " + previous);
}
}
- PairOfSameType<HRegionInfo> mergeRegions = MetaTableAccessor.getMergeRegions(result);
- for (HRegionInfo mergeRegion : new HRegionInfo[] {
+ PairOfSameType<RegionInfo> mergeRegions = MetaTableAccessor.getMergeRegions(result);
+ for (RegionInfo mergeRegion : new RegionInfo[] {
mergeRegions.getFirst(), mergeRegions.getSecond() }) {
if (mergeRegion != null) {
// This region is already been merged
@@ -3751,14 +3765,14 @@ public class HBaseFsck extends Configured implements Closeable {
static class MetaEntry extends HRegionInfo {
ServerName regionServer; // server hosting this region
long modTime; // timestamp of most recent modification metadata
- HRegionInfo splitA, splitB; //split daughters
+ RegionInfo splitA, splitB; //split daughters
- public MetaEntry(HRegionInfo rinfo, ServerName regionServer, long modTime) {
+ public MetaEntry(RegionInfo rinfo, ServerName regionServer, long modTime) {
this(rinfo, regionServer, modTime, null, null);
}
- public MetaEntry(HRegionInfo rinfo, ServerName regionServer, long modTime,
- HRegionInfo splitA, HRegionInfo splitB) {
+ public MetaEntry(RegionInfo rinfo, ServerName regionServer, long modTime,
+ RegionInfo splitA, RegionInfo splitB) {
super(rinfo);
this.regionServer = regionServer;
this.modTime = modTime;
@@ -3800,7 +3814,7 @@ public class HBaseFsck extends Configured implements Closeable {
* Stores the regioninfo entries from HDFS
*/
static class HdfsEntry {
- HRegionInfo hri;
+ RegionInfo hri;
Path hdfsRegionDir = null;
long hdfsRegionDirModTime = 0;
boolean hdfsRegioninfoFilePresent = false;
@@ -3811,7 +3825,7 @@ public class HBaseFsck extends Configured implements Closeable {
* Stores the regioninfo retrieved from Online region servers.
*/
static class OnlineEntry {
- HRegionInfo hri;
+ RegionInfo hri;
ServerName hsa;
@Override
@@ -3831,8 +3845,8 @@ public class HBaseFsck extends Configured implements Closeable {
private List<ServerName> deployedOn = Lists.newArrayList(); // info on RS's
private boolean skipChecks = false; // whether to skip further checks to this region info.
private boolean isMerged = false;// whether this region has already been merged into another one
- private int deployedReplicaId = HRegionInfo.DEFAULT_REPLICA_ID;
- private HRegionInfo primaryHRIForDeployedReplica = null;
+ private int deployedReplicaId = RegionInfo.DEFAULT_REPLICA_ID;
+ private RegionInfo primaryHRIForDeployedReplica = null;
HbckInfo(MetaEntry metaEntry) {
this.metaEntry = metaEntry;
@@ -3842,7 +3856,7 @@ public class HBaseFsck extends Configured implements Closeable {
return metaEntry != null? metaEntry.getReplicaId(): deployedReplicaId;
}
- public synchronized void addServer(HRegionInfo hri, ServerName server) {
+ public synchronized void addServer(RegionInfo hri, ServerName server) {
OnlineEntry rse = new OnlineEntry() ;
rse.hri = hri;
rse.hsa = server;
@@ -3937,7 +3951,7 @@ public class HBaseFsck extends Configured implements Closeable {
}
}
- public HRegionInfo getPrimaryHRIForDeployedReplica() {
+ public RegionInfo getPrimaryHRIForDeployedReplica() {
return primaryHRIForDeployedReplica;
}
@@ -3969,7 +3983,7 @@ public class HBaseFsck extends Configured implements Closeable {
return hdfsEntry.hdfsRegionDirModTime;
}
- HRegionInfo getHdfsHRI() {
+ RegionInfo getHdfsHRI() {
if (hdfsEntry == null) {
return null;
}
@@ -4271,13 +4285,13 @@ public class HBaseFsck extends Configured implements Closeable {
BlockingInterface server = connection.getAdmin(rsinfo);
// list all online regions from this region server
- List<HRegionInfo> regions = ProtobufUtil.getOnlineRegions(server);
+ List<RegionInfo> regions = ProtobufUtil.getOnlineRegions(server);
regions = filterRegions(regions);
if (details) {
errors.detail("RegionServer: " + rsinfo.getServerName() +
" number of regions: " + regions.size());
- for (HRegionInfo rinfo: regions) {
+ for (RegionInfo rinfo: regions) {
errors.detail(" " + rinfo.getRegionNameAsString() +
" id: " + rinfo.getRegionId() +
" encoded_name: " + rinfo.getEncodedName() +
@@ -4287,7 +4301,7 @@ public class HBaseFsck extends Configured implements Closeable {
}
// check to see if the existence of this region matches the region in META
- for (HRegionInfo r:regions) {
+ for (RegionInfo r:regions) {
HbckInfo hbi = hbck.getOrCreateInfo(r.getEncodedName());
hbi.addServer(r, rsinfo);
}
@@ -4299,9 +4313,9 @@ public class HBaseFsck extends Configured implements Closeable {
return null;
}
- private List<HRegionInfo> filterRegions(List<HRegionInfo> regions) {
- List<HRegionInfo> ret = Lists.newArrayList();
- for (HRegionInfo hri : regions) {
+ private List<RegionInfo> filterRegions(List<RegionInfo> regions) {
+ List<RegionInfo> ret = Lists.newArrayList();
+ for (RegionInfo hri : regions) {
if (hri.isMetaTable() || (!hbck.checkMetaOnly
&& hbck.isTableIncluded(hri.getTable()))) {
ret.add(hri);
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/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 651d6e5..afb6c5b 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
@@ -28,23 +28,23 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.ClusterStatus.Option;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.hbase.ClusterStatus.Option;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.ClusterConnection;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.master.RegionState;
import org.apache.hadoop.hbase.master.ServerManager;
import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.yetus.audience.InterfaceAudience;
import org.apache.zookeeper.KeeperException;
/**
@@ -64,18 +64,16 @@ public class HBaseFsckRepair {
* @param region Region to undeploy
* @param servers list of Servers to undeploy from
*/
- public static void fixMultiAssignment(Connection connection, HRegionInfo region,
+ public static void fixMultiAssignment(Connection connection, RegionInfo region,
List<ServerName> servers)
throws IOException, KeeperException, InterruptedException {
- HRegionInfo actualRegion = new HRegionInfo(region);
-
// Close region on the servers silently
for(ServerName server : servers) {
- closeRegionSilentlyAndWait(connection, server, actualRegion);
+ closeRegionSilentlyAndWait(connection, server, region);
}
// Force ZK node to OFFLINE so master assigns
- forceOfflineInZK(connection.getAdmin(), actualRegion);
+ forceOfflineInZK(connection.getAdmin(), region);
}
/**
@@ -90,12 +88,10 @@ public class HBaseFsckRepair {
* @throws IOException
* @throws KeeperException
*/
- public static void fixUnassigned(Admin admin, HRegionInfo region)
+ public static void fixUnassigned(Admin admin, RegionInfo region)
throws IOException, KeeperException, InterruptedException {
- HRegionInfo actualRegion = new HRegionInfo(region);
-
// Force ZK node to OFFLINE so master assigns
- forceOfflineInZK(admin, actualRegion);
+ forceOfflineInZK(admin, region);
}
/**
@@ -103,14 +99,12 @@ public class HBaseFsckRepair {
* in ZK to have HBCK_CODE_NAME as the server. This is a special case in
* the AssignmentManager that attempts an assign call by the master.
*
- * @see org.apache.hadoop.hbase.master.AssignementManager#handleHBCK
- *
* This doesn't seem to work properly in the updated version of 0.92+'s hbck
* so we use assign to force the region into transition. This has the
- * side-effect of requiring a HRegionInfo that considers regionId (timestamp)
+ * side-effect of requiring a RegionInfo that considers regionId (timestamp)
* in comparators that is addressed by HBASE-5563.
*/
- private static void forceOfflineInZK(Admin admin, final HRegionInfo region)
+ private static void forceOfflineInZK(Admin admin, final RegionInfo region)
throws ZooKeeperConnectionException, KeeperException, IOException, InterruptedException {
admin.assign(region.getRegionName());
}
@@ -119,7 +113,7 @@ public class HBaseFsckRepair {
* Should we check all assignments or just not in RIT?
*/
public static void waitUntilAssigned(Admin admin,
- HRegionInfo region) throws IOException, InterruptedException {
+ RegionInfo region) throws IOException, InterruptedException {
long timeout = admin.getConfiguration().getLong("hbase.hbck.assign.timeout", 120000);
long expiration = timeout + EnvironmentEdgeManager.currentTime();
while (EnvironmentEdgeManager.currentTime() < expiration) {
@@ -127,7 +121,7 @@ public class HBaseFsckRepair {
boolean inTransition = false;
for (RegionState rs : admin.getClusterStatus(EnumSet.of(Option.REGIONS_IN_TRANSITION))
.getRegionsInTransition()) {
- if (rs.getRegion().equals(region)) {
+ if (RegionInfo.COMPARATOR.compare(rs.getRegion(), region) == 0) {
inTransition = true;
break;
}
@@ -155,7 +149,7 @@ public class HBaseFsckRepair {
*/
@SuppressWarnings("deprecation")
public static void closeRegionSilentlyAndWait(Connection connection,
- ServerName server, HRegionInfo region) throws IOException, InterruptedException {
+ ServerName server, RegionInfo region) throws IOException, InterruptedException {
long timeout = connection.getConfiguration()
.getLong("hbase.hbck.close.timeout", 120000);
ServerManager.closeRegionSilentlyAndWait((ClusterConnection)connection, server,
@@ -163,10 +157,10 @@ public class HBaseFsckRepair {
}
/**
- * Puts the specified HRegionInfo into META with replica related columns
+ * Puts the specified RegionInfo into META with replica related columns
*/
public static void fixMetaHoleOnlineAndAddReplicas(Configuration conf,
- HRegionInfo hri, Collection<ServerName> servers, int numReplicas) throws IOException {
+ RegionInfo hri, Collection<ServerName> servers, int numReplicas) throws IOException {
Connection conn = ConnectionFactory.createConnection(conf);
Table meta = conn.getTable(TableName.META_TABLE_NAME);
Put put = MetaTableAccessor.makePutFromRegionInfo(hri);
@@ -191,7 +185,7 @@ public class HBaseFsckRepair {
* Creates, flushes, and closes a new region.
*/
public static HRegion createHDFSRegionDir(Configuration conf,
- HRegionInfo hri, TableDescriptor htd) throws IOException {
+ RegionInfo hri, TableDescriptor htd) throws IOException {
// Create HRegion
Path root = FSUtils.getRootDir(conf);
HRegion region = HRegion.createHRegion(hri, root, conf, htd, null);
@@ -204,7 +198,7 @@ public class HBaseFsckRepair {
/*
* Remove parent
*/
- public static void removeParentInMeta(Configuration conf, HRegionInfo hri) throws IOException {
+ public static void removeParentInMeta(Configuration conf, RegionInfo hri) throws IOException {
Connection conn = ConnectionFactory.createConnection(conf);
MetaTableAccessor.deleteRegion(conn, hri);
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HFileArchiveUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HFileArchiveUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HFileArchiveUtil.java
index 8e3e105..fb99cba 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HFileArchiveUtil.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HFileArchiveUtil.java
@@ -21,9 +21,9 @@ import java.io.IOException;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HStore;
@@ -61,7 +61,7 @@ public class HFileArchiveUtil {
* not be archived
*/
public static Path getStoreArchivePath(Configuration conf,
- HRegionInfo region,
+ RegionInfo region,
Path tabledir,
byte[] family) throws IOException {
return getStoreArchivePath(conf, region, family);
@@ -76,7 +76,7 @@ public class HFileArchiveUtil {
* not be archived
*/
public static Path getStoreArchivePath(Configuration conf,
- HRegionInfo region,
+ RegionInfo region,
byte[] family) throws IOException {
Path rootDir = FSUtils.getRootDir(conf);
Path tableArchiveDir = getTableArchivePath(rootDir, region.getTable());
@@ -146,7 +146,7 @@ public class HFileArchiveUtil {
}
/**
- * Get the full path to the archive directory on the configured
+ * Get the full path to the archive directory on the configured
* {@link org.apache.hadoop.hbase.master.MasterFileSystem}
* @param conf to look for archive directory name and root directory. Cannot be null. Notes for
* testing: requires a FileSystem root directory to be specified.
@@ -158,7 +158,7 @@ public class HFileArchiveUtil {
}
/**
- * Get the full path to the archive directory on the configured
+ * Get the full path to the archive directory on the configured
* {@link org.apache.hadoop.hbase.master.MasterFileSystem}
* @param rootdir {@link Path} to the root directory where hbase files are stored (for building
* the archive path)
@@ -167,7 +167,7 @@ public class HFileArchiveUtil {
private static Path getArchivePath(final Path rootdir) {
return new Path(rootdir, HConstants.HFILE_ARCHIVE_DIRECTORY);
}
-
+
/*
* @return table name given archive file path
*/
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ModifyRegionUtils.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ModifyRegionUtils.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ModifyRegionUtils.java
index 53c6d7a..fe33c24 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ModifyRegionUtils.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ModifyRegionUtils.java
@@ -34,13 +34,14 @@ import java.util.concurrent.TimeUnit;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.yetus.audience.InterfaceAudience;
/**
* Utility methods for interacting with the regions.
@@ -57,27 +58,36 @@ public abstract class ModifyRegionUtils {
}
public interface RegionEditTask {
- void editRegion(final HRegionInfo region) throws IOException;
+ void editRegion(final RegionInfo region) throws IOException;
}
- public static HRegionInfo[] createHRegionInfos(TableDescriptor tableDescriptor,
+ public static RegionInfo[] createRegionInfos(TableDescriptor tableDescriptor,
byte[][] splitKeys) {
long regionId = System.currentTimeMillis();
- HRegionInfo[] hRegionInfos = null;
+ RegionInfo[] hRegionInfos = null;
if (splitKeys == null || splitKeys.length == 0) {
- hRegionInfos = new HRegionInfo[]{
- new HRegionInfo(tableDescriptor.getTableName(), null, null, false, regionId)
+ hRegionInfos = new RegionInfo[]{
+ RegionInfoBuilder.newBuilder(tableDescriptor.getTableName())
+ .setStartKey(null)
+ .setEndKey(null)
+ .setSplit(false)
+ .setRegionId(regionId)
+ .build()
};
} else {
int numRegions = splitKeys.length + 1;
- hRegionInfos = new HRegionInfo[numRegions];
+ hRegionInfos = new RegionInfo[numRegions];
byte[] startKey = null;
byte[] endKey = null;
for (int i = 0; i < numRegions; i++) {
endKey = (i == splitKeys.length) ? null : splitKeys[i];
hRegionInfos[i] =
- new HRegionInfo(tableDescriptor.getTableName(), startKey, endKey,
- false, regionId);
+ RegionInfoBuilder.newBuilder(tableDescriptor.getTableName())
+ .setStartKey(startKey)
+ .setEndKey(endKey)
+ .setSplit(false)
+ .setRegionId(regionId)
+ .build();
startKey = endKey;
}
}
@@ -91,12 +101,12 @@ public abstract class ModifyRegionUtils {
* @param conf {@link Configuration}
* @param rootDir Root directory for HBase instance
* @param tableDescriptor description of the table
- * @param newRegions {@link HRegionInfo} that describes the regions to create
+ * @param newRegions {@link RegionInfo} that describes the regions to create
* @param task {@link RegionFillTask} custom code to populate region after creation
* @throws IOException
*/
- public static List<HRegionInfo> createRegions(final Configuration conf, final Path rootDir,
- final TableDescriptor tableDescriptor, final HRegionInfo[] newRegions,
+ public static List<RegionInfo> createRegions(final Configuration conf, final Path rootDir,
+ final TableDescriptor tableDescriptor, final RegionInfo[] newRegions,
final RegionFillTask task) throws IOException {
if (newRegions == null) return null;
int regionNumber = newRegions.length;
@@ -117,22 +127,22 @@ public abstract class ModifyRegionUtils {
* @param conf {@link Configuration}
* @param rootDir Root directory for HBase instance
* @param tableDescriptor description of the table
- * @param newRegions {@link HRegionInfo} that describes the regions to create
+ * @param newRegions {@link RegionInfo} that describes the regions to create
* @param task {@link RegionFillTask} custom code to populate region after creation
* @throws IOException
*/
- public static List<HRegionInfo> createRegions(final ThreadPoolExecutor exec,
+ public static List<RegionInfo> createRegions(final ThreadPoolExecutor exec,
final Configuration conf, final Path rootDir,
- final TableDescriptor tableDescriptor, final HRegionInfo[] newRegions,
+ final TableDescriptor tableDescriptor, final RegionInfo[] newRegions,
final RegionFillTask task) throws IOException {
if (newRegions == null) return null;
int regionNumber = newRegions.length;
- CompletionService<HRegionInfo> completionService = new ExecutorCompletionService<>(exec);
- List<HRegionInfo> regionInfos = new ArrayList<>();
- for (final HRegionInfo newRegion : newRegions) {
- completionService.submit(new Callable<HRegionInfo>() {
+ CompletionService<RegionInfo> completionService = new ExecutorCompletionService<>(exec);
+ List<RegionInfo> regionInfos = new ArrayList<>();
+ for (final RegionInfo newRegion : newRegions) {
+ completionService.submit(new Callable<RegionInfo>() {
@Override
- public HRegionInfo call() throws IOException {
+ public RegionInfo call() throws IOException {
return createRegion(conf, rootDir, tableDescriptor, newRegion, task);
}
});
@@ -156,12 +166,12 @@ public abstract class ModifyRegionUtils {
* @param conf {@link Configuration}
* @param rootDir Root directory for HBase instance
* @param tableDescriptor description of the table
- * @param newRegion {@link HRegionInfo} that describes the region to create
+ * @param newRegion {@link RegionInfo} that describes the region to create
* @param task {@link RegionFillTask} custom code to populate region after creation
* @throws IOException
*/
- public static HRegionInfo createRegion(final Configuration conf, final Path rootDir,
- final TableDescriptor tableDescriptor, final HRegionInfo newRegion,
+ public static RegionInfo createRegion(final Configuration conf, final Path rootDir,
+ final TableDescriptor tableDescriptor, final RegionInfo newRegion,
final RegionFillTask task) throws IOException {
// 1. Create HRegion
// The WAL subsystem will use the default rootDir rather than the passed in rootDir
@@ -185,14 +195,14 @@ public abstract class ModifyRegionUtils {
* Execute the task on the specified set of regions.
*
* @param exec Thread Pool Executor
- * @param regions {@link HRegionInfo} that describes the regions to edit
+ * @param regions {@link RegionInfo} that describes the regions to edit
* @param task {@link RegionFillTask} custom code to edit the region
* @throws IOException
*/
public static void editRegions(final ThreadPoolExecutor exec,
- final Collection<HRegionInfo> regions, final RegionEditTask task) throws IOException {
+ final Collection<RegionInfo> regions, final RegionEditTask task) throws IOException {
final ExecutorCompletionService<Void> completionService = new ExecutorCompletionService<>(exec);
- for (final HRegionInfo hri: regions) {
+ for (final RegionInfo hri: regions) {
completionService.submit(new Callable<Void>() {
@Override
public Void call() throws IOException {
@@ -203,7 +213,7 @@ public abstract class ModifyRegionUtils {
}
try {
- for (HRegionInfo hri: regions) {
+ for (RegionInfo hri: regions) {
completionService.take().get();
}
} catch (InterruptedException e) {
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionMover.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionMover.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionMover.java
index 4d9e24c..0f36a7b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionMover.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionMover.java
@@ -43,20 +43,19 @@ import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import org.apache.commons.cli.CommandLine;
-import org.apache.hadoop.conf.Configuration;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.ClusterStatus.Option;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.hbase.ClusterStatus.Option;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
@@ -64,6 +63,7 @@ import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.yetus.audience.InterfaceAudience;
/**
* Tool for loading/unloading regions to/from given regionserver This tool can be run from Command
@@ -253,7 +253,7 @@ public class RegionMover extends AbstractHBaseTool {
public Boolean call() throws IOException {
Connection conn = ConnectionFactory.createConnection(rm.conf);
try {
- List<HRegionInfo> regionsToMove = readRegionsFromFile(rm.filename);
+ List<RegionInfo> regionsToMove = readRegionsFromFile(rm.filename);
if (regionsToMove.isEmpty()) {
LOG.info("No regions to load.Exiting");
return true;
@@ -313,7 +313,7 @@ public class RegionMover extends AbstractHBaseTool {
private class Unload implements Callable<Boolean> {
- List<HRegionInfo> movedRegions = Collections.synchronizedList(new ArrayList<HRegionInfo>());
+ List<RegionInfo> movedRegions = Collections.synchronizedList(new ArrayList<RegionInfo>());
private RegionMover rm;
public Unload(RegionMover rm) {
@@ -366,9 +366,9 @@ public class RegionMover extends AbstractHBaseTool {
}
private void loadRegions(Admin admin, String hostname, int port,
- List<HRegionInfo> regionsToMove, boolean ack) throws Exception {
+ List<RegionInfo> regionsToMove, boolean ack) throws Exception {
String server = null;
- List<HRegionInfo> movedRegions = Collections.synchronizedList(new ArrayList<HRegionInfo>());
+ List<RegionInfo> movedRegions = Collections.synchronizedList(new ArrayList<RegionInfo>());
int maxWaitInSeconds =
admin.getConfiguration().getInt(SERVERSTART_WAIT_MAX_KEY, DEFAULT_SERVERSTART_WAIT_MAX);
long maxWait = EnvironmentEdgeManager.currentTime() + maxWaitInSeconds * 1000;
@@ -402,7 +402,7 @@ public class RegionMover extends AbstractHBaseTool {
List<Future<Boolean>> taskList = new ArrayList<>();
int counter = 0;
while (counter < regionsToMove.size()) {
- HRegionInfo region = regionsToMove.get(counter);
+ RegionInfo region = regionsToMove.get(counter);
String currentServer = getServerNameForRegion(admin, region);
if (currentServer == null) {
LOG.warn("Could not get server for Region:" + region.getEncodedName() + " moving on");
@@ -462,8 +462,8 @@ public class RegionMover extends AbstractHBaseTool {
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="DLS_DEAD_LOCAL_STORE",
justification="FB is wrong; its size is read")
private void unloadRegions(Admin admin, String server, ArrayList<String> regionServers,
- boolean ack, List<HRegionInfo> movedRegions) throws Exception {
- List<HRegionInfo> regionsToMove = new ArrayList<>();// FindBugs: DLS_DEAD_LOCAL_STORE
+ boolean ack, List<RegionInfo> movedRegions) throws Exception {
+ List<RegionInfo> regionsToMove = new ArrayList<>();// FindBugs: DLS_DEAD_LOCAL_STORE
regionsToMove = getRegions(this.conf, server);
if (regionsToMove.isEmpty()) {
LOG.info("No Regions to move....Quitting now");
@@ -540,13 +540,13 @@ public class RegionMover extends AbstractHBaseTool {
*/
private class MoveWithAck implements Callable<Boolean> {
private Admin admin;
- private HRegionInfo region;
+ private RegionInfo region;
private String targetServer;
- private List<HRegionInfo> movedRegions;
+ private List<RegionInfo> movedRegions;
private String sourceServer;
- public MoveWithAck(Admin admin, HRegionInfo regionInfo, String sourceServer,
- String targetServer, List<HRegionInfo> movedRegions) {
+ public MoveWithAck(Admin admin, RegionInfo regionInfo, String sourceServer,
+ String targetServer, List<RegionInfo> movedRegions) {
this.admin = admin;
this.region = regionInfo;
this.targetServer = targetServer;
@@ -605,13 +605,13 @@ public class RegionMover extends AbstractHBaseTool {
*/
private static class MoveWithoutAck implements Callable<Boolean> {
private Admin admin;
- private HRegionInfo region;
+ private RegionInfo region;
private String targetServer;
- private List<HRegionInfo> movedRegions;
+ private List<RegionInfo> movedRegions;
private String sourceServer;
- public MoveWithoutAck(Admin admin, HRegionInfo regionInfo, String sourceServer,
- String targetServer, List<HRegionInfo> movedRegions) {
+ public MoveWithoutAck(Admin admin, RegionInfo regionInfo, String sourceServer,
+ String targetServer, List<RegionInfo> movedRegions) {
this.admin = admin;
this.region = regionInfo;
this.targetServer = targetServer;
@@ -637,8 +637,8 @@ public class RegionMover extends AbstractHBaseTool {
}
}
- private List<HRegionInfo> readRegionsFromFile(String filename) throws IOException {
- List<HRegionInfo> regions = new ArrayList<>();
+ private List<RegionInfo> readRegionsFromFile(String filename) throws IOException {
+ List<RegionInfo> regions = new ArrayList<>();
File f = new File(filename);
if (!f.exists()) {
return regions;
@@ -651,7 +651,7 @@ public class RegionMover extends AbstractHBaseTool {
int numRegions = dis.readInt();
int index = 0;
while (index < numRegions) {
- regions.add(HRegionInfo.parseFromOrNull(Bytes.readByteArray(dis)));
+ regions.add(RegionInfo.parseFromOrNull(Bytes.readByteArray(dis)));
index++;
}
} catch (IOException e) {
@@ -675,10 +675,10 @@ public class RegionMover extends AbstractHBaseTool {
* @return List of Regions online on the server
* @throws IOException
*/
- private List<HRegionInfo> getRegions(Configuration conf, String server) throws IOException {
+ private List<RegionInfo> getRegions(Configuration conf, String server) throws IOException {
Connection conn = ConnectionFactory.createConnection(conf);
try {
- return conn.getAdmin().getOnlineRegions(ServerName.valueOf(server));
+ return conn.getAdmin().getRegions(ServerName.valueOf(server));
} finally {
conn.close();
}
@@ -691,15 +691,15 @@ public class RegionMover extends AbstractHBaseTool {
* @param movedRegions
* @throws IOException
*/
- private void writeFile(String filename, List<HRegionInfo> movedRegions) throws IOException {
+ private void writeFile(String filename, List<RegionInfo> movedRegions) throws IOException {
FileOutputStream fos = null;
DataOutputStream dos = null;
try {
fos = new FileOutputStream(filename);
dos = new DataOutputStream(fos);
dos.writeInt(movedRegions.size());
- for (HRegionInfo region : movedRegions) {
- Bytes.writeByteArray(dos, region.toByteArray());
+ for (RegionInfo region : movedRegions) {
+ Bytes.writeByteArray(dos, RegionInfo.toByteArray(region));
}
} catch (IOException e) {
LOG.error("ERROR: Was Not able to write regions moved to output file but moved "
@@ -846,7 +846,7 @@ public class RegionMover extends AbstractHBaseTool {
* @param region
* @throws IOException
*/
- private void isSuccessfulScan(Admin admin, HRegionInfo region) throws IOException {
+ private void isSuccessfulScan(Admin admin, RegionInfo region) throws IOException {
Scan scan = new Scan(region.getStartKey());
scan.setBatch(1);
scan.setCaching(1);
@@ -877,7 +877,7 @@ public class RegionMover extends AbstractHBaseTool {
* @return true if region is hosted on serverName otherwise false
* @throws IOException
*/
- private boolean isSameServer(Admin admin, HRegionInfo region, String serverName)
+ private boolean isSameServer(Admin admin, RegionInfo region, String serverName)
throws IOException {
String serverForRegion = getServerNameForRegion(admin, region);
if (serverForRegion != null && serverForRegion.equals(serverName)) {
@@ -894,7 +894,7 @@ public class RegionMover extends AbstractHBaseTool {
* @return regionServer hosting the given region
* @throws IOException
*/
- private String getServerNameForRegion(Admin admin, HRegionInfo region) throws IOException {
+ private String getServerNameForRegion(Admin admin, RegionInfo region) throws IOException {
String server = null;
if (!admin.isTableEnabled(region.getTable())) {
return null;
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ServerRegionReplicaUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ServerRegionReplicaUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ServerRegionReplicaUtil.java
index 34a9759..9b61b8b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ServerRegionReplicaUtil.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ServerRegionReplicaUtil.java
@@ -25,18 +25,15 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ReplicationPeerNotFoundException;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
-import org.apache.hadoop.hbase.client.replication.ReplicationAdmin;
import org.apache.hadoop.hbase.io.HFileLink;
import org.apache.hadoop.hbase.io.Reference;
-import org.apache.hadoop.hbase.quotas.MasterQuotaManager;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
-import org.apache.hadoop.hbase.replication.ReplicationException;
import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
import org.apache.hadoop.hbase.replication.regionserver.RegionReplicaReplicationEndpoint;
import org.apache.hadoop.hbase.zookeeper.ZKConfig;
@@ -82,9 +79,9 @@ public class ServerRegionReplicaUtil extends RegionReplicaUtil {
/**
* Returns the regionInfo object to use for interacting with the file system.
- * @return An HRegionInfo object to interact with the filesystem
+ * @return An RegionInfo object to interact with the filesystem
*/
- public static HRegionInfo getRegionInfoForFs(HRegionInfo regionInfo) {
+ public static RegionInfo getRegionInfoForFs(RegionInfo regionInfo) {
if (regionInfo == null) {
return null;
}
@@ -121,11 +118,11 @@ public class ServerRegionReplicaUtil extends RegionReplicaUtil {
* @throws IOException
*/
public static StoreFileInfo getStoreFileInfo(Configuration conf, FileSystem fs,
- HRegionInfo regionInfo, HRegionInfo regionInfoForFs, String familyName, Path path)
+ RegionInfo regionInfo, RegionInfo regionInfoForFs, String familyName, Path path)
throws IOException {
// if this is a primary region, just return the StoreFileInfo constructed from path
- if (regionInfo.equals(regionInfoForFs)) {
+ if (RegionInfo.COMPARATOR.compare(regionInfo, regionInfoForFs) == 0) {
return new StoreFileInfo(conf, fs, path);
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DisabledWALProvider.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DisabledWALProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DisabledWALProvider.java
index a6d43d6..652aa2f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DisabledWALProvider.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DisabledWALProvider.java
@@ -32,14 +32,14 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil;
-import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
-
-// imports for things that haven't moved from regionserver.wal yet.
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
import org.apache.hadoop.hbase.regionserver.wal.WALCoprocessorHost;
import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.yetus.audience.InterfaceAudience;
+
+// imports for things that haven't moved from regionserver.wal yet.
/**
* No-op implementation of {@link WALProvider} used when the WAL is disabled.
@@ -109,7 +109,7 @@ class DisabledWALProvider implements WALProvider {
public void registerWALActionsListener(final WALActionsListener listener) {
listeners.add(listener);
}
-
+
@Override
public boolean unregisterWALActionsListener(final WALActionsListener listener) {
return listeners.remove(listener);
@@ -161,7 +161,7 @@ class DisabledWALProvider implements WALProvider {
}
@Override
- public long append(HRegionInfo info, WALKey key, WALEdit edits, boolean inMemstore)
+ public long append(RegionInfo info, WALKey key, WALEdit edits, boolean inMemstore)
throws IOException {
if (!this.listeners.isEmpty()) {
final long start = System.nanoTime();
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WAL.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WAL.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WAL.java
index 9ec58ab..886ec78 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WAL.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WAL.java
@@ -25,17 +25,19 @@ import java.util.Map;
import java.util.Set;
import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.yetus.audience.InterfaceStability;
-// imports we use from yet-to-be-moved regionsever.wal
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.regionserver.wal.CompressionContext;
import org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException;
import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
import org.apache.hadoop.hbase.regionserver.wal.WALCoprocessorHost;
import org.apache.hadoop.hbase.replication.regionserver.WALFileLengthProvider;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.yetus.audience.InterfaceStability;
+
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
+// imports we use from yet-to-be-moved regionsever.wal
+
/**
* A Write Ahead Log (WAL) provides service for reading, writing waledits. This interface provides
* APIs for WAL users (such as RegionServer) to use the WAL (do append, sync, etc).
@@ -66,7 +68,7 @@ public interface WAL extends Closeable, WALFileLengthProvider {
*
* @return If lots of logs, flush the returned regions so next time through we
* can clean logs. Returns null if nothing to flush. Names are actual
- * region names as returned by {@link HRegionInfo#getEncodedName()}
+ * region names as returned by {@link RegionInfo#getEncodedName()}
*/
byte[][] rollWriter() throws FailedLogCloseException, IOException;
@@ -82,7 +84,7 @@ public interface WAL extends Closeable, WALFileLengthProvider {
* been written to the current writer
* @return If lots of logs, flush the returned regions so next time through we
* can clean logs. Returns null if nothing to flush. Names are actual
- * region names as returned by {@link HRegionInfo#getEncodedName()}
+ * region names as returned by {@link RegionInfo#getEncodedName()}
*/
byte[][] rollWriter(boolean force) throws FailedLogCloseException, IOException;
@@ -114,7 +116,7 @@ public interface WAL extends Closeable, WALFileLengthProvider {
* @return Returns a 'transaction id' and <code>key</code> will have the region edit/sequence id
* in it.
*/
- long append(HRegionInfo info, WALKey key, WALEdit edits, boolean inMemstore) throws IOException;
+ long append(RegionInfo info, WALKey key, WALEdit edits, boolean inMemstore) throws IOException;
/**
* updates the seuence number of a specific store.
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALEdit.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALEdit.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALEdit.java
index 6a30f9c..260e6db 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALEdit.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALEdit.java
@@ -26,20 +26,20 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.KeyValue;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.codec.Codec;
import org.apache.hadoop.hbase.io.HeapSize;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.CompactionDescriptor;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.RegionEventDescriptor;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.ClassSize;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.CompactionDescriptor;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.RegionEventDescriptor;
/**
@@ -192,7 +192,7 @@ public class WALEdit implements HeapSize {
return sb.toString();
}
- public static WALEdit createFlushWALEdit(HRegionInfo hri, FlushDescriptor f) {
+ public static WALEdit createFlushWALEdit(RegionInfo hri, FlushDescriptor f) {
KeyValue kv = new KeyValue(getRowForRegion(hri), METAFAMILY, FLUSH,
EnvironmentEdgeManager.currentTime(), f.toByteArray());
return new WALEdit().add(kv);
@@ -205,7 +205,7 @@ public class WALEdit implements HeapSize {
return null;
}
- public static WALEdit createRegionEventWALEdit(HRegionInfo hri,
+ public static WALEdit createRegionEventWALEdit(RegionInfo hri,
RegionEventDescriptor regionEventDesc) {
KeyValue kv = new KeyValue(getRowForRegion(hri), METAFAMILY, REGION_EVENT,
EnvironmentEdgeManager.currentTime(), regionEventDesc.toByteArray());
@@ -224,14 +224,14 @@ public class WALEdit implements HeapSize {
* @param c
* @return A WALEdit that has <code>c</code> serialized as its value
*/
- public static WALEdit createCompaction(final HRegionInfo hri, final CompactionDescriptor c) {
+ public static WALEdit createCompaction(final RegionInfo hri, final CompactionDescriptor c) {
byte [] pbbytes = c.toByteArray();
KeyValue kv = new KeyValue(getRowForRegion(hri), METAFAMILY, COMPACTION,
EnvironmentEdgeManager.currentTime(), pbbytes);
return new WALEdit().add(kv); //replication scope null so that this won't be replicated
}
- public static byte[] getRowForRegion(HRegionInfo hri) {
+ public static byte[] getRowForRegion(RegionInfo hri) {
byte[] startKey = hri.getStartKey();
if (startKey.length == 0) {
// empty row key is not allowed in mutations because it is both the start key and the end key
@@ -265,11 +265,11 @@ public class WALEdit implements HeapSize {
/**
* Create a bulk loader WALEdit
*
- * @param hri The HRegionInfo for the region in which we are bulk loading
+ * @param hri The RegionInfo for the region in which we are bulk loading
* @param bulkLoadDescriptor The descriptor for the Bulk Loader
* @return The WALEdit for the BulkLoad
*/
- public static WALEdit createBulkLoadEvent(HRegionInfo hri,
+ public static WALEdit createBulkLoadEvent(RegionInfo hri,
WALProtos.BulkLoadDescriptor bulkLoadDescriptor) {
KeyValue kv = new KeyValue(getRowForRegion(hri),
METAFAMILY,
[14/19] hbase git commit: HBASE-18839 Apply RegionInfo to code base
Posted by ch...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionPlacementMaintainer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionPlacementMaintainer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionPlacementMaintainer.java
index d23cf7d..afd402b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionPlacementMaintainer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionPlacementMaintainer.java
@@ -40,31 +40,32 @@ import org.apache.commons.cli.ParseException;
import org.apache.commons.lang3.StringUtils;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
-import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.hbase.ClusterStatus.Option;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.ClusterStatus.Option;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.ClusterConnection;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.favored.FavoredNodeAssignmentHelper;
import org.apache.hadoop.hbase.favored.FavoredNodesPlan;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesResponse;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.MunkresAssignment;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.log4j.Level;
import org.apache.log4j.Logger;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesResponse;
/**
* A tool that is used for manipulating and viewing favored nodes information
@@ -197,12 +198,12 @@ public class RegionPlacementMaintainer {
Map<String, Map<String, Float>> regionLocalityMap, FavoredNodesPlan plan,
boolean munkresForSecondaryAndTertiary) throws IOException {
// Get the all the regions for the current table
- List<HRegionInfo> regions =
+ List<RegionInfo> regions =
assignmentSnapshot.getTableToRegionMap().get(tableName);
int numRegions = regions.size();
// Get the current assignment map
- Map<HRegionInfo, ServerName> currentAssignmentMap =
+ Map<RegionInfo, ServerName> currentAssignmentMap =
assignmentSnapshot.getRegionToRegionServerMap();
// Get the all the region servers
@@ -257,12 +258,12 @@ public class RegionPlacementMaintainer {
// Compute the total rack locality for each region in each rack. The total
// rack locality is the sum of the localities of a region on all servers in
// a rack.
- Map<String, Map<HRegionInfo, Float>> rackRegionLocality = new HashMap<>();
+ Map<String, Map<RegionInfo, Float>> rackRegionLocality = new HashMap<>();
for (int i = 0; i < numRegions; i++) {
- HRegionInfo region = regions.get(i);
+ RegionInfo region = regions.get(i);
for (int j = 0; j < regionSlots; j += slotsPerServer) {
String rack = rackManager.getRack(servers.get(j / slotsPerServer));
- Map<HRegionInfo, Float> rackLocality = rackRegionLocality.get(rack);
+ Map<RegionInfo, Float> rackLocality = rackRegionLocality.get(rack);
if (rackLocality == null) {
rackLocality = new HashMap<>();
rackRegionLocality.put(rack, rackLocality);
@@ -417,18 +418,18 @@ public class RegionPlacementMaintainer {
LOG.info("Assignment plan for secondary and tertiary generated " +
"using MunkresAssignment");
} else {
- Map<HRegionInfo, ServerName> primaryRSMap = new HashMap<>();
+ Map<RegionInfo, ServerName> primaryRSMap = new HashMap<>();
for (int i = 0; i < numRegions; i++) {
primaryRSMap.put(regions.get(i), servers.get(primaryAssignment[i] / slotsPerServer));
}
FavoredNodeAssignmentHelper favoredNodeHelper =
new FavoredNodeAssignmentHelper(servers, conf);
favoredNodeHelper.initialize();
- Map<HRegionInfo, ServerName[]> secondaryAndTertiaryMap =
+ Map<RegionInfo, ServerName[]> secondaryAndTertiaryMap =
favoredNodeHelper.placeSecondaryAndTertiaryWithRestrictions(primaryRSMap);
for (int i = 0; i < numRegions; i++) {
List<ServerName> favoredServers = new ArrayList<>(FavoredNodeAssignmentHelper.FAVORED_NODES_NUM);
- HRegionInfo currentRegion = regions.get(i);
+ RegionInfo currentRegion = regions.get(i);
ServerName s = primaryRSMap.get(currentRegion);
favoredServers.add(ServerName.valueOf(s.getHostname(), s.getPort(),
ServerName.NON_STARTCODE));
@@ -467,7 +468,7 @@ public class RegionPlacementMaintainer {
FavoredNodesPlan plan = new FavoredNodesPlan();
// Get the table to region mapping
- Map<TableName, List<HRegionInfo>> tableToRegionMap =
+ Map<TableName, List<RegionInfo>> tableToRegionMap =
assignmentSnapshot.getTableToRegionMap();
LOG.info("Start to generate the new assignment plan for the " +
+ tableToRegionMap.keySet().size() + " tables" );
@@ -635,8 +636,8 @@ public class RegionPlacementMaintainer {
try {
LOG.info("Start to update the hbase:meta with the new assignment plan");
Map<String, List<ServerName>> assignmentMap = plan.getAssignmentMap();
- Map<HRegionInfo, List<ServerName>> planToUpdate = new HashMap<>(assignmentMap.size());
- Map<String, HRegionInfo> regionToRegionInfoMap =
+ Map<RegionInfo, List<ServerName>> planToUpdate = new HashMap<>(assignmentMap.size());
+ Map<String, RegionInfo> regionToRegionInfoMap =
getRegionAssignmentSnapshot().getRegionNameToRegionInfoMap();
for (Map.Entry<String, List<ServerName>> entry : assignmentMap.entrySet()) {
planToUpdate.put(regionToRegionInfoMap.get(entry.getKey()), entry.getValue());
@@ -659,21 +660,21 @@ public class RegionPlacementMaintainer {
throws IOException{
LOG.info("Start to update the region servers with the new assignment plan");
// Get the region to region server map
- Map<ServerName, List<HRegionInfo>> currentAssignment =
+ Map<ServerName, List<RegionInfo>> currentAssignment =
this.getRegionAssignmentSnapshot().getRegionServerToRegionMap();
// track of the failed and succeeded updates
int succeededNum = 0;
Map<ServerName, Exception> failedUpdateMap = new HashMap<>();
- for (Map.Entry<ServerName, List<HRegionInfo>> entry :
+ for (Map.Entry<ServerName, List<RegionInfo>> entry :
currentAssignment.entrySet()) {
- List<Pair<HRegionInfo, List<ServerName>>> regionUpdateInfos = new ArrayList<>();
+ List<Pair<RegionInfo, List<ServerName>>> regionUpdateInfos = new ArrayList<>();
try {
// Keep track of the favored updates for the current region server
FavoredNodesPlan singleServerPlan = null;
// Find out all the updates for the current region server
- for (HRegionInfo region : entry.getValue()) {
+ for (RegionInfo region : entry.getValue()) {
List<ServerName> favoredServerList = plan.getFavoredNodes(region);
if (favoredServerList != null &&
favoredServerList.size() == FavoredNodeAssignmentHelper.FAVORED_NODES_NUM) {
@@ -746,7 +747,7 @@ public class RegionPlacementMaintainer {
throws IOException {
Map<TableName, Integer> movesPerTable = new HashMap<>();
SnapshotOfRegionAssignmentFromMeta snapshot = this.getRegionAssignmentSnapshot();
- Map<TableName, List<HRegionInfo>> tableToRegions = snapshot
+ Map<TableName, List<RegionInfo>> tableToRegions = snapshot
.getTableToRegionMap();
FavoredNodesPlan oldPlan = snapshot.getExistingAssignmentPlan();
Set<TableName> tables = snapshot.getTableSet();
@@ -756,8 +757,8 @@ public class RegionPlacementMaintainer {
&& !this.targetTableSet.contains(table)) {
continue;
}
- List<HRegionInfo> regions = tableToRegions.get(table);
- for (HRegionInfo region : regions) {
+ List<RegionInfo> regions = tableToRegions.get(table);
+ for (RegionInfo region : regions) {
List<ServerName> oldServers = oldPlan.getFavoredNodes(region);
List<ServerName> newServers = newPlan.getFavoredNodes(region);
if (oldServers != null && newServers != null) {
@@ -789,7 +790,7 @@ public class RegionPlacementMaintainer {
SnapshotOfRegionAssignmentFromMeta snapshot = this.getRegionAssignmentSnapshot();
FavoredNodesPlan oldPlan = snapshot.getExistingAssignmentPlan();
Set<TableName> tables = snapshot.getTableSet();
- Map<TableName, List<HRegionInfo>> tableToRegionsMap = snapshot.getTableToRegionMap();
+ Map<TableName, List<RegionInfo>> tableToRegionsMap = snapshot.getTableToRegionMap();
for (TableName table : tables) {
float[] deltaLocality = new float[3];
float[] locality = new float[3];
@@ -797,13 +798,13 @@ public class RegionPlacementMaintainer {
&& !this.targetTableSet.contains(table)) {
continue;
}
- List<HRegionInfo> regions = tableToRegionsMap.get(table);
+ List<RegionInfo> regions = tableToRegionsMap.get(table);
System.out.println("==================================================");
System.out.println("Assignment Plan Projection Report For Table: " + table);
System.out.println("\t Total regions: " + regions.size());
System.out.println("\t" + movesPerTable.get(table)
+ " primaries will move due to their primary has changed");
- for (HRegionInfo currentRegion : regions) {
+ for (RegionInfo currentRegion : regions) {
Map<String, Float> regionLocality = regionLocalityMap.get(currentRegion
.getEncodedName());
if (regionLocality == null) {
@@ -881,7 +882,7 @@ public class RegionPlacementMaintainer {
SnapshotOfRegionAssignmentFromMeta snapshot = this.getRegionAssignmentSnapshot();
FavoredNodesPlan assignmentPlan = snapshot.getExistingAssignmentPlan();
Set<TableName> tables = snapshot.getTableSet();
- Map<TableName, List<HRegionInfo>> tableToRegionsMap = snapshot
+ Map<TableName, List<RegionInfo>> tableToRegionsMap = snapshot
.getTableToRegionMap();
for (TableName table : tables) {
float[] locality = new float[3];
@@ -889,8 +890,8 @@ public class RegionPlacementMaintainer {
&& !this.targetTableSet.contains(table)) {
continue;
}
- List<HRegionInfo> regions = tableToRegionsMap.get(table);
- for (HRegionInfo currentRegion : regions) {
+ List<RegionInfo> regions = tableToRegionsMap.get(table);
+ for (RegionInfo currentRegion : regions) {
Map<String, Float> regionLocality = regionLocalityMap.get(currentRegion
.getEncodedName());
if (regionLocality == null) {
@@ -1094,7 +1095,7 @@ public class RegionPlacementMaintainer {
LOG.info("Going to update the region " + regionName + " with the new favored nodes " +
favoredNodesStr);
List<ServerName> favoredNodes = null;
- HRegionInfo regionInfo =
+ RegionInfo regionInfo =
rp.getRegionAssignmentSnapshot().getRegionNameToRegionInfoMap().get(regionName);
if (regionInfo == null) {
LOG.error("Cannot find the region " + regionName + " from the META");
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionPlan.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionPlan.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionPlan.java
index dbeaa58..10252df 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionPlan.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionPlan.java
@@ -20,10 +20,10 @@ package org.apache.hadoop.hbase.master;
import java.io.Serializable;
import java.util.Comparator;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.ServerName;
/**
* Stores the plan for the move of an individual region.
@@ -38,7 +38,7 @@ import org.apache.hadoop.hbase.ServerName;
@InterfaceAudience.LimitedPrivate("Coprocessors")
@InterfaceStability.Evolving
public class RegionPlan implements Comparable<RegionPlan> {
- private final HRegionInfo hri;
+ private final RegionInfo hri;
private final ServerName source;
private ServerName dest;
@@ -66,7 +66,7 @@ public class RegionPlan implements Comparable<RegionPlan> {
* @param source regionserver region should be moved from
* @param dest regionserver region should be moved to
*/
- public RegionPlan(final HRegionInfo hri, ServerName source, ServerName dest) {
+ public RegionPlan(final RegionInfo hri, ServerName source, ServerName dest) {
this.hri = hri;
this.source = source;
this.dest = dest;
@@ -103,7 +103,7 @@ public class RegionPlan implements Comparable<RegionPlan> {
return this.hri.getEncodedName();
}
- public HRegionInfo getRegionInfo() {
+ public RegionInfo getRegionInfo() {
return this.hri;
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
index 4dba66e..b63d55a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
@@ -43,32 +43,32 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.ClockOutOfSyncException;
import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.NotServingRegionException;
import org.apache.hadoop.hbase.RegionLoad;
import org.apache.hadoop.hbase.ServerLoad;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.YouAreDeadException;
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
-import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.ClusterConnection;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.RetriesExhaustedException;
import org.apache.hadoop.hbase.ipc.HBaseRpcController;
import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
-import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionStoreSequenceIds;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.StoreSequenceId;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStartupRequest;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.yetus.audience.InterfaceAudience;
import org.apache.zookeeper.KeeperException;
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionStoreSequenceIds;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.StoreSequenceId;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStartupRequest;
/**
* The ServerManager class manages info about region servers.
@@ -257,7 +257,7 @@ public class ServerManager {
private void updateLastFlushedSequenceIds(ServerName sn, ServerLoad hsl) {
Map<byte[], RegionLoad> regionsLoad = hsl.getRegionsLoad();
for (Entry<byte[], RegionLoad> entry : regionsLoad.entrySet()) {
- byte[] encodedRegionName = Bytes.toBytes(HRegionInfo.encodeRegionName(entry.getKey()));
+ byte[] encodedRegionName = Bytes.toBytes(RegionInfo.encodeRegionName(entry.getKey()));
Long existingValue = flushedSequenceIdByRegion.get(encodedRegionName);
long l = entry.getValue().getCompleteSequenceId();
// Don't let smaller sequence ids override greater sequence ids.
@@ -714,7 +714,7 @@ public class ServerManager {
* @param region region to warmup
*/
public void sendRegionWarmup(ServerName server,
- HRegionInfo region) {
+ RegionInfo region) {
if (server == null) return;
try {
AdminService.BlockingInterface admin = getRsAdmin(server);
@@ -732,7 +732,7 @@ public class ServerManager {
* to close the region. This bypasses the active hmaster.
*/
public static void closeRegionSilentlyAndWait(ClusterConnection connection,
- ServerName server, HRegionInfo region, long timeout) throws IOException, InterruptedException {
+ ServerName server, RegionInfo region, long timeout) throws IOException, InterruptedException {
AdminService.BlockingInterface rs = connection.getAdmin(server);
HBaseRpcController controller = connection.getRpcControllerFactory().newController();
try {
@@ -744,7 +744,7 @@ public class ServerManager {
while (System.currentTimeMillis() < expiration) {
controller.reset();
try {
- HRegionInfo rsRegion =
+ RegionInfo rsRegion =
ProtobufUtil.getRegionInfo(controller, rs, region.getRegionName());
if (rsRegion == null) return;
} catch (IOException ioe) {
@@ -1023,14 +1023,14 @@ public class ServerManager {
/**
* Called by delete table and similar to notify the ServerManager that a region was removed.
*/
- public void removeRegion(final HRegionInfo regionInfo) {
+ public void removeRegion(final RegionInfo regionInfo) {
final byte[] encodedName = regionInfo.getEncodedNameAsBytes();
storeFlushedSequenceIdsByRegion.remove(encodedName);
flushedSequenceIdByRegion.remove(encodedName);
}
@VisibleForTesting
- public boolean isRegionInServerManagerStates(final HRegionInfo hri) {
+ public boolean isRegionInServerManagerStates(final RegionInfo hri) {
final byte[] encodedName = hri.getEncodedNameAsBytes();
return (storeFlushedSequenceIdsByRegion.containsKey(encodedName)
|| flushedSequenceIdByRegion.containsKey(encodedName));
@@ -1039,8 +1039,8 @@ public class ServerManager {
/**
* Called by delete table and similar to notify the ServerManager that a region was removed.
*/
- public void removeRegions(final List<HRegionInfo> regions) {
- for (HRegionInfo hri: regions) {
+ public void removeRegions(final List<RegionInfo> regions) {
+ for (RegionInfo hri: regions) {
removeRegion(hri);
}
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SnapshotOfRegionAssignmentFromMeta.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SnapshotOfRegionAssignmentFromMeta.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SnapshotOfRegionAssignmentFromMeta.java
index 403a24e..5d889a4 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SnapshotOfRegionAssignmentFromMeta.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SnapshotOfRegionAssignmentFromMeta.java
@@ -34,19 +34,19 @@ import java.util.TreeMap;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.hbase.favored.FavoredNodeAssignmentHelper;
-import org.apache.hadoop.hbase.favored.FavoredNodesPlan;
import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.MetaTableAccessor;
+import org.apache.hadoop.hbase.MetaTableAccessor.Visitor;
import org.apache.hadoop.hbase.RegionLocations;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.MetaTableAccessor;
-import org.apache.hadoop.hbase.MetaTableAccessor.Visitor;
import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.favored.FavoredNodeAssignmentHelper;
+import org.apache.hadoop.hbase.favored.FavoredNodesPlan;
+import org.apache.yetus.audience.InterfaceAudience;
/**
* Used internally for reading meta and constructing datastructures that are
@@ -62,18 +62,18 @@ public class SnapshotOfRegionAssignmentFromMeta {
private final Connection connection;
/** the table name to region map */
- private final Map<TableName, List<HRegionInfo>> tableToRegionMap;
+ private final Map<TableName, List<RegionInfo>> tableToRegionMap;
/** the region to region server map */
- //private final Map<HRegionInfo, ServerName> regionToRegionServerMap;
- private Map<HRegionInfo, ServerName> regionToRegionServerMap;
+ //private final Map<RegionInfo, ServerName> regionToRegionServerMap;
+ private Map<RegionInfo, ServerName> regionToRegionServerMap;
/** the region name to region info map */
- private final Map<String, HRegionInfo> regionNameToRegionInfoMap;
+ private final Map<String, RegionInfo> regionNameToRegionInfoMap;
/** the regionServer to region map */
- private final Map<ServerName, List<HRegionInfo>> currentRSToRegionMap;
- private final Map<ServerName, List<HRegionInfo>> secondaryRSToRegionMap;
- private final Map<ServerName, List<HRegionInfo>> teritiaryRSToRegionMap;
- private final Map<ServerName, List<HRegionInfo>> primaryRSToRegionMap;
+ private final Map<ServerName, List<RegionInfo>> currentRSToRegionMap;
+ private final Map<ServerName, List<RegionInfo>> secondaryRSToRegionMap;
+ private final Map<ServerName, List<RegionInfo>> teritiaryRSToRegionMap;
+ private final Map<ServerName, List<RegionInfo>> primaryRSToRegionMap;
/** the existing assignment plan in the hbase:meta region */
private final FavoredNodesPlan existingAssignmentPlan;
private final Set<TableName> disabledTables;
@@ -113,7 +113,7 @@ public class SnapshotOfRegionAssignmentFromMeta {
if (result == null || result.isEmpty()) return true;
RegionLocations rl = MetaTableAccessor.getRegionLocations(result);
if (rl == null) return true;
- HRegionInfo hri = rl.getRegionLocation(0).getRegionInfo();
+ RegionInfo hri = rl.getRegionLocation(0).getRegionInfo();
if (hri == null) return true;
if (hri.getTable() == null) return true;
if (disabledTables.contains(hri.getTable())) {
@@ -172,13 +172,13 @@ public class SnapshotOfRegionAssignmentFromMeta {
"snapshot");
}
- private void addRegion(HRegionInfo regionInfo) {
+ private void addRegion(RegionInfo regionInfo) {
// Process the region name to region info map
regionNameToRegionInfoMap.put(regionInfo.getRegionNameAsString(), regionInfo);
// Process the table to region map
TableName tableName = regionInfo.getTable();
- List<HRegionInfo> regionList = tableToRegionMap.get(tableName);
+ List<RegionInfo> regionList = tableToRegionMap.get(tableName);
if (regionList == null) {
regionList = new ArrayList<>();
}
@@ -187,14 +187,14 @@ public class SnapshotOfRegionAssignmentFromMeta {
tableToRegionMap.put(tableName, regionList);
}
- private void addAssignment(HRegionInfo regionInfo, ServerName server) {
+ private void addAssignment(RegionInfo regionInfo, ServerName server) {
// Process the region to region server map
regionToRegionServerMap.put(regionInfo, server);
if (server == null) return;
// Process the region server to region map
- List<HRegionInfo> regionList = currentRSToRegionMap.get(server);
+ List<RegionInfo> regionList = currentRSToRegionMap.get(server);
if (regionList == null) {
regionList = new ArrayList<>();
}
@@ -202,9 +202,9 @@ public class SnapshotOfRegionAssignmentFromMeta {
currentRSToRegionMap.put(server, regionList);
}
- private void addPrimaryAssignment(HRegionInfo regionInfo, ServerName server) {
+ private void addPrimaryAssignment(RegionInfo regionInfo, ServerName server) {
// Process the region server to region map
- List<HRegionInfo> regionList = primaryRSToRegionMap.get(server);
+ List<RegionInfo> regionList = primaryRSToRegionMap.get(server);
if (regionList == null) {
regionList = new ArrayList<>();
}
@@ -212,9 +212,9 @@ public class SnapshotOfRegionAssignmentFromMeta {
primaryRSToRegionMap.put(server, regionList);
}
- private void addSecondaryAssignment(HRegionInfo regionInfo, ServerName server) {
+ private void addSecondaryAssignment(RegionInfo regionInfo, ServerName server) {
// Process the region server to region map
- List<HRegionInfo> regionList = secondaryRSToRegionMap.get(server);
+ List<RegionInfo> regionList = secondaryRSToRegionMap.get(server);
if (regionList == null) {
regionList = new ArrayList<>();
}
@@ -222,9 +222,9 @@ public class SnapshotOfRegionAssignmentFromMeta {
secondaryRSToRegionMap.put(server, regionList);
}
- private void addTeritiaryAssignment(HRegionInfo regionInfo, ServerName server) {
+ private void addTeritiaryAssignment(RegionInfo regionInfo, ServerName server) {
// Process the region server to region map
- List<HRegionInfo> regionList = teritiaryRSToRegionMap.get(server);
+ List<RegionInfo> regionList = teritiaryRSToRegionMap.get(server);
if (regionList == null) {
regionList = new ArrayList<>();
}
@@ -236,7 +236,7 @@ public class SnapshotOfRegionAssignmentFromMeta {
* Get the regioninfo for a region
* @return the regioninfo
*/
- public Map<String, HRegionInfo> getRegionNameToRegionInfoMap() {
+ public Map<String, RegionInfo> getRegionNameToRegionInfoMap() {
return this.regionNameToRegionInfoMap;
}
@@ -244,7 +244,7 @@ public class SnapshotOfRegionAssignmentFromMeta {
* Get regions for tables
* @return a mapping from table to regions
*/
- public Map<TableName, List<HRegionInfo>> getTableToRegionMap() {
+ public Map<TableName, List<RegionInfo>> getTableToRegionMap() {
return tableToRegionMap;
}
@@ -252,7 +252,7 @@ public class SnapshotOfRegionAssignmentFromMeta {
* Get region to region server map
* @return region to region server map
*/
- public Map<HRegionInfo, ServerName> getRegionToRegionServerMap() {
+ public Map<RegionInfo, ServerName> getRegionToRegionServerMap() {
return regionToRegionServerMap;
}
@@ -260,7 +260,7 @@ public class SnapshotOfRegionAssignmentFromMeta {
* Get regionserver to region map
* @return regionserver to region map
*/
- public Map<ServerName, List<HRegionInfo>> getRegionServerToRegionMap() {
+ public Map<ServerName, List<RegionInfo>> getRegionServerToRegionMap() {
return currentRSToRegionMap;
}
@@ -280,15 +280,15 @@ public class SnapshotOfRegionAssignmentFromMeta {
return this.tableToRegionMap.keySet();
}
- public Map<ServerName, List<HRegionInfo>> getSecondaryToRegionInfoMap() {
+ public Map<ServerName, List<RegionInfo>> getSecondaryToRegionInfoMap() {
return this.secondaryRSToRegionMap;
}
- public Map<ServerName, List<HRegionInfo>> getTertiaryToRegionInfoMap() {
+ public Map<ServerName, List<RegionInfo>> getTertiaryToRegionInfoMap() {
return this.teritiaryRSToRegionMap;
}
- public Map<ServerName, List<HRegionInfo>> getPrimaryToRegionInfoMap() {
+ public Map<ServerName, List<RegionInfo>> getPrimaryToRegionInfoMap() {
return this.primaryRSToRegionMap;
}
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java
index bc05c0c..8027b6a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java
@@ -24,8 +24,6 @@ import static org.apache.hadoop.hbase.master.SplitLogManager.TerminationStatus.F
import static org.apache.hadoop.hbase.master.SplitLogManager.TerminationStatus.IN_PROGRESS;
import static org.apache.hadoop.hbase.master.SplitLogManager.TerminationStatus.SUCCESS;
-import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
-
import java.io.IOException;
import java.io.InterruptedIOException;
import java.util.ArrayList;
@@ -48,22 +46,24 @@ import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.PathFilter;
import org.apache.hadoop.hbase.ChoreService;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ScheduledChore;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.SplitLogCounters;
import org.apache.hadoop.hbase.Stoppable;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.coordination.BaseCoordinatedStateManager;
import org.apache.hadoop.hbase.coordination.SplitLogManagerCoordination;
import org.apache.hadoop.hbase.coordination.SplitLogManagerCoordination.SplitLogManagerDetails;
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
import org.apache.hadoop.hbase.monitoring.TaskMonitor;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
/**
* Distributes the task of log splitting to the available region servers.
@@ -545,7 +545,7 @@ public class SplitLogManager {
getSplitLogManagerCoordination().setRecoveryMode(isForInitialization);
}
- public void markRegionsRecovering(ServerName server, Set<HRegionInfo> userRegions)
+ public void markRegionsRecovering(ServerName server, Set<RegionInfo> userRegions)
throws InterruptedIOException, IOException {
if (userRegions == null || (!isLogReplaying())) {
return;
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignProcedure.java
index 4a0f2d8..01a5f83 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignProcedure.java
@@ -20,12 +20,12 @@
package org.apache.hadoop.hbase.master.assignment;
import java.io.IOException;
+
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.RetriesExhaustedException;
import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.exceptions.UnexpectedStateException;
@@ -38,6 +38,8 @@ import org.apache.hadoop.hbase.procedure2.ProcedureMetrics;
import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
import org.apache.hadoop.hbase.procedure2.RemoteProcedureDispatcher.RemoteOperation;
+import org.apache.yetus.audience.InterfaceAudience;
+
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.AssignRegionStateData;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RegionTransitionState;
@@ -87,17 +89,17 @@ public class AssignProcedure extends RegionTransitionProcedure {
super();
}
- public AssignProcedure(final HRegionInfo regionInfo) {
+ public AssignProcedure(final RegionInfo regionInfo) {
this(regionInfo, false);
}
- public AssignProcedure(final HRegionInfo regionInfo, final boolean forceNewPlan) {
+ public AssignProcedure(final RegionInfo regionInfo, final boolean forceNewPlan) {
super(regionInfo);
this.forceNewPlan = forceNewPlan;
this.targetServer = null;
}
- public AssignProcedure(final HRegionInfo regionInfo, final ServerName destinationServer) {
+ public AssignProcedure(final RegionInfo regionInfo, final ServerName destinationServer) {
super(regionInfo);
this.forceNewPlan = false;
this.targetServer = destinationServer;
@@ -123,7 +125,7 @@ public class AssignProcedure extends RegionTransitionProcedure {
throws IOException {
final AssignRegionStateData.Builder state = AssignRegionStateData.newBuilder()
.setTransitionState(getTransitionState())
- .setRegionInfo(HRegionInfo.convert(getRegionInfo()));
+ .setRegionInfo(ProtobufUtil.toRegionInfo(getRegionInfo()));
if (forceNewPlan) {
state.setForceNewPlan(true);
}
@@ -138,7 +140,7 @@ public class AssignProcedure extends RegionTransitionProcedure {
throws IOException {
final AssignRegionStateData state = serializer.deserialize(AssignRegionStateData.class);
setTransitionState(state.getTransitionState());
- setRegionInfo(HRegionInfo.convert(state.getRegionInfo()));
+ setRegionInfo(ProtobufUtil.toRegionInfo(state.getRegionInfo()));
forceNewPlan = state.getForceNewPlan();
if (state.hasTargetServer()) {
this.targetServer = ProtobufUtil.toServerName(state.getTargetServer());
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
index f96c1a5..021d411 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
@@ -40,17 +40,16 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseIOException;
import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.PleaseHoldException;
import org.apache.hadoop.hbase.RegionException;
import org.apache.hadoop.hbase.RegionStateListener;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.YouAreDeadException;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.exceptions.UnexpectedStateException;
-import org.apache.hadoop.hbase.master.balancer.FavoredStochasticBalancer;
import org.apache.hadoop.hbase.favored.FavoredNodesManager;
import org.apache.hadoop.hbase.favored.FavoredNodesPromoter;
import org.apache.hadoop.hbase.master.AssignmentListener;
@@ -66,7 +65,7 @@ import org.apache.hadoop.hbase.master.TableStateManager;
import org.apache.hadoop.hbase.master.assignment.RegionStates.RegionStateNode;
import org.apache.hadoop.hbase.master.assignment.RegionStates.ServerState;
import org.apache.hadoop.hbase.master.assignment.RegionStates.ServerStateNode;
-// TODO: why are they here?
+import org.apache.hadoop.hbase.master.balancer.FavoredStochasticBalancer;
import org.apache.hadoop.hbase.master.normalizer.RegionNormalizer;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureScheduler;
@@ -77,19 +76,21 @@ import org.apache.hadoop.hbase.procedure2.ProcedureEvent;
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
import org.apache.hadoop.hbase.procedure2.ProcedureInMemoryChore;
import org.apache.hadoop.hbase.procedure2.util.StringUtils;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionResponse;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.util.VersionInfo;
+import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionResponse;
+
+// TODO: why are they here?
/**
* The AssignmentManager is the coordinator for region assign/unassign operations.
@@ -256,7 +257,7 @@ public class AssignmentManager implements ServerListener {
if (hasProcExecutor) {
getProcedureScheduler().suspendEvent(metaLoadEvent);
setFailoverCleanupDone(false);
- for (HRegionInfo hri: getMetaRegionSet()) {
+ for (RegionInfo hri: getMetaRegionSet()) {
setMetaInitialized(hri, false);
}
}
@@ -326,7 +327,7 @@ public class AssignmentManager implements ServerListener {
return regionStateStore;
}
- public List<ServerName> getFavoredNodes(final HRegionInfo regionInfo) {
+ public List<ServerName> getFavoredNodes(final RegionInfo regionInfo) {
return this.shouldAssignRegionsWithFavoredNodes?
((FavoredStochasticBalancer)getBalancer()).getFavoredNodes(regionInfo):
ServerName.EMPTY_SERVER_LIST;
@@ -351,7 +352,7 @@ public class AssignmentManager implements ServerListener {
// ============================================================================================
// META Helpers
// ============================================================================================
- private boolean isMetaRegion(final HRegionInfo regionInfo) {
+ private boolean isMetaRegion(final RegionInfo regionInfo) {
return regionInfo.isMetaRegion();
}
@@ -359,8 +360,8 @@ public class AssignmentManager implements ServerListener {
return getMetaRegionFromName(regionName) != null;
}
- public HRegionInfo getMetaRegionFromName(final byte[] regionName) {
- for (HRegionInfo hri: getMetaRegionSet()) {
+ public RegionInfo getMetaRegionFromName(final byte[] regionName) {
+ for (RegionInfo hri: getMetaRegionSet()) {
if (Bytes.equals(hri.getRegionName(), regionName)) {
return hri;
}
@@ -369,7 +370,7 @@ public class AssignmentManager implements ServerListener {
}
public boolean isCarryingMeta(final ServerName serverName) {
- for (HRegionInfo hri: getMetaRegionSet()) {
+ for (RegionInfo hri: getMetaRegionSet()) {
if (isCarryingRegion(serverName, hri)) {
return true;
}
@@ -377,23 +378,23 @@ public class AssignmentManager implements ServerListener {
return false;
}
- private boolean isCarryingRegion(final ServerName serverName, final HRegionInfo regionInfo) {
+ private boolean isCarryingRegion(final ServerName serverName, final RegionInfo regionInfo) {
// TODO: check for state?
final RegionStateNode node = regionStates.getRegionNode(regionInfo);
return(node != null && serverName.equals(node.getRegionLocation()));
}
- private HRegionInfo getMetaForRegion(final HRegionInfo regionInfo) {
+ private RegionInfo getMetaForRegion(final RegionInfo regionInfo) {
//if (regionInfo.isMetaRegion()) return regionInfo;
// TODO: handle multiple meta. if the region provided is not meta lookup
// which meta the region belongs to.
- return HRegionInfo.FIRST_META_REGIONINFO;
+ return RegionInfoBuilder.FIRST_META_REGIONINFO;
}
// TODO: handle multiple meta.
- private static final Set<HRegionInfo> META_REGION_SET =
- Collections.singleton(HRegionInfo.FIRST_META_REGIONINFO);
- public Set<HRegionInfo> getMetaRegionSet() {
+ private static final Set<RegionInfo> META_REGION_SET =
+ Collections.singleton(RegionInfoBuilder.FIRST_META_REGIONINFO);
+ public Set<RegionInfo> getMetaRegionSet() {
return META_REGION_SET;
}
@@ -411,15 +412,15 @@ public class AssignmentManager implements ServerListener {
public boolean waitMetaInitialized(final Procedure proc) {
// TODO: handle multiple meta. should this wait on all meta?
// this is used by the ServerCrashProcedure...
- return waitMetaInitialized(proc, HRegionInfo.FIRST_META_REGIONINFO);
+ return waitMetaInitialized(proc, RegionInfoBuilder.FIRST_META_REGIONINFO);
}
- public boolean waitMetaInitialized(final Procedure proc, final HRegionInfo regionInfo) {
+ public boolean waitMetaInitialized(final Procedure proc, final RegionInfo regionInfo) {
return getProcedureScheduler().waitEvent(
getMetaInitializedEvent(getMetaForRegion(regionInfo)), proc);
}
- private void setMetaInitialized(final HRegionInfo metaRegionInfo, final boolean isInitialized) {
+ private void setMetaInitialized(final RegionInfo metaRegionInfo, final boolean isInitialized) {
assert isMetaRegion(metaRegionInfo) : "unexpected non-meta region " + metaRegionInfo;
final ProcedureEvent metaInitEvent = getMetaInitializedEvent(metaRegionInfo);
if (isInitialized) {
@@ -429,7 +430,7 @@ public class AssignmentManager implements ServerListener {
}
}
- private ProcedureEvent getMetaInitializedEvent(final HRegionInfo metaRegionInfo) {
+ private ProcedureEvent getMetaInitializedEvent(final RegionInfo metaRegionInfo) {
assert isMetaRegion(metaRegionInfo) : "unexpected non-meta region " + metaRegionInfo;
// TODO: handle multiple meta.
return metaInitializedEvent;
@@ -451,11 +452,11 @@ public class AssignmentManager implements ServerListener {
// ============================================================================================
// TODO: Sync helpers
// ============================================================================================
- public void assignMeta(final HRegionInfo metaRegionInfo) throws IOException {
+ public void assignMeta(final RegionInfo metaRegionInfo) throws IOException {
assignMeta(metaRegionInfo, null);
}
- public void assignMeta(final HRegionInfo metaRegionInfo, final ServerName serverName)
+ public void assignMeta(final RegionInfo metaRegionInfo, final ServerName serverName)
throws IOException {
assert isMetaRegion(metaRegionInfo) : "unexpected non-meta region " + metaRegionInfo;
AssignProcedure proc;
@@ -490,9 +491,9 @@ public class AssignmentManager implements ServerListener {
// handling.
continue;
}
- List<HRegionInfo> regionsShouldMove = getCarryingSystemTables(server);
+ List<RegionInfo> regionsShouldMove = getCarryingSystemTables(server);
if (!regionsShouldMove.isEmpty()) {
- for (HRegionInfo regionInfo : regionsShouldMove) {
+ for (RegionInfo regionInfo : regionsShouldMove) {
// null value for dest forces destination server to be selected by balancer
RegionPlan plan = new RegionPlan(regionInfo, server, null);
if (regionInfo.isMetaRegion()) {
@@ -514,31 +515,31 @@ public class AssignmentManager implements ServerListener {
}).start();
}
- private List<HRegionInfo> getCarryingSystemTables(ServerName serverName) {
+ private List<RegionInfo> getCarryingSystemTables(ServerName serverName) {
Set<RegionStateNode> regions = this.getRegionStates().getServerNode(serverName).getRegions();
if (regions == null) {
return new ArrayList<>();
}
return regions.stream()
.map(RegionStateNode::getRegionInfo)
- .filter(HRegionInfo::isSystemTable)
+ .filter(RegionInfo::isSystemTable)
.collect(Collectors.toList());
}
- public void assign(final HRegionInfo regionInfo) throws IOException {
+ public void assign(final RegionInfo regionInfo) throws IOException {
assign(regionInfo, true);
}
- public void assign(final HRegionInfo regionInfo, final boolean forceNewPlan) throws IOException {
+ public void assign(final RegionInfo regionInfo, final boolean forceNewPlan) throws IOException {
AssignProcedure proc = createAssignProcedure(regionInfo, forceNewPlan);
ProcedureSyncWait.submitAndWaitProcedure(master.getMasterProcedureExecutor(), proc);
}
- public void unassign(final HRegionInfo regionInfo) throws IOException {
+ public void unassign(final RegionInfo regionInfo) throws IOException {
unassign(regionInfo, false);
}
- public void unassign(final HRegionInfo regionInfo, final boolean forceNewPlan)
+ public void unassign(final RegionInfo regionInfo, final boolean forceNewPlan)
throws IOException {
// TODO: rename this reassign
RegionStateNode node = this.regionStates.getRegionNode(regionInfo);
@@ -557,13 +558,13 @@ public class AssignmentManager implements ServerListener {
}
@VisibleForTesting
- public boolean waitForAssignment(final HRegionInfo regionInfo) throws IOException {
+ public boolean waitForAssignment(final RegionInfo regionInfo) throws IOException {
return waitForAssignment(regionInfo, Long.MAX_VALUE);
}
@VisibleForTesting
// TODO: Remove this?
- public boolean waitForAssignment(final HRegionInfo regionInfo, final long timeout)
+ public boolean waitForAssignment(final RegionInfo regionInfo, final long timeout)
throws IOException {
RegionStateNode node = null;
// This method can be called before the regionInfo has made it into the regionStateMap
@@ -597,16 +598,16 @@ public class AssignmentManager implements ServerListener {
// RegionTransition procedures helpers
// ============================================================================================
- public AssignProcedure[] createAssignProcedures(final Collection<HRegionInfo> regionInfo) {
+ public AssignProcedure[] createAssignProcedures(final Collection<RegionInfo> regionInfo) {
return createAssignProcedures(regionInfo, false);
}
- public AssignProcedure[] createAssignProcedures(final Collection<HRegionInfo> regionInfo,
+ public AssignProcedure[] createAssignProcedures(final Collection<RegionInfo> regionInfo,
final boolean forceNewPlan) {
if (regionInfo.isEmpty()) return null;
final AssignProcedure[] procs = new AssignProcedure[regionInfo.size()];
int index = 0;
- for (HRegionInfo hri: regionInfo) {
+ for (RegionInfo hri: regionInfo) {
procs[index++] = createAssignProcedure(hri, forceNewPlan);
}
return procs;
@@ -629,10 +630,10 @@ public class AssignmentManager implements ServerListener {
return procs.toArray(UNASSIGNED_PROCEDURE_FOR_TYPE_INFO);
}
- public MoveRegionProcedure[] createReopenProcedures(final Collection<HRegionInfo> regionInfo) {
+ public MoveRegionProcedure[] createReopenProcedures(final Collection<RegionInfo> regionInfo) {
final MoveRegionProcedure[] procs = new MoveRegionProcedure[regionInfo.size()];
int index = 0;
- for (HRegionInfo hri: regionInfo) {
+ for (RegionInfo hri: regionInfo) {
final ServerName serverName = regionStates.getRegionServerOfRegion(hri);
final RegionPlan plan = new RegionPlan(hri, serverName, serverName);
procs[index++] = createMoveRegionProcedure(plan);
@@ -664,21 +665,21 @@ public class AssignmentManager implements ServerListener {
return createReopenProcedures(regionStates.getRegionsOfTable(tableName));
}
- public AssignProcedure createAssignProcedure(final HRegionInfo regionInfo,
+ public AssignProcedure createAssignProcedure(final RegionInfo regionInfo,
final boolean forceNewPlan) {
AssignProcedure proc = new AssignProcedure(regionInfo, forceNewPlan);
proc.setOwner(getProcedureEnvironment().getRequestUser().getShortName());
return proc;
}
- public AssignProcedure createAssignProcedure(final HRegionInfo regionInfo,
+ public AssignProcedure createAssignProcedure(final RegionInfo regionInfo,
final ServerName targetServer) {
AssignProcedure proc = new AssignProcedure(regionInfo, targetServer);
proc.setOwner(getProcedureEnvironment().getRequestUser().getShortName());
return proc;
}
- public UnassignProcedure createUnassignProcedure(final HRegionInfo regionInfo,
+ public UnassignProcedure createUnassignProcedure(final RegionInfo regionInfo,
final ServerName destinationServer, final boolean force) {
// If destinationServer is null, figure it.
ServerName sn = destinationServer != null? destinationServer:
@@ -707,13 +708,13 @@ public class AssignmentManager implements ServerListener {
}
- public SplitTableRegionProcedure createSplitProcedure(final HRegionInfo regionToSplit,
+ public SplitTableRegionProcedure createSplitProcedure(final RegionInfo regionToSplit,
final byte[] splitKey) throws IOException {
return new SplitTableRegionProcedure(getProcedureEnvironment(), regionToSplit, splitKey);
}
- public MergeTableRegionsProcedure createMergeProcedure(final HRegionInfo regionToMergeA,
- final HRegionInfo regionToMergeB) throws IOException {
+ public MergeTableRegionsProcedure createMergeProcedure(final RegionInfo regionToMergeA,
+ final RegionInfo regionToMergeB) throws IOException {
return new MergeTableRegionsProcedure(getProcedureEnvironment(), regionToMergeA,regionToMergeB);
}
@@ -721,10 +722,10 @@ public class AssignmentManager implements ServerListener {
* Delete the region states. This is called by "DeleteTable"
*/
public void deleteTable(final TableName tableName) throws IOException {
- final ArrayList<HRegionInfo> regions = regionStates.getTableRegionsInfo(tableName);
+ final ArrayList<RegionInfo> regions = regionStates.getTableRegionsInfo(tableName);
regionStateStore.deleteRegions(regions);
for (int i = 0; i < regions.size(); ++i) {
- final HRegionInfo regionInfo = regions.get(i);
+ final RegionInfo regionInfo = regions.get(i);
// we expect the region to be offline
regionStates.removeFromOfflineRegions(regionInfo);
regionStates.deleteRegion(regionInfo);
@@ -748,7 +749,7 @@ public class AssignmentManager implements ServerListener {
case FAILED_OPEN:
case CLOSED:
assert transition.getRegionInfoCount() == 1 : transition;
- final HRegionInfo hri = HRegionInfo.convert(transition.getRegionInfo(0));
+ final RegionInfo hri = ProtobufUtil.toRegionInfo(transition.getRegionInfo(0));
updateRegionTransition(serverName, transition.getTransitionCode(), hri,
transition.hasOpenSeqNum() ? transition.getOpenSeqNum() : HConstants.NO_SEQNUM);
break;
@@ -757,9 +758,9 @@ public class AssignmentManager implements ServerListener {
case SPLIT:
case SPLIT_REVERTED:
assert transition.getRegionInfoCount() == 3 : transition;
- final HRegionInfo parent = HRegionInfo.convert(transition.getRegionInfo(0));
- final HRegionInfo splitA = HRegionInfo.convert(transition.getRegionInfo(1));
- final HRegionInfo splitB = HRegionInfo.convert(transition.getRegionInfo(2));
+ final RegionInfo parent = ProtobufUtil.toRegionInfo(transition.getRegionInfo(0));
+ final RegionInfo splitA = ProtobufUtil.toRegionInfo(transition.getRegionInfo(1));
+ final RegionInfo splitB = ProtobufUtil.toRegionInfo(transition.getRegionInfo(2));
updateRegionSplitTransition(serverName, transition.getTransitionCode(),
parent, splitA, splitB);
break;
@@ -768,9 +769,9 @@ public class AssignmentManager implements ServerListener {
case MERGED:
case MERGE_REVERTED:
assert transition.getRegionInfoCount() == 3 : transition;
- final HRegionInfo merged = HRegionInfo.convert(transition.getRegionInfo(0));
- final HRegionInfo mergeA = HRegionInfo.convert(transition.getRegionInfo(1));
- final HRegionInfo mergeB = HRegionInfo.convert(transition.getRegionInfo(2));
+ final RegionInfo merged = ProtobufUtil.toRegionInfo(transition.getRegionInfo(0));
+ final RegionInfo mergeA = ProtobufUtil.toRegionInfo(transition.getRegionInfo(1));
+ final RegionInfo mergeB = ProtobufUtil.toRegionInfo(transition.getRegionInfo(2));
updateRegionMergeTransition(serverName, transition.getTransitionCode(),
merged, mergeA, mergeB);
break;
@@ -789,7 +790,7 @@ public class AssignmentManager implements ServerListener {
}
private void updateRegionTransition(final ServerName serverName, final TransitionCode state,
- final HRegionInfo regionInfo, final long seqId)
+ final RegionInfo regionInfo, final long seqId)
throws PleaseHoldException, UnexpectedStateException {
checkFailoverCleanupCompleted(regionInfo);
@@ -830,7 +831,7 @@ public class AssignmentManager implements ServerListener {
}
private void updateRegionSplitTransition(final ServerName serverName, final TransitionCode state,
- final HRegionInfo parent, final HRegionInfo hriA, final HRegionInfo hriB)
+ final RegionInfo parent, final RegionInfo hriA, final RegionInfo hriB)
throws IOException {
checkFailoverCleanupCompleted(parent);
@@ -863,7 +864,7 @@ public class AssignmentManager implements ServerListener {
}
private void updateRegionMergeTransition(final ServerName serverName, final TransitionCode state,
- final HRegionInfo merged, final HRegionInfo hriA, final HRegionInfo hriB) throws IOException {
+ final RegionInfo merged, final RegionInfo hriA, final RegionInfo hriB) throws IOException {
checkFailoverCleanupCompleted(merged);
if (state != TransitionCode.READY_TO_MERGE) {
@@ -935,7 +936,7 @@ public class AssignmentManager implements ServerListener {
final Set<byte[]> regionNames) {
try {
for (byte[] regionName: regionNames) {
- final HRegionInfo hri = getMetaRegionFromName(regionName);
+ final RegionInfo hri = getMetaRegionFromName(regionName);
if (hri == null) {
if (LOG.isTraceEnabled()) {
LOG.trace("Skip online report for region=" + Bytes.toStringBinary(regionName) +
@@ -1094,12 +1095,12 @@ public class AssignmentManager implements ServerListener {
return m != null? m.values(): Collections.EMPTY_SET;
}
- public boolean isRegionOverThreshold(final HRegionInfo regionInfo) {
+ public boolean isRegionOverThreshold(final RegionInfo regionInfo) {
Map<String, RegionState> m = this.ritsOverThreshold;
return m != null && m.containsKey(regionInfo.getEncodedName());
}
- public boolean isRegionTwiceOverThreshold(final HRegionInfo regionInfo) {
+ public boolean isRegionTwiceOverThreshold(final RegionInfo regionInfo) {
Map<String, RegionState> m = this.ritsOverThreshold;
if (m == null) return false;
final RegionState state = m.get(regionInfo.getEncodedName());
@@ -1138,7 +1139,7 @@ public class AssignmentManager implements ServerListener {
metrics.updateRITCountOverThreshold(ritStat.getTotalRITsOverThreshold());
}
- private void handleRegionOverStuckWarningThreshold(final HRegionInfo regionInfo) {
+ private void handleRegionOverStuckWarningThreshold(final RegionInfo regionInfo) {
final RegionStateNode regionNode = regionStates.getRegionNode(regionInfo);
//if (regionNode.isStuck()) {
LOG.warn("TODO Handle stuck in transition: " + regionNode);
@@ -1176,7 +1177,7 @@ public class AssignmentManager implements ServerListener {
// TODO: use a thread pool
regionStateStore.visitMeta(new RegionStateStore.RegionStateVisitor() {
@Override
- public void visitRegionState(final HRegionInfo regionInfo, final State state,
+ public void visitRegionState(final RegionInfo regionInfo, final State state,
final ServerName regionLocation, final ServerName lastHost, final long openSeqNum) {
final RegionStateNode regionNode = regionStates.getOrCreateRegionNode(regionInfo);
synchronized (regionNode) {
@@ -1211,7 +1212,7 @@ public class AssignmentManager implements ServerListener {
boolean failover = !master.getServerManager().getDeadServers().isEmpty();
final Set<ServerName> offlineServersWithOnlineRegions = new HashSet<ServerName>();
- final ArrayList<HRegionInfo> regionsToAssign = new ArrayList<HRegionInfo>();
+ final ArrayList<RegionInfo> regionsToAssign = new ArrayList<RegionInfo>();
long st, et;
st = System.currentTimeMillis();
@@ -1246,7 +1247,7 @@ public class AssignmentManager implements ServerListener {
// assign offline regions
st = System.currentTimeMillis();
- for (HRegionInfo regionInfo: getOrderedRegions(regionsToAssign)) {
+ for (RegionInfo regionInfo: getOrderedRegions(regionsToAssign)) {
master.getMasterProcedureExecutor().submitProcedure(
createAssignProcedure(regionInfo, false));
}
@@ -1285,7 +1286,7 @@ public class AssignmentManager implements ServerListener {
* @param hri region to check if it is already rebuild
* @throws PleaseHoldException if the failover cleanup is not completed
*/
- private void checkFailoverCleanupCompleted(final HRegionInfo hri) throws PleaseHoldException {
+ private void checkFailoverCleanupCompleted(final RegionInfo hri) throws PleaseHoldException {
if (!isRunning()) {
throw new PleaseHoldException("AssignmentManager not running");
}
@@ -1317,18 +1318,18 @@ public class AssignmentManager implements ServerListener {
" to dead servers, submitted shutdown handler to be executed meta=" + carryingMeta);
}
- public void offlineRegion(final HRegionInfo regionInfo) {
+ public void offlineRegion(final RegionInfo regionInfo) {
// TODO used by MasterRpcServices ServerCrashProcedure
final RegionStateNode node = regionStates.getRegionNode(regionInfo);
if (node != null) node.offline();
}
- public void onlineRegion(final HRegionInfo regionInfo, final ServerName serverName) {
+ public void onlineRegion(final RegionInfo regionInfo, final ServerName serverName) {
// TODO used by TestSplitTransactionOnCluster.java
}
- public Map<ServerName, List<HRegionInfo>> getSnapShotOfAssignment(
- final Collection<HRegionInfo> regions) {
+ public Map<ServerName, List<RegionInfo>> getSnapShotOfAssignment(
+ final Collection<RegionInfo> regions) {
return regionStates.getSnapShotOfAssignment(regions);
}
@@ -1359,13 +1360,13 @@ public class AssignmentManager implements ServerListener {
* @param regions
* @return A list of regions with system regions at front
*/
- public List<HRegionInfo> getOrderedRegions(
- final List<HRegionInfo> regions) {
+ public List<RegionInfo> getOrderedRegions(
+ final List<RegionInfo> regions) {
if (regions == null) return Collections.emptyList();
- List<HRegionInfo> systemList = new ArrayList<>();
- List<HRegionInfo> userList = new ArrayList<>();
- for (HRegionInfo hri : regions) {
+ List<RegionInfo> systemList = new ArrayList<>();
+ List<RegionInfo> userList = new ArrayList<>();
+ for (RegionInfo hri : regions) {
if (hri.isSystemTable()) systemList.add(hri);
else userList.add(hri);
}
@@ -1395,11 +1396,11 @@ public class AssignmentManager implements ServerListener {
return regionStates.getRegionsInTransition();
}
- public List<HRegionInfo> getAssignedRegions() {
+ public List<RegionInfo> getAssignedRegions() {
return regionStates.getAssignedRegions();
}
- public HRegionInfo getRegionInfo(final byte[] regionName) {
+ public RegionInfo getRegionInfo(final byte[] regionName) {
final RegionStateNode regionState = regionStates.getRegionNodeFromName(regionName);
return regionState != null ? regionState.getRegionInfo() : null;
}
@@ -1407,7 +1408,7 @@ public class AssignmentManager implements ServerListener {
// ============================================================================================
// TODO: Region Status update
// ============================================================================================
- private void sendRegionOpenedNotification(final HRegionInfo regionInfo,
+ private void sendRegionOpenedNotification(final RegionInfo regionInfo,
final ServerName serverName) {
getBalancer().regionOnline(regionInfo, serverName);
if (!this.listeners.isEmpty()) {
@@ -1417,7 +1418,7 @@ public class AssignmentManager implements ServerListener {
}
}
- private void sendRegionClosedNotification(final HRegionInfo regionInfo) {
+ private void sendRegionClosedNotification(final RegionInfo regionInfo) {
getBalancer().regionOffline(regionInfo);
if (!this.listeners.isEmpty()) {
for (AssignmentListener listener : this.listeners) {
@@ -1454,7 +1455,7 @@ public class AssignmentManager implements ServerListener {
}
public void markRegionAsOpened(final RegionStateNode regionNode) throws IOException {
- final HRegionInfo hri = regionNode.getRegionInfo();
+ final RegionInfo hri = regionNode.getRegionInfo();
synchronized (regionNode) {
State state = regionNode.transitionState(State.OPEN, RegionStates.STATES_EXPECTED_ON_OPEN);
if (isMetaRegion(hri)) {
@@ -1473,7 +1474,7 @@ public class AssignmentManager implements ServerListener {
}
public void markRegionAsClosing(final RegionStateNode regionNode) throws IOException {
- final HRegionInfo hri = regionNode.getRegionInfo();
+ final RegionInfo hri = regionNode.getRegionInfo();
synchronized (regionNode) {
State state = regionNode.transitionState(State.CLOSING, RegionStates.STATES_EXPECTED_ON_CLOSE);
// Set meta has not initialized early. so people trying to create/edit tables will wait
@@ -1496,7 +1497,7 @@ public class AssignmentManager implements ServerListener {
}
public void markRegionAsClosed(final RegionStateNode regionNode) throws IOException {
- final HRegionInfo hri = regionNode.getRegionInfo();
+ final RegionInfo hri = regionNode.getRegionInfo();
synchronized (regionNode) {
State state = regionNode.transitionState(State.CLOSED, RegionStates.STATES_EXPECTED_ON_CLOSE);
regionStates.removeRegionFromServer(regionNode.getRegionLocation(), regionNode);
@@ -1509,8 +1510,8 @@ public class AssignmentManager implements ServerListener {
}
}
- public void markRegionAsSplit(final HRegionInfo parent, final ServerName serverName,
- final HRegionInfo daughterA, final HRegionInfo daughterB)
+ public void markRegionAsSplit(final RegionInfo parent, final ServerName serverName,
+ final RegionInfo daughterA, final RegionInfo daughterB)
throws IOException {
// Update hbase:meta. Parent will be marked offline and split up in hbase:meta.
// The parent stays in regionStates until cleared when removed by CatalogJanitor.
@@ -1535,8 +1536,8 @@ public class AssignmentManager implements ServerListener {
* by the catalog janitor running against hbase:meta. It notices when the merged region no
* longer holds references to the old regions.
*/
- public void markRegionAsMerged(final HRegionInfo child, final ServerName serverName,
- final HRegionInfo mother, final HRegionInfo father) throws IOException {
+ public void markRegionAsMerged(final RegionInfo child, final ServerName serverName,
+ final RegionInfo mother, final RegionInfo father) throws IOException {
final RegionStateNode node = regionStates.getOrCreateRegionNode(child);
node.setState(State.MERGED);
regionStates.deleteRegion(mother);
@@ -1552,7 +1553,7 @@ public class AssignmentManager implements ServerListener {
* Favored nodes should be applied only when FavoredNodes balancer is configured and the region
* belongs to a non-system table.
*/
- private boolean shouldAssignFavoredNodes(HRegionInfo region) {
+ private boolean shouldAssignFavoredNodes(RegionInfo region) {
return this.shouldAssignRegionsWithFavoredNodes &&
FavoredNodesManager.isFavoredNodeApplicable(region);
}
@@ -1622,8 +1623,8 @@ public class AssignmentManager implements ServerListener {
}
@edu.umd.cs.findbugs.annotations.SuppressWarnings("WA_AWAIT_NOT_IN_LOOP")
- private HashMap<HRegionInfo, RegionStateNode> waitOnAssignQueue() {
- HashMap<HRegionInfo, RegionStateNode> regions = null;
+ private HashMap<RegionInfo, RegionStateNode> waitOnAssignQueue() {
+ HashMap<RegionInfo, RegionStateNode> regions = null;
assignQueueLock.lock();
try {
@@ -1633,7 +1634,7 @@ public class AssignmentManager implements ServerListener {
if (!isRunning()) return null;
assignQueueFullCond.await(assignDispatchWaitMillis, TimeUnit.MILLISECONDS);
- regions = new HashMap<HRegionInfo, RegionStateNode>(pendingAssignQueue.size());
+ regions = new HashMap<RegionInfo, RegionStateNode>(pendingAssignQueue.size());
for (RegionStateNode regionNode: pendingAssignQueue) {
regions.put(regionNode.getRegionInfo(), regionNode);
}
@@ -1648,7 +1649,7 @@ public class AssignmentManager implements ServerListener {
}
private void processAssignQueue() {
- final HashMap<HRegionInfo, RegionStateNode> regions = waitOnAssignQueue();
+ final HashMap<RegionInfo, RegionStateNode> regions = waitOnAssignQueue();
if (regions == null || regions.size() == 0 || !isRunning()) {
return;
}
@@ -1658,13 +1659,13 @@ public class AssignmentManager implements ServerListener {
}
// TODO: Optimize balancer. pass a RegionPlan?
- final HashMap<HRegionInfo, ServerName> retainMap = new HashMap<>();
- final List<HRegionInfo> userRRList = new ArrayList<>();
+ final HashMap<RegionInfo, ServerName> retainMap = new HashMap<>();
+ final List<RegionInfo> userRRList = new ArrayList<>();
// regions for system tables requiring reassignment
- final List<HRegionInfo> sysRRList = new ArrayList<>();
+ final List<RegionInfo> sysRRList = new ArrayList<>();
for (RegionStateNode regionNode : regions.values()) {
boolean sysTable = regionNode.isSystemTable();
- final List<HRegionInfo> rrList = sysTable ? sysRRList : userRRList;
+ final List<RegionInfo> rrList = sysTable ? sysRRList : userRRList;
if (regionNode.getRegionLocation() != null) {
retainMap.put(regionNode.getRegionInfo(), regionNode.getRegionLocation());
@@ -1711,8 +1712,8 @@ public class AssignmentManager implements ServerListener {
processAssignmentPlans(regions, retainMap, userRRList, servers);
}
- private void processAssignmentPlans(final HashMap<HRegionInfo, RegionStateNode> regions,
- final HashMap<HRegionInfo, ServerName> retainMap, final List<HRegionInfo> rrList,
+ private void processAssignmentPlans(final HashMap<RegionInfo, RegionStateNode> regions,
+ final HashMap<RegionInfo, ServerName> retainMap, final List<RegionInfo> rrList,
final List<ServerName> servers) {
boolean isTraceEnabled = LOG.isTraceEnabled();
if (isTraceEnabled) {
@@ -1736,7 +1737,7 @@ public class AssignmentManager implements ServerListener {
// TODO: Do we need to split retain and round-robin?
// the retain seems to fallback to round-robin/random if the region is not in the map.
if (!rrList.isEmpty()) {
- Collections.sort(rrList);
+ Collections.sort(rrList, RegionInfo.COMPARATOR);
if (isTraceEnabled) {
LOG.trace("round robin regions=" + rrList);
}
@@ -1749,8 +1750,8 @@ public class AssignmentManager implements ServerListener {
}
}
- private void acceptPlan(final HashMap<HRegionInfo, RegionStateNode> regions,
- final Map<ServerName, List<HRegionInfo>> plan) throws HBaseIOException {
+ private void acceptPlan(final HashMap<RegionInfo, RegionStateNode> regions,
+ final Map<ServerName, List<RegionInfo>> plan) throws HBaseIOException {
final ProcedureEvent[] events = new ProcedureEvent[regions.size()];
final long st = System.currentTimeMillis();
@@ -1761,9 +1762,9 @@ public class AssignmentManager implements ServerListener {
if (plan.isEmpty()) return;
int evcount = 0;
- for (Map.Entry<ServerName, List<HRegionInfo>> entry: plan.entrySet()) {
+ for (Map.Entry<ServerName, List<RegionInfo>> entry: plan.entrySet()) {
final ServerName server = entry.getKey();
- for (HRegionInfo hri: entry.getValue()) {
+ for (RegionInfo hri: entry.getValue()) {
final RegionStateNode regionNode = regions.get(hri);
regionNode.setRegionLocation(server);
events[evcount++] = regionNode.getProcedureEvent();
@@ -1778,11 +1779,11 @@ public class AssignmentManager implements ServerListener {
}
}
- private void addToPendingAssignment(final HashMap<HRegionInfo, RegionStateNode> regions,
- final Collection<HRegionInfo> pendingRegions) {
+ private void addToPendingAssignment(final HashMap<RegionInfo, RegionStateNode> regions,
+ final Collection<RegionInfo> pendingRegions) {
assignQueueLock.lock();
try {
- for (HRegionInfo hri: pendingRegions) {
+ for (RegionInfo hri: pendingRegions) {
pendingAssignQueue.add(regions.get(hri));
}
} finally {
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/GCMergedRegionsProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/GCMergedRegionsProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/GCMergedRegionsProcedure.java
index 6ca00da..37521cc 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/GCMergedRegionsProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/GCMergedRegionsProcedure.java
@@ -18,17 +18,20 @@
package org.apache.hadoop.hbase.master.assignment;
import java.io.IOException;
+
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.TableName;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.master.procedure.AbstractStateMachineTableProcedure;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
import org.apache.hadoop.hbase.procedure2.ProcedureYieldException;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.GCMergedRegionsState;
@@ -43,14 +46,14 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.G
public class GCMergedRegionsProcedure
extends AbstractStateMachineTableProcedure<GCMergedRegionsState> {
private static final Log LOG = LogFactory.getLog(GCMergedRegionsProcedure.class);
- private HRegionInfo father;
- private HRegionInfo mother;
- private HRegionInfo mergedChild;
+ private RegionInfo father;
+ private RegionInfo mother;
+ private RegionInfo mergedChild;
public GCMergedRegionsProcedure(final MasterProcedureEnv env,
- final HRegionInfo mergedChild,
- final HRegionInfo father,
- final HRegionInfo mother) {
+ final RegionInfo mergedChild,
+ final RegionInfo father,
+ final RegionInfo mother) {
super(env);
this.father = father;
this.mother = mother;
@@ -100,7 +103,7 @@ extends AbstractStateMachineTableProcedure<GCMergedRegionsState> {
private GCRegionProcedure[] createGCRegionProcedures(final MasterProcedureEnv env) {
GCRegionProcedure [] procs = new GCRegionProcedure[2];
int index = 0;
- for (HRegionInfo hri: new HRegionInfo [] {this.father, this.mother}) {
+ for (RegionInfo hri: new RegionInfo [] {this.father, this.mother}) {
GCRegionProcedure proc = new GCRegionProcedure(env, hri);
proc.setOwner(env.getRequestUser().getShortName());
procs[index++] = proc;
@@ -135,9 +138,9 @@ extends AbstractStateMachineTableProcedure<GCMergedRegionsState> {
super.serializeStateData(serializer);
final MasterProcedureProtos.GCMergedRegionsStateData.Builder msg =
MasterProcedureProtos.GCMergedRegionsStateData.newBuilder().
- setParentA(HRegionInfo.convert(this.father)).
- setParentB(HRegionInfo.convert(this.mother)).
- setMergedChild(HRegionInfo.convert(this.mergedChild));
+ setParentA(ProtobufUtil.toRegionInfo(this.father)).
+ setParentB(ProtobufUtil.toRegionInfo(this.mother)).
+ setMergedChild(ProtobufUtil.toRegionInfo(this.mergedChild));
serializer.serialize(msg.build());
}
@@ -147,9 +150,9 @@ extends AbstractStateMachineTableProcedure<GCMergedRegionsState> {
super.deserializeStateData(serializer);
final MasterProcedureProtos.GCMergedRegionsStateData msg =
serializer.deserialize(MasterProcedureProtos.GCMergedRegionsStateData.class);
- this.father = HRegionInfo.convert(msg.getParentA());
- this.mother = HRegionInfo.convert(msg.getParentB());
- this.mergedChild = HRegionInfo.convert(msg.getMergedChild());
+ this.father = ProtobufUtil.toRegionInfo(msg.getParentA());
+ this.mother = ProtobufUtil.toRegionInfo(msg.getParentB());
+ this.mergedChild = ProtobufUtil.toRegionInfo(msg.getMergedChild());
}
@Override
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/GCRegionProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/GCRegionProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/GCRegionProcedure.java
index bc3cc0f..805b870 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/GCRegionProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/GCRegionProcedure.java
@@ -18,13 +18,13 @@
package org.apache.hadoop.hbase.master.assignment;
import java.io.IOException;
+
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.backup.HFileArchiver;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.favored.FavoredNodesManager;
import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.master.procedure.AbstractStateMachineRegionProcedure;
@@ -32,10 +32,12 @@ import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
import org.apache.hadoop.hbase.procedure2.ProcedureYieldException;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.GCRegionState;
+import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.GCRegionState;
/**
* GC a Region that is no longer in use. It has been split or merged away.
@@ -47,7 +49,7 @@ import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
public class GCRegionProcedure extends AbstractStateMachineRegionProcedure<GCRegionState> {
private static final Log LOG = LogFactory.getLog(GCRegionProcedure.class);
- public GCRegionProcedure(final MasterProcedureEnv env, final HRegionInfo hri) {
+ public GCRegionProcedure(final MasterProcedureEnv env, final RegionInfo hri) {
super(env, hri);
}
@@ -135,7 +137,7 @@ public class GCRegionProcedure extends AbstractStateMachineRegionProcedure<GCReg
// Double serialization of regionname. Superclass is also serializing. Fix.
final MasterProcedureProtos.GCRegionStateData.Builder msg =
MasterProcedureProtos.GCRegionStateData.newBuilder()
- .setRegionInfo(HRegionInfo.convert(getRegion()));
+ .setRegionInfo(ProtobufUtil.toRegionInfo(getRegion()));
serializer.serialize(msg.build());
}
@@ -145,7 +147,7 @@ public class GCRegionProcedure extends AbstractStateMachineRegionProcedure<GCReg
super.deserializeStateData(serializer);
final MasterProcedureProtos.GCRegionStateData msg =
serializer.deserialize(MasterProcedureProtos.GCRegionStateData.class);
- setRegion(HRegionInfo.convert(msg.getRegionInfo()));
+ setRegion(ProtobufUtil.toRegionInfo(msg.getRegionInfo()));
}
@Override
[05/19] hbase git commit: HBASE-18839 Apply RegionInfo to code base
Posted by ch...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterOperationsForRegionReplicas.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterOperationsForRegionReplicas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterOperationsForRegionReplicas.java
index a8b9998..b73c873 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterOperationsForRegionReplicas.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterOperationsForRegionReplicas.java
@@ -19,12 +19,9 @@
package org.apache.hadoop.hbase.master;
import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
import java.io.IOException;
-import java.util.Collection;
import java.util.EnumSet;
-import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
@@ -38,18 +35,18 @@ import org.apache.hadoop.hbase.ClusterStatus.Option;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.MetaTableAccessor;
+import org.apache.hadoop.hbase.MetaTableAccessor.Visitor;
import org.apache.hadoop.hbase.RegionLocations;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.MetaTableAccessor;
-import org.apache.hadoop.hbase.MetaTableAccessor.Visitor;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Table;
@@ -107,7 +104,7 @@ public class TestMasterOperationsForRegionReplicas {
ADMIN.createTable(desc, Bytes.toBytes("A"), Bytes.toBytes("Z"), numRegions);
validateNumberOfRowsInMeta(tableName, numRegions, ADMIN.getConnection());
- List<HRegionInfo> hris = MetaTableAccessor.getTableRegions(
+ List<RegionInfo> hris = MetaTableAccessor.getTableRegions(
ADMIN.getConnection(), tableName);
assert(hris.size() == numRegions * numReplica);
} finally {
@@ -129,12 +126,12 @@ public class TestMasterOperationsForRegionReplicas {
TEST_UTIL.waitTableEnabled(tableName);
validateNumberOfRowsInMeta(tableName, numRegions, ADMIN.getConnection());
- List<HRegionInfo> hris = MetaTableAccessor.getTableRegions(ADMIN.getConnection(), tableName);
+ List<RegionInfo> hris = MetaTableAccessor.getTableRegions(ADMIN.getConnection(), tableName);
assert(hris.size() == numRegions * numReplica);
// check that the master created expected number of RegionState objects
for (int i = 0; i < numRegions; i++) {
for (int j = 0; j < numReplica; j++) {
- HRegionInfo replica = RegionReplicaUtil.getRegionInfoForReplica(hris.get(i), j);
+ RegionInfo replica = RegionReplicaUtil.getRegionInfoForReplica(hris.get(i), j);
RegionState state = TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager()
.getRegionStates().getRegionState(replica);
assert (state != null);
@@ -145,7 +142,7 @@ public class TestMasterOperationsForRegionReplicas {
int numRows = 0;
for (Result result : metaRows) {
RegionLocations locations = MetaTableAccessor.getRegionLocations(result);
- HRegionInfo hri = locations.getRegionLocation().getRegionInfo();
+ RegionInfo hri = locations.getRegionLocation().getRegionInfo();
if (!hri.getTable().equals(tableName)) continue;
numRows += 1;
HRegionLocation[] servers = locations.getRegionLocations();
@@ -168,7 +165,7 @@ public class TestMasterOperationsForRegionReplicas {
TEST_UTIL.getHBaseClusterInterface().waitForActiveAndReadyMaster();
for (int i = 0; i < numRegions; i++) {
for (int j = 0; j < numReplica; j++) {
- HRegionInfo replica = RegionReplicaUtil.getRegionInfoForReplica(hris.get(i), j);
+ RegionInfo replica = RegionReplicaUtil.getRegionInfoForReplica(hris.get(i), j);
RegionState state = TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager()
.getRegionStates().getRegionState(replica);
assert (state != null);
@@ -205,7 +202,7 @@ public class TestMasterOperationsForRegionReplicas {
ADMIN.enableTable(tableName);
LOG.info(ADMIN.getTableDescriptor(tableName).toString());
assert(ADMIN.isTableEnabled(tableName));
- List<HRegionInfo> regions = TEST_UTIL.getMiniHBaseCluster().getMaster()
+ List<RegionInfo> regions = TEST_UTIL.getMiniHBaseCluster().getMaster()
.getAssignmentManager().getRegionStates().getRegionsOfTable(tableName);
assertTrue("regions.size=" + regions.size() + ", numRegions=" + numRegions + ", numReplica=" + numReplica,
regions.size() == numRegions * (numReplica + 1));
@@ -225,10 +222,10 @@ public class TestMasterOperationsForRegionReplicas {
//just check that the number of default replica regions in the meta table are the same
//as the number of regions the table was created with, and the count of the
//replicas is numReplica for each region
- Map<HRegionInfo, Integer> defaultReplicas = new HashMap<>();
- for (HRegionInfo hri : hris) {
+ Map<RegionInfo, Integer> defaultReplicas = new HashMap<>();
+ for (RegionInfo hri : hris) {
Integer i;
- HRegionInfo regionReplica0 = RegionReplicaUtil.getRegionInfoForDefaultReplica(hri);
+ RegionInfo regionReplica0 = RegionReplicaUtil.getRegionInfoForDefaultReplica(hri);
defaultReplicas.put(regionReplica0,
(i = defaultReplicas.get(regionReplica0)) == null ? 1 : i + 1);
}
@@ -256,8 +253,8 @@ public class TestMasterOperationsForRegionReplicas {
ADMIN.createTable(desc, Bytes.toBytes("A"), Bytes.toBytes("Z"), numRegions);
TEST_UTIL.waitTableEnabled(tableName);
Set<byte[]> tableRows = new HashSet<>();
- List<HRegionInfo> hris = MetaTableAccessor.getTableRegions(ADMIN.getConnection(), tableName);
- for (HRegionInfo hri : hris) {
+ List<RegionInfo> hris = MetaTableAccessor.getTableRegions(ADMIN.getConnection(), tableName);
+ for (RegionInfo hri : hris) {
tableRows.add(hri.getRegionName());
}
ADMIN.disableTable(tableName);
@@ -279,7 +276,7 @@ public class TestMasterOperationsForRegionReplicas {
// get back the desired number of replicas for the regions
ADMIN.enableTable(tableName);
assert(ADMIN.isTableEnabled(tableName));
- List<HRegionInfo> regions = TEST_UTIL.getMiniHBaseCluster().getMaster()
+ List<RegionInfo> regions = TEST_UTIL.getMiniHBaseCluster().getMaster()
.getAssignmentManager().getRegionStates().getRegionsOfTable(tableName);
assert(regions.size() == numRegions * numReplica);
} finally {
@@ -288,9 +285,9 @@ public class TestMasterOperationsForRegionReplicas {
}
}
- private String printRegions(List<HRegionInfo> regions) {
+ private String printRegions(List<RegionInfo> regions) {
StringBuffer strBuf = new StringBuffer();
- for (HRegionInfo r : regions) {
+ for (RegionInfo r : regions) {
strBuf.append(" ____ " + r.toString());
}
return strBuf.toString();
@@ -303,7 +300,7 @@ public class TestMasterOperationsForRegionReplicas {
Visitor visitor = new Visitor() {
@Override
public boolean visit(Result r) throws IOException {
- if (MetaTableAccessor.getHRegionInfo(r).getTable().equals(table)) count.incrementAndGet();
+ if (MetaTableAccessor.getRegionInfo(r).getTable().equals(table)) count.incrementAndGet();
return true;
}
};
@@ -316,16 +313,16 @@ public class TestMasterOperationsForRegionReplicas {
SnapshotOfRegionAssignmentFromMeta snapshot = new SnapshotOfRegionAssignmentFromMeta(
connection);
snapshot.initialize();
- Map<HRegionInfo, ServerName> regionToServerMap = snapshot.getRegionToRegionServerMap();
+ Map<RegionInfo, ServerName> regionToServerMap = snapshot.getRegionToRegionServerMap();
assert(regionToServerMap.size() == numRegions * numReplica + 1); //'1' for the namespace
- Map<ServerName, List<HRegionInfo>> serverToRegionMap = snapshot.getRegionServerToRegionMap();
- for (Map.Entry<ServerName, List<HRegionInfo>> entry : serverToRegionMap.entrySet()) {
+ Map<ServerName, List<RegionInfo>> serverToRegionMap = snapshot.getRegionServerToRegionMap();
+ for (Map.Entry<ServerName, List<RegionInfo>> entry : serverToRegionMap.entrySet()) {
if (entry.getKey().equals(util.getHBaseCluster().getMaster().getServerName())) {
continue;
}
- List<HRegionInfo> regions = entry.getValue();
+ List<RegionInfo> regions = entry.getValue();
Set<byte[]> setOfStartKeys = new HashSet<>();
- for (HRegionInfo region : regions) {
+ for (RegionInfo region : regions) {
byte[] startKey = region.getStartKey();
if (region.getTable().equals(table)) {
setOfStartKeys.add(startKey); //ignore other tables
@@ -343,11 +340,11 @@ public class TestMasterOperationsForRegionReplicas {
SnapshotOfRegionAssignmentFromMeta snapshot = new SnapshotOfRegionAssignmentFromMeta(
connection);
snapshot.initialize();
- Map<HRegionInfo, ServerName> regionToServerMap = snapshot.getRegionToRegionServerMap();
+ Map<RegionInfo, ServerName> regionToServerMap = snapshot.getRegionToRegionServerMap();
assertEquals(regionToServerMap.size(), numRegions * numReplica + 1); //'1' for the namespace
- Map<ServerName, List<HRegionInfo>> serverToRegionMap = snapshot.getRegionServerToRegionMap();
+ Map<ServerName, List<RegionInfo>> serverToRegionMap = snapshot.getRegionServerToRegionMap();
assertEquals(serverToRegionMap.keySet().size(), 2); // 1 rs + 1 master
- for (Map.Entry<ServerName, List<HRegionInfo>> entry : serverToRegionMap.entrySet()) {
+ for (Map.Entry<ServerName, List<RegionInfo>> entry : serverToRegionMap.entrySet()) {
if (entry.getKey().equals(TEST_UTIL.getHBaseCluster().getMaster().getServerName())) {
continue;
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterTransitions.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterTransitions.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterTransitions.java
index 1d8a761..042a462 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterTransitions.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterTransitions.java
@@ -24,11 +24,11 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.RegionLocator;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
@@ -101,13 +101,13 @@ public class TestMasterTransitions {
private final HServerAddress metaAddress;
private final MiniHBaseCluster cluster;
private final int otherServerIndex;
- private final HRegionInfo hri;
+ private final RegionInfo hri;
private int closeCount = 0;
static final int SERVER_DURATION = 3 * 1000;
static final int CLOSE_DURATION = 1 * 1000;
HBase2428Listener(final MiniHBaseCluster c, final HServerAddress metaAddress,
- final HRegionInfo closingHRI, final int otherServerIndex) {
+ final RegionInfo closingHRI, final int otherServerIndex) {
this.cluster = c;
this.metaAddress = metaAddress;
this.hri = closingHRI;
@@ -211,7 +211,7 @@ public class TestMasterTransitions {
final HRegionServer metaHRS = cluster.getRegionServer(metaIndex);
// Get a region out on the otherServer.
- final HRegionInfo hri =
+ final RegionInfo hri =
otherServer.getOnlineRegions().iterator().next().getRegionInfo();
// Add our RegionServerOperationsListener
@@ -311,7 +311,7 @@ public class TestMasterTransitions {
private final Collection<HRegion> copyOfOnlineRegions;
// This is the region that was in transition on the server we aborted. Test
// passes if this region comes back online successfully.
- private HRegionInfo regionToFind;
+ private RegionInfo regionToFind;
HBase2482Listener(final HRegionServer victim) {
this.victim = victim;
@@ -443,7 +443,7 @@ public class TestMasterTransitions {
return countOfRegions;
}
- private void assertRegionIsBackOnline(final HRegionInfo hri)
+ private void assertRegionIsBackOnline(final RegionInfo hri)
throws IOException {
// Region should have an entry in its startkey because of addRowToEachRegion.
byte [] row = getStartKey(hri);
@@ -490,7 +490,7 @@ public class TestMasterTransitions {
scan.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
ResultScanner s = meta.getScanner(scan);
for (Result r = null; (r = s.next()) != null;) {
- HRegionInfo hri = MetaTableAccessor.getHRegionInfo(r);
+ RegionInfo hri = MetaTableAccessor.getRegionInfo(r);
if (hri == null) break;
if (!hri.getTable().equals(TABLENAME)) {
continue;
@@ -518,7 +518,7 @@ public class TestMasterTransitions {
* @param hri
* @return Start key for hri (If start key is '', then return 'aaa'.
*/
- private static byte [] getStartKey(final HRegionInfo hri) {
+ private static byte [] getStartKey(final RegionInfo hri) {
return Bytes.equals(HConstants.EMPTY_START_ROW, hri.getStartKey())?
Bytes.toBytes("aaa"): hri.getStartKey();
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionPlacement.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionPlacement.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionPlacement.java
index b6f3869..c0464a4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionPlacement.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionPlacement.java
@@ -40,7 +40,6 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MetaTableAccessor;
@@ -50,6 +49,7 @@ import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.RegionLocator;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.TableDescriptor;
@@ -81,7 +81,7 @@ public class TestRegionPlacement {
private static Position[] positions = Position.values();
private int lastRegionOnPrimaryRSCount = 0;
private int REGION_NUM = 10;
- private Map<HRegionInfo, ServerName[]> favoredNodesAssignmentPlan = new HashMap<>();
+ private Map<RegionInfo, ServerName[]> favoredNodesAssignmentPlan = new HashMap<>();
@BeforeClass
public static void setupBeforeClass() throws Exception {
@@ -204,9 +204,9 @@ public class TestRegionPlacement {
} while (ServerName.isSameAddress(metaServer, serverToKill) || isNamespaceServer ||
TEST_UTIL.getHBaseCluster().getRegionServer(killIndex).getNumberOfOnlineRegions() == 0);
LOG.debug("Stopping RS " + serverToKill);
- Map<HRegionInfo, Pair<ServerName, ServerName>> regionsToVerify = new HashMap<>();
+ Map<RegionInfo, Pair<ServerName, ServerName>> regionsToVerify = new HashMap<>();
// mark the regions to track
- for (Map.Entry<HRegionInfo, ServerName[]> entry : favoredNodesAssignmentPlan.entrySet()) {
+ for (Map.Entry<RegionInfo, ServerName[]> entry : favoredNodesAssignmentPlan.entrySet()) {
ServerName s = entry.getValue()[0];
if (ServerName.isSameAddress(s, serverToKill)) {
regionsToVerify.put(entry.getKey(), new Pair<>(entry.getValue()[1], entry.getValue()[2]));
@@ -225,7 +225,7 @@ public class TestRegionPlacement {
curr = TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager().getNumRegionsOpened();
}
// now verify
- for (Map.Entry<HRegionInfo, Pair<ServerName, ServerName>> entry : regionsToVerify.entrySet()) {
+ for (Map.Entry<RegionInfo, Pair<ServerName, ServerName>> entry : regionsToVerify.entrySet()) {
ServerName newDestination = TEST_UTIL.getHBaseCluster().getMaster()
.getAssignmentManager().getRegionStates().getRegionServerOfRegion(entry.getKey());
Pair<ServerName, ServerName> secondaryTertiaryServers = entry.getValue();
@@ -300,7 +300,7 @@ public class TestRegionPlacement {
FavoredNodesPlan.Position p1, FavoredNodesPlan.Position p2) throws IOException {
FavoredNodesPlan shuffledPlan = new FavoredNodesPlan();
- Map<String, HRegionInfo> regionToHRegion =
+ Map<String, RegionInfo> regionToHRegion =
rp.getRegionAssignmentSnapshot().getRegionNameToRegionInfoMap();
for (Map.Entry<String, List<ServerName>> entry :
plan.getAssignmentMap().entrySet()) {
@@ -470,7 +470,7 @@ public class TestRegionPlacement {
public boolean visit(Result result) throws IOException {
try {
@SuppressWarnings("deprecation")
- HRegionInfo info = MetaTableAccessor.getHRegionInfo(result);
+ RegionInfo info = MetaTableAccessor.getRegionInfo(result);
if(info.getTable().getNamespaceAsString()
.equals(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR)) {
return true;
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionPlacement2.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionPlacement2.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionPlacement2.java
index 86aca40..87f5ba3 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionPlacement2.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionPlacement2.java
@@ -17,7 +17,7 @@
*/
package org.apache.hadoop.hbase.master;
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertTrue;
import java.util.ArrayList;
import java.util.Collection;
@@ -28,15 +28,16 @@ import java.util.Set;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.favored.FavoredNodeAssignmentHelper;
-import org.apache.hadoop.hbase.favored.FavoredNodeLoadBalancer;
-import org.apache.hadoop.hbase.favored.FavoredNodesPlan.Position;
import org.apache.hadoop.hbase.HBaseIOException;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.favored.FavoredNodeAssignmentHelper;
+import org.apache.hadoop.hbase.favored.FavoredNodeLoadBalancer;
+import org.apache.hadoop.hbase.favored.FavoredNodesPlan.Position;
import org.apache.hadoop.hbase.master.balancer.LoadBalancerFactory;
import org.apache.hadoop.hbase.testclassification.MasterTests;
import org.apache.hadoop.hbase.testclassification.MediumTests;
@@ -84,10 +85,10 @@ public class TestRegionPlacement2 {
ServerName server = TEST_UTIL.getMiniHBaseCluster().getRegionServer(i).getServerName();
servers.add(server);
}
- List<HRegionInfo> regions = new ArrayList<>(1);
- HRegionInfo region = new HRegionInfo(TableName.valueOf(name.getMethodName()));
+ List<RegionInfo> regions = new ArrayList<>(1);
+ RegionInfo region = RegionInfoBuilder.newBuilder(TableName.valueOf(name.getMethodName())).build();
regions.add(region);
- Map<ServerName,List<HRegionInfo>> assignmentMap = balancer.roundRobinAssignment(regions,
+ Map<ServerName,List<RegionInfo>> assignmentMap = balancer.roundRobinAssignment(regions,
servers);
Set<ServerName> serverBefore = assignmentMap.keySet();
List<ServerName> favoredNodesBefore =
@@ -145,8 +146,8 @@ public class TestRegionPlacement2 {
ServerName server = TEST_UTIL.getMiniHBaseCluster().getRegionServer(i).getServerName();
servers.add(server);
}
- List<HRegionInfo> regions = new ArrayList<>(1);
- HRegionInfo region = new HRegionInfo(TableName.valueOf(name.getMethodName()));
+ List<RegionInfo> regions = new ArrayList<>(1);
+ RegionInfo region = RegionInfoBuilder.newBuilder(TableName.valueOf(name.getMethodName())).build();
regions.add(region);
ServerName serverBefore = balancer.randomAssignment(region, servers);
List<ServerName> favoredNodesBefore =
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRestartCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRestartCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRestartCluster.java
index 4eb4e53..9809090 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRestartCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRestartCluster.java
@@ -29,22 +29,19 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableExistsException;
import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.master.assignment.RegionStates;
-import org.apache.hadoop.hbase.net.Address;
-import org.apache.hadoop.hbase.shaded.com.google.common.net.HostAndPort;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.testclassification.MasterTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.JVMClusterUtil;
import org.apache.hadoop.hbase.util.Threads;
import org.junit.After;
-import org.junit.Ignore;
import org.junit.Test;
import org.junit.experimental.categories.Category;
@@ -78,7 +75,7 @@ public class TestRestartCluster {
UTIL.waitTableEnabled(TABLE);
}
- List<HRegionInfo> allRegions = MetaTableAccessor.getAllRegions(UTIL.getConnection(), false);
+ List<RegionInfo> allRegions = MetaTableAccessor.getAllRegions(UTIL.getConnection(), false);
assertEquals(4, allRegions.size());
LOG.info("\n\nShutting down cluster");
@@ -135,7 +132,7 @@ public class TestRestartCluster {
SnapshotOfRegionAssignmentFromMeta snapshot = new SnapshotOfRegionAssignmentFromMeta(
master.getConnection());
snapshot.initialize();
- Map<HRegionInfo, ServerName> regionToRegionServerMap
+ Map<RegionInfo, ServerName> regionToRegionServerMap
= snapshot.getRegionToRegionServerMap();
MiniHBaseCluster cluster = UTIL.getHBaseCluster();
@@ -201,10 +198,10 @@ public class TestRestartCluster {
snapshot = new SnapshotOfRegionAssignmentFromMeta(master.getConnection());
snapshot.initialize();
- Map<HRegionInfo, ServerName> newRegionToRegionServerMap =
+ Map<RegionInfo, ServerName> newRegionToRegionServerMap =
snapshot.getRegionToRegionServerMap();
assertEquals(regionToRegionServerMap.size(), newRegionToRegionServerMap.size());
- for (Map.Entry<HRegionInfo, ServerName> entry: newRegionToRegionServerMap.entrySet()) {
+ for (Map.Entry<RegionInfo, ServerName> entry: newRegionToRegionServerMap.entrySet()) {
if (TableName.NAMESPACE_TABLE_NAME.equals(entry.getKey().getTable())) continue;
ServerName oldServer = regionToRegionServerMap.get(entry.getKey());
ServerName currentServer = entry.getValue();
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRollingRestart.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRollingRestart.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRollingRestart.java
index 80c6f3a..89feadf 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRollingRestart.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRollingRestart.java
@@ -31,13 +31,12 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.RegionLocator;
import org.apache.hadoop.hbase.client.Table;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.testclassification.MasterTests;
import org.apache.hadoop.hbase.util.Bytes;
@@ -48,6 +47,8 @@ import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.junit.rules.TestName;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+
/**
* Tests the restarting of everything as done during rolling restarts.
*/
@@ -262,7 +263,7 @@ public class TestRollingRestart {
NavigableSet<String> online = new TreeSet<>();
NavigableSet<String> doubled = new TreeSet<>();
for (RegionServerThread rst : cluster.getLiveRegionServerThreads()) {
- for (HRegionInfo region : ProtobufUtil.getOnlineRegions(
+ for (RegionInfo region : ProtobufUtil.getOnlineRegions(
rst.getRegionServer().getRSRpcServices())) {
if(!online.add(region.getRegionNameAsString())) {
doubled.add(region.getRegionNameAsString());
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestWarmupRegion.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestWarmupRegion.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestWarmupRegion.java
index 58be83b..a3de52d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestWarmupRegion.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestWarmupRegion.java
@@ -27,13 +27,13 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.Waiter;
import org.apache.hadoop.hbase.client.CompactionState;
import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
@@ -131,7 +131,7 @@ public class TestWarmupRegion {
public void run() {
HRegionServer rs = TEST_UTIL.getMiniHBaseCluster().getRegionServer(0);
HRegion region = TEST_UTIL.getMiniHBaseCluster().getRegions(TABLENAME).get(0);
- HRegionInfo info = region.getRegionInfo();
+ RegionInfo info = region.getRegionInfo();
try {
HTableDescriptor htd = table.getTableDescriptor();
@@ -155,7 +155,7 @@ public class TestWarmupRegion {
public void testWarmup() throws Exception {
int serverid = 0;
HRegion region = TEST_UTIL.getMiniHBaseCluster().getRegions(TABLENAME).get(0);
- HRegionInfo info = region.getRegionInfo();
+ RegionInfo info = region.getRegionInfo();
runwarmup();
for (int i = 0; i < 10; i++) {
HRegionServer rs = TEST_UTIL.getMiniHBaseCluster().getRegionServer(serverid);
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/AssignmentTestingUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/AssignmentTestingUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/AssignmentTestingUtil.java
index 479b299..459e4ee 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/AssignmentTestingUtil.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/AssignmentTestingUtil.java
@@ -17,20 +17,20 @@
*/
package org.apache.hadoop.hbase.master.assignment;
+import static org.junit.Assert.assertEquals;
+
import java.util.Set;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.yetus.audience.InterfaceStability;
-import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.Waiter.ExplainingPredicate;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.util.Threads;
-
-import static org.junit.Assert.assertEquals;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.yetus.audience.InterfaceStability;
@InterfaceAudience.Private
@InterfaceStability.Evolving
@@ -40,7 +40,7 @@ public abstract class AssignmentTestingUtil {
private AssignmentTestingUtil() {}
public static void waitForRegionToBeInTransition(final HBaseTestingUtility util,
- final HRegionInfo hri) throws Exception {
+ final RegionInfo hri) throws Exception {
while (!getMaster(util).getAssignmentManager().getRegionStates().isRegionInTransition(hri)) {
Threads.sleep(10);
}
@@ -85,14 +85,14 @@ public abstract class AssignmentTestingUtil {
}
public static ServerName crashRsWithRegion(final HBaseTestingUtility util,
- final HRegionInfo hri, final boolean kill) throws Exception {
+ final RegionInfo hri, final boolean kill) throws Exception {
ServerName serverName = getServerHoldingRegion(util, hri);
crashRs(util, serverName, kill);
return serverName;
}
public static ServerName getServerHoldingRegion(final HBaseTestingUtility util,
- final HRegionInfo hri) throws Exception {
+ final RegionInfo hri) throws Exception {
ServerName serverName = util.getMiniHBaseCluster().getServerHoldingRegion(
hri.getTable(), hri.getRegionName());
ServerName amServerName = getMaster(util).getAssignmentManager().getRegionStates()
@@ -107,7 +107,7 @@ public abstract class AssignmentTestingUtil {
public static boolean isServerHoldingMeta(final HBaseTestingUtility util,
final ServerName serverName) throws Exception {
- for (HRegionInfo hri: getMetaRegions(util)) {
+ for (RegionInfo hri: getMetaRegions(util)) {
if (serverName.equals(getServerHoldingRegion(util, hri))) {
return true;
}
@@ -115,7 +115,7 @@ public abstract class AssignmentTestingUtil {
return false;
}
- public static Set<HRegionInfo> getMetaRegions(final HBaseTestingUtility util) {
+ public static Set<RegionInfo> getMetaRegions(final HBaseTestingUtility util) {
return getMaster(util).getAssignmentManager().getMetaRegionSet();
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/MockMasterServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/MockMasterServices.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/MockMasterServices.java
index 592794f..073216c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/MockMasterServices.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/MockMasterServices.java
@@ -26,7 +26,6 @@ import java.util.SortedSet;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.CoordinatedStateManager;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerLoad;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableDescriptors;
@@ -35,6 +34,8 @@ import org.apache.hadoop.hbase.YouAreDeadException;
import org.apache.hadoop.hbase.client.ClusterConnection;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
import org.apache.hadoop.hbase.client.HConnectionTestingUtility;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
import org.apache.hadoop.hbase.master.LoadBalancer;
@@ -54,6 +55,11 @@ import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
import org.apache.hadoop.hbase.procedure2.store.NoopProcedureStore;
import org.apache.hadoop.hbase.procedure2.store.ProcedureStore;
import org.apache.hadoop.hbase.security.Superusers;
+import org.apache.hadoop.hbase.util.FSUtils;
+import org.mockito.Mockito;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
@@ -66,10 +72,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateResp
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionAction;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionActionResult;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ResultOrException;
-import org.apache.hadoop.hbase.util.FSUtils;
-import org.mockito.Mockito;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
/**
* A mocked master services.
@@ -156,7 +158,7 @@ public class MockMasterServices extends MockNoopMasterServices {
this.connection =
HConnectionTestingUtility.getMockedConnectionAndDecorate(getConfiguration(),
Mockito.mock(AdminProtos.AdminService.BlockingInterface.class), ri, MOCK_MASTER_SERVERNAME,
- HRegionInfo.FIRST_META_REGIONINFO);
+ RegionInfoBuilder.FIRST_META_REGIONINFO);
// Set hbase.rootdir into test dir.
Path rootdir = FSUtils.getRootDir(getConfiguration());
FSUtils.setRootDir(getConfiguration(), rootdir);
@@ -291,7 +293,7 @@ public class MockMasterServices extends MockNoopMasterServices {
}
@Override
- public void updateRegionLocation(HRegionInfo regionInfo, State state, ServerName regionLocation,
+ public void updateRegionLocation(RegionInfo regionInfo, State state, ServerName regionLocation,
ServerName lastHost, long openSeqNum, long pid) throws IOException {
}
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentManager.java
index 69f8667..dba5d97 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentManager.java
@@ -43,10 +43,11 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.CategoryBasedTimeout;
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.NotServingRegionException;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
import org.apache.hadoop.hbase.client.RetriesExhaustedException;
import org.apache.hadoop.hbase.exceptions.UnexpectedStateException;
import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
@@ -56,7 +57,6 @@ import org.apache.hadoop.hbase.master.procedure.MasterProcedureConstants;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureScheduler;
import org.apache.hadoop.hbase.master.procedure.ProcedureSyncWait;
import org.apache.hadoop.hbase.master.procedure.RSProcedureDispatcher;
-import org.apache.hadoop.hbase.master.procedure.ServerCrashException;
import org.apache.hadoop.hbase.procedure2.Procedure;
import org.apache.hadoop.hbase.procedure2.ProcedureMetrics;
import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
@@ -64,19 +64,6 @@ import org.apache.hadoop.hbase.procedure2.store.wal.WALProcedureStore;
import org.apache.hadoop.hbase.procedure2.util.StringUtils;
import org.apache.hadoop.hbase.regionserver.RegionServerAbortedException;
import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ExecuteProceduresRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ExecuteProceduresResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest.RegionOpenInfo;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionResponse.RegionOpeningState;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest;
import org.apache.hadoop.hbase.testclassification.MasterTests;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.util.Bytes;
@@ -94,6 +81,19 @@ import org.junit.rules.ExpectedException;
import org.junit.rules.TestName;
import org.junit.rules.TestRule;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ExecuteProceduresRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ExecuteProceduresResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest.RegionOpenInfo;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionResponse.RegionOpeningState;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest;
+
@Category({MasterTests.class, MediumTests.class})
public class TestAssignmentManager {
private static final Log LOG = LogFactory.getLog(TestAssignmentManager.class);
@@ -152,7 +152,7 @@ public class TestAssignmentManager {
private void setUpMeta() throws Exception {
rsDispatcher.setMockRsExecutor(new GoodRsExecutor());
- am.assign(HRegionInfo.FIRST_META_REGIONINFO);
+ am.assign(RegionInfoBuilder.FIRST_META_REGIONINFO);
am.wakeMetaLoadedEvent();
am.setFailoverCleanupDone(true);
}
@@ -172,7 +172,12 @@ public class TestAssignmentManager {
@Ignore @Test // TODO
public void testGoodSplit() throws Exception {
TableName tableName = TableName.valueOf(this.name.getMethodName());
- HRegionInfo hri = new HRegionInfo(tableName, Bytes.toBytes(0), Bytes.toBytes(2), false, 0);
+ RegionInfo hri = RegionInfoBuilder.newBuilder(tableName)
+ .setStartKey(Bytes.toBytes(0))
+ .setEndKey(Bytes.toBytes(2))
+ .setSplit(false)
+ .setRegionId(0)
+ .build();
SplitTableRegionProcedure split =
new SplitTableRegionProcedure(this.master.getMasterProcedureExecutor().getEnvironment(),
hri, Bytes.toBytes(1));
@@ -209,7 +214,7 @@ public class TestAssignmentManager {
@Test
public void testAssignAndCrashBeforeResponse() throws Exception {
final TableName tableName = TableName.valueOf("testAssignAndCrashBeforeResponse");
- final HRegionInfo hri = createRegionInfo(tableName, 1);
+ final RegionInfo hri = createRegionInfo(tableName, 1);
rsDispatcher.setMockRsExecutor(new HangThenRSCrashExecutor());
AssignProcedure proc = am.createAssignProcedure(hri, false);
waitOnFuture(submitProcedure(proc));
@@ -218,7 +223,7 @@ public class TestAssignmentManager {
@Test
public void testUnassignAndCrashBeforeResponse() throws Exception {
final TableName tableName = TableName.valueOf("testAssignAndCrashBeforeResponse");
- final HRegionInfo hri = createRegionInfo(tableName, 1);
+ final RegionInfo hri = createRegionInfo(tableName, 1);
rsDispatcher.setMockRsExecutor(new HangOnCloseThenRSCrashExecutor());
for (int i = 0; i < HangOnCloseThenRSCrashExecutor.TYPES_OF_FAILURE; i++) {
AssignProcedure assign = am.createAssignProcedure(hri, false);
@@ -232,7 +237,7 @@ public class TestAssignmentManager {
@Test
public void testAssignWithRandExec() throws Exception {
final TableName tableName = TableName.valueOf("testAssignWithRandExec");
- final HRegionInfo hri = createRegionInfo(tableName, 1);
+ final RegionInfo hri = createRegionInfo(tableName, 1);
rsDispatcher.setMockRsExecutor(new RandRsExecutor());
// Loop a bunch of times so we hit various combos of exceptions.
@@ -246,7 +251,7 @@ public class TestAssignmentManager {
@Ignore @Test // Disabled for now. Since HBASE-18551, this mock is insufficient.
public void testSocketTimeout() throws Exception {
final TableName tableName = TableName.valueOf(this.name.getMethodName());
- final HRegionInfo hri = createRegionInfo(tableName, 1);
+ final RegionInfo hri = createRegionInfo(tableName, 1);
// collect AM metrics before test
collectAssignmentManagerMetrics();
@@ -272,7 +277,7 @@ public class TestAssignmentManager {
private void testRetriesExhaustedFailure(final TableName tableName,
final MockRSExecutor executor) throws Exception {
- final HRegionInfo hri = createRegionInfo(tableName, 1);
+ final RegionInfo hri = createRegionInfo(tableName, 1);
// collect AM metrics before test
collectAssignmentManagerMetrics();
@@ -335,7 +340,7 @@ public class TestAssignmentManager {
private void testFailedOpen(final TableName tableName,
final MockRSExecutor executor) throws Exception {
- final HRegionInfo hri = createRegionInfo(tableName, 1);
+ final RegionInfo hri = createRegionInfo(tableName, 1);
// Test Assign operation failure
rsDispatcher.setMockRsExecutor(executor);
@@ -376,7 +381,7 @@ public class TestAssignmentManager {
@Test
public void testAssignAnAssignedRegion() throws Exception {
final TableName tableName = TableName.valueOf("testAssignAnAssignedRegion");
- final HRegionInfo hri = createRegionInfo(tableName, 1);
+ final RegionInfo hri = createRegionInfo(tableName, 1);
// collect AM metrics before test
collectAssignmentManagerMetrics();
@@ -406,7 +411,7 @@ public class TestAssignmentManager {
@Test
public void testUnassignAnUnassignedRegion() throws Exception {
final TableName tableName = TableName.valueOf("testUnassignAnUnassignedRegion");
- final HRegionInfo hri = createRegionInfo(tableName, 1);
+ final RegionInfo hri = createRegionInfo(tableName, 1);
// collect AM metrics before test
collectAssignmentManagerMetrics();
@@ -482,26 +487,31 @@ public class TestAssignmentManager {
}
private AssignProcedure createAndSubmitAssign(TableName tableName, int regionId) {
- HRegionInfo hri = createRegionInfo(tableName, regionId);
+ RegionInfo hri = createRegionInfo(tableName, regionId);
AssignProcedure proc = am.createAssignProcedure(hri, false);
master.getMasterProcedureExecutor().submitProcedure(proc);
return proc;
}
private UnassignProcedure createAndSubmitUnassign(TableName tableName, int regionId) {
- HRegionInfo hri = createRegionInfo(tableName, regionId);
+ RegionInfo hri = createRegionInfo(tableName, regionId);
UnassignProcedure proc = am.createUnassignProcedure(hri, null, false);
master.getMasterProcedureExecutor().submitProcedure(proc);
return proc;
}
- private HRegionInfo createRegionInfo(final TableName tableName, final long regionId) {
- return new HRegionInfo(tableName,
- Bytes.toBytes(regionId), Bytes.toBytes(regionId + 1), false, 0);
+ private RegionInfo createRegionInfo(final TableName tableName, final long regionId) {
+ return RegionInfoBuilder.newBuilder(tableName)
+ .setStartKey(Bytes.toBytes(regionId))
+ .setEndKey(Bytes.toBytes(regionId + 1))
+ .setSplit(false)
+ .setRegionId(0)
+ .build();
}
private void sendTransitionReport(final ServerName serverName,
- final RegionInfo regionInfo, final TransitionCode state) throws IOException {
+ final org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo regionInfo,
+ final TransitionCode state) throws IOException {
ReportRegionStateTransitionRequest.Builder req =
ReportRegionStateTransitionRequest.newBuilder();
req.setServer(ProtobufUtil.toServerName(serverName));
@@ -568,7 +578,7 @@ public class TestAssignmentManager {
regions = new ConcurrentSkipListSet<byte[]>(Bytes.BYTES_COMPARATOR);
regionsToRegionServers.put(server, regions);
}
- HRegionInfo hri = HRegionInfo.convert(openReq.getRegion());
+ RegionInfo hri = ProtobufUtil.toRegionInfo(openReq.getRegion());
if (regions.contains(hri.getRegionName())) {
throw new UnsupportedOperationException(hri.getRegionNameAsString());
}
@@ -579,8 +589,8 @@ public class TestAssignmentManager {
@Override
protected CloseRegionResponse execCloseRegion(ServerName server, byte[] regionName)
throws IOException {
- HRegionInfo hri = am.getRegionInfo(regionName);
- sendTransitionReport(server, HRegionInfo.convert(hri), TransitionCode.CLOSED);
+ RegionInfo hri = am.getRegionInfo(regionName);
+ sendTransitionReport(server, ProtobufUtil.toRegionInfo(hri), TransitionCode.CLOSED);
return CloseRegionResponse.newBuilder().setClosed(true).build();
}
}
@@ -745,8 +755,8 @@ public class TestAssignmentManager {
CloseRegionResponse.Builder resp = CloseRegionResponse.newBuilder();
boolean closed = rand.nextBoolean();
if (closed) {
- HRegionInfo hri = am.getRegionInfo(regionName);
- sendTransitionReport(server, HRegionInfo.convert(hri), TransitionCode.CLOSED);
+ RegionInfo hri = am.getRegionInfo(regionName);
+ sendTransitionReport(server, ProtobufUtil.toRegionInfo(hri), TransitionCode.CLOSED);
}
resp.setClosed(closed);
return resp.build();
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentOnRSCrash.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentOnRSCrash.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentOnRSCrash.java
index e4cec45..72fb7ad 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentOnRSCrash.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentOnRSCrash.java
@@ -18,33 +18,32 @@
package org.apache.hadoop.hbase.master.assignment;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertTrue;
+
import java.io.IOException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Table;
-import org.apache.hadoop.hbase.testclassification.MasterTests;
-import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureConstants;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.testclassification.MasterTests;
import org.apache.hadoop.hbase.util.Bytes;
-
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import org.junit.experimental.categories.Category;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotEquals;
-import static org.junit.Assert.assertTrue;
-
@Category({MasterTests.class, LargeTests.class})
public class TestAssignmentOnRSCrash {
private static final Log LOG = LogFactory.getLog(TestAssignmentOnRSCrash.class);
@@ -102,7 +101,7 @@ public class TestAssignmentOnRSCrash {
throws Exception {
final int NROWS = 100;
int nkilled = 0;
- for (HRegionInfo hri: UTIL.getHBaseAdmin().getTableRegions(TEST_TABLE)) {
+ for (RegionInfo hri: UTIL.getHBaseAdmin().getTableRegions(TEST_TABLE)) {
ServerName serverName = AssignmentTestingUtil.getServerHoldingRegion(UTIL, hri);
if (AssignmentTestingUtil.isServerHoldingMeta(UTIL, serverName)) continue;
@@ -141,7 +140,7 @@ public class TestAssignmentOnRSCrash {
private void testCrashRsWithMetaRegion(final boolean kill) throws Exception {
int nkilled = 0;
- for (HRegionInfo hri: AssignmentTestingUtil.getMetaRegions(UTIL)) {
+ for (RegionInfo hri: AssignmentTestingUtil.getMetaRegions(UTIL)) {
ServerName serverName = AssignmentTestingUtil.crashRsWithRegion(UTIL, hri, kill);
// wait for region to enter in transition and then to get out of transition
@@ -159,7 +158,7 @@ public class TestAssignmentOnRSCrash {
assertTrue("expected RSs to be killed", nkilled > 0);
}
- private void testInsert(final HRegionInfo hri, final int nrows) throws IOException {
+ private void testInsert(final RegionInfo hri, final int nrows) throws IOException {
final Table table = UTIL.getConnection().getTable(hri.getTable());
for (int i = 0; i < nrows; ++i) {
final byte[] row = Bytes.add(hri.getStartKey(), Bytes.toBytes(i));
@@ -169,7 +168,7 @@ public class TestAssignmentOnRSCrash {
}
}
- public int testGet(final HRegionInfo hri, final int nrows) throws IOException {
+ public int testGet(final RegionInfo hri, final int nrows) throws IOException {
int nresults = 0;
final Table table = UTIL.getConnection().getTable(hri.getTable());
for (int i = 0; i < nrows; ++i) {
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestMergeTableRegionsProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestMergeTableRegionsProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestMergeTableRegionsProcedure.java
index dcc0702..28d07aa 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestMergeTableRegionsProcedure.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestMergeTableRegionsProcedure.java
@@ -30,11 +30,11 @@ import org.apache.hadoop.hbase.CategoryBasedTimeout;
import org.apache.hadoop.hbase.HBaseTestingUtility;
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.MetaTableAccessor;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureConstants;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureTestingUtility;
@@ -146,9 +146,9 @@ public class TestMergeTableRegionsProcedure {
final TableName tableName = TableName.valueOf(this.name.getMethodName());
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
- List<HRegionInfo> tableRegions = createTable(tableName);
+ List<RegionInfo> tableRegions = createTable(tableName);
- HRegionInfo[] regionsToMerge = new HRegionInfo[2];
+ RegionInfo[] regionsToMerge = new RegionInfo[2];
regionsToMerge[0] = tableRegions.get(0);
regionsToMerge[1] = tableRegions.get(1);
@@ -169,7 +169,7 @@ public class TestMergeTableRegionsProcedure {
assertEquals(unassignSubmittedCount + 2, unassignProcMetrics.getSubmittedCounter().getCount());
assertEquals(unassignFailedCount, unassignProcMetrics.getFailedCounter().getCount());
- Pair<HRegionInfo, HRegionInfo> pair =
+ Pair<RegionInfo, RegionInfo> pair =
MetaTableAccessor.getRegionsFromMergeQualifier(UTIL.getConnection(),
proc.getMergedRegion().getRegionName());
assertTrue(pair.getFirst() != null && pair.getSecond() != null);
@@ -194,10 +194,10 @@ public class TestMergeTableRegionsProcedure {
final TableName tableName = TableName.valueOf("testMergeRegionsConcurrently");
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
- List<HRegionInfo> tableRegions = createTable(tableName);
+ List<RegionInfo> tableRegions = createTable(tableName);
- HRegionInfo[] regionsToMerge1 = new HRegionInfo[2];
- HRegionInfo[] regionsToMerge2 = new HRegionInfo[2];
+ RegionInfo[] regionsToMerge1 = new RegionInfo[2];
+ RegionInfo[] regionsToMerge2 = new RegionInfo[2];
regionsToMerge1[0] = tableRegions.get(0);
regionsToMerge1[1] = tableRegions.get(1);
regionsToMerge2[0] = tableRegions.get(2);
@@ -229,12 +229,12 @@ public class TestMergeTableRegionsProcedure {
final TableName tableName = TableName.valueOf("testRecoveryAndDoubleExecution");
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
- List<HRegionInfo> tableRegions = createTable(tableName);
+ List<RegionInfo> tableRegions = createTable(tableName);
ProcedureTestingUtility.waitNoProcedureRunning(procExec);
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
- HRegionInfo[] regionsToMerge = new HRegionInfo[2];
+ RegionInfo[] regionsToMerge = new RegionInfo[2];
regionsToMerge[0] = tableRegions.get(0);
regionsToMerge[1] = tableRegions.get(1);
@@ -253,12 +253,12 @@ public class TestMergeTableRegionsProcedure {
final TableName tableName = TableName.valueOf("testRollbackAndDoubleExecution");
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
- List<HRegionInfo> tableRegions = createTable(tableName);
+ List<RegionInfo> tableRegions = createTable(tableName);
ProcedureTestingUtility.waitNoProcedureRunning(procExec);
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
- HRegionInfo[] regionsToMerge = new HRegionInfo[2];
+ RegionInfo[] regionsToMerge = new RegionInfo[2];
regionsToMerge[0] = tableRegions.get(0);
regionsToMerge[1] = tableRegions.get(1);
@@ -272,7 +272,7 @@ public class TestMergeTableRegionsProcedure {
MasterProcedureTestingUtility.testRollbackAndDoubleExecution(procExec, procId, numberOfSteps);
}
- private List<HRegionInfo> createTable(final TableName tableName)
+ private List<RegionInfo> createTable(final TableName tableName)
throws Exception {
HTableDescriptor desc = new HTableDescriptor(tableName);
desc.addFamily(new HColumnDescriptor(FAMILY));
@@ -284,10 +284,10 @@ public class TestMergeTableRegionsProcedure {
return assertRegionCount(tableName, initialRegionCount);
}
- public List<HRegionInfo> assertRegionCount(final TableName tableName, final int nregions)
+ public List<RegionInfo> assertRegionCount(final TableName tableName, final int nregions)
throws Exception {
UTIL.waitUntilNoRegionsInTransition();
- List<HRegionInfo> tableRegions = admin.getTableRegions(tableName);
+ List<RegionInfo> tableRegions = admin.getRegions(tableName);
assertEquals(nregions, tableRegions.size());
return tableRegions;
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionStates.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionStates.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionStates.java
index 0087ecd..5f1d09a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionStates.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionStates.java
@@ -18,11 +18,22 @@
package org.apache.hadoop.hbase.master.assignment;
+import static org.junit.Assert.assertEquals;
+
+import java.lang.Thread.UncaughtExceptionHandler;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.Future;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
import org.apache.hadoop.hbase.procedure2.util.StringUtils;
import org.apache.hadoop.hbase.testclassification.MasterTests;
import org.apache.hadoop.hbase.testclassification.MediumTests;
@@ -35,16 +46,6 @@ import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
-import java.lang.Thread.UncaughtExceptionHandler;
-import java.util.List;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutorCompletionService;
-import java.util.concurrent.Future;
-import java.util.concurrent.ThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
-
-import static org.junit.Assert.assertEquals;
-
@Category({MasterTests.class, MediumTests.class})
public class TestRegionStates {
private static final Log LOG = LogFactory.getLog(TestRegionStates.class);
@@ -97,7 +98,7 @@ public class TestRegionStates {
@Test
public void testRegionDoubleCreation() throws Exception {
- // NOTE: HRegionInfo sort by table first, so we are relying on that
+ // NOTE: RegionInfo sort by table first, so we are relying on that
final TableName TABLE_NAME_A = TableName.valueOf("testOrderedByTableA");
final TableName TABLE_NAME_B = TableName.valueOf("testOrderedByTableB");
final TableName TABLE_NAME_C = TableName.valueOf("testOrderedByTableC");
@@ -129,7 +130,7 @@ public class TestRegionStates {
private void checkTableRegions(final RegionStates stateMap,
final TableName tableName, final int nregions) {
- List<HRegionInfo> hris = stateMap.getRegionsOfTable(tableName, true);
+ List<RegionInfo> hris = stateMap.getRegionsOfTable(tableName, true);
assertEquals(nregions, hris.size());
for (int i = 1; i < hris.size(); ++i) {
long a = Bytes.toLong(hris.get(i - 1).getStartKey());
@@ -143,9 +144,12 @@ public class TestRegionStates {
executorService.submit(new Callable<Object>() {
@Override
public Object call() {
- HRegionInfo hri = new HRegionInfo(tableName,
- Bytes.toBytes(regionId), Bytes.toBytes(regionId + 1), false, 0);
- return stateMap.getOrCreateRegionNode(hri);
+ return stateMap.getOrCreateRegionNode(RegionInfoBuilder.newBuilder(tableName)
+ .setStartKey(Bytes.toBytes(regionId))
+ .setEndKey(Bytes.toBytes(regionId + 1))
+ .setSplit(false)
+ .setRegionId(0)
+ .build());
}
});
}
@@ -155,9 +159,13 @@ public class TestRegionStates {
return stateMap.getOrCreateRegionNode(createRegionInfo(tableName, regionId));
}
- private HRegionInfo createRegionInfo(final TableName tableName, final long regionId) {
- return new HRegionInfo(tableName,
- Bytes.toBytes(regionId), Bytes.toBytes(regionId + 1), false, 0);
+ private RegionInfo createRegionInfo(final TableName tableName, final long regionId) {
+ return RegionInfoBuilder.newBuilder(tableName)
+ .setStartKey(Bytes.toBytes(regionId))
+ .setEndKey(Bytes.toBytes(regionId + 1))
+ .setSplit(false)
+ .setRegionId(0)
+ .build();
}
@Test
@@ -172,7 +180,7 @@ public class TestRegionStates {
executorService.submit(new Callable<Object>() {
@Override
public Object call() {
- HRegionInfo hri = createRegionInfo(TABLE_NAME, regionId);
+ RegionInfo hri = createRegionInfo(TABLE_NAME, regionId);
return stateMap.getOrCreateRegionNode(hri);
}
});
@@ -189,7 +197,7 @@ public class TestRegionStates {
executorService.submit(new Callable<Object>() {
@Override
public Object call() {
- HRegionInfo hri = createRegionInfo(TABLE_NAME, regionId);
+ RegionInfo hri = createRegionInfo(TABLE_NAME, regionId);
return stateMap.getRegionState(hri);
}
});
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestSplitTableRegionProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestSplitTableRegionProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestSplitTableRegionProcedure.java
index ae08f12..7b6e977 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestSplitTableRegionProcedure.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestSplitTableRegionProcedure.java
@@ -34,7 +34,6 @@ import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.Waiter;
@@ -42,10 +41,11 @@ import org.apache.hadoop.hbase.client.CompactionState;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Table;
-import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureConstants;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureTestingUtility;
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
import org.apache.hadoop.hbase.procedure2.ProcedureMetrics;
@@ -141,7 +141,7 @@ public class TestSplitTableRegionProcedure {
final TableName tableName = TableName.valueOf(name.getMethodName());
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
- HRegionInfo [] regions = MasterProcedureTestingUtility.createTable(
+ RegionInfo [] regions = MasterProcedureTestingUtility.createTable(
procExec, tableName, null, ColumnFamilyName1, ColumnFamilyName2);
insertData(tableName);
int splitRowNum = startRowNum + rowCount / 2;
@@ -175,7 +175,7 @@ public class TestSplitTableRegionProcedure {
final TableName tableName = TableName.valueOf(name.getMethodName());
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
- HRegionInfo [] regions = MasterProcedureTestingUtility.createTable(
+ RegionInfo [] regions = MasterProcedureTestingUtility.createTable(
procExec, tableName, null, ColumnFamilyName1, ColumnFamilyName2);
int splitRowNum = startRowNum + rowCount / 2;
byte[] splitKey = Bytes.toBytes("" + splitRowNum);
@@ -205,7 +205,7 @@ public class TestSplitTableRegionProcedure {
final TableName tableName = TableName.valueOf(name.getMethodName());
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
- HRegionInfo [] regions = MasterProcedureTestingUtility.createTable(
+ RegionInfo [] regions = MasterProcedureTestingUtility.createTable(
procExec, tableName, null, ColumnFamilyName1, ColumnFamilyName2);
insertData(tableName);
// Split to two daughters with one of them only has 1 row
@@ -236,7 +236,7 @@ public class TestSplitTableRegionProcedure {
final TableName tableName = TableName.valueOf(name.getMethodName());
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
- HRegionInfo [] regions = MasterProcedureTestingUtility.createTable(
+ RegionInfo [] regions = MasterProcedureTestingUtility.createTable(
procExec, tableName, null, ColumnFamilyName1, ColumnFamilyName2);
insertData(tableName);
// Split to two daughters with one of them only has 1 row
@@ -272,7 +272,7 @@ public class TestSplitTableRegionProcedure {
final TableName tableName = TableName.valueOf(name.getMethodName());
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
- HRegionInfo [] regions = MasterProcedureTestingUtility.createTable(
+ RegionInfo [] regions = MasterProcedureTestingUtility.createTable(
procExec, tableName, null, ColumnFamilyName1, ColumnFamilyName2);
insertData(tableName);
// Split to two daughters with one of them only has 1 row
@@ -318,7 +318,7 @@ public class TestSplitTableRegionProcedure {
final TableName tableName = TableName.valueOf(name.getMethodName());
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
- HRegionInfo [] regions = MasterProcedureTestingUtility.createTable(
+ RegionInfo [] regions = MasterProcedureTestingUtility.createTable(
procExec, tableName, null, ColumnFamilyName1, ColumnFamilyName2);
insertData(tableName);
@@ -347,7 +347,7 @@ public class TestSplitTableRegionProcedure {
final TableName tableName = TableName.valueOf(name.getMethodName());
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
- HRegionInfo [] regions = MasterProcedureTestingUtility.createTable(
+ RegionInfo [] regions = MasterProcedureTestingUtility.createTable(
procExec, tableName, null, ColumnFamilyName1, ColumnFamilyName2);
insertData(tableName);
int splitRowNum = startRowNum + rowCount / 2;
@@ -387,7 +387,7 @@ public class TestSplitTableRegionProcedure {
final TableName tableName = TableName.valueOf(name.getMethodName());
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
- HRegionInfo [] regions = MasterProcedureTestingUtility.createTable(
+ RegionInfo [] regions = MasterProcedureTestingUtility.createTable(
procExec, tableName, null, ColumnFamilyName1, ColumnFamilyName2);
insertData(tableName);
int splitRowNum = startRowNum + rowCount / 2;
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/BalancerTestBase.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/BalancerTestBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/BalancerTestBase.java
index 7f97322..ee2e433 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/BalancerTestBase.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/BalancerTestBase.java
@@ -39,9 +39,10 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
import org.apache.hadoop.hbase.master.RackManager;
import org.apache.hadoop.hbase.master.RegionPlan;
@@ -52,7 +53,7 @@ import org.junit.BeforeClass;
/**
* Class used to be the base of unit tests on load balancers. It gives helper
- * methods to create maps of {@link ServerName} to lists of {@link HRegionInfo}
+ * methods to create maps of {@link ServerName} to lists of {@link RegionInfo}
* and to check list of region plans.
*
*/
@@ -244,20 +245,20 @@ public class BalancerTestBase {
/**
* Checks whether region replicas are not hosted on the same host.
*/
- public void assertRegionReplicaPlacement(Map<ServerName, List<HRegionInfo>> serverMap, RackManager rackManager) {
- TreeMap<String, Set<HRegionInfo>> regionsPerHost = new TreeMap<>();
- TreeMap<String, Set<HRegionInfo>> regionsPerRack = new TreeMap<>();
+ public void assertRegionReplicaPlacement(Map<ServerName, List<RegionInfo>> serverMap, RackManager rackManager) {
+ TreeMap<String, Set<RegionInfo>> regionsPerHost = new TreeMap<>();
+ TreeMap<String, Set<RegionInfo>> regionsPerRack = new TreeMap<>();
- for (Entry<ServerName, List<HRegionInfo>> entry : serverMap.entrySet()) {
+ for (Entry<ServerName, List<RegionInfo>> entry : serverMap.entrySet()) {
String hostname = entry.getKey().getHostname();
- Set<HRegionInfo> infos = regionsPerHost.get(hostname);
+ Set<RegionInfo> infos = regionsPerHost.get(hostname);
if (infos == null) {
infos = new HashSet<>();
regionsPerHost.put(hostname, infos);
}
- for (HRegionInfo info : entry.getValue()) {
- HRegionInfo primaryInfo = RegionReplicaUtil.getRegionInfoForDefaultReplica(info);
+ for (RegionInfo info : entry.getValue()) {
+ RegionInfo primaryInfo = RegionReplicaUtil.getRegionInfoForDefaultReplica(info);
if (!infos.add(primaryInfo)) {
Assert.fail("Two or more region replicas are hosted on the same host after balance");
}
@@ -268,16 +269,16 @@ public class BalancerTestBase {
return;
}
- for (Entry<ServerName, List<HRegionInfo>> entry : serverMap.entrySet()) {
+ for (Entry<ServerName, List<RegionInfo>> entry : serverMap.entrySet()) {
String rack = rackManager.getRack(entry.getKey());
- Set<HRegionInfo> infos = regionsPerRack.get(rack);
+ Set<RegionInfo> infos = regionsPerRack.get(rack);
if (infos == null) {
infos = new HashSet<>();
regionsPerRack.put(rack, infos);
}
- for (HRegionInfo info : entry.getValue()) {
- HRegionInfo primaryInfo = RegionReplicaUtil.getRegionInfoForDefaultReplica(info);
+ for (RegionInfo info : entry.getValue()) {
+ RegionInfo primaryInfo = RegionReplicaUtil.getRegionInfoForDefaultReplica(info);
if (!infos.add(primaryInfo)) {
Assert.fail("Two or more region replicas are hosted on the same rack after balance");
}
@@ -298,9 +299,9 @@ public class BalancerTestBase {
+ " min=" + min + "]";
}
- protected List<ServerAndLoad> convertToList(final Map<ServerName, List<HRegionInfo>> servers) {
+ protected List<ServerAndLoad> convertToList(final Map<ServerName, List<RegionInfo>> servers) {
List<ServerAndLoad> list = new ArrayList<>(servers.size());
- for (Map.Entry<ServerName, List<HRegionInfo>> e : servers.entrySet()) {
+ for (Map.Entry<ServerName, List<RegionInfo>> e : servers.entrySet()) {
list.add(new ServerAndLoad(e.getKey(), e.getValue().size()));
}
return list;
@@ -333,7 +334,7 @@ public class BalancerTestBase {
*/
protected List<ServerAndLoad> reconcile(List<ServerAndLoad> list,
List<RegionPlan> plans,
- Map<ServerName, List<HRegionInfo>> servers) {
+ Map<ServerName, List<RegionInfo>> servers) {
List<ServerAndLoad> result = new ArrayList<>(list.size());
Map<ServerName, ServerAndLoad> map = new HashMap<>(list.size());
@@ -366,7 +367,7 @@ public class BalancerTestBase {
map.put(sn, sal);
}
- protected TreeMap<ServerName, List<HRegionInfo>> mockClusterServers(int[] mockCluster) {
+ protected TreeMap<ServerName, List<RegionInfo>> mockClusterServers(int[] mockCluster) {
return mockClusterServers(mockCluster, -1);
}
@@ -375,42 +376,42 @@ public class BalancerTestBase {
mockClusterServers(mockCluster, -1), null, null, null);
}
- protected TreeMap<ServerName, List<HRegionInfo>> mockClusterServers(int[] mockCluster, int numTables) {
+ protected TreeMap<ServerName, List<RegionInfo>> mockClusterServers(int[] mockCluster, int numTables) {
int numServers = mockCluster.length;
- TreeMap<ServerName, List<HRegionInfo>> servers = new TreeMap<>();
+ TreeMap<ServerName, List<RegionInfo>> servers = new TreeMap<>();
for (int i = 0; i < numServers; i++) {
int numRegions = mockCluster[i];
ServerAndLoad sal = randomServer(0);
- List<HRegionInfo> regions = randomRegions(numRegions, numTables);
+ List<RegionInfo> regions = randomRegions(numRegions, numTables);
servers.put(sal.getServerName(), regions);
}
return servers;
}
- protected TreeMap<ServerName, List<HRegionInfo>> mockUniformClusterServers(int[] mockCluster) {
+ protected TreeMap<ServerName, List<RegionInfo>> mockUniformClusterServers(int[] mockCluster) {
int numServers = mockCluster.length;
- TreeMap<ServerName, List<HRegionInfo>> servers = new TreeMap<>();
+ TreeMap<ServerName, List<RegionInfo>> servers = new TreeMap<>();
for (int i = 0; i < numServers; i++) {
int numRegions = mockCluster[i];
ServerAndLoad sal = randomServer(0);
- List<HRegionInfo> regions = uniformRegions(numRegions);
+ List<RegionInfo> regions = uniformRegions(numRegions);
servers.put(sal.getServerName(), regions);
}
return servers;
}
- protected HashMap<TableName, TreeMap<ServerName, List<HRegionInfo>>> mockClusterServersWithTables(Map<ServerName, List<HRegionInfo>> clusterServers) {
- HashMap<TableName, TreeMap<ServerName, List<HRegionInfo>>> result = new HashMap<>();
- for (Map.Entry<ServerName, List<HRegionInfo>> entry : clusterServers.entrySet()) {
+ protected HashMap<TableName, TreeMap<ServerName, List<RegionInfo>>> mockClusterServersWithTables(Map<ServerName, List<RegionInfo>> clusterServers) {
+ HashMap<TableName, TreeMap<ServerName, List<RegionInfo>>> result = new HashMap<>();
+ for (Map.Entry<ServerName, List<RegionInfo>> entry : clusterServers.entrySet()) {
ServerName sal = entry.getKey();
- List<HRegionInfo> regions = entry.getValue();
- for (HRegionInfo hri : regions){
- TreeMap<ServerName, List<HRegionInfo>> servers = result.get(hri.getTable());
+ List<RegionInfo> regions = entry.getValue();
+ for (RegionInfo hri : regions){
+ TreeMap<ServerName, List<RegionInfo>> servers = result.get(hri.getTable());
if (servers == null) {
servers = new TreeMap<>();
result.put(hri.getTable(), servers);
}
- List<HRegionInfo> hrilist = servers.get(sal);
+ List<RegionInfo> hrilist = servers.get(sal);
if (hrilist == null) {
hrilist = new ArrayList<>();
servers.put(sal, hrilist);
@@ -418,7 +419,7 @@ public class BalancerTestBase {
hrilist.add(hri);
}
}
- for(Map.Entry<TableName, TreeMap<ServerName, List<HRegionInfo>>> entry : result.entrySet()){
+ for(Map.Entry<TableName, TreeMap<ServerName, List<RegionInfo>>> entry : result.entrySet()){
for(ServerName srn : clusterServers.keySet()){
if (!entry.getValue().containsKey(srn)) entry.getValue().put(srn, new ArrayList<>());
}
@@ -426,14 +427,14 @@ public class BalancerTestBase {
return result;
}
- private Queue<HRegionInfo> regionQueue = new LinkedList<>();
+ private Queue<RegionInfo> regionQueue = new LinkedList<>();
- protected List<HRegionInfo> randomRegions(int numRegions) {
+ protected List<RegionInfo> randomRegions(int numRegions) {
return randomRegions(numRegions, -1);
}
- protected List<HRegionInfo> randomRegions(int numRegions, int numTables) {
- List<HRegionInfo> regions = new ArrayList<>(numRegions);
+ protected List<RegionInfo> randomRegions(int numRegions, int numTables) {
+ List<RegionInfo> regions = new ArrayList<>(numRegions);
byte[] start = new byte[16];
byte[] end = new byte[16];
rand.nextBytes(start);
@@ -447,14 +448,19 @@ public class BalancerTestBase {
Bytes.putInt(end, 0, (numRegions << 1) + 1);
TableName tableName =
TableName.valueOf("table" + (numTables > 0 ? rand.nextInt(numTables) : i));
- HRegionInfo hri = new HRegionInfo(tableName, start, end, false, regionId++);
+ RegionInfo hri = RegionInfoBuilder.newBuilder(tableName)
+ .setStartKey(start)
+ .setEndKey(end)
+ .setSplit(false)
+ .setRegionId(regionId++)
+ .build();
regions.add(hri);
}
return regions;
}
- protected List<HRegionInfo> uniformRegions(int numRegions) {
- List<HRegionInfo> regions = new ArrayList<>(numRegions);
+ protected List<RegionInfo> uniformRegions(int numRegions) {
+ List<RegionInfo> regions = new ArrayList<>(numRegions);
byte[] start = new byte[16];
byte[] end = new byte[16];
rand.nextBytes(start);
@@ -464,13 +470,17 @@ public class BalancerTestBase {
Bytes.putInt(end, 0, (numRegions << 1) + 1);
TableName tableName =
TableName.valueOf("table" + i);
- HRegionInfo hri = new HRegionInfo(tableName, start, end, false);
+ RegionInfo hri = RegionInfoBuilder.newBuilder(tableName)
+ .setStartKey(start)
+ .setEndKey(end)
+ .setSplit(false)
+ .build();
regions.add(hri);
}
return regions;
}
- protected void returnRegions(List<HRegionInfo> regions) {
+ protected void returnRegions(List<RegionInfo> regions) {
regionQueue.addAll(regions);
}
@@ -510,12 +520,12 @@ public class BalancerTestBase {
int replication,
int numTables,
boolean assertFullyBalanced, boolean assertFullyBalancedForReplicas) {
- Map<ServerName, List<HRegionInfo>> serverMap =
+ Map<ServerName, List<RegionInfo>> serverMap =
createServerMap(numNodes, numRegions, numRegionsPerServer, replication, numTables);
testWithCluster(serverMap, null, assertFullyBalanced, assertFullyBalancedForReplicas);
}
- protected void testWithCluster(Map<ServerName, List<HRegionInfo>> serverMap,
+ protected void testWithCluster(Map<ServerName, List<RegionInfo>> serverMap,
RackManager rackManager, boolean assertFullyBalanced, boolean assertFullyBalancedForReplicas) {
List<ServerAndLoad> list = convertToList(serverMap);
LOG.info("Mock Cluster : " + printMock(list) + " " + printStats(list));
@@ -545,7 +555,7 @@ public class BalancerTestBase {
}
}
- protected Map<ServerName, List<HRegionInfo>> createServerMap(int numNodes,
+ protected Map<ServerName, List<RegionInfo>> createServerMap(int numNodes,
int numRegions,
int numRegionsPerServer,
int replication,
@@ -558,10 +568,10 @@ public class BalancerTestBase {
cluster[i] = numRegionsPerServer;
}
cluster[cluster.length - 1] = numRegions - ((cluster.length - 1) * numRegionsPerServer);
- Map<ServerName, List<HRegionInfo>> clusterState = mockClusterServers(cluster, numTables);
+ Map<ServerName, List<RegionInfo>> clusterState = mockClusterServers(cluster, numTables);
if (replication > 0) {
// replicate the regions to the same servers
- for (List<HRegionInfo> regions : clusterState.values()) {
+ for (List<RegionInfo> regions : clusterState.values()) {
int length = regions.size();
for (int i = 0; i < length; i++) {
for (int r = 1; r < replication ; r++) {
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/LoadBalancerPerformanceEvaluation.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/LoadBalancerPerformanceEvaluation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/LoadBalancerPerformanceEvaluation.java
index 8f43607..2b40ea7 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/LoadBalancerPerformanceEvaluation.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/LoadBalancerPerformanceEvaluation.java
@@ -18,8 +18,14 @@
package org.apache.hadoop.hbase.master.balancer;
-import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
-import org.apache.hadoop.hbase.shaded.com.google.common.base.Stopwatch;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
import org.apache.commons.cli.CommandLine;
import org.apache.commons.cli.Option;
import org.apache.commons.logging.Log;
@@ -27,21 +33,17 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HBaseCommonTestingUtility;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
import org.apache.hadoop.hbase.master.LoadBalancer;
import org.apache.hadoop.hbase.util.AbstractHBaseTool;
import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.TimeUnit;
+import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
+import org.apache.hadoop.hbase.shaded.com.google.common.base.Stopwatch;
/**
* Tool to test performance of different {@link org.apache.hadoop.hbase.master.LoadBalancer}
@@ -80,9 +82,9 @@ public class LoadBalancerPerformanceEvaluation extends AbstractHBaseTool {
// data
private List<ServerName> servers;
- private List<HRegionInfo> regions;
- private Map<HRegionInfo, ServerName> regionServerMap;
- private Map<ServerName, List<HRegionInfo>> serverRegionMap;
+ private List<RegionInfo> regions;
+ private Map<RegionInfo, ServerName> regionServerMap;
+ private Map<ServerName, List<RegionInfo>> serverRegionMap;
// Non-default configurations.
private void setupConf() {
@@ -101,7 +103,12 @@ public class LoadBalancerPerformanceEvaluation extends AbstractHBaseTool {
Bytes.putInt(start, 0, i);
Bytes.putInt(end, 0, i + 1);
TableName tableName = TableName.valueOf("LoadBalancerPerfTable");
- HRegionInfo hri = new HRegionInfo(tableName, start, end, false, i);
+ RegionInfo hri = RegionInfoBuilder.newBuilder(tableName)
+ .setStartKey(start)
+ .setEndKey(end)
+ .setSplit(false)
+ .setRegionId(i)
+ .build();
regions.add(hri);
regionServerMap.put(hri, null);
}
[02/19] hbase git commit: HBASE-18839 Apply RegionInfo to code base
Posted by ch...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionInfoBuilder.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionInfoBuilder.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionInfoBuilder.java
index 497ecc4..5e553dc 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionInfoBuilder.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionInfoBuilder.java
@@ -86,7 +86,7 @@ public class TestRegionInfoBuilder {
RegionInfo ri = RegionInfoBuilder.FIRST_META_REGIONINFO;
byte [] bytes = RegionInfo.toByteArray(ri);
RegionInfo pbri = RegionInfo.parseFrom(bytes);
- assertTrue(ri.equals(pbri));
+ assertTrue(RegionInfo.COMPARATOR.compare(ri, pbri) == 0);
}
@Test
@@ -286,7 +286,7 @@ public class TestRegionInfoBuilder {
.setReplicaId(replicaId).build();
// convert two times, compare
- RegionInfo convertedRi = ProtobufUtil.toRegionInfo(ProtobufUtil.toProtoRegionInfo(ri));
+ RegionInfo convertedRi = ProtobufUtil.toRegionInfo(ProtobufUtil.toRegionInfo(ri));
assertEquals(ri, convertedRi);
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransactionOnCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransactionOnCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransactionOnCluster.java
index b594a2f..15c6b76 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransactionOnCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransactionOnCluster.java
@@ -40,7 +40,6 @@ import org.apache.hadoop.hbase.CategoryBasedTimeout;
import org.apache.hadoop.hbase.CoordinatedStateManager;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.ServerName;
@@ -49,6 +48,7 @@ import org.apache.hadoop.hbase.UnknownRegionException;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
@@ -62,11 +62,6 @@ import org.apache.hadoop.hbase.master.RegionState;
import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
import org.apache.hadoop.hbase.master.assignment.RegionStates;
import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
-import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController;
-import org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionResponse;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
import org.apache.hadoop.hbase.util.Bytes;
@@ -87,6 +82,11 @@ import org.junit.rules.TestName;
import org.junit.rules.TestRule;
import org.apache.hadoop.hbase.shaded.com.google.common.base.Joiner;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionResponse;
@Category({RegionServerTests.class, MediumTests.class})
public class TestRegionMergeTransactionOnCluster {
@@ -142,14 +142,14 @@ public class TestRegionMergeTransactionOnCluster {
INITIAL_REGION_NUM - 1);
// Merge 2nd and 3th region
- PairOfSameType<HRegionInfo> mergedRegions =
+ PairOfSameType<RegionInfo> mergedRegions =
mergeRegionsAndVerifyRegionNum(MASTER, tableName, 1, 2,
INITIAL_REGION_NUM - 2);
verifyRowCount(table, ROWSIZE);
// Randomly choose one of the two merged regions
- HRegionInfo hri = RandomUtils.nextBoolean() ?
+ RegionInfo hri = RandomUtils.nextBoolean() ?
mergedRegions.getFirst() : mergedRegions.getSecond();
MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
AssignmentManager am = cluster.getMaster().getAssignmentManager();
@@ -204,9 +204,9 @@ public class TestRegionMergeTransactionOnCluster {
verifyRowCount(table, ROWSIZE);
table.close();
- List<Pair<HRegionInfo, ServerName>> tableRegions = MetaTableAccessor
+ List<Pair<RegionInfo, ServerName>> tableRegions = MetaTableAccessor
.getTableRegionsAndLocations(MASTER.getConnection(), tableName);
- HRegionInfo mergedRegionInfo = tableRegions.get(0).getFirst();
+ RegionInfo mergedRegionInfo = tableRegions.get(0).getFirst();
TableDescriptor tableDescriptor = MASTER.getTableDescriptors().get(
tableName);
Result mergedRegionResult = MetaTableAccessor.getRegionResult(
@@ -219,9 +219,9 @@ public class TestRegionMergeTransactionOnCluster {
HConstants.MERGEB_QUALIFIER) != null);
// merging regions' directory are in the file system all the same
- PairOfSameType<HRegionInfo> p = MetaTableAccessor.getMergeRegions(mergedRegionResult);
- HRegionInfo regionA = p.getFirst();
- HRegionInfo regionB = p.getSecond();
+ PairOfSameType<RegionInfo> p = MetaTableAccessor.getMergeRegions(mergedRegionResult);
+ RegionInfo regionA = p.getFirst();
+ RegionInfo regionB = p.getSecond();
FileSystem fs = MASTER.getMasterFileSystem().getFileSystem();
Path rootDir = MASTER.getMasterFileSystem().getRootDir();
@@ -315,10 +315,10 @@ public class TestRegionMergeTransactionOnCluster {
// Create table and load data.
Table table = createTableAndLoadData(MASTER, tableName);
AssignmentManager am = MASTER.getAssignmentManager();
- List<HRegionInfo> regions = am.getRegionStates().getRegionsOfTable(tableName);
+ List<RegionInfo> regions = am.getRegionStates().getRegionsOfTable(tableName);
// Fake offline one region
- HRegionInfo a = regions.get(0);
- HRegionInfo b = regions.get(1);
+ RegionInfo a = regions.get(0);
+ RegionInfo b = regions.get(1);
am.unassign(b);
am.offlineRegion(b);
try {
@@ -362,21 +362,21 @@ public class TestRegionMergeTransactionOnCluster {
final TableName tableName = TableName.valueOf(name.getMethodName());
// Create table and load data.
createTableAndLoadData(MASTER, tableName, 5, 2);
- List<Pair<HRegionInfo, ServerName>> initialRegionToServers =
+ List<Pair<RegionInfo, ServerName>> initialRegionToServers =
MetaTableAccessor.getTableRegionsAndLocations(
TEST_UTIL.getConnection(), tableName);
// Merge 1st and 2nd region
- PairOfSameType<HRegionInfo> mergedRegions = mergeRegionsAndVerifyRegionNum(MASTER, tableName,
+ PairOfSameType<RegionInfo> mergedRegions = mergeRegionsAndVerifyRegionNum(MASTER, tableName,
0, 2, 5 * 2 - 2);
- List<Pair<HRegionInfo, ServerName>> currentRegionToServers =
+ List<Pair<RegionInfo, ServerName>> currentRegionToServers =
MetaTableAccessor.getTableRegionsAndLocations(
TEST_UTIL.getConnection(), tableName);
- List<HRegionInfo> initialRegions = new ArrayList<>();
- for (Pair<HRegionInfo, ServerName> p : initialRegionToServers) {
+ List<RegionInfo> initialRegions = new ArrayList<>();
+ for (Pair<RegionInfo, ServerName> p : initialRegionToServers) {
initialRegions.add(p.getFirst());
}
- List<HRegionInfo> currentRegions = new ArrayList<>();
- for (Pair<HRegionInfo, ServerName> p : currentRegionToServers) {
+ List<RegionInfo> currentRegions = new ArrayList<>();
+ for (Pair<RegionInfo, ServerName> p : currentRegionToServers) {
currentRegions.add(p.getFirst());
}
assertTrue(initialRegions.contains(mergedRegions.getFirst())); //this is the first region
@@ -396,23 +396,23 @@ public class TestRegionMergeTransactionOnCluster {
mergedRegions.getSecond(), 1))); //replica of the merged region
}
- private PairOfSameType<HRegionInfo> mergeRegionsAndVerifyRegionNum(
+ private PairOfSameType<RegionInfo> mergeRegionsAndVerifyRegionNum(
HMaster master, TableName tablename,
int regionAnum, int regionBnum, int expectedRegionNum) throws Exception {
- PairOfSameType<HRegionInfo> mergedRegions =
+ PairOfSameType<RegionInfo> mergedRegions =
requestMergeRegion(master, tablename, regionAnum, regionBnum);
waitAndVerifyRegionNum(master, tablename, expectedRegionNum);
return mergedRegions;
}
- private PairOfSameType<HRegionInfo> requestMergeRegion(
+ private PairOfSameType<RegionInfo> requestMergeRegion(
HMaster master, TableName tablename,
int regionAnum, int regionBnum) throws Exception {
- List<Pair<HRegionInfo, ServerName>> tableRegions = MetaTableAccessor
+ List<Pair<RegionInfo, ServerName>> tableRegions = MetaTableAccessor
.getTableRegionsAndLocations(
TEST_UTIL.getConnection(), tablename);
- HRegionInfo regionA = tableRegions.get(regionAnum).getFirst();
- HRegionInfo regionB = tableRegions.get(regionBnum).getFirst();
+ RegionInfo regionA = tableRegions.get(regionAnum).getFirst();
+ RegionInfo regionB = tableRegions.get(regionBnum).getFirst();
ADMIN.mergeRegionsAsync(
regionA.getEncodedNameAsBytes(),
regionB.getEncodedNameAsBytes(), false);
@@ -421,8 +421,8 @@ public class TestRegionMergeTransactionOnCluster {
private void waitAndVerifyRegionNum(HMaster master, TableName tablename,
int expectedRegionNum) throws Exception {
- List<Pair<HRegionInfo, ServerName>> tableRegionsInMeta;
- List<HRegionInfo> tableRegionsInMaster;
+ List<Pair<RegionInfo, ServerName>> tableRegionsInMeta;
+ List<RegionInfo> tableRegionsInMaster;
long timeout = System.currentTimeMillis() + waitTime;
while (System.currentTimeMillis() < timeout) {
tableRegionsInMeta =
@@ -472,7 +472,7 @@ public class TestRegionMergeTransactionOnCluster {
// Sleep here is an ugly hack to allow region transitions to finish
long timeout = System.currentTimeMillis() + waitTime;
- List<Pair<HRegionInfo, ServerName>> tableRegions;
+ List<Pair<RegionInfo, ServerName>> tableRegions;
while (System.currentTimeMillis() < timeout) {
tableRegions = MetaTableAccessor.getTableRegionsAndLocations(
TEST_UTIL.getConnection(), tablename);
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerAbort.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerAbort.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerAbort.java
index 1e3db70..6ff9f75 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerAbort.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerAbort.java
@@ -18,6 +18,12 @@
package org.apache.hadoop.hbase.regionserver;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
@@ -25,13 +31,12 @@ import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
@@ -43,12 +48,12 @@ import org.apache.hadoop.hbase.coprocessor.RegionServerCoprocessor;
import org.apache.hadoop.hbase.coprocessor.RegionServerCoprocessorEnvironment;
import org.apache.hadoop.hbase.coprocessor.RegionServerObserver;
import org.apache.hadoop.hbase.master.HMaster;
-import org.apache.hadoop.hbase.wal.WALEdit;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.JVMClusterUtil;
import org.apache.hadoop.hbase.wal.WAL;
+import org.apache.hadoop.hbase.wal.WALEdit;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.junit.After;
@@ -206,13 +211,13 @@ public class TestRegionServerAbort {
*/
public static class ErrorThrowingHRegion extends HRegion {
public ErrorThrowingHRegion(Path tableDir, WAL wal, FileSystem fs, Configuration confParam,
- HRegionInfo regionInfo, TableDescriptor htd,
+ RegionInfo regionInfo, TableDescriptor htd,
RegionServerServices rsServices) {
super(tableDir, wal, fs, confParam, regionInfo, htd, rsServices);
}
public ErrorThrowingHRegion(HRegionFileSystem fs, WAL wal, Configuration confParam,
- HTableDescriptor htd, RegionServerServices rsServices) {
+ TableDescriptor htd, RegionServerServices rsServices) {
super(fs, wal, confParam, htd, rsServices);
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerRegionSpaceUseReport.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerRegionSpaceUseReport.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerRegionSpaceUseReport.java
index 7c16d32..44ab24e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerRegionSpaceUseReport.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerRegionSpaceUseReport.java
@@ -27,16 +27,18 @@ import static org.mockito.Mockito.mock;
import java.util.HashMap;
import java.util.Map;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportRequest;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.Test;
import org.junit.experimental.categories.Category;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportRequest;
+
/**
* Test class for isolated (non-cluster) tests surrounding the report
* of Region space use to the Master by RegionServers.
@@ -47,10 +49,20 @@ public class TestRegionServerRegionSpaceUseReport {
@Test
public void testConversion() {
TableName tn = TableName.valueOf("table1");
- HRegionInfo hri1 = new HRegionInfo(tn, Bytes.toBytes("a"), Bytes.toBytes("b"));
- HRegionInfo hri2 = new HRegionInfo(tn, Bytes.toBytes("b"), Bytes.toBytes("c"));
- HRegionInfo hri3 = new HRegionInfo(tn, Bytes.toBytes("c"), Bytes.toBytes("d"));
- Map<HRegionInfo,Long> sizes = new HashMap<>();
+
+ RegionInfo hri1 = RegionInfoBuilder.newBuilder(tn)
+ .setStartKey(Bytes.toBytes("a"))
+ .setEndKey(Bytes.toBytes("b"))
+ .build();
+ RegionInfo hri2 = RegionInfoBuilder.newBuilder(tn)
+ .setStartKey(Bytes.toBytes("b"))
+ .setEndKey(Bytes.toBytes("c"))
+ .build();
+ RegionInfo hri3 = RegionInfoBuilder.newBuilder(tn)
+ .setStartKey(Bytes.toBytes("c"))
+ .setEndKey(Bytes.toBytes("d"))
+ .build();
+ Map<RegionInfo,Long> sizes = new HashMap<>();
sizes.put(hri1, 1024L * 1024L);
sizes.put(hri2, 1024L * 1024L * 8L);
sizes.put(hri3, 1024L * 1024L * 32L);
@@ -58,13 +70,12 @@ public class TestRegionServerRegionSpaceUseReport {
// Call the real method to convert the map into a protobuf
HRegionServer rs = mock(HRegionServer.class);
doCallRealMethod().when(rs).buildRegionSpaceUseReportRequest(any(Map.class));
- doCallRealMethod().when(rs).convertRegionSize(any(HRegionInfo.class), anyLong());
+ doCallRealMethod().when(rs).convertRegionSize(any(RegionInfo.class), anyLong());
RegionSpaceUseReportRequest requests = rs.buildRegionSpaceUseReportRequest(sizes);
assertEquals(sizes.size(), requests.getSpaceUseCount());
for (RegionSpaceUse spaceUse : requests.getSpaceUseList()) {
- RegionInfo ri = spaceUse.getRegionInfo();
- HRegionInfo hri = HRegionInfo.convert(ri);
+ RegionInfo hri = ProtobufUtil.toRegionInfo(spaceUse.getRegionInfo());
Long expectedSize = sizes.remove(hri);
assertNotNull("Could not find size for HRI: " + hri, expectedSize);
assertEquals(expectedSize.longValue(), spaceUse.getRegionSize());
@@ -77,7 +88,7 @@ public class TestRegionServerRegionSpaceUseReport {
// Call the real method to convert the map into a protobuf
HRegionServer rs = mock(HRegionServer.class);
doCallRealMethod().when(rs).buildRegionSpaceUseReportRequest(any(Map.class));
- doCallRealMethod().when(rs).convertRegionSize(any(HRegionInfo.class), anyLong());
+ doCallRealMethod().when(rs).convertRegionSize(any(RegionInfo.class), anyLong());
rs.buildRegionSpaceUseReportRequest(null);
}
@@ -85,14 +96,17 @@ public class TestRegionServerRegionSpaceUseReport {
@Test(expected = NullPointerException.class)
public void testMalformedMap() {
TableName tn = TableName.valueOf("table1");
- HRegionInfo hri1 = new HRegionInfo(tn, Bytes.toBytes("a"), Bytes.toBytes("b"));
- Map<HRegionInfo,Long> sizes = new HashMap<>();
+ RegionInfo hri1 = RegionInfoBuilder.newBuilder(tn)
+ .setStartKey(Bytes.toBytes("a"))
+ .setEndKey(Bytes.toBytes("b"))
+ .build();
+ Map<RegionInfo,Long> sizes = new HashMap<>();
sizes.put(hri1, null);
// Call the real method to convert the map into a protobuf
HRegionServer rs = mock(HRegionServer.class);
doCallRealMethod().when(rs).buildRegionSpaceUseReportRequest(any(Map.class));
- doCallRealMethod().when(rs).convertRegionSize(any(HRegionInfo.class), anyLong());
+ doCallRealMethod().when(rs).convertRegionSize(any(RegionInfo.class), anyLong());
rs.buildRegionSpaceUseReportRequest(sizes);
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerHeartbeatMessages.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerHeartbeatMessages.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerHeartbeatMessages.java
index 00e6231..7907e13 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerHeartbeatMessages.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerHeartbeatMessages.java
@@ -21,9 +21,6 @@ import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
-import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController;
-import org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
-
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
@@ -40,12 +37,12 @@ import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.CoordinatedStateManager;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.HTestConst;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
@@ -54,8 +51,6 @@ import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.filter.Filter;
import org.apache.hadoop.hbase.filter.FilterBase;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanResponse;
import org.apache.hadoop.hbase.regionserver.HRegion.RegionScannerImpl;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.util.Bytes;
@@ -69,6 +64,11 @@ import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanResponse;
+
/**
* Here we test to make sure that scans return the expected Results when the server is sending the
* Client heartbeat messages. Heartbeat messages are essentially keep-alive messages (they prevent
@@ -446,12 +446,12 @@ public class TestScannerHeartbeatMessages {
private static volatile boolean sleepBetweenColumnFamilies = false;
public HeartbeatHRegion(Path tableDir, WAL wal, FileSystem fs, Configuration confParam,
- HRegionInfo regionInfo, TableDescriptor htd, RegionServerServices rsServices) {
+ RegionInfo regionInfo, TableDescriptor htd, RegionServerServices rsServices) {
super(tableDir, wal, fs, confParam, regionInfo, htd, rsServices);
}
public HeartbeatHRegion(HRegionFileSystem fs, WAL wal, Configuration confParam,
- HTableDescriptor htd, RegionServerServices rsServices) {
+ TableDescriptor htd, RegionServerServices rsServices) {
super(fs, wal, confParam, htd, rsServices);
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java
index 57f50b6..1a69be3 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java
@@ -47,7 +47,6 @@ import org.apache.hadoop.hbase.CoprocessorEnvironment;
import org.apache.hadoop.hbase.HBaseTestingUtility;
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.MasterNotRunningException;
import org.apache.hadoop.hbase.MetaTableAccessor;
@@ -62,6 +61,7 @@ import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
@@ -154,10 +154,10 @@ public class TestSplitTransactionOnCluster {
}
}
- private HRegionInfo getAndCheckSingleTableRegion(final List<HRegion> regions)
+ private RegionInfo getAndCheckSingleTableRegion(final List<HRegion> regions)
throws IOException, InterruptedException {
assertEquals(1, regions.size());
- HRegionInfo hri = regions.get(0).getRegionInfo();
+ RegionInfo hri = regions.get(0).getRegionInfo();
try {
cluster.getMaster().getAssignmentManager().waitForAssignment(hri, 600000);
} catch (NoSuchProcedureException e) {
@@ -184,7 +184,7 @@ public class TestSplitTransactionOnCluster {
// Create table then get the single region for our new table.
Table t = createTableAndWait(tableName, Bytes.toBytes("cf"));
final List<HRegion> regions = cluster.getRegions(tableName);
- final HRegionInfo hri = getAndCheckSingleTableRegion(regions);
+ final RegionInfo hri = getAndCheckSingleTableRegion(regions);
insertData(tableName, admin, t);
t.close();
@@ -299,7 +299,7 @@ public class TestSplitTransactionOnCluster {
// Create table then get the single region for our new table.
Table t = createTableAndWait(tableName, HConstants.CATALOG_FAMILY);
List<HRegion> regions = cluster.getRegions(tableName);
- HRegionInfo hri = getAndCheckSingleTableRegion(regions);
+ RegionInfo hri = getAndCheckSingleTableRegion(regions);
int tableRegionIndex = ensureTableRegionNotOnSameServerAsMeta(admin, hri);
@@ -355,7 +355,7 @@ public class TestSplitTransactionOnCluster {
// Create table then get the single region for our new table.
Table t = createTableAndWait(tableName, HConstants.CATALOG_FAMILY);
List<HRegion> regions = cluster.getRegions(tableName);
- HRegionInfo hri = getAndCheckSingleTableRegion(regions);
+ RegionInfo hri = getAndCheckSingleTableRegion(regions);
int tableRegionIndex = ensureTableRegionNotOnSameServerAsMeta(admin, hri);
@@ -376,7 +376,7 @@ public class TestSplitTransactionOnCluster {
List<HRegion> daughters = checkAndGetDaughters(tableName);
// Now split one of the daughters.
regionCount = ProtobufUtil.getOnlineRegions(server.getRSRpcServices()).size();
- HRegionInfo daughter = daughters.get(0).getRegionInfo();
+ RegionInfo daughter = daughters.get(0).getRegionInfo();
LOG.info("Daughter we are going to split: " + daughter);
// Compact first to ensure we have cleaned up references -- else the split
// will fail.
@@ -384,7 +384,7 @@ public class TestSplitTransactionOnCluster {
daughters = cluster.getRegions(tableName);
HRegion daughterRegion = null;
for (HRegion r: daughters) {
- if (r.getRegionInfo().equals(daughter)) {
+ if (RegionInfo.COMPARATOR.compare(r.getRegionInfo(), daughter) == 0) {
daughterRegion = r;
LOG.info("Found matching HRI: " + daughterRegion);
break;
@@ -451,11 +451,11 @@ public class TestSplitTransactionOnCluster {
admin.flush(userTableName);
}
admin.majorCompact(userTableName);
- List<HRegionInfo> regionsOfTable =
+ List<RegionInfo> regionsOfTable =
cluster.getMaster().getAssignmentManager().getRegionStates()
.getRegionsOfTable(userTableName);
assertEquals(1, regionsOfTable.size());
- HRegionInfo hRegionInfo = regionsOfTable.get(0);
+ RegionInfo hRegionInfo = regionsOfTable.get(0);
Put p = new Put("row6".getBytes());
p.addColumn("col".getBytes(), "ql".getBytes(), "val".getBytes());
table.put(p);
@@ -511,7 +511,7 @@ public class TestSplitTransactionOnCluster {
// Create table then get the single region for our new table.
Table t = createTableAndWait(tableName, HConstants.CATALOG_FAMILY);
List<HRegion> regions = cluster.getRegions(tableName);
- HRegionInfo hri = getAndCheckSingleTableRegion(regions);
+ RegionInfo hri = getAndCheckSingleTableRegion(regions);
int tableRegionIndex = ensureTableRegionNotOnSameServerAsMeta(admin, hri);
@@ -660,7 +660,7 @@ public class TestSplitTransactionOnCluster {
// Create table then get the single region for our new table.
createTableAndWait(tableName, HConstants.CATALOG_FAMILY);
List<HRegion> regions = cluster.getRegions(tableName);
- HRegionInfo hri = getAndCheckSingleTableRegion(regions);
+ RegionInfo hri = getAndCheckSingleTableRegion(regions);
ensureTableRegionNotOnSameServerAsMeta(admin, hri);
int regionServerIndex = cluster.getServerWith(regions.get(0).getRegionInfo()
.getRegionName());
@@ -809,7 +809,7 @@ public class TestSplitTransactionOnCluster {
return master;
}
- private void split(final HRegionInfo hri, final HRegionServer server, final int regionCount)
+ private void split(final RegionInfo hri, final HRegionServer server, final int regionCount)
throws IOException, InterruptedException {
this.admin.splitRegion(hri.getRegionName());
for (int i = 0; this.cluster.getRegions(hri.getTable()).size() <= regionCount && i < 60; i++) {
@@ -833,7 +833,7 @@ public class TestSplitTransactionOnCluster {
* @throws InterruptedException
*/
private int ensureTableRegionNotOnSameServerAsMeta(final Admin admin,
- final HRegionInfo hri)
+ final RegionInfo hri)
throws IOException, MasterNotRunningException,
ZooKeeperConnectionException, InterruptedException {
// Now make sure that the table region is not on same server as that hosting
@@ -901,8 +901,8 @@ public class TestSplitTransactionOnCluster {
private void printOutRegions(final HRegionServer hrs, final String prefix)
throws IOException {
- List<HRegionInfo> regions = ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices());
- for (HRegionInfo region: regions) {
+ List<RegionInfo> regions = ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices());
+ for (RegionInfo region: regions) {
LOG.info(prefix + region.getRegionNameAsString());
}
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestFSWAL.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestFSWAL.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestFSWAL.java
index bc66c33..d5e8c1c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestFSWAL.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestFSWAL.java
@@ -18,10 +18,10 @@
package org.apache.hadoop.hbase.regionserver.wal;
import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
-import static org.junit.Assert.assertNotEquals;
import java.io.IOException;
import java.util.ArrayList;
@@ -45,12 +45,13 @@ import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
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.KeyValue;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
import org.apache.hadoop.hbase.coprocessor.SampleRegionWALCoprocessor;
@@ -150,7 +151,7 @@ public abstract class AbstractTestFSWAL {
}
}
- protected void addEdits(WAL log, HRegionInfo hri, HTableDescriptor htd, int times,
+ protected void addEdits(WAL log, RegionInfo hri, HTableDescriptor htd, int times,
MultiVersionConcurrencyControl mvcc, NavigableMap<byte[], Integer> scopes)
throws IOException {
final byte[] row = Bytes.toBytes("row");
@@ -250,10 +251,14 @@ public abstract class AbstractTestFSWAL {
new HTableDescriptor(TableName.valueOf("t1")).addFamily(new HColumnDescriptor("row"));
HTableDescriptor t2 =
new HTableDescriptor(TableName.valueOf("t2")).addFamily(new HColumnDescriptor("row"));
- HRegionInfo hri1 =
- new HRegionInfo(t1.getTableName(), HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW);
- HRegionInfo hri2 =
- new HRegionInfo(t2.getTableName(), HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW);
+ RegionInfo hri1 = RegionInfoBuilder.newBuilder(t1.getTableName())
+ .setStartKey(HConstants.EMPTY_START_ROW)
+ .setEndKey(HConstants.EMPTY_END_ROW)
+ .build();
+ RegionInfo hri2 = RegionInfoBuilder.newBuilder(t2.getTableName())
+ .setStartKey(HConstants.EMPTY_START_ROW)
+ .setEndKey(HConstants.EMPTY_END_ROW)
+ .build();
// add edits and roll the wal
MultiVersionConcurrencyControl mvcc = new MultiVersionConcurrencyControl();
NavigableMap<byte[], Integer> scopes1 = new TreeMap<>(Bytes.BYTES_COMPARATOR);
@@ -354,7 +359,7 @@ public abstract class AbstractTestFSWAL {
public void testFlushSequenceIdIsGreaterThanAllEditsInHFile() throws IOException {
String testName = currentTest.getMethodName();
final TableName tableName = TableName.valueOf(testName);
- final HRegionInfo hri = new HRegionInfo(tableName);
+ final RegionInfo hri = RegionInfoBuilder.newBuilder(tableName).build();
final byte[] rowName = tableName.getName();
final HTableDescriptor htd = new HTableDescriptor(tableName);
htd.addFamily(new HColumnDescriptor("f"));
@@ -408,7 +413,7 @@ public abstract class AbstractTestFSWAL {
// Now make appends run slow.
goslow.set(true);
for (int i = 0; i < countPerFamily; i++) {
- final HRegionInfo info = region.getRegionInfo();
+ final RegionInfo info = region.getRegionInfo();
final WALKey logkey = new WALKey(info.getEncodedNameAsBytes(), tableName,
System.currentTimeMillis(), clusterIds, -1, -1, region.getMVCC(), scopes);
wal.append(info, logkey, edits, true);
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestLogRolling.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestLogRolling.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestLogRolling.java
index 4c3d2a5..8ff1823 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestLogRolling.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestLogRolling.java
@@ -30,7 +30,6 @@ import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.hbase.HBaseTestingUtility;
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.MiniHBaseCluster;
import org.apache.hadoop.hbase.ServerName;
@@ -38,6 +37,7 @@ import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
@@ -189,8 +189,7 @@ public abstract class AbstractTestLogRolling {
this.tableName = getName();
// TODO: Why does this write data take for ever?
startAndWriteData();
- HRegionInfo region = server.getRegions(TableName.valueOf(tableName)).get(0)
- .getRegionInfo();
+ RegionInfo region = server.getRegions(TableName.valueOf(tableName)).get(0).getRegionInfo();
final WAL log = server.getWAL(region);
LOG.info("after writing there are " + AbstractFSWALProvider.getNumRolledLogFiles(log) + " log files");
assertLogFileSize(log);
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncLogRolling.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncLogRolling.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncLogRolling.java
index bba08a7..73f70a7 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncLogRolling.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncLogRolling.java
@@ -21,9 +21,9 @@ import static org.junit.Assert.assertEquals;
import java.io.IOException;
-import org.apache.hadoop.hbase.CategoryBasedTimeout;
import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.CategoryBasedTimeout;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.testclassification.VerySlowRegionServerTests;
@@ -58,7 +58,7 @@ public class TestAsyncLogRolling extends AbstractTestLogRolling {
TEST_UTIL.waitUntilAllRegionsAssigned(table.getName());
doPut(table, 1);
server = TEST_UTIL.getRSForFirstRegionInTable(table.getName());
- HRegionInfo hri = server.getRegions(table.getName()).get(0).getRegionInfo();
+ RegionInfo hri = server.getRegions(table.getName()).get(0).getRegionInfo();
AsyncFSWAL wal = (AsyncFSWAL) server.getWAL(hri);
int numRolledLogFiles = AsyncFSWALProvider.getNumRolledLogFiles(wal);
DatanodeInfo[] dnInfos = wal.getPipeline();
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java
index de6fdb3..295ac2f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java
@@ -38,10 +38,10 @@ import org.apache.hadoop.hbase.CategoryBasedTimeout;
import org.apache.hadoop.hbase.Cell;
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.TableName;
import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
@@ -140,7 +140,7 @@ public class TestLogRolling extends AbstractTestLogRolling {
Table table = TEST_UTIL.getConnection().getTable(desc.getTableName());
server = TEST_UTIL.getRSForFirstRegionInTable(desc.getTableName());
- HRegionInfo region = server.getRegions(desc.getTableName()).get(0).getRegionInfo();
+ RegionInfo region = server.getRegions(desc.getTableName()).get(0).getRegionInfo();
final FSHLog log = (FSHLog) server.getWAL(region);
final AtomicBoolean lowReplicationHookCalled = new AtomicBoolean(false);
@@ -248,7 +248,7 @@ public class TestLogRolling extends AbstractTestLogRolling {
Table table = TEST_UTIL.getConnection().getTable(desc.getTableName());
server = TEST_UTIL.getRSForFirstRegionInTable(desc.getTableName());
- HRegionInfo region = server.getRegions(desc.getTableName()).get(0).getRegionInfo();
+ RegionInfo region = server.getRegions(desc.getTableName()).get(0).getRegionInfo();
final WAL log = server.getWAL(region);
final List<Path> paths = new ArrayList<>(1);
final List<Integer> preLogRolledCalled = new ArrayList<>();
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationEndpoint.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationEndpoint.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationEndpoint.java
index a0562bf..6396228 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationEndpoint.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationEndpoint.java
@@ -18,9 +18,12 @@
package org.apache.hadoop.hbase.replication;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
import java.io.IOException;
import java.util.ArrayList;
-import java.util.Arrays;
import java.util.List;
import java.util.UUID;
import java.util.concurrent.atomic.AtomicBoolean;
@@ -30,14 +33,19 @@ import java.util.concurrent.atomic.AtomicReference;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.Waiter;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.regionserver.HRegion;
-import org.apache.hadoop.hbase.replication.regionserver.*;
+import org.apache.hadoop.hbase.replication.regionserver.HBaseInterClusterReplicationEndpoint;
+import org.apache.hadoop.hbase.replication.regionserver.MetricsReplicationGlobalSourceSource;
+import org.apache.hadoop.hbase.replication.regionserver.MetricsReplicationSourceImpl;
+import org.apache.hadoop.hbase.replication.regionserver.MetricsReplicationSourceSource;
+import org.apache.hadoop.hbase.replication.regionserver.MetricsReplicationSourceSourceImpl;
+import org.apache.hadoop.hbase.replication.regionserver.MetricsSource;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.testclassification.ReplicationTests;
import org.apache.hadoop.hbase.util.Bytes;
@@ -53,10 +61,6 @@ import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
-
/**
* Tests ReplicationSource and ReplicationEndpoint interactions
*/
@@ -202,7 +206,7 @@ public class TestReplicationEndpoint extends TestReplicationBase {
// Make sure edits are spread across regions because we do region based batching
// before shipping edits.
for(HRegion region: regions) {
- HRegionInfo hri = region.getRegionInfo();
+ RegionInfo hri = region.getRegionInfo();
byte[] row = hri.getStartKey();
for (int i = 0; i < 100; i++) {
if (row.length > 0) {
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSerialReplication.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSerialReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSerialReplication.java
index b5b24d9..7a8e639 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSerialReplication.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSerialReplication.java
@@ -19,6 +19,8 @@
*/
package org.apache.hadoop.hbase.replication;
+import static org.junit.Assert.assertEquals;
+
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
@@ -32,13 +34,13 @@ import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
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.HTestConst;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
@@ -60,8 +62,6 @@ import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.junit.rules.TestName;
-import static org.junit.Assert.assertEquals;
-
@Category({ ReplicationTests.class, LargeTests.class })
public class TestSerialReplication {
private static final Log LOG = LogFactory.getLog(TestSerialReplication.class);
@@ -271,7 +271,7 @@ public class TestSerialReplication {
put.addColumn(famName, VALUE, VALUE);
t1.put(put);
}
- List<Pair<HRegionInfo, ServerName>> regions =
+ List<Pair<RegionInfo, ServerName>> regions =
MetaTableAccessor.getTableRegionsAndLocations(utility1.getConnection(), tableName);
utility1.getAdmin().mergeRegionsAsync(regions.get(0).getFirst().getRegionName(),
regions.get(1).getFirst().getRegionName(), true);
@@ -336,10 +336,10 @@ public class TestSerialReplication {
}
private void moveRegion(Table table, int index) throws IOException {
- List<Pair<HRegionInfo, ServerName>> regions =
+ List<Pair<RegionInfo, ServerName>> regions =
MetaTableAccessor.getTableRegionsAndLocations(utility1.getConnection(), table.getName());
assertEquals(1, regions.size());
- HRegionInfo regionInfo = regions.get(0).getFirst();
+ RegionInfo regionInfo = regions.get(0).getFirst();
ServerName name = utility1.getHBaseCluster().getRegionServer(index).getServerName();
utility1.getAdmin()
.move(regionInfo.getEncodedNameAsBytes(), Bytes.toBytes(name.getServerName()));
@@ -354,12 +354,12 @@ public class TestSerialReplication {
}
private void balanceTwoRegions(Table table) throws Exception {
- List<Pair<HRegionInfo, ServerName>> regions =
+ List<Pair<RegionInfo, ServerName>> regions =
MetaTableAccessor.getTableRegionsAndLocations(utility1.getConnection(), table.getName());
assertEquals(2, regions.size());
- HRegionInfo regionInfo1 = regions.get(0).getFirst();
+ RegionInfo regionInfo1 = regions.get(0).getFirst();
ServerName name1 = utility1.getHBaseCluster().getRegionServer(0).getServerName();
- HRegionInfo regionInfo2 = regions.get(1).getFirst();
+ RegionInfo regionInfo2 = regions.get(1).getFirst();
ServerName name2 = utility1.getHBaseCluster().getRegionServer(1).getServerName();
utility1.getAdmin()
.move(regionInfo1.getEncodedNameAsBytes(), Bytes.toBytes(name1.getServerName()));
@@ -377,7 +377,7 @@ public class TestSerialReplication {
private void waitTableHasRightNumberOfRegions(TableName tableName, int num) throws IOException {
while (true) {
- List<Pair<HRegionInfo, ServerName>> regions =
+ List<Pair<RegionInfo, ServerName>> regions =
MetaTableAccessor.getTableRegionsAndLocations(utility1.getConnection(), tableName);
if (regions.size() == num) {
return;
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/SnapshotTestingUtils.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/SnapshotTestingUtils.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/SnapshotTestingUtils.java
index 3f0b5b4..c0cb9e7 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/SnapshotTestingUtils.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/SnapshotTestingUtils.java
@@ -22,8 +22,8 @@ import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import java.io.IOException;
-import java.util.Collections;
import java.util.ArrayList;
+import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
@@ -33,23 +33,23 @@ import java.util.TreeSet;
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.FSDataOutputStream;
-import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.PathFilter;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotEnabledException;
-import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.BufferedMutator;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
import org.apache.hadoop.hbase.client.SnapshotDescription;
import org.apache.hadoop.hbase.client.SnapshotType;
@@ -64,18 +64,20 @@ import org.apache.hadoop.hbase.mob.MobUtils;
import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.regionserver.Region;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneResponse;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSTableDescriptors;
-import org.apache.hadoop.hbase.util.FSVisitor;
import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.FSVisitor;
import org.apache.hadoop.hbase.util.MD5Hash;
+import org.apache.yetus.audience.InterfaceAudience;
import org.junit.Assert;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest;
+
import com.google.protobuf.ServiceException;
/**
@@ -221,7 +223,7 @@ public final class SnapshotTestingUtils {
SnapshotReferenceUtil.visitRegionStoreFiles(regionManifest,
new SnapshotReferenceUtil.StoreFileVisitor() {
@Override
- public void storeFile(final HRegionInfo regionInfo, final String family,
+ public void storeFile(final RegionInfo regionInfo, final String family,
final SnapshotRegionManifest.StoreFile storeFile) throws IOException {
snapshotFamilies.add(Bytes.toBytes(family));
}
@@ -243,7 +245,7 @@ public final class SnapshotTestingUtils {
}
// check the region snapshot for all the regions
- List<HRegionInfo> regions = admin.getTableRegions(tableName);
+ List<RegionInfo> regions = admin.getRegions(tableName);
// remove the non-default regions
RegionReplicaUtil.removeNonDefaultRegions(regions);
boolean hasMob = regionManifests.containsKey(MobUtils.getMobRegionInfo(tableName)
@@ -255,7 +257,7 @@ public final class SnapshotTestingUtils {
// region manifest. we should exclude the parent regions.
int regionCountExclusiveSplitParent = 0;
for (SnapshotRegionManifest snapshotRegionManifest : regionManifests.values()) {
- HRegionInfo hri = HRegionInfo.convert(snapshotRegionManifest.getRegionInfo());
+ RegionInfo hri = ProtobufUtil.toRegionInfo(snapshotRegionManifest.getRegionInfo());
if (hri.isOffline() && (hri.isSplit() || hri.isSplitParent())) {
continue;
}
@@ -265,7 +267,7 @@ public final class SnapshotTestingUtils {
}
// Verify Regions (redundant check, see MasterSnapshotVerifier)
- for (HRegionInfo info : regions) {
+ for (RegionInfo info : regions) {
String regionName = info.getEncodedName();
assertTrue("Missing region name: '" + regionName + "'", regionManifests.containsKey(regionName));
}
@@ -449,7 +451,7 @@ public final class SnapshotTestingUtils {
SnapshotReferenceUtil.visitTableStoreFiles(conf, fs, snapshotDir, snapshotDesc,
new SnapshotReferenceUtil.StoreFileVisitor() {
@Override
- public void storeFile(final HRegionInfo regionInfo, final String family,
+ public void storeFile(final RegionInfo regionInfo, final String family,
final SnapshotRegionManifest.StoreFile storeFile) throws IOException {
String region = regionInfo.getEncodedName();
String hfile = storeFile.getName();
@@ -477,11 +479,11 @@ public final class SnapshotTestingUtils {
private final Path rootDir;
static class RegionData {
- public HRegionInfo hri;
+ public RegionInfo hri;
public Path tableDir;
public Path[] files;
- public RegionData(final Path tableDir, final HRegionInfo hri, final int nfiles) {
+ public RegionData(final Path tableDir, final RegionInfo hri, final int nfiles) {
this.tableDir = tableDir;
this.hri = hri;
this.files = new Path[nfiles];
@@ -711,7 +713,10 @@ public final class SnapshotTestingUtils {
byte[] endKey = Bytes.toBytes(1 + i * 2);
// First region, simple with one plain hfile.
- HRegionInfo hri = new HRegionInfo(htd.getTableName(), startKey, endKey);
+ RegionInfo hri = RegionInfoBuilder.newBuilder(htd.getTableName())
+ .setStartKey(startKey)
+ .setEndKey(endKey)
+ .build();
HRegionFileSystem rfs = HRegionFileSystem.createRegionOnFileSystem(conf, fs, tableDir, hri);
regions[i] = new RegionData(tableDir, hri, 3);
for (int j = 0; j < regions[i].files.length; ++j) {
@@ -723,7 +728,7 @@ public final class SnapshotTestingUtils {
// This region contains a reference to the hfile in the first region.
startKey = Bytes.toBytes(2 + i * 2);
endKey = Bytes.toBytes(3 + i * 2);
- hri = new HRegionInfo(htd.getTableName());
+ hri = RegionInfoBuilder.newBuilder(htd.getTableName()).build();
rfs = HRegionFileSystem.createRegionOnFileSystem(conf, fs, tableDir, hri);
regions[i+1] = new RegionData(tableDir, hri, regions[i].files.length);
for (int j = 0; j < regions[i].files.length; ++j) {
@@ -883,12 +888,12 @@ public final class SnapshotTestingUtils {
public static void verifyReplicasCameOnline(TableName tableName, Admin admin,
int regionReplication) throws IOException {
- List<HRegionInfo> regions = admin.getTableRegions(tableName);
- HashSet<HRegionInfo> set = new HashSet<>();
- for (HRegionInfo hri : regions) {
+ List<RegionInfo> regions = admin.getRegions(tableName);
+ HashSet<RegionInfo> set = new HashSet<>();
+ for (RegionInfo hri : regions) {
set.add(RegionReplicaUtil.getRegionInfoForDefaultReplica(hri));
for (int i = 0; i < regionReplication; i++) {
- HRegionInfo replica = RegionReplicaUtil.getRegionInfoForReplica(hri, i);
+ RegionInfo replica = RegionReplicaUtil.getRegionInfoForReplica(hri, i);
if (!regions.contains(replica)) {
Assert.fail(replica + " is not contained in the list of online regions");
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestLoadIncrementalHFilesSplitRecovery.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestLoadIncrementalHFilesSplitRecovery.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestLoadIncrementalHFilesSplitRecovery.java
index 414a6cb..86e5eb3 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestLoadIncrementalHFilesSplitRecovery.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestLoadIncrementalHFilesSplitRecovery.java
@@ -40,7 +40,6 @@ import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.ServerName;
@@ -52,6 +51,8 @@ import org.apache.hadoop.hbase.client.ClusterConnection;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
import org.apache.hadoop.hbase.client.RegionLocator;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
@@ -63,12 +64,6 @@ import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.regionserver.TestHRegionServerBulkLoad;
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.Multimap;
-import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController;
-import org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileRequest;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.testclassification.MiscTests;
import org.apache.hadoop.hbase.util.Bytes;
@@ -82,6 +77,13 @@ import org.junit.experimental.categories.Category;
import org.junit.rules.TestName;
import org.mockito.Mockito;
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.Multimap;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileRequest;
+
/**
* Test cases for the atomic load error handling of the bulk load functionality.
*/
@@ -201,7 +203,7 @@ public class TestLoadIncrementalHFilesSplitRecovery {
// need to call regions server to by synchronous but isn't visible.
HRegionServer hrs = util.getRSForFirstRegionInTable(table);
- for (HRegionInfo hri : ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices())) {
+ for (RegionInfo hri : ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices())) {
if (hri.getTable().equals(table)) {
util.getAdmin().splitRegionAsync(hri.getRegionName(), rowkey(ROWCOUNT / 2));
// ProtobufUtil.split(null, hrs.getRSRpcServices(), hri, rowkey(ROWCOUNT / 2));
@@ -212,7 +214,7 @@ public class TestLoadIncrementalHFilesSplitRecovery {
int regions;
do {
regions = 0;
- for (HRegionInfo hri : ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices())) {
+ for (RegionInfo hri : ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices())) {
if (hri.getTable().equals(table)) {
regions++;
}
@@ -359,7 +361,7 @@ public class TestLoadIncrementalHFilesSplitRecovery {
Mockito.when(c.getConfiguration()).thenReturn(conf);
Mockito.doNothing().when(c).close();
// Make it so we return a particular location when asked.
- final HRegionLocation loc = new HRegionLocation(HRegionInfo.FIRST_META_REGIONINFO,
+ final HRegionLocation loc = new HRegionLocation(RegionInfoBuilder.FIRST_META_REGIONINFO,
ServerName.valueOf("example.org", 1234, 0));
Mockito.when(
c.getRegionLocation((TableName) Mockito.any(), (byte[]) Mockito.any(), Mockito.anyBoolean()))
@@ -578,8 +580,8 @@ public class TestLoadIncrementalHFilesSplitRecovery {
dir = buildBulkFiles(tableName, 3);
// Mess it up by leaving a hole in the hbase:meta
- List<HRegionInfo> regionInfos = MetaTableAccessor.getTableRegions(connection, tableName);
- for (HRegionInfo regionInfo : regionInfos) {
+ List<RegionInfo> regionInfos = MetaTableAccessor.getTableRegions(connection, tableName);
+ for (RegionInfo regionInfo : regionInfos) {
if (Bytes.equals(regionInfo.getStartKey(), HConstants.EMPTY_BYTE_ARRAY)) {
MetaTableAccessor.deleteRegion(connection, regionInfo);
break;
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java
index d10e6e7..e499ff0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java
@@ -18,6 +18,9 @@
*/
package org.apache.hadoop.hbase.util;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
@@ -42,7 +45,6 @@ import org.apache.hadoop.hbase.ClusterStatus.Option;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MetaTableAccessor;
@@ -55,6 +57,7 @@ import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
import org.apache.hadoop.hbase.client.RegionLocator;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Table;
@@ -68,8 +71,6 @@ import org.apache.hadoop.hbase.master.assignment.RegionStates;
import org.apache.hadoop.hbase.mob.MobFileName;
import org.apache.hadoop.hbase.mob.MobUtils;
import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
import org.apache.hadoop.hbase.util.HBaseFsck.ErrorReporter;
import org.apache.hadoop.hbase.util.HBaseFsck.HbckInfo;
import org.apache.hadoop.hbase.util.HBaseFsck.TableInfo;
@@ -77,8 +78,8 @@ import org.apache.hadoop.hbase.util.hbck.HFileCorruptionChecker;
import org.apache.zookeeper.KeeperException;
import org.junit.rules.TestName;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
/**
* This is the base class for HBaseFsck's ability to detect reasons for inconsistent tables.
@@ -117,11 +118,14 @@ public class BaseTestHBaseFsck {
/**
* Create a new region in META.
*/
- protected HRegionInfo createRegion(final HTableDescriptor
+ protected RegionInfo createRegion(final HTableDescriptor
htd, byte[] startKey, byte[] endKey)
throws IOException {
Table meta = connection.getTable(TableName.META_TABLE_NAME, tableExecutorService);
- HRegionInfo hri = new HRegionInfo(htd.getTableName(), startKey, endKey);
+ RegionInfo hri = RegionInfoBuilder.newBuilder(htd.getTableName())
+ .setStartKey(startKey)
+ .setEndKey(endKey)
+ .build();
MetaTableAccessor.addRegionToMeta(meta, hri);
meta.close();
return hri;
@@ -142,7 +146,7 @@ public class BaseTestHBaseFsck {
* remove its state from the Master.
*/
protected void undeployRegion(Connection conn, ServerName sn,
- HRegionInfo hri) throws IOException, InterruptedException {
+ RegionInfo hri) throws IOException, InterruptedException {
try {
HBaseFsckRepair.closeRegionSilentlyAndWait(conn, sn, hri);
if (!hri.isMetaTable()) {
@@ -163,7 +167,7 @@ public class BaseTestHBaseFsck {
byte[] startKey, byte[] endKey, boolean unassign, boolean metaRow,
boolean hdfs) throws IOException, InterruptedException {
deleteRegion(conf, htd, startKey, endKey, unassign, metaRow, hdfs, false,
- HRegionInfo.DEFAULT_REPLICA_ID);
+ RegionInfo.DEFAULT_REPLICA_ID);
}
/**
@@ -187,7 +191,7 @@ public class BaseTestHBaseFsck {
}
for (HRegionLocation location : locations) {
- HRegionInfo hri = location.getRegionInfo();
+ RegionInfo hri = location.getRegionInfo();
ServerName hsa = location.getServerName();
if (Bytes.compareTo(hri.getStartKey(), startKey) == 0
&& Bytes.compareTo(hri.getEndKey(), endKey) == 0
@@ -340,9 +344,9 @@ public class BaseTestHBaseFsck {
AdminProtos.AdminService.BlockingInterface server = connection.getAdmin(hsi);
// list all online regions from this region server
- List<HRegionInfo> regions = ProtobufUtil.getOnlineRegions(server);
+ List<RegionInfo> regions = ProtobufUtil.getOnlineRegions(server);
List<String> regionNames = new ArrayList<>(regions.size());
- for (HRegionInfo hri : regions) {
+ for (RegionInfo hri : regions) {
regionNames.add(hri.getRegionNameAsString());
}
mm.put(hsi, regionNames);
@@ -353,7 +357,7 @@ public class BaseTestHBaseFsck {
/**
* Returns the HSI a region info is on.
*/
- ServerName findDeployedHSI(Map<ServerName, List<String>> mm, HRegionInfo hri) {
+ ServerName findDeployedHSI(Map<ServerName, List<String>> mm, RegionInfo hri) {
for (Map.Entry<ServerName,List <String>> e : mm.entrySet()) {
if (e.getValue().contains(hri.getRegionNameAsString())) {
return e.getKey();
@@ -562,7 +566,7 @@ public class BaseTestHBaseFsck {
HRegionLocation metaLocation = connection.getRegionLocator(TableName.META_TABLE_NAME)
.getRegionLocation(HConstants.EMPTY_START_ROW);
ServerName hsa = metaLocation.getServerName();
- HRegionInfo hri = metaLocation.getRegionInfo();
+ RegionInfo hri = metaLocation.getRegionInfo();
if (unassign) {
LOG.info("Undeploying meta region " + hri + " from server " + hsa);
try (Connection unmanagedConnection = ConnectionFactory.createConnection(conf)) {
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckOneRS.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckOneRS.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckOneRS.java
index e8148d7..7463da1 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckOneRS.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckOneRS.java
@@ -50,7 +50,6 @@ import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MetaTableAccessor;
@@ -58,14 +57,19 @@ import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.ClusterConnection;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
import org.apache.hadoop.hbase.client.RegionLocator;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
import org.apache.hadoop.hbase.client.replication.ReplicationAdmin;
import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
import org.apache.hadoop.hbase.io.hfile.TestHFile;
@@ -303,10 +307,10 @@ public class TestHBaseFsckOneRS extends BaseTestHBaseFsck {
// Mess it up by creating an overlap in the metadata
admin.disableTable(tableName);
deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("A"), Bytes.toBytes("B"), true,
- true, false, true, HRegionInfo.DEFAULT_REPLICA_ID);
+ true, false, true, RegionInfo.DEFAULT_REPLICA_ID);
admin.enableTable(tableName);
- HRegionInfo hriOverlap =
+ RegionInfo hriOverlap =
createRegion(tbl.getTableDescriptor(), Bytes.toBytes("A2"), Bytes.toBytes("B"));
TEST_UTIL.assignRegion(hriOverlap);
@@ -346,7 +350,7 @@ public class TestHBaseFsckOneRS extends BaseTestHBaseFsck {
assertEquals(ROWKEYS.length, countRows());
// Mess it up by creating an overlap in the metadata
- HRegionInfo hriOverlap =
+ RegionInfo hriOverlap =
createRegion(tbl.getTableDescriptor(), Bytes.toBytes("A2"), Bytes.toBytes("B2"));
TEST_UTIL.assignRegion(hriOverlap);
@@ -421,13 +425,14 @@ public class TestHBaseFsckOneRS extends BaseTestHBaseFsck {
byte[][] SPLIT_KEYS = new byte[][] { new byte[0], Bytes.toBytes("aaa"),
Bytes.toBytes("bbb"), Bytes.toBytes("ccc"), Bytes.toBytes("ddd") };
- HTableDescriptor htdDisabled = new HTableDescriptor(tableName);
- htdDisabled.addFamily(new HColumnDescriptor(FAM));
+ TableDescriptor htdDisabled = TableDescriptorBuilder.newBuilder(tableName)
+ .addColumnFamily(ColumnFamilyDescriptorBuilder.of(FAM))
+ .build();
// Write the .tableinfo
FSTableDescriptors fstd = new FSTableDescriptors(conf);
fstd.createTableDescriptor(htdDisabled);
- List<HRegionInfo> disabledRegions =
+ List<RegionInfo> disabledRegions =
TEST_UTIL.createMultiRegionsInMeta(conf, htdDisabled, SPLIT_KEYS);
// Let's just assign everything to first RS
@@ -439,7 +444,7 @@ public class TestHBaseFsckOneRS extends BaseTestHBaseFsck {
// Disable the table and close its regions
admin.disableTable(tableName);
- HRegionInfo region = disabledRegions.remove(0);
+ RegionInfo region = disabledRegions.remove(0);
byte[] regionName = region.getRegionName();
// The region should not be assigned currently
@@ -622,8 +627,8 @@ public class TestHBaseFsckOneRS extends BaseTestHBaseFsck {
try(RegionLocator rl = connection.getRegionLocator(tbl.getName())) {
// make sure data in regions, if in wal only there is no data loss
admin.flush(tableName);
- HRegionInfo region1 = rl.getRegionLocation(Bytes.toBytes("A")).getRegionInfo();
- HRegionInfo region2 = rl.getRegionLocation(Bytes.toBytes("B")).getRegionInfo();
+ RegionInfo region1 = rl.getRegionLocation(Bytes.toBytes("A")).getRegionInfo();
+ RegionInfo region2 = rl.getRegionLocation(Bytes.toBytes("B")).getRegionInfo();
int regionCountBeforeMerge = rl.getAllRegionLocations().size();
@@ -818,15 +823,18 @@ public class TestHBaseFsckOneRS extends BaseTestHBaseFsck {
// Create a new meta entry to fake it as a split parent.
meta = connection.getTable(TableName.META_TABLE_NAME, tableExecutorService);
- HRegionInfo hri = location.getRegionInfo();
-
- HRegionInfo a = new HRegionInfo(tbl.getName(),
- Bytes.toBytes("B"), Bytes.toBytes("BM"));
- HRegionInfo b = new HRegionInfo(tbl.getName(),
- Bytes.toBytes("BM"), Bytes.toBytes("C"));
-
- hri.setOffline(true);
- hri.setSplit(true);
+ RegionInfo a = RegionInfoBuilder.newBuilder(tbl.getName())
+ .setStartKey(Bytes.toBytes("B"))
+ .setEndKey(Bytes.toBytes("BM"))
+ .build();
+ RegionInfo b = RegionInfoBuilder.newBuilder(tbl.getName())
+ .setStartKey(Bytes.toBytes("BM"))
+ .setEndKey(Bytes.toBytes("C"))
+ .build();
+ RegionInfo hri = RegionInfoBuilder.newBuilder(location.getRegion())
+ .setOffline(true)
+ .setSplit(true)
+ .build();
MetaTableAccessor.addRegionToMeta(meta, hri, a, b);
meta.close();
@@ -897,7 +905,7 @@ public class TestHBaseFsckOneRS extends BaseTestHBaseFsck {
HRegionLocation location = rl.getRegionLocation(Bytes.toBytes("B"));
meta = connection.getTable(TableName.META_TABLE_NAME, tableExecutorService);
- HRegionInfo hri = location.getRegionInfo();
+ RegionInfo hri = location.getRegionInfo();
// do a regular split
byte[] regionName = location.getRegionInfo().getRegionName();
@@ -916,7 +924,7 @@ public class TestHBaseFsckOneRS extends BaseTestHBaseFsck {
Get get = new Get(hri.getRegionName());
Result result = meta.get(get);
assertNotNull(result);
- assertNotNull(MetaTableAccessor.getHRegionInfo(result));
+ assertNotNull(MetaTableAccessor.getRegionInfo(result));
assertEquals(ROWKEYS.length, countRows());
@@ -949,7 +957,7 @@ public class TestHBaseFsckOneRS extends BaseTestHBaseFsck {
try(RegionLocator rl = connection.getRegionLocator(tbl.getName())) {
HRegionLocation location = rl.getRegionLocation(Bytes.toBytes("B"));
- HRegionInfo hri = location.getRegionInfo();
+ RegionInfo hri = location.getRegionInfo();
// Disable CatalogJanitor to prevent it from cleaning up the parent region
// after split.
@@ -960,7 +968,7 @@ public class TestHBaseFsckOneRS extends BaseTestHBaseFsck {
admin.splitRegion(location.getRegionInfo().getRegionName(), Bytes.toBytes("BM"));
TestEndToEndSplitTransaction.blockUntilRegionSplit(conf, 60000, regionName, true);
- PairOfSameType<HRegionInfo> daughters = MetaTableAccessor.getDaughterRegions(
+ PairOfSameType<RegionInfo> daughters = MetaTableAccessor.getDaughterRegions(
meta.get(new Get(regionName)));
// Delete daughter regions from meta, but not hdfs, unassign it.
@@ -1002,7 +1010,7 @@ public class TestHBaseFsckOneRS extends BaseTestHBaseFsck {
Get get = new Get(hri.getRegionName());
Result result = meta.get(get);
assertNotNull(result);
- assertNotNull(MetaTableAccessor.getHRegionInfo(result));
+ assertNotNull(MetaTableAccessor.getRegionInfo(result));
assertEquals(ROWKEYS.length, countRows());
@@ -1116,7 +1124,7 @@ public class TestHBaseFsckOneRS extends BaseTestHBaseFsck {
// Mess it up by closing a region
deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("A"), Bytes.toBytes("B"), true,
- false, false, false, HRegionInfo.DEFAULT_REPLICA_ID);
+ false, false, false, RegionInfo.DEFAULT_REPLICA_ID);
// verify there is no other errors
HBaseFsck hbck = doFsck(conf, false);
@@ -1172,7 +1180,7 @@ public class TestHBaseFsckOneRS extends BaseTestHBaseFsck {
// Mess it up by deleting a region from the metadata
deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("A"),
- Bytes.toBytes("B"), false, true, false, false, HRegionInfo.DEFAULT_REPLICA_ID);
+ Bytes.toBytes("B"), false, true, false, false, RegionInfo.DEFAULT_REPLICA_ID);
// verify there is no other errors
HBaseFsck hbck = doFsck(conf, false);
@@ -1234,10 +1242,10 @@ public class TestHBaseFsckOneRS extends BaseTestHBaseFsck {
// Mess it up by creating an overlap in the metadata
admin.disableTable(tableName);
deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("A"), Bytes.toBytes("B"), true,
- true, false, true, HRegionInfo.DEFAULT_REPLICA_ID);
+ true, false, true, RegionInfo.DEFAULT_REPLICA_ID);
admin.enableTable(tableName);
- HRegionInfo hriOverlap =
+ RegionInfo hriOverlap =
createRegion(tbl.getTableDescriptor(), Bytes.toBytes("A2"), Bytes.toBytes("B"));
TEST_UTIL.assignRegion(hriOverlap);
@@ -1365,7 +1373,7 @@ public class TestHBaseFsckOneRS extends BaseTestHBaseFsck {
assertEquals(ROWKEYS.length, countRows());
// Now let's mess it up, by adding a region with a duplicate startkey
- HRegionInfo hriDupe =
+ RegionInfo hriDupe =
createRegion(tbl.getTableDescriptor(), Bytes.toBytes("B"), Bytes.toBytes("B"));
TEST_UTIL.assignRegion(hriDupe);
@@ -1410,7 +1418,7 @@ public class TestHBaseFsckOneRS extends BaseTestHBaseFsck {
@Override
public boolean visit(Result rowResult) throws IOException {
- HRegionInfo hri = MetaTableAccessor.getHRegionInfo(rowResult);
+ RegionInfo hri = MetaTableAccessor.getRegionInfo(rowResult);
if (hri != null && !hri.getTable().isSystemTable()) {
Delete delete = new Delete(rowResult.getRow());
delete.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
@@ -1535,7 +1543,7 @@ public class TestHBaseFsckOneRS extends BaseTestHBaseFsck {
// Mess it up by leaving a hole in the meta data
admin.disableTable(tableName);
deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("B"), Bytes.toBytes("C"), true,
- true, false, true, HRegionInfo.DEFAULT_REPLICA_ID);
+ true, false, true, RegionInfo.DEFAULT_REPLICA_ID);
admin.enableTable(tableName);
HBaseFsck hbck = doFsck(conf, false);
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/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 1971049..341bd34 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
@@ -17,23 +17,42 @@
*/
package org.apache.hadoop.hbase.util;
+import static org.apache.hadoop.hbase.util.hbck.HbckTestingUtil.assertErrors;
+import static org.apache.hadoop.hbase.util.hbck.HbckTestingUtil.assertNoErrors;
+import static org.apache.hadoop.hbase.util.hbck.HbckTestingUtil.doFsck;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.EnumSet;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.Set;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.hbase.ClusterStatus.Option;
import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.ClusterStatus.Option;
import org.apache.hadoop.hbase.client.ClusterConnection;
import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
import org.apache.hadoop.hbase.client.Table;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
-import org.apache.hadoop.hbase.HRegionLocation;
-
import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
-import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.testclassification.MiscTests;
import org.junit.AfterClass;
@@ -45,22 +64,6 @@ import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.junit.rules.TestName;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.EnumSet;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.NavigableMap;
-import java.util.Set;
-import java.util.concurrent.ScheduledThreadPoolExecutor;
-import java.util.concurrent.SynchronousQueue;
-import java.util.concurrent.ThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
-
-import static org.apache.hadoop.hbase.util.hbck.HbckTestingUtil.*;
-import static org.junit.Assert.*;
-
@Ignore
@Category({MiscTests.class, LargeTests.class})
public class TestHBaseFsckReplicas extends BaseTestHBaseFsck {
@@ -189,15 +192,15 @@ public class TestHBaseFsckReplicas extends BaseTestHBaseFsck {
// for the master to treat the request for assignment as valid; the master
// checks the region is valid either from its memory or meta)
Table meta = connection.getTable(TableName.META_TABLE_NAME, tableExecutorService);
- List<HRegionInfo> regions = admin.getTableRegions(tableName);
+ List<RegionInfo> regions = admin.getRegions(tableName);
byte[] startKey = Bytes.toBytes("B");
byte[] endKey = Bytes.toBytes("C");
byte[] metaKey = null;
- HRegionInfo newHri = null;
- for (HRegionInfo h : regions) {
+ RegionInfo newHri = null;
+ for (RegionInfo h : regions) {
if (Bytes.compareTo(h.getStartKey(), startKey) == 0 &&
Bytes.compareTo(h.getEndKey(), endKey) == 0 &&
- h.getReplicaId() == HRegionInfo.DEFAULT_REPLICA_ID) {
+ h.getReplicaId() == RegionInfo.DEFAULT_REPLICA_ID) {
metaKey = h.getRegionName();
//create a hri with replicaId as 2 (since we already have replicas with replicaid 0 and 1)
newHri = RegionReplicaUtil.getRegionInfoForReplica(h, 2);
@@ -241,15 +244,15 @@ public class TestHBaseFsckReplicas extends BaseTestHBaseFsck {
public void testNotInHdfsWithReplicas() throws Exception {
final TableName tableName = TableName.valueOf(name.getMethodName());
try {
- HRegionInfo[] oldHris = new HRegionInfo[2];
+ RegionInfo[] oldHris = new RegionInfo[2];
setupTableWithRegionReplica(tableName, 2);
assertEquals(ROWKEYS.length, countRows());
- NavigableMap<HRegionInfo, ServerName> map =
+ NavigableMap<RegionInfo, ServerName> map =
MetaTableAccessor.allTableRegions(TEST_UTIL.getConnection(),
tbl.getName());
int i = 0;
// store the HRIs of the regions we will mess up
- for (Map.Entry<HRegionInfo, ServerName> m : map.entrySet()) {
+ for (Map.Entry<RegionInfo, ServerName> m : map.entrySet()) {
if (m.getKey().getStartKey().length > 0 &&
m.getKey().getStartKey()[0] == Bytes.toBytes("B")[0]) {
LOG.debug("Initially server hosting " + m.getKey() + " is " + m.getValue());
@@ -276,11 +279,11 @@ public class TestHBaseFsckReplicas extends BaseTestHBaseFsck {
// the following code checks whether the old primary/secondary has
// been unassigned and the new primary/secondary has been assigned
i = 0;
- HRegionInfo[] newHris = new HRegionInfo[2];
+ RegionInfo[] newHris = new RegionInfo[2];
// get all table's regions from meta
map = MetaTableAccessor.allTableRegions(TEST_UTIL.getConnection(), tbl.getName());
// get the HRIs of the new regions (hbck created new regions for fixing the hdfs mess-up)
- for (Map.Entry<HRegionInfo, ServerName> m : map.entrySet()) {
+ for (Map.Entry<RegionInfo, ServerName> m : map.entrySet()) {
if (m.getKey().getStartKey().length > 0 &&
m.getKey().getStartKey()[0] == Bytes.toBytes("B")[0]) {
newHris[i++] = m.getKey();
@@ -289,9 +292,9 @@ public class TestHBaseFsckReplicas extends BaseTestHBaseFsck {
// get all the online regions in the regionservers
Collection<ServerName> servers =
admin.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS)).getServers();
- Set<HRegionInfo> onlineRegions = new HashSet<>();
+ Set<RegionInfo> onlineRegions = new HashSet<>();
for (ServerName s : servers) {
- List<HRegionInfo> list = admin.getOnlineRegions(s);
+ List<RegionInfo> list = admin.getRegions(s);
onlineRegions.addAll(list);
}
// the new HRIs must be a subset of the online regions
@@ -325,7 +328,7 @@ public class TestHBaseFsckReplicas extends BaseTestHBaseFsck {
admin.enableCatalogJanitor(false);
meta = connection.getTable(TableName.META_TABLE_NAME, tableExecutorService);
HRegionLocation loc = this.connection.getRegionLocation(table, SPLITS[0], false);
- HRegionInfo hriParent = loc.getRegionInfo();
+ RegionInfo hriParent = loc.getRegionInfo();
// Split Region A just before B
this.connection.getAdmin().split(table, Bytes.toBytes("A@"));
[09/19] hbase git commit: HBASE-18839 Apply RegionInfo to code base
Posted by ch...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerServices.java
index 1d31d5a..b21d55a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerServices.java
@@ -23,21 +23,21 @@ import java.util.List;
import java.util.Map;
import java.util.concurrent.ConcurrentMap;
-import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.hbase.Abortable;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.Server;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.locking.EntityLock;
import org.apache.hadoop.hbase.executor.ExecutorService;
import org.apache.hadoop.hbase.ipc.RpcServerInterface;
import org.apache.hadoop.hbase.quotas.RegionServerRpcQuotaManager;
import org.apache.hadoop.hbase.quotas.RegionServerSpaceQuotaManager;
import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
import org.apache.hadoop.hbase.wal.WAL;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.zookeeper.KeeperException;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
+
import com.google.protobuf.Service;
/**
@@ -49,7 +49,7 @@ public interface RegionServerServices
/** @return the WAL for a particular region. Pass null for getting the
* default (common) WAL */
- WAL getWAL(HRegionInfo regionInfo) throws IOException;
+ WAL getWAL(RegionInfo regionInfo) throws IOException;
/** @return the List of WALs that are used by this server
* Doesn't include the meta WAL
@@ -127,11 +127,11 @@ public interface RegionServerServices
private final TransitionCode code;
private final long openSeqNum;
private final long masterSystemTime;
- private final HRegionInfo[] hris;
+ private final RegionInfo[] hris;
@InterfaceAudience.Private
public RegionStateTransitionContext(TransitionCode code, long openSeqNum, long masterSystemTime,
- HRegionInfo... hris) {
+ RegionInfo... hris) {
this.code = code;
this.openSeqNum = openSeqNum;
this.masterSystemTime = masterSystemTime;
@@ -146,7 +146,7 @@ public interface RegionServerServices
public long getMasterSystemTime() {
return masterSystemTime;
}
- public HRegionInfo[] getHris() {
+ public RegionInfo[] getHris() {
return hris;
}
}
@@ -161,14 +161,14 @@ public interface RegionServerServices
* @deprecated use {@link #reportRegionStateTransition(RegionStateTransitionContext)}
*/
@Deprecated
- boolean reportRegionStateTransition(TransitionCode code, long openSeqNum, HRegionInfo... hris);
+ boolean reportRegionStateTransition(TransitionCode code, long openSeqNum, RegionInfo... hris);
/**
* Notify master that a handler requests to change a region state
* @deprecated use {@link #reportRegionStateTransition(RegionStateTransitionContext)}
*/
@Deprecated
- boolean reportRegionStateTransition(TransitionCode code, HRegionInfo... hris);
+ boolean reportRegionStateTransition(TransitionCode code, RegionInfo... hris);
/**
* Returns a reference to the region server's RPC server
@@ -244,7 +244,7 @@ public interface RegionServerServices
/**
* Master based locks on namespaces/tables/regions.
*/
- EntityLock regionLock(List<HRegionInfo> regionInfos, String description,
+ EntityLock regionLock(List<RegionInfo> regionInfos, String description,
Abortable abort) throws IOException;
/**
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServicesForStores.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServicesForStores.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServicesForStores.java
index 9b4a32a..aea92f8 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServicesForStores.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServicesForStores.java
@@ -23,9 +23,9 @@ import java.util.concurrent.ThreadFactory;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.wal.WAL;
+import org.apache.yetus.audience.InterfaceAudience;
/**
* Services a Store needs from a Region.
@@ -67,7 +67,7 @@ public class RegionServicesForStores {
region.addAndGetMemstoreSize(size);
}
- public HRegionInfo getRegionInfo() {
+ public RegionInfo getRegionInfo() {
return region.getRegionInfo();
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionSplitPolicy.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionSplitPolicy.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionSplitPolicy.java
index 71b7b9b..3155379 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionSplitPolicy.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionSplitPolicy.java
@@ -131,7 +131,7 @@ public abstract class RegionSplitPolicy extends Configured {
}
/**
- * In {@link HRegionFileSystem#splitStoreFile(org.apache.hadoop.hbase.HRegionInfo, String,
+ * In {@link HRegionFileSystem#splitStoreFile(org.apache.hadoop.hbase.client.RegionInfo, String,
* HStoreFile, byte[], boolean, RegionSplitPolicy)} we are not creating the split reference
* if split row not lies in the StoreFile range. But in some use cases we may need to create
* the split reference even when the split row not lies in the range. This method can be used
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitRequest.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitRequest.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitRequest.java
index c1c3f58..0227f5b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitRequest.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitRequest.java
@@ -22,14 +22,15 @@ import java.security.PrivilegedAction;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.TableName;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
import org.apache.hadoop.hbase.security.User;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
/**
* Handles processing region splits. Put in a queue, owned by HRegionServer.
@@ -37,7 +38,7 @@ import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
@InterfaceAudience.Private
class SplitRequest implements Runnable {
private static final Log LOG = LogFactory.getLog(SplitRequest.class);
- private final HRegionInfo parent;
+ private final RegionInfo parent;
private final byte[] midKey;
private final HRegionServer server;
private final User user;
@@ -72,8 +73,14 @@ class SplitRequest implements Runnable {
private void requestRegionSplit() {
final TableName table = parent.getTable();
- final HRegionInfo hri_a = new HRegionInfo(table, parent.getStartKey(), midKey);
- final HRegionInfo hri_b = new HRegionInfo(table, midKey, parent.getEndKey());
+ final RegionInfo hri_a = RegionInfoBuilder.newBuilder(table)
+ .setStartKey(parent.getStartKey())
+ .setEndKey(midKey)
+ .build();
+ final RegionInfo hri_b = RegionInfoBuilder.newBuilder(table)
+ .setStartKey(midKey)
+ .setEndKey(parent.getEndKey())
+ .build();
// Send the split request to the master. the master will do the validation on the split-key.
// The parent region will be unassigned and the two new regions will be assigned.
// hri_a and hri_b objects may not reflect the regions that will be created, those objects
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/CloseMetaHandler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/CloseMetaHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/CloseMetaHandler.java
index cc3663e..38097ba 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/CloseMetaHandler.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/CloseMetaHandler.java
@@ -18,11 +18,11 @@
*/
package org.apache.hadoop.hbase.regionserver.handler;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.Server;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.executor.EventType;
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
+import org.apache.yetus.audience.InterfaceAudience;
/**
* Handles closing of the meta region on a region server.
@@ -33,7 +33,7 @@ public class CloseMetaHandler extends CloseRegionHandler {
// Called when regionserver determines its to go down; not master orchestrated
public CloseMetaHandler(final Server server,
final RegionServerServices rsServices,
- final HRegionInfo regionInfo,
+ final RegionInfo regionInfo,
final boolean abort) {
super(server, rsServices, regionInfo, abort, EventType.M_RS_CLOSE_META, null);
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/CloseRegionHandler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/CloseRegionHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/CloseRegionHandler.java
index 5f3e472..8c5ba9f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/CloseRegionHandler.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/CloseRegionHandler.java
@@ -22,15 +22,16 @@ import java.io.IOException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.executor.EventHandler;
import org.apache.hadoop.hbase.executor.EventType;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
/**
* Handles closing of a region on a region server.
@@ -45,7 +46,7 @@ public class CloseRegionHandler extends EventHandler {
private static final Log LOG = LogFactory.getLog(CloseRegionHandler.class);
private final RegionServerServices rsServices;
- private final HRegionInfo regionInfo;
+ private final RegionInfo regionInfo;
// If true, the hosting server is aborting. Region close process is different
// when we are aborting.
@@ -62,14 +63,14 @@ public class CloseRegionHandler extends EventHandler {
*/
public CloseRegionHandler(final Server server,
final RegionServerServices rsServices,
- final HRegionInfo regionInfo, final boolean abort,
+ final RegionInfo regionInfo, final boolean abort,
ServerName destination) {
this(server, rsServices, regionInfo, abort,
EventType.M_RS_CLOSE_REGION, destination);
}
protected CloseRegionHandler(final Server server,
- final RegionServerServices rsServices, HRegionInfo regionInfo,
+ final RegionServerServices rsServices, RegionInfo regionInfo,
boolean abort, EventType eventType, ServerName destination) {
super(server, eventType);
this.server = server;
@@ -79,7 +80,7 @@ public class CloseRegionHandler extends EventHandler {
this.destination = destination;
}
- public HRegionInfo getRegionInfo() {
+ public RegionInfo getRegionInfo() {
return regionInfo;
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenMetaHandler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenMetaHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenMetaHandler.java
index 018ceef..ca5f9e1 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenMetaHandler.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenMetaHandler.java
@@ -18,12 +18,12 @@
*/
package org.apache.hadoop.hbase.regionserver.handler;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.Server;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.executor.EventType;
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
+import org.apache.yetus.audience.InterfaceAudience;
/**
* Handles opening of a meta region on a region server.
@@ -33,7 +33,7 @@ import org.apache.hadoop.hbase.regionserver.RegionServerServices;
@InterfaceAudience.Private
public class OpenMetaHandler extends OpenRegionHandler {
public OpenMetaHandler(final Server server,
- final RegionServerServices rsServices, HRegionInfo regionInfo,
+ final RegionServerServices rsServices, RegionInfo regionInfo,
final TableDescriptor htd, long masterSystemTime) {
super(server, rsServices, regionInfo, htd, masterSystemTime, EventType.M_RS_OPEN_META);
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenPriorityRegionHandler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenPriorityRegionHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenPriorityRegionHandler.java
index 279a561..1861a2b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenPriorityRegionHandler.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenPriorityRegionHandler.java
@@ -18,12 +18,12 @@
package org.apache.hadoop.hbase.regionserver.handler;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.Server;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.executor.EventType;
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
+import org.apache.yetus.audience.InterfaceAudience;
/**
* Handles opening of a high priority region on a region server.
@@ -33,7 +33,7 @@ import org.apache.hadoop.hbase.regionserver.RegionServerServices;
@InterfaceAudience.Private
public class OpenPriorityRegionHandler extends OpenRegionHandler {
public OpenPriorityRegionHandler(Server server, RegionServerServices rsServices,
- HRegionInfo regionInfo, TableDescriptor htd, long masterSystemTime) {
+ RegionInfo regionInfo, TableDescriptor htd, long masterSystemTime) {
super(server, rsServices, regionInfo, htd, masterSystemTime,
EventType.M_RS_OPEN_PRIORITY_REGION);
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRegionHandler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRegionHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRegionHandler.java
index 4729c22..147317c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRegionHandler.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRegionHandler.java
@@ -23,19 +23,20 @@ import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.Server;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.executor.EventHandler;
import org.apache.hadoop.hbase.executor.EventType;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.regionserver.RegionServerAccounting;
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
import org.apache.hadoop.hbase.regionserver.RegionServerServices.PostOpenDeployContext;
import org.apache.hadoop.hbase.util.CancelableProgressable;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
/**
* Handles opening of a region on a region server.
* <p>
@@ -47,18 +48,18 @@ public class OpenRegionHandler extends EventHandler {
protected final RegionServerServices rsServices;
- private final HRegionInfo regionInfo;
+ private final RegionInfo regionInfo;
private final TableDescriptor htd;
private final long masterSystemTime;
public OpenRegionHandler(final Server server,
- final RegionServerServices rsServices, HRegionInfo regionInfo,
+ final RegionServerServices rsServices, RegionInfo regionInfo,
TableDescriptor htd, long masterSystemTime) {
this(server, rsServices, regionInfo, htd, masterSystemTime, EventType.M_RS_OPEN_REGION);
}
protected OpenRegionHandler(final Server server,
- final RegionServerServices rsServices, final HRegionInfo regionInfo,
+ final RegionServerServices rsServices, final RegionInfo regionInfo,
final TableDescriptor htd, long masterSystemTime, EventType eventType) {
super(server, eventType);
this.rsServices = rsServices;
@@ -67,7 +68,7 @@ public class OpenRegionHandler extends EventHandler {
this.masterSystemTime = masterSystemTime;
}
- public HRegionInfo getRegionInfo() {
+ public RegionInfo getRegionInfo() {
return regionInfo;
}
@@ -319,7 +320,7 @@ public class OpenRegionHandler extends EventHandler {
}
private static boolean isRegionStillOpening(
- HRegionInfo regionInfo, RegionServerServices rsServices) {
+ RegionInfo regionInfo, RegionServerServices rsServices) {
byte[] encodedName = regionInfo.getEncodedNameAsBytes();
Boolean action = rsServices.getRegionsInTransitionInRS().get(encodedName);
return Boolean.TRUE.equals(action); // true means opening for RIT
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java
index 8157108..3868ba7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java
@@ -21,8 +21,6 @@ import static org.apache.hadoop.hbase.shaded.com.google.common.base.Precondition
import static org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions.checkNotNull;
import static org.apache.hadoop.hbase.wal.AbstractFSWALProvider.WAL_FILE_NAME_DELIMITER;
-import com.lmax.disruptor.RingBuffer;
-
import java.io.IOException;
import java.io.InterruptedIOException;
import java.lang.management.MemoryType;
@@ -57,12 +55,10 @@ import org.apache.hadoop.fs.PathFilter;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.exceptions.TimeoutIOException;
import org.apache.hadoop.hbase.io.util.MemorySizeUtil;
import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl;
-import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.CollectionUtils;
import org.apache.hadoop.hbase.util.DrainBarrier;
@@ -80,6 +76,11 @@ import org.apache.htrace.NullScope;
import org.apache.htrace.Span;
import org.apache.htrace.Trace;
import org.apache.htrace.TraceScope;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
+
+import com.lmax.disruptor.RingBuffer;
/**
* Implementation of {@link WAL} to go against {@link FileSystem}; i.e. keep WALs in HDFS. Only one
@@ -908,7 +909,7 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
}
// Coprocessor hook.
- if (!coprocessorHost.preWALWrite(entry.getHRegionInfo(), entry.getKey(), entry.getEdit())) {
+ if (!coprocessorHost.preWALWrite(entry.getRegionInfo(), entry.getKey(), entry.getEdit())) {
if (entry.getEdit().isReplay()) {
// Set replication scope null so that this won't be replicated
entry.getKey().serializeReplicationScope(false);
@@ -924,7 +925,7 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
highestUnsyncedTxid = entry.getTxid();
sequenceIdAccounting.update(encodedRegionName, entry.getFamilyNames(), regionSequenceId,
entry.isInMemstore());
- coprocessorHost.postWALWrite(entry.getHRegionInfo(), entry.getKey(), entry.getEdit());
+ coprocessorHost.postWALWrite(entry.getRegionInfo(), entry.getKey(), entry.getEdit());
// Update metrics.
postAppend(entry, EnvironmentEdgeManager.currentTime() - start);
numEntries.incrementAndGet();
@@ -958,7 +959,7 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
}
}
- protected long stampSequenceIdAndPublishToRingBuffer(HRegionInfo hri, WALKey key, WALEdit edits,
+ protected long stampSequenceIdAndPublishToRingBuffer(RegionInfo hri, WALKey key, WALEdit edits,
boolean inMemstore, RingBuffer<RingBufferTruck> ringBuffer)
throws IOException {
if (this.closed) {
@@ -1021,7 +1022,7 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
* this append; otherwise, you will just have to wait on the WriteEntry to get filled in.
*/
@Override
- public abstract long append(HRegionInfo info, WALKey key, WALEdit edits, boolean inMemstore)
+ public abstract long append(RegionInfo info, WALKey key, WALEdit edits, boolean inMemstore)
throws IOException;
protected abstract void doAppend(W writer, FSWALEntry entry) throws IOException;
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java
index 42183ec..cff3f70 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java
@@ -19,15 +19,6 @@ package org.apache.hadoop.hbase.regionserver.wal;
import static org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutputHelper.shouldRetryCreate;
-import org.apache.hadoop.hbase.shaded.com.google.common.util.concurrent.ThreadFactoryBuilder;
-import com.lmax.disruptor.RingBuffer;
-import com.lmax.disruptor.Sequence;
-import com.lmax.disruptor.Sequencer;
-
-import org.apache.hadoop.hbase.shaded.io.netty.channel.Channel;
-import org.apache.hadoop.hbase.shaded.io.netty.channel.EventLoop;
-import org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.SingleThreadEventExecutor;
-
import java.io.IOException;
import java.io.InterruptedIOException;
import java.lang.reflect.Field;
@@ -53,9 +44,8 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.ConnectionUtils;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.io.asyncfs.AsyncFSOutput;
import org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutputHelper.NameNodeException;
import org.apache.hadoop.hbase.wal.AsyncFSWALProvider;
@@ -68,6 +58,16 @@ import org.apache.htrace.NullScope;
import org.apache.htrace.Span;
import org.apache.htrace.Trace;
import org.apache.htrace.TraceScope;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hadoop.hbase.shaded.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.Channel;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.EventLoop;
+import org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.SingleThreadEventExecutor;
+
+import com.lmax.disruptor.RingBuffer;
+import com.lmax.disruptor.Sequence;
+import com.lmax.disruptor.Sequencer;
/**
* An asynchronous implementation of FSWAL.
@@ -547,7 +547,7 @@ public class AsyncFSWAL extends AbstractFSWAL<AsyncWriter> {
}
@Override
- public long append(HRegionInfo hri, WALKey key, WALEdit edits, boolean inMemstore)
+ public long append(RegionInfo hri, WALKey key, WALEdit edits, boolean inMemstore)
throws IOException {
long txid =
stampSequenceIdAndPublishToRingBuffer(hri, key, edits, inMemstore, waitingConsumePayloads);
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/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 7e0fc37..649e450 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
@@ -17,14 +17,6 @@
*/
package org.apache.hadoop.hbase.regionserver.wal;
-import com.lmax.disruptor.BlockingWaitStrategy;
-import com.lmax.disruptor.EventHandler;
-import com.lmax.disruptor.ExceptionHandler;
-import com.lmax.disruptor.LifecycleAware;
-import com.lmax.disruptor.TimeoutException;
-import com.lmax.disruptor.dsl.Disruptor;
-import com.lmax.disruptor.dsl.ProducerType;
-
import java.io.FileNotFoundException;
import java.io.IOException;
import java.io.OutputStream;
@@ -44,7 +36,7 @@ import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.ClassSize;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
@@ -69,6 +61,14 @@ import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
+import com.lmax.disruptor.BlockingWaitStrategy;
+import com.lmax.disruptor.EventHandler;
+import com.lmax.disruptor.ExceptionHandler;
+import com.lmax.disruptor.LifecycleAware;
+import com.lmax.disruptor.TimeoutException;
+import com.lmax.disruptor.dsl.Disruptor;
+import com.lmax.disruptor.dsl.ProducerType;
+
/**
* The default implementation of FSWAL.
*/
@@ -442,7 +442,7 @@ public class FSHLog extends AbstractFSWAL<Writer> {
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "NP_NULL_ON_SOME_PATH_EXCEPTION",
justification = "Will never be null")
@Override
- public long append(final HRegionInfo hri, final WALKey key, final WALEdit edits,
+ public long append(final RegionInfo hri, final WALKey key, final WALEdit edits,
final boolean inMemstore) throws IOException {
return stampSequenceIdAndPublishToRingBuffer(hri, key, edits, inMemstore,
disruptor.getRingBuffer());
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSWALEntry.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSWALEntry.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSWALEntry.java
index 7c46976..0c83374 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSWALEntry.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSWALEntry.java
@@ -17,21 +17,18 @@
*/
package org.apache.hadoop.hbase.regionserver.wal;
-
-import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
+import static java.util.stream.Collectors.toCollection;
import java.io.IOException;
import java.util.Collections;
import java.util.List;
import java.util.Set;
import java.util.TreeSet;
-import static java.util.stream.Collectors.toCollection;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellComparator;
import org.apache.hadoop.hbase.CellUtil;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.CollectionUtils;
@@ -39,6 +36,9 @@ import org.apache.hadoop.hbase.wal.WAL.Entry;
import org.apache.hadoop.hbase.wal.WALEdit;
import org.apache.hadoop.hbase.wal.WALKey;
import org.apache.htrace.Span;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
/**
* A WAL Entry for {@link AbstractFSWAL} implementation. Immutable.
@@ -54,17 +54,17 @@ class FSWALEntry extends Entry {
// they are only in memory and held here while passing over the ring buffer.
private final transient long txid;
private final transient boolean inMemstore;
- private final transient HRegionInfo hri;
+ private final transient RegionInfo regionInfo;
private final transient Set<byte[]> familyNames;
// The tracing span for this entry when writing WAL.
private transient Span span;
FSWALEntry(final long txid, final WALKey key, final WALEdit edit,
- final HRegionInfo hri, final boolean inMemstore) {
+ final RegionInfo regionInfo, final boolean inMemstore) {
super(key, edit);
this.inMemstore = inMemstore;
- this.hri = hri;
+ this.regionInfo = regionInfo;
this.txid = txid;
if (inMemstore) {
// construct familyNames here to reduce the work of log sinker.
@@ -96,8 +96,8 @@ class FSWALEntry extends Entry {
return this.inMemstore;
}
- HRegionInfo getHRegionInfo() {
- return this.hri;
+ RegionInfo getRegionInfo() {
+ return this.regionInfo;
}
/**
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/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 73ba776..34f93fa 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
@@ -26,7 +26,7 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.coprocessor.BaseEnvironment;
import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
import org.apache.hadoop.hbase.coprocessor.MetricsCoprocessor;
@@ -147,7 +147,7 @@ public class WALCoprocessorHost
* @return true if default behavior should be bypassed, false otherwise
* @throws IOException
*/
- public boolean preWALWrite(final HRegionInfo info, final WALKey logKey, final WALEdit logEdit)
+ public boolean preWALWrite(final RegionInfo info, final WALKey logKey, final WALEdit logEdit)
throws IOException {
return execOperationWithResult(false, coprocEnvironments.isEmpty() ? null :
new ObserverOperationWithResult<WALObserver, Boolean>(walObserverGetter) {
@@ -164,7 +164,7 @@ public class WALCoprocessorHost
* @param logEdit
* @throws IOException
*/
- public void postWALWrite(final HRegionInfo info, final WALKey logKey, final WALEdit logEdit)
+ public void postWALWrite(final RegionInfo info, final WALKey logKey, final WALEdit logEdit)
throws IOException {
execOperation(coprocEnvironments.isEmpty() ? null : new WALObserverOperation() {
@Override
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALUtil.java
index 89c80bf..816a3b8 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALUtil.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALUtil.java
@@ -24,18 +24,18 @@ import java.util.NavigableMap;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.CompactionDescriptor;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.RegionEventDescriptor;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl;
import org.apache.hadoop.hbase.wal.WAL;
import org.apache.hadoop.hbase.wal.WALEdit;
import org.apache.hadoop.hbase.wal.WALKey;
+import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.TextFormat;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.CompactionDescriptor;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.RegionEventDescriptor;
/**
* Helper methods to ease Region Server integration with the Write Ahead Log (WAL).
@@ -60,7 +60,7 @@ public class WALUtil {
* @param mvcc Used by WAL to get sequence Id for the waledit.
*/
public static WALKey writeCompactionMarker(WAL wal,
- NavigableMap<byte[], Integer> replicationScope, HRegionInfo hri, final CompactionDescriptor c,
+ NavigableMap<byte[], Integer> replicationScope, RegionInfo hri, final CompactionDescriptor c,
MultiVersionConcurrencyControl mvcc)
throws IOException {
WALKey walKey = writeMarker(wal, replicationScope, hri, WALEdit.createCompaction(hri, c), mvcc);
@@ -76,7 +76,7 @@ public class WALUtil {
* <p>This write is for internal use only. Not for external client consumption.
*/
public static WALKey writeFlushMarker(WAL wal, NavigableMap<byte[], Integer> replicationScope,
- HRegionInfo hri, final FlushDescriptor f, boolean sync, MultiVersionConcurrencyControl mvcc)
+ RegionInfo hri, final FlushDescriptor f, boolean sync, MultiVersionConcurrencyControl mvcc)
throws IOException {
WALKey walKey = doFullAppendTransaction(wal, replicationScope, hri,
WALEdit.createFlushWALEdit(hri, f), mvcc, sync);
@@ -91,7 +91,7 @@ public class WALUtil {
* This write is for internal use only. Not for external client consumption.
*/
public static WALKey writeRegionEventMarker(WAL wal,
- NavigableMap<byte[], Integer> replicationScope, HRegionInfo hri,
+ NavigableMap<byte[], Integer> replicationScope, RegionInfo hri,
final RegionEventDescriptor r, final MultiVersionConcurrencyControl mvcc)
throws IOException {
WALKey walKey = writeMarker(wal, replicationScope, hri,
@@ -113,7 +113,7 @@ public class WALUtil {
* @throws IOException We will throw an IOException if we can not append to the HLog.
*/
public static WALKey writeBulkLoadMarkerAndSync(final WAL wal,
- final NavigableMap<byte[], Integer> replicationScope, final HRegionInfo hri,
+ final NavigableMap<byte[], Integer> replicationScope, final RegionInfo hri,
final WALProtos.BulkLoadDescriptor desc, final MultiVersionConcurrencyControl mvcc)
throws IOException {
WALKey walKey = writeMarker(wal, replicationScope, hri, WALEdit.createBulkLoadEvent(hri, desc),
@@ -125,7 +125,7 @@ public class WALUtil {
}
private static WALKey writeMarker(final WAL wal,
- final NavigableMap<byte[], Integer> replicationScope, final HRegionInfo hri,
+ final NavigableMap<byte[], Integer> replicationScope, final RegionInfo hri,
final WALEdit edit, final MultiVersionConcurrencyControl mvcc)
throws IOException {
// If sync == true in below, then timeout is not used; safe to pass UNSPECIFIED_TIMEOUT
@@ -141,7 +141,7 @@ public class WALUtil {
* @return WALKey that was added to the WAL.
*/
public static WALKey doFullAppendTransaction(final WAL wal,
- final NavigableMap<byte[], Integer> replicationScope, final HRegionInfo hri,
+ final NavigableMap<byte[], Integer> replicationScope, final RegionInfo hri,
final WALEdit edit, final MultiVersionConcurrencyControl mvcc, final boolean sync)
throws IOException {
// TODO: Pass in current time to use?
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RegionReplicaReplicationEndpoint.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RegionReplicaReplicationEndpoint.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RegionReplicaReplicationEndpoint.java
index 77759b7..e1e2514 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RegionReplicaReplicationEndpoint.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RegionReplicaReplicationEndpoint.java
@@ -41,16 +41,15 @@ import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseIOException;
import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.RegionLocations;
import org.apache.hadoop.hbase.TableDescriptors;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotFoundException;
-import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.ClusterConnection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.RegionAdminServiceCallable;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
import org.apache.hadoop.hbase.client.RetryingCallable;
import org.apache.hadoop.hbase.client.RpcRetryingCallerFactory;
@@ -58,8 +57,6 @@ import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.ipc.HBaseRpcController;
import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
import org.apache.hadoop.hbase.protobuf.ReplicationProtbufUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryResponse;
import org.apache.hadoop.hbase.replication.BaseWALEntryFilter;
import org.apache.hadoop.hbase.replication.ChainWALEntryFilter;
import org.apache.hadoop.hbase.replication.HBaseReplicationEndpoint;
@@ -74,10 +71,13 @@ import org.apache.hadoop.hbase.wal.WALSplitter.PipelineController;
import org.apache.hadoop.hbase.wal.WALSplitter.RegionEntryBuffer;
import org.apache.hadoop.hbase.wal.WALSplitter.SinkWriter;
import org.apache.hadoop.util.StringUtils;
+import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.common.cache.Cache;
import org.apache.hadoop.hbase.shaded.com.google.common.cache.CacheBuilder;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryResponse;
/**
* A {@link org.apache.hadoop.hbase.replication.ReplicationEndpoint} endpoint
@@ -534,7 +534,7 @@ public class RegionReplicaReplicationEndpoint extends HBaseReplicationEndpoint {
for (int replicaId = 0; replicaId < locations.size(); replicaId++) {
HRegionLocation location = locations.getRegionLocation(replicaId);
if (!RegionReplicaUtil.isDefaultReplica(replicaId)) {
- HRegionInfo regionInfo = location == null
+ RegionInfo regionInfo = location == null
? RegionReplicaUtil.getRegionInfoForReplica(
locations.getDefaultRegionLocation().getRegionInfo(), replicaId)
: location.getRegionInfo();
@@ -615,7 +615,7 @@ public class RegionReplicaReplicationEndpoint extends HBaseReplicationEndpoint {
public RegionReplicaReplayCallable(ClusterConnection connection,
RpcControllerFactory rpcControllerFactory, TableName tableName,
- HRegionLocation location, HRegionInfo regionInfo, byte[] row,List<Entry> entries,
+ HRegionLocation location, RegionInfo regionInfo, byte[] row,List<Entry> entries,
AtomicLong skippedEntries) {
super(connection, rpcControllerFactory, location, tableName, row, regionInfo.getReplicaId());
this.entries = entries;
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/RestoreSnapshotHelper.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/RestoreSnapshotHelper.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/RestoreSnapshotHelper.java
index 61d2007..e08d547 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/RestoreSnapshotHelper.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/RestoreSnapshotHelper.java
@@ -34,40 +34,43 @@ import java.util.Set;
import java.util.TreeMap;
import java.util.concurrent.ThreadPoolExecutor;
-import org.apache.hadoop.hbase.client.TableDescriptor;
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.ListMultimap;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
-import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.backup.HFileArchiver;
-import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
import org.apache.hadoop.hbase.io.HFileLink;
import org.apache.hadoop.hbase.io.Reference;
import org.apache.hadoop.hbase.mob.MobUtils;
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
import org.apache.hadoop.hbase.monitoring.TaskMonitor;
-import org.apache.hadoop.hbase.security.access.AccessControlClient;
-import org.apache.hadoop.hbase.security.access.ShadedAccessControlUtil;
-import org.apache.hadoop.hbase.security.access.TablePermission;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
+import org.apache.hadoop.hbase.security.access.AccessControlClient;
+import org.apache.hadoop.hbase.security.access.ShadedAccessControlUtil;
+import org.apache.hadoop.hbase.security.access.TablePermission;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.ModifyRegionUtils;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.io.IOUtils;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.ListMultimap;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest;
/**
* Helper to Restore/Clone a Snapshot
@@ -193,14 +196,14 @@ public class RestoreSnapshotHelper {
// this instance, by removing the regions already present in the restore dir.
Set<String> regionNames = new HashSet<>(regionManifests.keySet());
- HRegionInfo mobRegion = MobUtils.getMobRegionInfo(snapshotManifest.getTableDescriptor()
+ RegionInfo mobRegion = MobUtils.getMobRegionInfo(snapshotManifest.getTableDescriptor()
.getTableName());
// Identify which region are still available and which not.
// NOTE: we rely upon the region name as: "table name, start key, end key"
- List<HRegionInfo> tableRegions = getTableRegions();
+ List<RegionInfo> tableRegions = getTableRegions();
if (tableRegions != null) {
monitor.rethrowException();
- for (HRegionInfo regionInfo: tableRegions) {
+ for (RegionInfo regionInfo: tableRegions) {
String regionName = regionInfo.getEncodedName();
if (regionNames.contains(regionName)) {
LOG.info("region to restore: " + regionName);
@@ -217,7 +220,7 @@ public class RestoreSnapshotHelper {
status.setStatus("Restoring table regions...");
if (regionNames.contains(mobRegion.getEncodedName())) {
// restore the mob region in case
- List<HRegionInfo> mobRegions = new ArrayList<>(1);
+ List<RegionInfo> mobRegions = new ArrayList<>(1);
mobRegions.add(mobRegion);
restoreHdfsMobRegions(exec, regionManifests, mobRegions);
regionNames.remove(mobRegion.getEncodedName());
@@ -234,7 +237,7 @@ public class RestoreSnapshotHelper {
// Regions to Add: present in the snapshot but not in the current table
if (regionNames.size() > 0) {
- List<HRegionInfo> regionsToAdd = new ArrayList<>(regionNames.size());
+ List<RegionInfo> regionsToAdd = new ArrayList<>(regionNames.size());
monitor.rethrowException();
// add the mob region
@@ -244,13 +247,13 @@ public class RestoreSnapshotHelper {
}
for (String regionName: regionNames) {
LOG.info("region to add: " + regionName);
- regionsToAdd.add(HRegionInfo.convert(regionManifests.get(regionName).getRegionInfo()));
+ regionsToAdd.add(ProtobufUtil.toRegionInfo(regionManifests.get(regionName).getRegionInfo()));
}
// Create new regions cloning from the snapshot
monitor.rethrowException();
status.setStatus("Cloning regions...");
- HRegionInfo[] clonedRegions = cloneHdfsRegions(exec, regionManifests, regionsToAdd);
+ RegionInfo[] clonedRegions = cloneHdfsRegions(exec, regionManifests, regionsToAdd);
metaChanges.setNewRegions(clonedRegions);
status.setStatus("Finished cloning regions.");
}
@@ -267,9 +270,9 @@ public class RestoreSnapshotHelper {
private final Map<String, Pair<String, String> > parentsMap;
private final TableDescriptor htd;
- private List<HRegionInfo> regionsToRestore = null;
- private List<HRegionInfo> regionsToRemove = null;
- private List<HRegionInfo> regionsToAdd = null;
+ private List<RegionInfo> regionsToRestore = null;
+ private List<RegionInfo> regionsToRemove = null;
+ private List<RegionInfo> regionsToAdd = null;
public RestoreMetaChanges(TableDescriptor htd, Map<String, Pair<String, String> > parentsMap) {
this.parentsMap = parentsMap;
@@ -301,7 +304,7 @@ public class RestoreSnapshotHelper {
* e.g MetaTableAccessor.addRegionsToMeta(...)
* @return the list of regions to add to META
*/
- public List<HRegionInfo> getRegionsToAdd() {
+ public List<RegionInfo> getRegionsToAdd() {
return this.regionsToAdd;
}
@@ -317,7 +320,7 @@ public class RestoreSnapshotHelper {
* The caller is responsible to add the regions to hbase:meta if not present.
* @return the list of regions restored
*/
- public List<HRegionInfo> getRegionsToRestore() {
+ public List<RegionInfo> getRegionsToRestore() {
return this.regionsToRestore;
}
@@ -334,11 +337,11 @@ public class RestoreSnapshotHelper {
* e.g. MetaTableAccessor.deleteRegions(...)
* @return the list of regions to remove from META
*/
- public List<HRegionInfo> getRegionsToRemove() {
+ public List<RegionInfo> getRegionsToRemove() {
return this.regionsToRemove;
}
- void setNewRegions(final HRegionInfo[] hris) {
+ void setNewRegions(final RegionInfo[] hris) {
if (hris != null) {
regionsToAdd = Arrays.asList(hris);
} else {
@@ -346,14 +349,14 @@ public class RestoreSnapshotHelper {
}
}
- void addRegionToRemove(final HRegionInfo hri) {
+ void addRegionToRemove(final RegionInfo hri) {
if (regionsToRemove == null) {
regionsToRemove = new LinkedList<>();
}
regionsToRemove.add(hri);
}
- void addRegionToRestore(final HRegionInfo hri) {
+ void addRegionToRestore(final RegionInfo hri) {
if (regionsToRestore == null) {
regionsToRestore = new LinkedList<>();
}
@@ -361,13 +364,13 @@ public class RestoreSnapshotHelper {
}
public void updateMetaParentRegions(Connection connection,
- final List<HRegionInfo> regionInfos) throws IOException {
+ final List<RegionInfo> regionInfos) throws IOException {
if (regionInfos == null || parentsMap.isEmpty()) return;
// Extract region names and offlined regions
- Map<String, HRegionInfo> regionsByName = new HashMap<>(regionInfos.size());
- List<HRegionInfo> parentRegions = new LinkedList<>();
- for (HRegionInfo regionInfo: regionInfos) {
+ Map<String, RegionInfo> regionsByName = new HashMap<>(regionInfos.size());
+ List<RegionInfo> parentRegions = new LinkedList<>();
+ for (RegionInfo regionInfo: regionInfos) {
if (regionInfo.isSplitParent()) {
parentRegions.add(regionInfo);
} else {
@@ -376,7 +379,7 @@ public class RestoreSnapshotHelper {
}
// Update Offline parents
- for (HRegionInfo regionInfo: parentRegions) {
+ for (RegionInfo regionInfo: parentRegions) {
Pair<String, String> daughters = parentsMap.get(regionInfo.getEncodedName());
if (daughters == null) {
// The snapshot contains an unreferenced region.
@@ -401,12 +404,12 @@ public class RestoreSnapshotHelper {
/**
* Remove specified regions from the file-system, using the archiver.
*/
- private void removeHdfsRegions(final ThreadPoolExecutor exec, final List<HRegionInfo> regions)
+ private void removeHdfsRegions(final ThreadPoolExecutor exec, final List<RegionInfo> regions)
throws IOException {
if (regions == null || regions.isEmpty()) return;
ModifyRegionUtils.editRegions(exec, regions, new ModifyRegionUtils.RegionEditTask() {
@Override
- public void editRegion(final HRegionInfo hri) throws IOException {
+ public void editRegion(final RegionInfo hri) throws IOException {
HFileArchiver.archiveRegion(conf, fs, hri);
}
});
@@ -417,11 +420,11 @@ public class RestoreSnapshotHelper {
*/
private void restoreHdfsRegions(final ThreadPoolExecutor exec,
final Map<String, SnapshotRegionManifest> regionManifests,
- final List<HRegionInfo> regions) throws IOException {
+ final List<RegionInfo> regions) throws IOException {
if (regions == null || regions.isEmpty()) return;
ModifyRegionUtils.editRegions(exec, regions, new ModifyRegionUtils.RegionEditTask() {
@Override
- public void editRegion(final HRegionInfo hri) throws IOException {
+ public void editRegion(final RegionInfo hri) throws IOException {
restoreRegion(hri, regionManifests.get(hri.getEncodedName()));
}
});
@@ -432,11 +435,11 @@ public class RestoreSnapshotHelper {
*/
private void restoreHdfsMobRegions(final ThreadPoolExecutor exec,
final Map<String, SnapshotRegionManifest> regionManifests,
- final List<HRegionInfo> regions) throws IOException {
+ final List<RegionInfo> regions) throws IOException {
if (regions == null || regions.isEmpty()) return;
ModifyRegionUtils.editRegions(exec, regions, new ModifyRegionUtils.RegionEditTask() {
@Override
- public void editRegion(final HRegionInfo hri) throws IOException {
+ public void editRegion(final RegionInfo hri) throws IOException {
restoreMobRegion(hri, regionManifests.get(hri.getEncodedName()));
}
});
@@ -457,7 +460,7 @@ public class RestoreSnapshotHelper {
* Restore region by removing files not in the snapshot
* and adding the missing ones from the snapshot.
*/
- private void restoreRegion(final HRegionInfo regionInfo,
+ private void restoreRegion(final RegionInfo regionInfo,
final SnapshotRegionManifest regionManifest) throws IOException {
restoreRegion(regionInfo, regionManifest, new Path(tableDir, regionInfo.getEncodedName()));
}
@@ -466,7 +469,7 @@ public class RestoreSnapshotHelper {
* Restore mob region by removing files not in the snapshot
* and adding the missing ones from the snapshot.
*/
- private void restoreMobRegion(final HRegionInfo regionInfo,
+ private void restoreMobRegion(final RegionInfo regionInfo,
final SnapshotRegionManifest regionManifest) throws IOException {
if (regionManifest == null) {
return;
@@ -479,7 +482,7 @@ public class RestoreSnapshotHelper {
* Restore region by removing files not in the snapshot
* and adding the missing ones from the snapshot.
*/
- private void restoreRegion(final HRegionInfo regionInfo,
+ private void restoreRegion(final RegionInfo regionInfo,
final SnapshotRegionManifest regionManifest, Path regionDir) throws IOException {
Map<String, List<SnapshotRegionManifest.StoreFile>> snapshotFiles =
getRegionHFileReferences(regionManifest);
@@ -562,18 +565,18 @@ public class RestoreSnapshotHelper {
* Clone specified regions. For each region create a new region
* and create a HFileLink for each hfile.
*/
- private HRegionInfo[] cloneHdfsRegions(final ThreadPoolExecutor exec,
+ private RegionInfo[] cloneHdfsRegions(final ThreadPoolExecutor exec,
final Map<String, SnapshotRegionManifest> regionManifests,
- final List<HRegionInfo> regions) throws IOException {
+ final List<RegionInfo> regions) throws IOException {
if (regions == null || regions.isEmpty()) return null;
- final Map<String, HRegionInfo> snapshotRegions = new HashMap<>(regions.size());
+ final Map<String, RegionInfo> snapshotRegions = new HashMap<>(regions.size());
// clone region info (change embedded tableName with the new one)
- HRegionInfo[] clonedRegionsInfo = new HRegionInfo[regions.size()];
+ RegionInfo[] clonedRegionsInfo = new RegionInfo[regions.size()];
for (int i = 0; i < clonedRegionsInfo.length; ++i) {
// clone the region info from the snapshot region info
- HRegionInfo snapshotRegionInfo = regions.get(i);
+ RegionInfo snapshotRegionInfo = regions.get(i);
clonedRegionsInfo[i] = cloneRegionInfo(snapshotRegionInfo);
// add the region name mapping between snapshot and cloned
@@ -591,7 +594,7 @@ public class RestoreSnapshotHelper {
tableDesc, clonedRegionsInfo, new ModifyRegionUtils.RegionFillTask() {
@Override
public void fillRegion(final HRegion region) throws IOException {
- HRegionInfo snapshotHri = snapshotRegions.get(region.getRegionInfo().getEncodedName());
+ RegionInfo snapshotHri = snapshotRegions.get(region.getRegionInfo().getEncodedName());
cloneRegion(region, snapshotHri, regionManifests.get(snapshotHri.getEncodedName()));
}
});
@@ -604,7 +607,7 @@ public class RestoreSnapshotHelper {
* and create a HFileLink for each hfile.
*/
private void cloneHdfsMobRegion(final Map<String, SnapshotRegionManifest> regionManifests,
- final HRegionInfo region) throws IOException {
+ final RegionInfo region) throws IOException {
// clone region info (change embedded tableName with the new one)
Path clonedRegionPath = MobUtils.getMobRegionPath(conf, tableDesc.getTableName());
cloneRegion(clonedRegionPath, region, regionManifests.get(region.getEncodedName()));
@@ -621,7 +624,7 @@ public class RestoreSnapshotHelper {
* @param regionDir {@link Path} cloned dir
* @param snapshotRegionInfo
*/
- private void cloneRegion(final Path regionDir, final HRegionInfo snapshotRegionInfo,
+ private void cloneRegion(final Path regionDir, final RegionInfo snapshotRegionInfo,
final SnapshotRegionManifest manifest) throws IOException {
final String tableName = tableDesc.getTableName().getNameAsString();
for (SnapshotRegionManifest.FamilyFiles familyFiles: manifest.getFamilyFilesList()) {
@@ -644,7 +647,7 @@ public class RestoreSnapshotHelper {
* @param region {@link HRegion} cloned
* @param snapshotRegionInfo
*/
- private void cloneRegion(final HRegion region, final HRegionInfo snapshotRegionInfo,
+ private void cloneRegion(final HRegion region, final RegionInfo snapshotRegionInfo,
final SnapshotRegionManifest manifest) throws IOException {
cloneRegion(new Path(tableDir, region.getRegionInfo().getEncodedName()), snapshotRegionInfo,
manifest);
@@ -663,7 +666,7 @@ public class RestoreSnapshotHelper {
* @param createBackRef - Whether back reference should be created. Defaults to true.
* @param storeFile store file name (can be a Reference, HFileLink or simple HFile)
*/
- private void restoreStoreFile(final Path familyDir, final HRegionInfo regionInfo,
+ private void restoreStoreFile(final Path familyDir, final RegionInfo regionInfo,
final SnapshotRegionManifest.StoreFile storeFile, final boolean createBackRef)
throws IOException {
String hfileName = storeFile.getName();
@@ -694,7 +697,7 @@ public class RestoreSnapshotHelper {
* @param regionInfo destination region info for the table
* @param storeFile reference file name
*/
- private void restoreReferenceFile(final Path familyDir, final HRegionInfo regionInfo,
+ private void restoreReferenceFile(final Path familyDir, final RegionInfo regionInfo,
final SnapshotRegionManifest.StoreFile storeFile) throws IOException {
String hfileName = storeFile.getName();
@@ -753,36 +756,38 @@ public class RestoreSnapshotHelper {
}
/**
- * Create a new {@link HRegionInfo} from the snapshot region info.
+ * Create a new {@link RegionInfo} from the snapshot region info.
* Keep the same startKey, endKey, regionId and split information but change
* the table name.
*
* @param snapshotRegionInfo Info for region to clone.
* @return the new HRegion instance
*/
- public HRegionInfo cloneRegionInfo(final HRegionInfo snapshotRegionInfo) {
+ public RegionInfo cloneRegionInfo(final RegionInfo snapshotRegionInfo) {
return cloneRegionInfo(tableDesc.getTableName(), snapshotRegionInfo);
}
- public static HRegionInfo cloneRegionInfo(TableName tableName, HRegionInfo snapshotRegionInfo) {
- HRegionInfo regionInfo = new HRegionInfo(tableName,
- snapshotRegionInfo.getStartKey(), snapshotRegionInfo.getEndKey(),
- snapshotRegionInfo.isSplit(), snapshotRegionInfo.getRegionId());
- regionInfo.setOffline(snapshotRegionInfo.isOffline());
- return regionInfo;
+ public static RegionInfo cloneRegionInfo(TableName tableName, RegionInfo snapshotRegionInfo) {
+ return RegionInfoBuilder.newBuilder(tableName)
+ .setStartKey(snapshotRegionInfo.getStartKey())
+ .setEndKey(snapshotRegionInfo.getEndKey())
+ .setSplit(snapshotRegionInfo.isSplit())
+ .setRegionId(snapshotRegionInfo.getRegionId())
+ .setOffline(snapshotRegionInfo.isOffline())
+ .build();
}
/**
* @return the set of the regions contained in the table
*/
- private List<HRegionInfo> getTableRegions() throws IOException {
+ private List<RegionInfo> getTableRegions() throws IOException {
LOG.debug("get table regions: " + tableDir);
FileStatus[] regionDirs = FSUtils.listStatus(fs, tableDir, new FSUtils.RegionDirFilter(fs));
if (regionDirs == null) return null;
- List<HRegionInfo> regions = new ArrayList<>(regionDirs.length);
+ List<RegionInfo> regions = new ArrayList<>(regionDirs.length);
for (int i = 0; i < regionDirs.length; ++i) {
- HRegionInfo hri = HRegionFileSystem.loadRegionInfoFileContent(fs, regionDirs[i].getPath());
+ RegionInfo hri = HRegionFileSystem.loadRegionInfoFileContent(fs, regionDirs[i].getPath());
regions.add(hri);
}
LOG.debug("found " + regions.size() + " regions for table=" +
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotInfo.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotInfo.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotInfo.java
index 9c8ca75..c76155c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotInfo.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotInfo.java
@@ -18,8 +18,8 @@
package org.apache.hadoop.hbase.snapshot;
-import java.io.IOException;
import java.io.FileNotFoundException;
+import java.io.IOException;
import java.net.URI;
import java.text.SimpleDateFormat;
import java.util.ArrayList;
@@ -36,25 +36,23 @@ import org.apache.commons.cli.CommandLine;
import org.apache.commons.cli.Option;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
-
-import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.hbase.client.SnapshotDescription;
-import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.SnapshotDescription;
+import org.apache.hadoop.hbase.io.HFileLink;
+import org.apache.hadoop.hbase.io.WALLink;
import org.apache.hadoop.hbase.util.AbstractHBaseTool;
+import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.util.StringUtils;
+import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.io.HFileLink;
-import org.apache.hadoop.hbase.io.WALLink;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest;
-import org.apache.hadoop.hbase.util.FSUtils;
/**
* Tool for dumping snapshot information.
@@ -294,7 +292,7 @@ public final class SnapshotInfo extends AbstractHBaseTool {
* @param filesMap store files map for all snapshots, it may be null
* @return the store file information
*/
- FileInfo addStoreFile(final HRegionInfo region, final String family,
+ FileInfo addStoreFile(final RegionInfo region, final String family,
final SnapshotRegionManifest.StoreFile storeFile,
final Map<Path, Integer> filesMap) throws IOException {
HFileLink link = HFileLink.build(conf, snapshotTable, region.getEncodedName(),
@@ -466,7 +464,7 @@ public final class SnapshotInfo extends AbstractHBaseTool {
"SnapshotInfo",
new SnapshotReferenceUtil.SnapshotVisitor() {
@Override
- public void storeFile(final HRegionInfo regionInfo, final String family,
+ public void storeFile(final RegionInfo regionInfo, final String family,
final SnapshotRegionManifest.StoreFile storeFile) throws IOException {
if (storeFile.hasReference()) return;
@@ -576,7 +574,7 @@ public final class SnapshotInfo extends AbstractHBaseTool {
SnapshotReferenceUtil.concurrentVisitReferencedFiles(conf, fs, manifest,
"SnapshotsStatsAggregation", new SnapshotReferenceUtil.SnapshotVisitor() {
@Override
- public void storeFile(final HRegionInfo regionInfo, final String family,
+ public void storeFile(final RegionInfo regionInfo, final String family,
final SnapshotRegionManifest.StoreFile storeFile) throws IOException {
if (!storeFile.hasReference()) {
stats.addStoreFile(regionInfo, family, storeFile, filesMap);
@@ -631,7 +629,7 @@ public final class SnapshotInfo extends AbstractHBaseTool {
SnapshotManifest manifest = SnapshotManifest.open(conf, fs, snapshotDir, snapshotDesc);
SnapshotReferenceUtil.concurrentVisitReferencedFiles(conf, fs, manifest, exec,
new SnapshotReferenceUtil.SnapshotVisitor() {
- @Override public void storeFile(final HRegionInfo regionInfo, final String family,
+ @Override public void storeFile(final RegionInfo regionInfo, final String family,
final SnapshotRegionManifest.StoreFile storeFile) throws IOException {
if (!storeFile.hasReference()) {
HFileLink link = HFileLink.build(conf, snapshot.getTableName(),
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifest.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifest.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifest.java
index b371b3e..1a7c7f0 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifest.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifest.java
@@ -36,8 +36,8 @@ import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.errorhandling.ForeignExceptionSnare;
import org.apache.hadoop.hbase.mob.MobUtils;
@@ -142,7 +142,7 @@ public final class SnapshotManifest {
}
interface RegionVisitor<TRegion, TFamily> {
- TRegion regionOpen(final HRegionInfo regionInfo) throws IOException;
+ TRegion regionOpen(final RegionInfo regionInfo) throws IOException;
void regionClose(final TRegion region) throws IOException;
TFamily familyOpen(final TRegion region, final byte[] familyName) throws IOException;
@@ -164,7 +164,7 @@ public final class SnapshotManifest {
}
}
- public void addMobRegion(HRegionInfo regionInfo) throws IOException {
+ public void addMobRegion(RegionInfo regionInfo) throws IOException {
// Get the ManifestBuilder/RegionVisitor
RegionVisitor visitor = createRegionVisitor(desc);
@@ -173,7 +173,7 @@ public final class SnapshotManifest {
}
@VisibleForTesting
- protected void addMobRegion(HRegionInfo regionInfo, RegionVisitor visitor) throws IOException {
+ protected void addMobRegion(RegionInfo regionInfo, RegionVisitor visitor) throws IOException {
// 1. dump region meta info into the snapshot directory
LOG.debug("Storing mob region '" + regionInfo + "' region-info for snapshot.");
Object regionData = visitor.regionOpen(regionInfo);
@@ -258,7 +258,7 @@ public final class SnapshotManifest {
* Creates a 'manifest' for the specified region, by reading directly from the disk.
* This is used by the "offline snapshot" when the table is disabled.
*/
- public void addRegion(final Path tableDir, final HRegionInfo regionInfo) throws IOException {
+ public void addRegion(final Path tableDir, final RegionInfo regionInfo) throws IOException {
// Get the ManifestBuilder/RegionVisitor
RegionVisitor visitor = createRegionVisitor(desc);
@@ -267,7 +267,7 @@ public final class SnapshotManifest {
}
@VisibleForTesting
- protected void addRegion(final Path tableDir, final HRegionInfo regionInfo, RegionVisitor visitor)
+ protected void addRegion(final Path tableDir, final RegionInfo regionInfo, RegionVisitor visitor)
throws IOException {
boolean isMobRegion = MobUtils.isMobRegionInfo(regionInfo);
try {
@@ -566,11 +566,11 @@ public final class SnapshotManifest {
* Extract the region encoded name from the region manifest
*/
static String getRegionNameFromManifest(final SnapshotRegionManifest manifest) {
- byte[] regionName = HRegionInfo.createRegionName(
+ byte[] regionName = RegionInfo.createRegionName(
ProtobufUtil.toTableName(manifest.getRegionInfo().getTableName()),
manifest.getRegionInfo().getStartKey().toByteArray(),
manifest.getRegionInfo().getRegionId(), true);
- return HRegionInfo.encodeRegionName(regionName);
+ return RegionInfo.encodeRegionName(regionName);
}
/*
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifestV1.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifestV1.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifestV1.java
index ab9e8b2..61cbbd1 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifestV1.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifestV1.java
@@ -34,15 +34,17 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.yetus.audience.InterfaceAudience;
+
import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.FSUtils;
/**
* DO NOT USE DIRECTLY. USE {@link SnapshotManifest}.
@@ -74,7 +76,7 @@ public final class SnapshotManifestV1 {
this.fs = fs;
}
- public HRegionFileSystem regionOpen(final HRegionInfo regionInfo) throws IOException {
+ public HRegionFileSystem regionOpen(final RegionInfo regionInfo) throws IOException {
HRegionFileSystem snapshotRegionFs = HRegionFileSystem.createRegionOnFileSystem(conf,
fs, snapshotDir, regionInfo);
return snapshotRegionFs;
@@ -126,7 +128,7 @@ public final class SnapshotManifestV1 {
completionService.submit(new Callable<SnapshotRegionManifest>() {
@Override
public SnapshotRegionManifest call() throws IOException {
- HRegionInfo hri = HRegionFileSystem.loadRegionInfoFileContent(fs, region.getPath());
+ RegionInfo hri = HRegionFileSystem.loadRegionInfoFileContent(fs, region.getPath());
return buildManifestFromDisk(conf, fs, snapshotDir, hri);
}
});
@@ -154,14 +156,14 @@ public final class SnapshotManifestV1 {
}
static SnapshotRegionManifest buildManifestFromDisk(final Configuration conf,
- final FileSystem fs, final Path tableDir, final HRegionInfo regionInfo) throws IOException {
+ final FileSystem fs, final Path tableDir, final RegionInfo regionInfo) throws IOException {
HRegionFileSystem regionFs = HRegionFileSystem.openRegionFromFileSystem(conf, fs,
tableDir, regionInfo, true);
SnapshotRegionManifest.Builder manifest = SnapshotRegionManifest.newBuilder();
// 1. dump region meta info into the snapshot directory
LOG.debug("Storing region-info for snapshot.");
- manifest.setRegionInfo(HRegionInfo.convert(regionInfo));
+ manifest.setRegionInfo(ProtobufUtil.toRegionInfo(regionInfo));
// 2. iterate through all the stores in the region
LOG.debug("Creating references for hfiles");
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifestV2.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifestV2.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifestV2.java
index af610bc..561eb77 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifestV2.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifestV2.java
@@ -18,22 +18,17 @@
package org.apache.hadoop.hbase.snapshot;
-import org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream;
-import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
-import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
-
import java.io.IOException;
import java.io.InterruptedIOException;
import java.util.ArrayList;
import java.util.List;
import java.util.concurrent.Callable;
-import java.util.concurrent.Executor;
import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
import java.util.concurrent.ExecutorCompletionService;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
-import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
@@ -41,11 +36,17 @@ import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.PathFilter;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest;
/**
* DO NOT USE DIRECTLY. USE {@link SnapshotManifest}.
@@ -77,9 +78,9 @@ public final class SnapshotManifestV2 {
this.fs = fs;
}
- public SnapshotRegionManifest.Builder regionOpen(final HRegionInfo regionInfo) {
+ public SnapshotRegionManifest.Builder regionOpen(final RegionInfo regionInfo) {
SnapshotRegionManifest.Builder manifest = SnapshotRegionManifest.newBuilder();
- manifest.setRegionInfo(HRegionInfo.convert(regionInfo));
+ manifest.setRegionInfo(ProtobufUtil.toRegionInfo(regionInfo));
return manifest;
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotReferenceUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotReferenceUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotReferenceUtil.java
index 04798c1..0cca62f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotReferenceUtil.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotReferenceUtil.java
@@ -25,26 +25,27 @@ import java.util.HashSet;
import java.util.List;
import java.util.Set;
import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutorService;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.ExecutorService;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
-import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.io.HFileLink;
import org.apache.hadoop.hbase.mob.MobUtils;
+import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
+import org.apache.hadoop.hbase.util.HFileArchiveUtil;
+import org.apache.yetus.audience.InterfaceAudience;
+
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest;
-import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
-import org.apache.hadoop.hbase.util.HFileArchiveUtil;
/**
* Utility methods for interacting with the snapshot referenced files.
@@ -54,7 +55,7 @@ public final class SnapshotReferenceUtil {
private static final Log LOG = LogFactory.getLog(SnapshotReferenceUtil.class);
public interface StoreFileVisitor {
- void storeFile(final HRegionInfo regionInfo, final String familyName,
+ void storeFile(final RegionInfo regionInfo, final String familyName,
final SnapshotRegionManifest.StoreFile storeFile) throws IOException;
}
@@ -131,7 +132,7 @@ public final class SnapshotReferenceUtil {
*/
static void visitRegionStoreFiles(final SnapshotRegionManifest manifest,
final StoreFileVisitor visitor) throws IOException {
- HRegionInfo regionInfo = HRegionInfo.convert(manifest.getRegionInfo());
+ RegionInfo regionInfo = ProtobufUtil.toRegionInfo(manifest.getRegionInfo());
for (SnapshotRegionManifest.FamilyFiles familyFiles: manifest.getFamilyFilesList()) {
String familyName = familyFiles.getFamilyName().toStringUtf8();
for (SnapshotRegionManifest.StoreFile storeFile: familyFiles.getStoreFilesList()) {
@@ -171,7 +172,7 @@ public final class SnapshotReferenceUtil {
final Path snapshotDir = manifest.getSnapshotDir();
concurrentVisitReferencedFiles(conf, fs, manifest, "VerifySnapshot", new StoreFileVisitor() {
@Override
- public void storeFile(final HRegionInfo regionInfo, final String family,
+ public void storeFile(final RegionInfo regionInfo, final String family,
final SnapshotRegionManifest.StoreFile storeFile) throws IOException {
verifyStoreFile(conf, fs, snapshotDir, snapshotDesc, regionInfo, family, storeFile);
}
@@ -245,14 +246,14 @@ public final class SnapshotReferenceUtil {
* @param fs {@link FileSystem}
* @param snapshotDir {@link Path} to the Snapshot directory of the snapshot to verify
* @param snapshot the {@link SnapshotDescription} of the snapshot to verify
- * @param regionInfo {@link HRegionInfo} of the region that contains the store file
+ * @param regionInfo {@link RegionInfo} of the region that contains the store file
* @param family family that contains the store file
* @param storeFile the store file to verify
* @throws CorruptedSnapshotException if the snapshot is corrupted
* @throws IOException if an error occurred while scanning the directory
*/
private static void verifyStoreFile(final Configuration conf, final FileSystem fs,
- final Path snapshotDir, final SnapshotDescription snapshot, final HRegionInfo regionInfo,
+ final Path snapshotDir, final SnapshotDescription snapshot, final RegionInfo regionInfo,
final String family, final SnapshotRegionManifest.StoreFile storeFile) throws IOException {
TableName table = TableName.valueOf(snapshot.getTable());
String fileName = storeFile.getName();
@@ -347,7 +348,7 @@ public final class SnapshotReferenceUtil {
final Set<String> names = new HashSet<>();
visitTableStoreFiles(conf, fs, snapshotDir, snapshotDesc, new StoreFileVisitor() {
@Override
- public void storeFile(final HRegionInfo regionInfo, final String family,
+ public void storeFile(final RegionInfo regionInfo, final String family,
final SnapshotRegionManifest.StoreFile storeFile) throws IOException {
String hfile = storeFile.getName();
if (HFileLink.isHFileLink(hfile)) {